From 7fe38535228ea5f4414837dee2e03a70d031bc1e Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Fri, 10 May 2024 21:22:54 -0700 Subject: [PATCH 001/115] autorepair v2 framework --- .../pages/managing/operating/metrics.adoc | 50 ++ .../org/apache/cassandra/config/Config.java | 4 + .../cassandra/config/DatabaseDescriptor.java | 17 +- .../apache/cassandra/config/DurationSpec.java | 3 +- .../cassandra/cql3/UntypedResultSet.java | 7 + .../statements/ModificationStatement.java | 2 +- .../statements/schema/TableAttributes.java | 9 + .../cassandra/metrics/AutoRepairMetrics.java | 165 ++++ .../metrics/AutoRepairMetricsManager.java | 34 + .../repair/RepairMessageVerbHandler.java | 7 + .../repair/autorepair/AutoRepair.java | 402 +++++++++ .../repair/autorepair/AutoRepairConfig.java | 408 +++++++++ .../repair/autorepair/AutoRepairKeyspace.java | 84 ++ .../repair/autorepair/AutoRepairState.java | 349 ++++++++ .../repair/autorepair/AutoRepairUtils.java | 811 ++++++++++++++++++ .../DefaultAutoRepairTokenSplitter.java | 88 ++ .../IAutoRepairTokenRangeSplitter.java | 31 + .../cassandra/schema/AutoRepairParams.java | 145 ++++ .../cassandra/schema/SchemaKeyspace.java | 14 +- .../apache/cassandra/schema/TableParams.java | 57 +- .../service/ActiveRepairService.java | 51 ++ .../cassandra/service/AutoRepairService.java | 147 ++++ .../service/AutoRepairServiceMBean.java | 60 ++ .../cassandra/service/CassandraDaemon.java | 2 + .../cassandra/service/StorageService.java | 15 +- .../org/apache/cassandra/tools/NodeProbe.java | 85 ++ .../org/apache/cassandra/tools/NodeTool.java | 2 + .../tools/nodetool/GetAutoRepairConfig.java | 77 ++ .../tools/nodetool/SetAutoRepairConfig.java | 156 ++++ .../apache/cassandra/utils/FBUtilities.java | 14 + test/unit/org/apache/cassandra/Util.java | 14 + .../config/YamlConfigurationLoaderTest.java | 31 +- .../autorepair/AutoRepairConfigTest.java | 453 ++++++++++ .../autorepair/AutoRepairKeyspaceTest.java | 64 ++ .../AutoRepairParameterizedTest.java | 581 +++++++++++++ .../AutoRepairStateFactoryTest.java | 52 ++ .../autorepair/AutoRepairStateTest.java | 382 +++++++++ .../repair/autorepair/AutoRepairTest.java | 119 +++ .../autorepair/AutoRepairUtilsTest.java | 478 +++++++++++ .../service/ActiveRepairServiceTest.java | 49 +- 40 files changed, 5497 insertions(+), 22 deletions(-) create mode 100644 src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java create mode 100644 src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java create mode 100644 src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java create mode 100644 src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java create mode 100644 src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.java create mode 100644 src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java create mode 100644 src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java create mode 100644 src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java create mode 100644 src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java create mode 100644 src/java/org/apache/cassandra/schema/AutoRepairParams.java create mode 100644 src/java/org/apache/cassandra/service/AutoRepairService.java create mode 100644 src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java create mode 100644 src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java create mode 100644 src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java create mode 100644 test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java create mode 100644 test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java create mode 100644 test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java create mode 100644 test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java create mode 100644 test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java create mode 100644 test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java create mode 100644 test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java diff --git a/doc/modules/cassandra/pages/managing/operating/metrics.adoc b/doc/modules/cassandra/pages/managing/operating/metrics.adoc index 78dbd165867a..bce50953e01e 100644 --- a/doc/modules/cassandra/pages/managing/operating/metrics.adoc +++ b/doc/modules/cassandra/pages/managing/operating/metrics.adoc @@ -1078,6 +1078,56 @@ partitions processed per logged batch partitions processed per unlogged batch |=== +== Automated Repair Metrics + +Metrics specifc to automated repair. + +Reported name format: + +*Metric Name*:: +`org.apache.cassandra.metrics.AutoRepairMetrics.` +*JMX MBean*:: +`org.apache.cassandra.metrics:type=AutoRepairMetrics name=` + +[cols=",,",options="header",] +|=== +|Name |Type |Description +|RepairsInProgress |Gauge |Repair is in progress +on the node + +|NodeRepairTimeInSec |Gauge |Time taken to repair +the node + +|ClusterRepairTimeInSec |Gauge |Time taken to repair +the entire Cassandra cluster + +|SkippedTablesCount |Gauge |Number of tables skipped +on the node + +|LongestUnrepairedSec |Gauge |Time since the last repair +ran on the node + +|FailedTablesCount |Gauge |Number of tables encountered +failure during repair on the node + +|TotalMVTablesConsideredForRepair |Gauge |Number of materialized +views considered on the node + +|TotalDisabledRepairTables |Gauge |Number of tables on which +the automated repair has been disabled on the node + +|RepairTurnMyTurn |Counter |Represents the node's turn to repair + +|RepairTurnMyTurnDueToPriority |Counter |Represents the node's turn to repair +due to priority set in the automated repair + +|RepairTurnMyTurnForceRepair |Counter |Represents the node's turn to repair +due to force repair set in the automated repair + + +|=== + + == JVM Metrics JVM metrics such as memory and garbage collection statistics can either diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 119cfbfb7da0..17afdaaa3a73 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -32,6 +32,8 @@ import com.google.common.base.Joiner; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; + +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -352,6 +354,7 @@ public MemtableOptions() // The number of executors to use for building secondary indexes public volatile int concurrent_index_builders = 2; + public volatile double incremental_repair_disk_headroom_reject_ratio = 0.2; // at least 20% of disk must be unused to run incremental repair /** * @deprecated retry support removed on CASSANDRA-10992 @@ -947,6 +950,7 @@ public static void setClientMode(boolean clientMode) public volatile boolean password_validator_reconfiguration_enabled = true; public volatile CustomGuardrailConfig password_validator = new CustomGuardrailConfig(); + public volatile AutoRepairConfig auto_repair = new AutoRepairConfig(); /** * The variants of paxos implementation and semantics supported by Cassandra. diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 784a07932034..874e59f81204 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -62,7 +62,6 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.google.common.util.concurrent.RateLimiter; - import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -107,6 +106,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.SeedProvider; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.security.AbstractCryptoProvider; import org.apache.cassandra.security.EncryptionContext; import org.apache.cassandra.security.JREProvider; @@ -5349,4 +5349,19 @@ public static boolean isPasswordValidatorReconfigurationEnabled() { return conf.password_validator_reconfiguration_enabled; } + + public static AutoRepairConfig getAutoRepairConfig() + { + return conf.auto_repair; + } + + public static double getIncrementalRepairDiskHeadroomRejectRatio() + { + return conf.incremental_repair_disk_headroom_reject_ratio; + } + + public static void setIncrementalRepairDiskHeadroomRejectRatio(double value) + { + conf.incremental_repair_disk_headroom_reject_ratio = value; + } } diff --git a/src/java/org/apache/cassandra/config/DurationSpec.java b/src/java/org/apache/cassandra/config/DurationSpec.java index 3a9a03ab1fb2..4e2952a3a196 100644 --- a/src/java/org/apache/cassandra/config/DurationSpec.java +++ b/src/java/org/apache/cassandra/config/DurationSpec.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.config; +import java.io.Serializable; import java.util.Arrays; import java.util.Objects; import java.util.concurrent.TimeUnit; @@ -41,7 +42,7 @@ * users the opportunity to be able to provide config with a unit of their choice in cassandra.yaml as per the available * options. (CASSANDRA-15234) */ -public abstract class DurationSpec +public abstract class DurationSpec implements Serializable { /** * The Regexp used to parse the duration provided as String. diff --git a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java index a0201c500a39..7291f101eda2 100644 --- a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java +++ b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java @@ -449,6 +449,13 @@ public long getLong(String column) return LongType.instance.compose(data.get(column)); } + // this function will return the default value if the row doesn't have that column or the column data is null + // This function is used to avoid the nullpointerexception + public long getLong(String column, long ifNull) { + ByteBuffer bytes = data.get(column); + return bytes == null ? ifNull : LongType.instance.compose(bytes); + } + public Set getSet(String column, AbstractType type) { ByteBuffer raw = data.get(column); diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java index e3662a609571..f82e05e2e9d1 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java @@ -81,7 +81,7 @@ public abstract class ModificationStatement implements CQLStatement.SingleKeyspa public static final String CUSTOM_EXPRESSIONS_NOT_ALLOWED = "Custom index expressions cannot be used in WHERE clauses for UPDATE or DELETE statements"; - private static final ColumnIdentifier CAS_RESULT_COLUMN = new ColumnIdentifier("[applied]", false); + public static final ColumnIdentifier CAS_RESULT_COLUMN = new ColumnIdentifier("[applied]", false); protected final StatementType type; diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index 87af6b840b00..540f1036e2f5 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -25,6 +25,9 @@ import org.apache.cassandra.cql3.statements.PropertyDefinitions; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.SyntaxException; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.schema.AutoRepairParams; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.schema.CompressionParams; @@ -151,6 +154,12 @@ private TableParams build(TableParams.Builder builder) if (hasOption(READ_REPAIR)) builder.readRepair(ReadRepairStrategy.fromString(getString(READ_REPAIR))); + if (hasOption(Option.AUTOMATED_REPAIR_FULL)) + builder.automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, getMap(Option.AUTOMATED_REPAIR_FULL))); + + if (hasOption(Option.AUTOMATED_REPAIR_INCREMENTAL)) + builder.automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, getMap(Option.AUTOMATED_REPAIR_INCREMENTAL))); + return builder.build(); } diff --git a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java new file mode 100644 index 000000000000..cd46c462fb66 --- /dev/null +++ b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.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.metrics; + +import com.codahale.metrics.Counter; +import com.codahale.metrics.Gauge; +import com.google.common.annotations.VisibleForTesting; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.repair.autorepair.AutoRepair; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + +/** + * Metrics related to AutoRepair. + */ +public class AutoRepairMetrics +{ + public Gauge repairsInProgress; + public Gauge nodeRepairTimeInSec; + public Gauge clusterRepairTimeInSec; + public Gauge skippedTablesCount; + public Gauge longestUnrepairedSec; + public Gauge failedTablesCount; + public Counter repairTurnMyTurn; + public Counter repairTurnMyTurnDueToPriority; + public Counter repairTurnMyTurnForceRepair; + public Gauge totalMVTablesConsideredForRepair; + public Gauge totalDisabledRepairTables; + + public AutoRepairMetrics(RepairType repairType) + { + AutoRepairMetricsFactory factory = new AutoRepairMetricsFactory(repairType); + + repairsInProgress = Metrics.register(factory.createMetricName("RepairsInProgress"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).isRepairInProgress() ? 1 : 0; + } + }); + + nodeRepairTimeInSec = Metrics.register(factory.createMetricName("NodeRepairTimeInSec"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getNodeRepairTimeInSec(); + } + }); + + clusterRepairTimeInSec = Metrics.register(factory.createMetricName("ClusterRepairTimeInSec"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getClusterRepairTimeInSec(); + } + }); + + skippedTablesCount = Metrics.register(factory.createMetricName("SkippedTablesCount"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getRepairSkippedTablesCount(); + } + }); + + longestUnrepairedSec = Metrics.register(factory.createMetricName("LongestUnrepairedSec"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getLongestUnrepairedSec(); + } + }); + + failedTablesCount = Metrics.register(factory.createMetricName("FailedTablesCount"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getRepairFailedTablesCount(); + } + }); + + repairTurnMyTurn = Metrics.counter(factory.createMetricName("RepairTurnMyTurn")); + repairTurnMyTurnDueToPriority = Metrics.counter(factory.createMetricName("RepairTurnMyTurnDueToPriority")); + repairTurnMyTurnForceRepair = Metrics.counter(factory.createMetricName("RepairTurnMyTurnForceRepair")); + + totalMVTablesConsideredForRepair = Metrics.register(factory.createMetricName("TotalMVTablesConsideredForRepair"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getTotalMVTablesConsideredForRepair(); + } + }); + + totalDisabledRepairTables = Metrics.register(factory.createMetricName("TotalDisabledRepairTables"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getTotalDisabledTablesRepairCount(); + } + }); + } + + public void recordTurn(AutoRepairUtils.RepairTurn turn) + { + switch (turn) + { + case MY_TURN: + repairTurnMyTurn.inc(); + break; + case MY_TURN_FORCE_REPAIR: + repairTurnMyTurnForceRepair.inc(); + break; + case MY_TURN_DUE_TO_PRIORITY: + repairTurnMyTurnDueToPriority.inc(); + break; + default: + throw new RuntimeException(String.format("Unrecoginized turn: %s", turn.name())); + } + } + + @VisibleForTesting + protected static class AutoRepairMetricsFactory implements MetricNameFactory + { + private static final String TYPE = "AutoRepair"; + @VisibleForTesting + protected final String repairType; + + protected AutoRepairMetricsFactory(RepairType repairType) + { + this.repairType = repairType.toString().toLowerCase(); + } + + @Override + public CassandraMetricsRegistry.MetricName createMetricName(String metricName) + { + StringBuilder mbeanName = new StringBuilder(); + mbeanName.append(DefaultNameFactory.GROUP_NAME).append(':'); + mbeanName.append("type=").append(TYPE); + mbeanName.append(",name=").append(metricName); + mbeanName.append(",repairType=").append(repairType); + + StringBuilder scope = new StringBuilder(); + scope.append("repairType=").append(repairType); + + return new CassandraMetricsRegistry.MetricName(DefaultNameFactory.GROUP_NAME, TYPE.toLowerCase(), + metricName, scope.toString(), mbeanName.toString()); + } + } +} diff --git a/src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java b/src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java new file mode 100644 index 000000000000..e293945c9846 --- /dev/null +++ b/src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.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.metrics; + +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class AutoRepairMetricsManager +{ + private static final Map metrics = new ConcurrentHashMap<>(); + + public static AutoRepairMetrics getMetrics(RepairType repairType) + { + return metrics.computeIfAbsent(repairType, k -> new AutoRepairMetrics(repairType)); + } +} diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java index ca823faa5a6f..d746c30ad70f 100644 --- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java +++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java @@ -110,6 +110,13 @@ public void doVerb(final Message message) sendFailureResponse(message); return; } + if (!ActiveRepairService.verifyDiskHeadroomThreshold(prepareMessage.parentRepairSession, prepareMessage.previewKind, prepareMessage.isIncremental)) + { + // error is logged in verifyDiskHeadroomThreshold + state.phase.fail("Not enough disk headroom to perform incremental repair"); + sendFailureResponse(message); + return; + } List columnFamilyStores = new ArrayList<>(prepareMessage.tableIds.size()); for (TableId tableId : prepareMessage.tableIds) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java new file mode 100644 index 000000000000..f6a08e73b16d --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -0,0 +1,402 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.ArrayList; +import java.util.EnumMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.FutureTask; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.repair.RepairCoordinator; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.Pair; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.concurrent.NamedThreadFactory; +import org.apache.cassandra.concurrent.ScheduledExecutorPlus; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.Tables; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn; + +import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; + +public class AutoRepair +{ + // Initial delay for repair session to start after setup + final static long INITIAL_REPAIR_DELAY_SEC = 30; + + private static final Logger logger = LoggerFactory.getLogger(AutoRepair.class); + + @VisibleForTesting + protected static Supplier timeFunc = System::currentTimeMillis; + + public static AutoRepair instance = new AutoRepair(); + + @VisibleForTesting + protected final Map repairExecutors; + @VisibleForTesting + protected final Map repairStates; + + protected final Map tokenRangeSplitters = new EnumMap<>(AutoRepairConfig.RepairType.class); + + + @VisibleForTesting + protected AutoRepair() + { + AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); + repairExecutors = new EnumMap<>(AutoRepairConfig.RepairType.class); + repairStates = new EnumMap<>(AutoRepairConfig.RepairType.class); + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + repairExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-Repair-" + repairType, Thread.NORM_PRIORITY)); + repairStates.put(repairType, AutoRepairConfig.RepairType.getAutoRepairState(repairType)); + tokenRangeSplitters.put(repairType, FBUtilities.newAutoRepairTokenRangeSplitter(config.getTokenRangeSplitter(repairType))); + } + } + + public void setup() + { + verifyIsSafeToEnable(); + + AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); + AutoRepairService.setup(); + AutoRepairUtils.setup(); + + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + repairExecutors.get(repairType).scheduleWithFixedDelay( + () -> repair(repairType, 60000), + INITIAL_REPAIR_DELAY_SEC, + config.getRepairCheckInterval().toSeconds(), + TimeUnit.SECONDS); + } + } + + @VisibleForTesting + protected void verifyIsSafeToEnable() + { + AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); + if (config.isAutoRepairEnabled(AutoRepairConfig.RepairType.incremental) && + (DatabaseDescriptor.getMaterializedViewsEnabled() || DatabaseDescriptor.isCDCEnabled())) + throw new ConfigurationException("Cannot enable incremental repair with materialized views or CDC enabled"); + } + + // repairAsync runs a repair session of the given type asynchronously. + public void repairAsync(AutoRepairConfig.RepairType repairType, long millisToWait) + { + repairExecutors.get(repairType).submit(() -> repair(repairType, millisToWait)); + } + + // repair runs a repair session of the given type synchronously. + public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + if (!config.isAutoRepairEnabled(repairType)) + { + logger.debug("Auto-repair is disabled for repair type {}", repairType); + return; + } + + + AutoRepairState repairState = repairStates.get(repairType); + + try + { + String localDC = DatabaseDescriptor.getLocalDataCenter(); + if (config.getIgnoreDCs(repairType).contains(localDC)) + { + logger.info("Not running repair as this node belongs to datacenter {}", localDC); + return; + } + + // refresh the longest unrepaired node + repairState.setLongestUnrepairedNode(AutoRepairUtils.getHostWithLongestUnrepairTime(repairType)); + + //consistency level to use for local query + UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); + RepairTurn turn = AutoRepairUtils.myTurnToRunRepair(repairType, myId); + if (turn == MY_TURN || turn == MY_TURN_DUE_TO_PRIORITY || turn == MY_TURN_FORCE_REPAIR) + { + repairState.recordTurn(turn); + // For normal auto repair, we will use primary range only repairs (Repair with -pr option). + // For some cases, we may set the auto_repair_primary_token_range_only flag to false then we will do repair + // without -pr. We may also do force repair for certain node that we want to repair all the data on one node + // When doing force repair, we want to repair without -pr. + boolean primaryRangeOnly = config.getRepairPrimaryTokenRangeOnly(repairType) + && turn != MY_TURN_FORCE_REPAIR; + repairState.setTotalTablesConsideredForRepair(0); + if (repairState.getLastRepairTime() != 0) + { + /** check if it is too soon to run repair. one of the reason we + * should not run frequent repair is because repair triggers + * memtable flush + */ + long timeElapsedSinceLastRepair = TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - repairState.getLastRepairTime()); + if (timeElapsedSinceLastRepair < config.getRepairMinInterval(repairType).toSeconds()) + { + logger.info("Too soon to run repair, last repair was done {} seconds ago", + timeElapsedSinceLastRepair); + return; + } + } + + long startTime = timeFunc.get(); + logger.info("My host id: {}, my turn to run repair...repair primary-ranges only? {}", myId, + config.getRepairPrimaryTokenRangeOnly(repairType)); + AutoRepairUtils.updateStartAutoRepairHistory(repairType, myId, timeFunc.get(), turn); + + repairState.setRepairKeyspaceCount(0); + repairState.setRepairTableSuccessCount(0); + repairState.setRepairFailedTablesCount(0); + repairState.setRepairSkippedTablesCount(0); + repairState.setRepairInProgress(true); + repairState.setTotalMVTablesConsideredForRepair(0); + for (Keyspace keyspace : Keyspace.all()) + { + Tables tables = keyspace.getMetadata().tables; + Iterator iter = tables.iterator(); + String keyspaceName = keyspace.getName(); + if (!AutoRepairUtils.checkNodeContainsKeyspaceReplica(keyspace)) + { + continue; + } + + repairState.setRepairKeyspaceCount(repairState.getRepairKeyspaceCount() + 1); + List tablesToBeRepaired = new ArrayList<>(); + while (iter.hasNext()) + { + repairState.setTotalTablesConsideredForRepair(repairState.getTotalTablesConsideredForRepair() + 1); + TableMetadata tableMetadata = iter.next(); + String tableName = tableMetadata.name; + tablesToBeRepaired.add(tableName); + + // See if we should repair MVs as well that are associated with this given table + List mvs = AutoRepairUtils.getAllMVs(repairType, keyspace, tableMetadata); + if (!mvs.isEmpty()) + { + tablesToBeRepaired.addAll(mvs); + repairState.setTotalMVTablesConsideredForRepair(repairState.getTotalMVTablesConsideredForRepair() + mvs.size()); + } + } + + for (String tableName : tablesToBeRepaired) + { + try + { + ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName); + if (!columnFamilyStore.metadata().params.automatedRepair.get(repairType).repairEnabled()) + { + logger.info("Repair is disabled for keyspace {} for tables: {}", keyspaceName, tableName); + repairState.setTotalDisabledTablesRepairCount(repairState.getTotalDisabledTablesRepairCount() + 1); + continue; + } + // this is done to make autorepair safe as running repair on table with more sstables + // may have its own challenges + int size = columnFamilyStore.getLiveSSTables().size(); + if (size > config.getRepairSSTableCountHigherThreshold(repairType)) + { + logger.info("Too many SSTables for repair, not doing repair on table {}.{} " + + "totalSSTables {}", keyspaceName, tableName, columnFamilyStore.getLiveSSTables().size()); + repairState.setRepairSkippedTablesCount(repairState.getRepairSkippedTablesCount() + 1); + continue; + } + + if (config.getRepairByKeyspace(repairType)) + { + logger.info("Repair keyspace {} for tables: {}", keyspaceName, tablesToBeRepaired); + } + else + { + logger.info("Repair table {}.{}", keyspaceName, tableName); + } + long tableStartTime = timeFunc.get(); + boolean repairSuccess = true; + Set> ranges = new HashSet<>(); + List> subRangesToBeRepaired = tokenRangeSplitters.get(repairType).getRange(repairType, primaryRangeOnly, keyspaceName, tableName); + int totalSubRanges = subRangesToBeRepaired.size(); + int totalProcessedSubRanges = 0; + for (Pair token : subRangesToBeRepaired) + { + if (!config.isAutoRepairEnabled(repairType)) + { + logger.error("Auto-repair for type {} is disabled hence not running repair", repairType); + repairState.setRepairInProgress(false); + return; + } + + if (config.getRepairByKeyspace(repairType)) + { + if (AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, tableStartTime, tablesToBeRepaired.size())) + { + repairState.setRepairSkippedTablesCount(repairState.getRepairSkippedTablesCount() + tablesToBeRepaired.size()); + logger.info("Keyspace took too much time to repair hence skipping it {}", + keyspaceName); + break; + } + } + else + { + if (AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, tableStartTime)) + { + repairState.setRepairSkippedTablesCount(repairState.getRepairSkippedTablesCount() + 1); + logger.info("Table took too much time to repair hence skipping it {}.{}", + keyspaceName, tableName); + break; + } + } + Token childStartToken = token.left; + Token childEndToken = token.right; + logger.debug("Current Token Left side {}, right side {}", childStartToken + .toString(), childEndToken.toString()); + + ranges.add(new Range<>(childStartToken, childEndToken)); + totalProcessedSubRanges++; + if ((totalProcessedSubRanges % config.getRepairThreads(repairType) == 0) || + (totalProcessedSubRanges == totalSubRanges)) + { + RepairCoordinator task = repairState.getRepairRunnable(keyspaceName, + config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : ImmutableList.of(tableName), + ranges, primaryRangeOnly); + repairState.resetWaitCondition(); + new Thread(NamedThreadFactory.createAnonymousThread(new FutureTask<>(task, null))).start(); + try + { + repairState.waitForRepairToComplete(); + } + catch (InterruptedException e) + { + logger.error("Exception in cond await:", e); + } + + //check repair status + if (repairState.isSuccess()) + { + logger.info("Repair completed for range {}-{} for {}.{}, total subranges: {}," + + "processed subranges: {}", childStartToken, childEndToken, + keyspaceName, config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, totalSubRanges, totalProcessedSubRanges); + } + else + { + repairSuccess = false; + //in future we can add retry, etc. + logger.info("Repair failed for range {}-{} for {}.{} total subranges: {}," + + "processed subranges: {}", childStartToken, childEndToken, + keyspaceName, config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, totalSubRanges, totalProcessedSubRanges); + } + ranges.clear(); + } + } + int touchedTables = config.getRepairByKeyspace(repairType) ? tablesToBeRepaired.size() : 1; + if (repairSuccess) + { + repairState.setRepairTableSuccessCount(repairState.getRepairTableSuccessCount() + touchedTables); + } + else + { + repairState.setRepairFailedTablesCount(repairState.getRepairFailedTablesCount() + touchedTables); + } + if (config.getRepairByKeyspace(repairType)) + { + logger.info("Repair completed for keyspace {}, tables: {}", keyspaceName, tablesToBeRepaired); + break; + } + else + { + logger.info("Repair completed for {}.{}", keyspaceName, tableName); + } + } + catch (Exception e) + { + logger.error("Exception while repairing keyspace {}:", keyspaceName, e); + } + } + } + + //if it was due to priority then remove it now + if (turn == MY_TURN_DUE_TO_PRIORITY) + { + logger.info("Remove current host from priority list"); + AutoRepairUtils.removePriorityStatus(repairType, myId); + } + + repairState.setNodeRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - startTime)); + long timeInHours = TimeUnit.SECONDS.toHours(repairState.getNodeRepairTimeInSec()); + logger.info("Local {} repair time {} hour(s), stats: repairKeyspaceCount {}, " + + "repairTableSuccessCount {}, repairTableFailureCount {}, " + + "repairTableSkipCount {}", repairType, timeInHours, repairState.getRepairKeyspaceCount(), + repairState.getRepairTableSuccessCount(), repairState.getRepairFailedTablesCount(), + repairState.getRepairSkippedTablesCount()); + if (repairState.getLastRepairTime() != 0) + { + repairState.setClusterRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - + repairState.getLastRepairTime())); + logger.info("Cluster repair time for repair type {}: {} day(s)", repairType, + TimeUnit.SECONDS.toDays(repairState.getClusterRepairTimeInSec())); + } + repairState.setLastRepairTime(timeFunc.get()); + if (timeInHours == 0 && millisToWait > 0) + { + //If repair finished quickly, happens for an empty instance, in such case + //wait for a minute so that the JMX metrics can detect the repairInProgress + logger.info("Wait for {} milliseconds for repair type {}.", millisToWait, repairType); + Thread.sleep(millisToWait); + } + repairState.setRepairInProgress(false); + AutoRepairUtils.updateFinishAutoRepairHistory(repairType, myId, timeFunc.get()); + } + else + { + logger.info("Waiting for my turn..."); + } + } + catch (Exception e) + { + logger.error("Exception in autorepair:", e); + } + } + + public AutoRepairState getRepairState(AutoRepairConfig.RepairType repairType) + { + return repairStates.get(repairType); + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java new file mode 100644 index 000000000000..bd20c1d3c548 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -0,0 +1,408 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.EnumMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.exceptions.ConfigurationException; + +public class AutoRepairConfig implements Serializable +{ + // enable/disable auto repair globally, overrides all other settings. Cannot be modified dynamically. + public final Boolean enabled; + // the interval in seconds between checks for eligible repair operations. Cannot be modified dynamically. + public final DurationSpec.IntSecondsBound repair_check_interval = new DurationSpec.IntSecondsBound("5m"); + // configures how long repair history is kept for a replaced node + public volatile DurationSpec.IntSecondsBound history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound("2h"); + // global_settings overides Options.defaultOptions for all repair types + public volatile Options global_settings; + + public enum RepairType + { + full, + incremental; + + public static AutoRepairState getAutoRepairState(RepairType repairType) + { + switch (repairType) + { + case full: + return new FullRepairState(); + case incremental: + return new IncrementalRepairState(); + } + + throw new IllegalArgumentException("Invalid repair type: " + repairType); + } + } + + // repair_type_overrides overrides the global_settings for a specific repair type + public volatile Map repair_type_overrides = new EnumMap<>(RepairType.class); + + public AutoRepairConfig() + { + this(false); + } + + public AutoRepairConfig(boolean enabled) + { + this.enabled = enabled; + global_settings = Options.getDefaultOptions(); + for (RepairType type : RepairType.values()) + { + repair_type_overrides.put(type, new Options()); + } + } + + public DurationSpec.IntSecondsBound getRepairCheckInterval() + { + return repair_check_interval; + } + + public boolean isAutoRepairSchedulingEnabled() + { + return enabled; + } + + public DurationSpec.IntSecondsBound getAutoRepairHistoryClearDeleteHostsBufferInterval() + { + return history_clear_delete_hosts_buffer_interval; + } + + public void setAutoRepairHistoryClearDeleteHostsBufferInterval(String duration) + { + history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound(duration); + } + + public boolean isAutoRepairEnabled(RepairType repairType) + { + return enabled && applyOverrides(repairType, opt -> opt.enabled); + } + + public void setAutoRepairEnabled(RepairType repairType, boolean enabled) + { + if (enabled && repairType == RepairType.incremental && + (DatabaseDescriptor.getMaterializedViewsEnabled() || DatabaseDescriptor.isCDCEnabled())) + throw new ConfigurationException("Cannot enable incremental repair with materialized views or CDC enabled"); + + ensureOverrides(repairType); + repair_type_overrides.get(repairType).enabled = enabled; + } + + public void setRepairByKeyspace(RepairType repairType, boolean repairByKeyspace) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).repair_by_keyspace = repairByKeyspace; + } + + public boolean getRepairByKeyspace(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.repair_by_keyspace); + } + + public int getRepairThreads(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.number_of_repair_threads); + } + + public void setRepairThreads(RepairType repairType, int repairThreads) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).number_of_repair_threads = repairThreads; + } + + public int getRepairSubRangeNum(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.number_of_subranges); + } + + public void setRepairSubRangeNum(RepairType repairType, int repairSubRanges) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).number_of_subranges = repairSubRanges; + } + + public DurationSpec.IntSecondsBound getRepairMinInterval(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.min_repair_interval); + } + + public void setRepairMinInterval(RepairType repairType, String minRepairInterval) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).min_repair_interval = new DurationSpec.IntSecondsBound(minRepairInterval); + } + + public int getRepairSSTableCountHigherThreshold(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.sstable_upper_threshold); + } + + public void setRepairSSTableCountHigherThreshold(RepairType repairType, int sstableHigherThreshold) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).sstable_upper_threshold = sstableHigherThreshold; + } + + public DurationSpec.IntSecondsBound getAutoRepairTableMaxRepairTime(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.table_max_repair_time); + } + + public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).table_max_repair_time = new DurationSpec.IntSecondsBound(autoRepairTableMaxRepairTime); + } + + public Set getIgnoreDCs(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.ignore_dcs); + } + + public void setIgnoreDCs(RepairType repairType, Set ignoreDCs) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).ignore_dcs = ignoreDCs; + } + + public boolean getRepairPrimaryTokenRangeOnly(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.repair_primary_token_range_only); + } + + public void setRepairPrimaryTokenRangeOnly(RepairType repairType, boolean primaryTokenRangeOnly) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).repair_primary_token_range_only = primaryTokenRangeOnly; + } + + public int getParallelRepairPercentageInGroup(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.parallel_repair_percentage_in_group); + } + + public void setParallelRepairPercentageInGroup(RepairType repairType, int percentageInGroup) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).parallel_repair_percentage_in_group = percentageInGroup; + } + + public int getParallelRepairCountInGroup(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.parallel_repair_count_in_group); + } + + public void setParallelRepairCountInGroup(RepairType repairType, int countInGroup) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).parallel_repair_count_in_group = countInGroup; + } + + public boolean getMVRepairEnabled(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.mv_repair_enabled); + } + + public void setMVRepairEnabled(RepairType repairType, boolean enabled) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).mv_repair_enabled = enabled; + } + + public void setForceRepairNewNode(RepairType repairType, boolean forceRepairNewNode) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).force_repair_new_node = forceRepairNewNode; + } + + public boolean getForceRepairNewNode(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.force_repair_new_node); + } + + public String getTokenRangeSplitter(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.token_range_splitter); + } + + // Options configures auto-repair behavior for a given repair type. + // All fields can be modified dynamically. + public static class Options implements Serializable + { + // The separator separating different DCs in repair_dc_groups + public static final String DC_GROUP_SEPARATOR = "\\|"; + + // defaultOptions defines the default auto-repair behavior when no overrides are defined + @VisibleForTesting + protected static final Options defaultOptions = getDefaultOptions(); + + public Options() + { + } + + @VisibleForTesting + protected static Options getDefaultOptions() + { + Options opts = new Options(); + + opts.enabled = false; + opts.repair_by_keyspace = false; + opts.number_of_subranges = 1; + opts.number_of_repair_threads = 1; + opts.parallel_repair_count_in_group = 1; + opts.parallel_repair_percentage_in_group = 0; + opts.sstable_upper_threshold = 10000; + opts.min_repair_interval = new DurationSpec.IntSecondsBound("24h"); + opts.ignore_dcs = new HashSet<>(); + opts.repair_primary_token_range_only = true; + opts.force_repair_new_node = false; + opts.table_max_repair_time = new DurationSpec.IntSecondsBound("6h"); + opts.mv_repair_enabled = true; + opts.token_range_splitter = DefaultAutoRepairTokenSplitter.class.getName(); + + return opts; + } + + // enable/disable auto repair for the given repair type + public volatile Boolean enabled; + // auto repair is default repair table by table, if this is enabled, the framework will repair all the tables in a keyspace in one go. + public volatile Boolean repair_by_keyspace; + // the number of subranges to split each to-be-repaired token range into, + // the higher this number, the smaller the repair sessions will be + // How many subranges to divide one range into? The default is 1. + // If you are using v-node, say 256, then the repair will always go one v-node range at a time, this parameter, additionally, will let us further subdivide a given v-node range into sub-ranges. + // With the value “1” and v-nodes of 256, a given table on a node will undergo the repair 256 times. But with a value “2,” the same table on a node will undergo a repair 512 times because every v-node range will be further divided by two. + // If you do not use v-nodes or the number of v-nodes is pretty small, say 8, setting this value to a higher number, say 16, will be useful to repair on a smaller range, and the chance of succeeding is higher. + public volatile Integer number_of_subranges; + // the number of repair threads to run for a given invoked Repair Job. + // Once the scheduler schedules one Job, then howmany threads to use inside that job will be controlled through this parameter. + // This is similar to -j for repair options for the nodetool repair command. + public volatile Integer number_of_repair_threads; + // the number of repair sessions that can run in parallel in a single group + // The number of nodes running repair parallelly. If parallelrepaircount is set, it will choose the larger value of the two. The default is 3. + // This configuration controls how many nodes would run repair in parallel. + // The value “3” means, at any given point in time, at most 3 nodes would be running repair in parallel. + // If one or more node(s) finish repair, then the framework automatically picks up the next candidate and ensures the maximum number of nodes running repair do not exceed “3”. + public volatile Integer parallel_repair_count_in_group; + // the number of repair sessions that can run in parallel in a single groupas a percentage + // of the total number of nodes in the group [0,100] + // The percentage of nodes in the cluster that run repair parallelly. If parallelrepaircount is set, it will choose the larger value of the two. + //The problem with a fixed number of nodes (the above property) is that in a large-scale environment, + // the nodes keep getting added/removed due to elasticity, so if we have a fixed number, then manual interventions would increase because, on a continuous basis,operators would have to adjust to meet the SLA. + //The default is 3%, which means that 3% of the nodes in the Cassandra cluster would be repaired in parallel. + // So now, if a fleet, an operator won't have to worry about changing the repair frequency, etc., as overall repair time will continue to remain the same even if nodes are added or removed due to elasticity. + // Extremely fewer manual interventions as it will rarely violate the repair SLA for customers + public volatile Integer parallel_repair_percentage_in_group; + // the upper threshold of SSTables allowed to participate in a single repair session + // Threshold to skip a table if it has too many sstables. The default is 10000. This means, if a table on a node has 10000 or more SSTables, then that table will be skipped. + // This is to avoid penalizing good neighbors with an outlier. + public volatile Integer sstable_upper_threshold; + // the minimum time in hours between repairs of the same token range + // The minimum number of hours to run one repair cycle is 24 hours. The default is 24 hours. + // This means that if auto repair finishes one round on one cluster within 24 hours, it won’t start a new round. + // This is applicable for extremely tiny clusters, say 3 nodes. + public volatile DurationSpec.IntSecondsBound min_repair_interval; + // specifies a denylist of datacenters to repair + // This is useful if you want to completely avoid running repairs in one or more data centers. By default, it is empty, i.e., the framework will repair nodes in all the datacenters. + public volatile Set ignore_dcs; + // Set this 'true' if AutoRepair should repair only the primary ranges owned by this node; else, 'false' + // It is the same as -pr in nodetool repair options. + public volatile Boolean repair_primary_token_range_only; + // configures whether to force immediate repair on new nodes + public volatile Boolean force_repair_new_node; + // the maximum time in seconds that a repair session can run for a single table + // Max time for repairing one table, if exceeded, skip the table. The default is 6 * 60 * 60, which is 6 hours. + // Let's say there is a Cassandra cluster in that there are 10 tables belonging to 10 different customers. + // Out of these 10 tables, 1 table is humongous. Repairing this 1 table, say, takes 5 days, in the worst case, but others could finish in just 1 hour. + // Then we would penalize 9 customers just because of one bad actor, and those 9 customers would ping an operator telling them they are violating SLA even if I am a neighbor, and it would require a lot of back-and-forth manual interventions, etc. + // So, the idea here is to penalize the outliers instead of good candidates. This can easily be configured with a higher value if we want to disable the functionality. + // Please note the repair will still run in parallel on other nodes, this is to address outliers on a given node. + public volatile DurationSpec.IntSecondsBound table_max_repair_time; + // the default is 'true'. MVs are mutated at LOCAL_ONE consistency level in Cassandra. + // This flag determines whether the auto-repair framework needs to run anti-entropy, a.k.a, repair on the MV table or not. + public volatile Boolean mv_repair_enabled; + // the default is DefaultAutoRepairTokenSplitter.class.getName(). The class should implement IAutoRepairTokenRangeSplitter. + // The default implementation splits the tokens based on the token ranges owned by this node divided by the number of 'number_of_subranges' + public volatile String token_range_splitter; + + public String toString() + { + return "Options{" + + "enabled=" + enabled + + ", repair_by_keyspace=" + repair_by_keyspace + + ", number_of_subranges=" + number_of_subranges + + ", number_of_repair_threads=" + number_of_repair_threads + + ", parallel_repair_count_in_group=" + parallel_repair_count_in_group + + ", parallel_repair_percentage_in_group=" + parallel_repair_percentage_in_group + + ", sstable_upper_threshold=" + sstable_upper_threshold + + ", min_repair_interval=" + min_repair_interval + + ", ignore_dcs=" + ignore_dcs + + ", repair_primary_token_range_only=" + repair_primary_token_range_only + + ", force_repair_new_node=" + force_repair_new_node + + ", table_max_repair_time=" + table_max_repair_time + + ", mv_repair_enabled=" + mv_repair_enabled + + ", token_range_splitter=" + token_range_splitter + + '}'; + } + } + + @VisibleForTesting + protected T applyOverrides(RepairType repairType, Function optionSupplier) + { + ArrayList optsProviders = new ArrayList<>(); + if (repair_type_overrides != null) + { + optsProviders.add(repair_type_overrides.get(repairType)); + } + optsProviders.add(global_settings); + optsProviders.add(Options.defaultOptions); + + return optsProviders.stream() + .map(opt -> Optional.ofNullable(opt).map(optionSupplier).orElse(null)) + .filter(Objects::nonNull) + .findFirst() + .orElse(null); + } + + protected void ensureOverrides(RepairType repairType) + { + if (repair_type_overrides == null) + { + repair_type_overrides = new EnumMap<>(RepairType.class); + } + + if (repair_type_overrides.get(repairType) == null) + { + repair_type_overrides.put(repairType, new Options()); + } + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.java new file mode 100644 index 000000000000..14cba17f6095 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.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.repair.autorepair; + +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.cql3.statements.schema.CreateTableStatement; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.Tables; + +public class AutoRepairKeyspace +{ + private AutoRepairKeyspace() + { + } + + /** + * Generation is used as a timestamp for automatic table creation on startup. + * If you make any changes to the tables below, make sure to increment the + * generation and document your change here. + */ + public static final long GENERATION = 0; + + public static final String AUTO_REPAIR_HISTORY = "auto_repair_history"; + + public static final String AUTO_REPAIR_PRIORITY = "auto_repair_priority"; + + private static final TableMetadata AutoRepairHistory = + parse(AUTO_REPAIR_HISTORY, + "Auto repair history for each node", + "CREATE TABLE %s (" + + "host_id uuid," + + "repair_type text," + + "repair_turn text," + + "repair_start_ts timestamp," + + "repair_finish_ts timestamp," + + "delete_hosts set," + + "delete_hosts_update_time timestamp," + + "force_repair boolean," + + "PRIMARY KEY (repair_type, host_id))"); + + private static final TableMetadata AutoRepairPriority = + parse(AUTO_REPAIR_PRIORITY, + "Auto repair priority for each group", + "CREATE TABLE %s (" + + "repair_type text," + + "repair_priority set," + + "PRIMARY KEY (repair_type))"); + + private static TableMetadata parse(String name, String description, String schema) + { + return CreateTableStatement.parse(String.format(schema, name), SchemaConstants.DISTRIBUTED_KEYSPACE_NAME) + .id(TableId.forSystemTable(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, name)) + .comment(description) + .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(90)) + .build(); + } + + public static KeyspaceMetadata metadata() + { + Tables tables = Tables.of(AutoRepairHistory, AutoRepairPriority); + return KeyspaceMetadata.create(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, KeyspaceParams.simple(1), tables); + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java new file mode 100644 index 000000000000..523a48545d59 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -0,0 +1,349 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.view.TableViews; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.metrics.AutoRepairMetricsManager; +import org.apache.cassandra.metrics.AutoRepairMetrics; +import org.apache.cassandra.repair.RepairCoordinator; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; +import org.apache.cassandra.repair.RepairParallelism; +import org.apache.cassandra.repair.messages.RepairOption; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.utils.concurrent.Condition; +import org.apache.cassandra.utils.progress.ProgressEvent; +import org.apache.cassandra.utils.progress.ProgressEventType; +import org.apache.cassandra.utils.progress.ProgressListener; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.text.SimpleDateFormat; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition; + +// AutoRepairState represents the state of automated repair for a given repair type. +public abstract class AutoRepairState implements ProgressListener +{ + protected static final Logger logger = LoggerFactory.getLogger(AutoRepairState.class); + private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS"); + @VisibleForTesting + protected static Supplier timeFunc = System::currentTimeMillis; + + @VisibleForTesting + protected final RepairType repairType; + @VisibleForTesting + protected int totalTablesConsideredForRepair = 0; + @VisibleForTesting + protected long lastRepairTimeInMs; + @VisibleForTesting + protected int nodeRepairTimeInSec = 0; + @VisibleForTesting + protected int clusterRepairTimeInSec = 0; + @VisibleForTesting + protected boolean repairInProgress = false; + @VisibleForTesting + protected int repairKeyspaceCount = 0; + @VisibleForTesting + protected int repairTableSuccessCount = 0; + @VisibleForTesting + protected int repairTableSkipCount = 0; + @VisibleForTesting + protected int repairTableFailureCount = 0; + @VisibleForTesting + protected int totalMVTablesConsideredForRepair = 0; + + @VisibleForTesting + protected int totalDisabledTablesRepairCount = 0; + @VisibleForTesting + protected AutoRepairHistory longestUnrepairedNode; + @VisibleForTesting + protected Condition condition = newOneTimeCondition(); + @VisibleForTesting + protected boolean success = true; + protected final AutoRepairMetrics metrics; + + protected AutoRepairState(RepairType repairType) + { + metrics = AutoRepairMetricsManager.getMetrics(repairType); + this.repairType = repairType; + } + + public abstract RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly); + + protected RepairCoordinator getRepairRunnable(String keyspace, RepairOption options) + { + RepairCoordinator task = new RepairCoordinator(StorageService.instance, StorageService.nextRepairCommand.incrementAndGet(), + options, keyspace); + + task.addProgressListener(this); + + return task; + } + + @Override + public void progress(String tag, ProgressEvent event) + { + ProgressEventType type = event.getType(); + String message = String.format("[%s] %s", format.format(System.currentTimeMillis()), event.getMessage()); + if (type == ProgressEventType.ERROR) + { + logger.error("Repair failure for {} repair: {}", repairType.toString(), message); + success = false; + condition.signalAll(); + } + if (type == ProgressEventType.PROGRESS) + { + message = message + " (progress: " + (int) event.getProgressPercentage() + "%)"; + logger.debug("Repair progress for {} repair: {}", repairType.toString(), message); + } + if (type == ProgressEventType.COMPLETE) + { + success = true; + condition.signalAll(); + } + } + + public void waitForRepairToComplete() throws InterruptedException + { + //if for some reason we don't hear back on repair progress for sometime + if (!condition.await(12, TimeUnit.HOURS)) + { + success = false; + } + } + + public long getLastRepairTime() + { + return lastRepairTimeInMs; + } + + public void setTotalTablesConsideredForRepair(int count) + { + totalTablesConsideredForRepair = count; + } + + public int getTotalTablesConsideredForRepair() + { + return totalTablesConsideredForRepair; + } + + public void setLastRepairTime(long lastRepairTime) + { + lastRepairTimeInMs = lastRepairTime; + } + + public int getClusterRepairTimeInSec() + { + return clusterRepairTimeInSec; + } + + public int getNodeRepairTimeInSec() + { + return nodeRepairTimeInSec; + } + + public void setRepairInProgress(boolean repairInProgress) + { + this.repairInProgress = repairInProgress; + } + + public boolean isRepairInProgress() + { + return repairInProgress; + } + + public void setRepairSkippedTablesCount(int count) + { + repairTableSkipCount = count; + } + + public int getRepairSkippedTablesCount() + { + return repairTableSkipCount; + } + + public int getLongestUnrepairedSec() + { + if (longestUnrepairedNode == null) + { + return 0; + } + return (int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - longestUnrepairedNode.getLastRepairFinishTime()); + } + + public void setRepairFailedTablesCount(int count) + { + repairTableFailureCount = count; + } + + public int getRepairFailedTablesCount() + { + return repairTableFailureCount; + } + + public void setTotalMVTablesConsideredForRepair(int count) + { + totalMVTablesConsideredForRepair = count; + } + + public int getTotalMVTablesConsideredForRepair() + { + return totalMVTablesConsideredForRepair; + } + + public void setNodeRepairTimeInSec(int elapsed) + { + nodeRepairTimeInSec = elapsed; + } + + public void setClusterRepairTimeInSec(int seconds) + { + clusterRepairTimeInSec = seconds; + } + + public void setRepairKeyspaceCount(int count) + { + repairKeyspaceCount = count; + } + + public void setRepairTableSuccessCount(int count) + { + repairTableSuccessCount = count; + } + + public int getRepairKeyspaceCount() + { + return repairKeyspaceCount; + } + + public int getRepairTableSuccessCount() + { + return repairTableSuccessCount; + } + + public void setLongestUnrepairedNode(AutoRepairHistory longestUnrepairedNode) + { + this.longestUnrepairedNode = longestUnrepairedNode; + } + + public boolean isSuccess() + { + return success; + } + + public void recordTurn(AutoRepairUtils.RepairTurn turn) + { + metrics.recordTurn(turn); + } + + public void setTotalDisabledTablesRepairCount(int count) + { + totalDisabledTablesRepairCount = count; + } + + public int getTotalDisabledTablesRepairCount() + { + return totalDisabledTablesRepairCount; + } + + public void resetWaitCondition() + { + condition = newOneTimeCondition(); + } +} + +class IncrementalRepairState extends AutoRepairState +{ + public IncrementalRepairState() + { + super(RepairType.incremental); + } + + @Override + public RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) + { + RepairOption option = new RepairOption(RepairParallelism.PARALLEL, primaryRangeOnly, true, false, + AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), ranges, + !ranges.isEmpty(), false, false, PreviewKind.NONE, true, true, false, false); + + option.getColumnFamilies().addAll(filterOutUnsafeTables(keyspace, tables)); + + return getRepairRunnable(keyspace, option); + } + + @VisibleForTesting + protected List filterOutUnsafeTables(String keyspaceName, List tables) + { + Keyspace keyspace = Keyspace.open(keyspaceName); + + return tables.stream() + .filter(table -> { + ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(table); + TableViews views = keyspace.viewManager.forTable(cfs.metadata()); + if (views != null && !views.isEmpty()) + { + logger.debug("Skipping incremental repair for {}.{} as it has materialized views", keyspaceName, table); + return false; + } + + if (cfs.metadata().params != null && cfs.metadata().params.cdc) + { + logger.debug("Skipping incremental repair for {}.{} as it has CDC enabled", keyspaceName, table); + return false; + } + + return true; + }).collect(Collectors.toList()); + } +} + +class FullRepairState extends AutoRepairState +{ + public FullRepairState() + { + super(RepairType.full); + } + + @Override + public RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) + { + RepairOption option = new RepairOption(RepairParallelism.PARALLEL, primaryRangeOnly, false, false, + AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), ranges, + !ranges.isEmpty(), false, false, PreviewKind.NONE, true, true, false, false); + + option.getColumnFamilies().addAll(tables); + + return getRepairRunnable(keyspace, option); + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java new file mode 100644 index 000000000000..cac3e6bf8d4c --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -0,0 +1,811 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.repair.autorepair; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; +import com.google.common.collect.Lists; + +import org.apache.cassandra.locator.LocalStrategy; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.cql3.statements.ModificationStatement; +import org.apache.cassandra.cql3.statements.SelectStatement; +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.locator.AbstractReplicationStrategy; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.NetworkTopologyStrategy; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.ViewMetadata; +import org.apache.cassandra.serializers.SetSerializer; +import org.apache.cassandra.serializers.UUIDSerializer; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.NodeAddresses; +import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.transport.Dispatcher; +import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.transport.messages.ResultMessage; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; + +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; + +/** + * This class serves as a utility class for AutoRepair. It contains various helper APIs + * to store/retrieve repair status, decide whose turn is next, etc. + */ +public class AutoRepairUtils +{ + private static final Logger logger = LoggerFactory.getLogger(AutoRepairUtils.class); + static final String COL_REPAIR_TYPE = "repair_type"; + static final String COL_HOST_ID = "host_id"; + static final String COL_REPAIR_START_TS = "repair_start_ts"; + static final String COL_REPAIR_FINISH_TS = "repair_finish_ts"; + static final String COL_REPAIR_PRIORITY = "repair_priority"; + static final String COL_DELETE_HOSTS = "delete_hosts"; // this set stores the host ids which think the row should be deleted + static final String COL_REPAIR_TURN = "repair_turn"; // this record the last repair turn. Normal turn or turn due to priority + static final String COL_DELETE_HOSTS_UPDATE_TIME = "delete_hosts_update_time"; // the time when delete hosts are upated + static final String COL_FORCE_REPAIR = "force_repair"; // if set to true, the node will do non-primary range rapair + + final static String SELECT_REPAIR_HISTORY = String.format( + "SELECT * FROM %s.%s WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, + AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE); + final static String SELECT_REPAIR_PRIORITY = String.format( + "SELECT * FROM %s.%s WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, + AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); + final static String DEL_REPAIR_PRIORITY = String.format( + "DELETE %s[?] FROM %s.%s WHERE %s = ?", COL_REPAIR_PRIORITY, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, + AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); + final static String ADD_PRIORITY_HOST = String.format( + "UPDATE %s.%s SET %s = %s + ? WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, + AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_TYPE); + + final static String INSERT_NEW_REPAIR_HISTORY = String.format( + "INSERT INTO %s.%s (%s, %s, %s, %s, %s, %s) values (?, ? ,?, ?, {}, ?) IF NOT EXISTS", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, + COL_HOST_ID, COL_REPAIR_START_TS, COL_REPAIR_FINISH_TS, COL_DELETE_HOSTS, COL_DELETE_HOSTS_UPDATE_TIME); + + final static String ADD_HOST_ID_TO_DELETE_HOSTS = String.format( + "UPDATE %s.%s SET %s = %s + ?, %s = ? WHERE %s = ? AND %s = ? IF EXISTS" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, + COL_DELETE_HOSTS, COL_DELETE_HOSTS_UPDATE_TIME, COL_REPAIR_TYPE, COL_HOST_ID); + + final static String DEL_AUTO_REPAIR_HISTORY = String.format( + "DELETE FROM %s.%s WHERE %s = ? AND %s = ?" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, + COL_HOST_ID); + + final static String RECORD_START_REPAIR_HISTORY = String.format( + "UPDATE %s.%s SET %s= ?, repair_turn = ? WHERE %s = ? AND %s = ?" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_START_TS, + COL_REPAIR_TYPE, COL_HOST_ID); + + final static String RECORD_FINISH_REPAIR_HISTORY = String.format( + + "UPDATE %s.%s SET %s= ?, %s=false WHERE %s = ? AND %s = ?" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_FINISH_TS, + COL_FORCE_REPAIR, COL_REPAIR_TYPE, COL_HOST_ID); + + final static String CLEAR_DELETE_HOSTS = String.format( + "UPDATE %s.%s SET %s= {} WHERE %s = ? AND %s = ?" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, + COL_REPAIR_TYPE, COL_HOST_ID); + + final static String SET_FORCE_REPAIR = String.format( + "UPDATE %s.%s SET %s=true WHERE %s = ? AND %s = ?" + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_FORCE_REPAIR, + COL_REPAIR_TYPE, COL_HOST_ID); + + static ModificationStatement delStatementRepairHistory; + static SelectStatement selectStatementRepairHistory; + static ModificationStatement delStatementPriorityStatus; + static SelectStatement selectStatementRepairPriority; + static ModificationStatement addPriorityHost; + static ModificationStatement insertNewRepairHistoryStatement; + static ModificationStatement recordStartRepairHistoryStatement; + static ModificationStatement recordFinishRepairHistoryStatement; + static ModificationStatement addHostIDToDeleteHostsStatement; + static ModificationStatement clearDeleteHostsStatement; + static ModificationStatement setForceRepairStatement; + static ConsistencyLevel internalQueryCL; + + public enum RepairTurn + { + MY_TURN, + NOT_MY_TURN, + MY_TURN_DUE_TO_PRIORITY, + MY_TURN_FORCE_REPAIR + } + + public static void setup() + { + selectStatementRepairHistory = (SelectStatement) QueryProcessor.getStatement(SELECT_REPAIR_HISTORY, ClientState + .forInternalCalls()); + selectStatementRepairPriority = (SelectStatement) QueryProcessor.getStatement(SELECT_REPAIR_PRIORITY, ClientState + .forInternalCalls()); + delStatementPriorityStatus = (ModificationStatement) QueryProcessor.getStatement(DEL_REPAIR_PRIORITY, ClientState + .forInternalCalls()); + addPriorityHost = (ModificationStatement) QueryProcessor.getStatement(ADD_PRIORITY_HOST, ClientState + .forInternalCalls()); + insertNewRepairHistoryStatement = (ModificationStatement) QueryProcessor.getStatement(INSERT_NEW_REPAIR_HISTORY, ClientState + .forInternalCalls()); + recordStartRepairHistoryStatement = (ModificationStatement) QueryProcessor.getStatement(RECORD_START_REPAIR_HISTORY, ClientState + .forInternalCalls()); + recordFinishRepairHistoryStatement = (ModificationStatement) QueryProcessor.getStatement(RECORD_FINISH_REPAIR_HISTORY, ClientState + .forInternalCalls()); + addHostIDToDeleteHostsStatement = (ModificationStatement) QueryProcessor.getStatement(ADD_HOST_ID_TO_DELETE_HOSTS, ClientState + .forInternalCalls()); + setForceRepairStatement = (ModificationStatement) QueryProcessor.getStatement(SET_FORCE_REPAIR, ClientState + .forInternalCalls()); + clearDeleteHostsStatement = (ModificationStatement) QueryProcessor.getStatement(CLEAR_DELETE_HOSTS, ClientState + .forInternalCalls()); + delStatementRepairHistory = (ModificationStatement) QueryProcessor.getStatement(DEL_AUTO_REPAIR_HISTORY, ClientState + .forInternalCalls()); + Keyspace autoRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); + internalQueryCL = autoRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? + ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.ONE; + } + + public static class AutoRepairHistory + { + UUID hostId; + String repairTurn; + long lastRepairStartTime; + long lastRepairFinishTime; + Set deleteHosts; + long deleteHostsUpdateTime; + boolean forceRepair; + + public AutoRepairHistory(UUID hostId, String repairTurn, long lastRepairStartTime, long lastRepairFinishTime, + Set deleteHosts, long deleteHostsUpateTime, boolean forceRepair) + { + this.hostId = hostId; + this.repairTurn = repairTurn; + this.lastRepairStartTime = lastRepairStartTime; + this.lastRepairFinishTime = lastRepairFinishTime; + this.deleteHosts = deleteHosts; + if (this.deleteHosts == null) + { + this.deleteHosts = new HashSet<>(); + } + this.deleteHostsUpdateTime = deleteHostsUpateTime; + this.forceRepair = forceRepair; + } + + public String toString() + { + return MoreObjects.toStringHelper(this). + add("hostId", hostId). + add("repairTurn", repairTurn). + add("lastRepairStartTime", lastRepairStartTime). + add("lastRepairFinishTime", lastRepairFinishTime). + add("deleteHosts", deleteHosts). + toString(); + } + + public boolean isRepairRunning() + { + // if a repair history record has start time laster than finish time, it means the repair is running + return lastRepairStartTime > lastRepairFinishTime; + } + + public long getLastRepairFinishTime() + { + return lastRepairFinishTime; + } + } + + public static class CurrentRepairStatus + { + public Set hostIdsWithOnGoingRepair; // hosts that is running repair + public Set hostIdsWithOnGoingForceRepair; // hosts that is running repair because of force repair + Set priority; + List historiesWithoutOnGoingRepair; // hosts that is NOT running repair + + public CurrentRepairStatus(List repairHistories, Set priority) + { + hostIdsWithOnGoingRepair = new HashSet<>(); + hostIdsWithOnGoingForceRepair = new HashSet<>(); + historiesWithoutOnGoingRepair = new ArrayList<>(); + + for (AutoRepairHistory history : repairHistories) + { + if (history.isRepairRunning()) + { + if (history.forceRepair) + { + hostIdsWithOnGoingForceRepair.add(history.hostId); + } + else + { + hostIdsWithOnGoingRepair.add(history.hostId); + } + } + else + { + historiesWithoutOnGoingRepair.add(history); + } + } + this.priority = priority; + } + + public String toString() + { + return MoreObjects.toStringHelper(this). + add("hostIdsWithOnGoingRepair", hostIdsWithOnGoingRepair). + add("hostIdsWithOnGoingForceRepair", hostIdsWithOnGoingForceRepair). + add("historiesWithoutOnGoingRepair", historiesWithoutOnGoingRepair). + add("priority", priority). + toString(); + } + } + + @VisibleForTesting + public static List getAutoRepairHistoryByGroupID(RepairType repairType) + { + UntypedResultSet repairHistoryResult; + + ResultMessage.Rows repairStatusRows = selectStatementRepairHistory.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()))), Dispatcher.RequestTime.forImmediateExecution()); + repairHistoryResult = UntypedResultSet.create(repairStatusRows.result); + + List repairHistories = new ArrayList<>(); + if (repairHistoryResult.size() > 0) + { + for (UntypedResultSet.Row row : repairHistoryResult) + { + UUID hostId = row.getUUID(COL_HOST_ID); + String repairTurn = null; + if (row.has(COL_REPAIR_TURN)) + repairTurn = row.getString(COL_REPAIR_TURN); + long lastRepairStartTime = row.getLong(COL_REPAIR_START_TS, 0); + long lastRepairFinishTime = row.getLong(COL_REPAIR_FINISH_TS, 0); + Set deleteHosts = row.getSet(COL_DELETE_HOSTS, UUIDType.instance); + long deleteHostsUpdateTime = row.getLong(COL_DELETE_HOSTS_UPDATE_TIME, 0); + Boolean forceRepair = row.has(COL_FORCE_REPAIR) ? row.getBoolean(COL_FORCE_REPAIR) : false; + repairHistories.add(new AutoRepairHistory(hostId, repairTurn, lastRepairStartTime, lastRepairFinishTime, + deleteHosts, deleteHostsUpdateTime, forceRepair)); + } + return repairHistories; + } + logger.info("No repair history found"); + return null; + } + + public static List getAutoRepairHistoryForLocalGroup(RepairType repairType) + { + return getAutoRepairHistoryByGroupID(repairType); + } + + // A host may add itself in delete hosts for some other hosts due to restart or some temp gossip issue. If a node's record + // delete_hosts is not growing for more than 2 hours, we consider it as a normal node so we clear the delete_hosts for that node + public static void clearDeleteHosts(RepairType repairType, UUID hostId) + { + clearDeleteHostsStatement.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostId))), Dispatcher.RequestTime.forImmediateExecution()); + } + + public static void setForceRepairNewNode(RepairType repairType) + { + // this function will be called when a node bootstrap finished + UUID hostId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); + // insert the data first + insertNewRepairHistory(repairType, System.currentTimeMillis(), System.currentTimeMillis()); + setForceRepair(repairType, hostId); + } + + public static void setForceRepair(RepairType repairType, Set hosts) + { + // this function is used by nodetool + for (InetAddressAndPort host : hosts) + { + UUID hostId = StorageService.instance.getHostIdForEndpoint(host); + setForceRepair(repairType, hostId); + } + } + + public static void setForceRepair(RepairType repairType, UUID hostId) + { + setForceRepairStatement.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostId))), + Dispatcher.RequestTime.forImmediateExecution()); + + logger.info("Set force repair repair type: {}, node: {}", repairType, hostId); + } + + public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType) + { + List autoRepairHistories = getAutoRepairHistoryForLocalGroup(repairType); + return getCurrentRepairStatus(repairType, autoRepairHistories); + } + + public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType, List autoRepairHistories) + { + if (autoRepairHistories != null) + { + CurrentRepairStatus status = new CurrentRepairStatus(autoRepairHistories, getPriorityHostIds(repairType)); + + return status; + } + return null; + } + + @VisibleForTesting + protected static TreeSet getHostIdsInCurrentRing(RepairType repairType, Set allNodesInRing) + { + TreeSet hostIdsInCurrentRing = new TreeSet<>(); + for (NodeAddresses node : allNodesInRing) + { + String nodeDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(node.broadcastAddress); + if (AutoRepairService.instance.getAutoRepairConfig().getIgnoreDCs(repairType).contains(nodeDC)) + { + logger.info("Ignore node {} because its datacenter is {}", node, nodeDC); + continue; + } + /** Check if endpoint state exists in gossip or not. If it + * does not then this maybe a ghost node so ignore it + */ + if (Gossiper.instance.isAlive(node.broadcastAddress)) + { + UUID hostId = StorageService.instance.getHostIdForEndpoint(node.broadcastAddress); + hostIdsInCurrentRing.add(hostId); + } + else + { + logger.info("Node is not present in Gossipe cache node {}, node data center {}", node, nodeDC); + } + } + return hostIdsInCurrentRing; + } + + public static TreeSet getHostIdsInCurrentRing(RepairType repairType) + { + Set allNodesInRing = ClusterMetadata.current().directory.addresses.values(); + return getHostIdsInCurrentRing(repairType, allNodesInRing); + } + + // This function will return the host ID for the node which has not been repaired for longest time + public static AutoRepairHistory getHostWithLongestUnrepairTime(RepairType repairType) + { + List autoRepairHistories = getAutoRepairHistoryForLocalGroup(repairType); + return getHostWithLongestUnrepairTime(autoRepairHistories); + } + + private static AutoRepairHistory getHostWithLongestUnrepairTime(List autoRepairHistories) + { + if (autoRepairHistories == null) + { + return null; + } + AutoRepairHistory rst = null; + long oldestTimestamp = Long.MAX_VALUE; + for (AutoRepairHistory autoRepairHistory : autoRepairHistories) + { + if (autoRepairHistory.lastRepairFinishTime < oldestTimestamp) + { + rst = autoRepairHistory; + oldestTimestamp = autoRepairHistory.lastRepairFinishTime; + } + } + return rst; + } + + public static int getMaxNumberOfNodeRunAutoRepairInGroup(RepairType repairType, int groupSize) + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + if (groupSize == 0) + { + return Math.max(config.getParallelRepairCountInGroup(repairType), 1); + } + // we will use the max number from config between auto_repair_parallel_repair_count_in_group and auto_repair_parallel_repair_percentage_in_group + int value = Math.max(groupSize * config.getParallelRepairPercentageInGroup(repairType) / 100, + config.getParallelRepairCountInGroup(repairType)); + // make sure at least one node getting repaired + return Math.max(1, value); + } + + @VisibleForTesting + public static RepairTurn myTurnToRunRepair(RepairType repairType, UUID myId) + { + try + { + Set allNodesInRing = ClusterMetadata.current().directory.addresses.values(); + logger.info("Total nodes in ring {}", allNodesInRing.size()); + TreeSet hostIdsInCurrentRing = getHostIdsInCurrentRing(repairType, allNodesInRing); + logger.info("Total nodes qualified for repair {}", hostIdsInCurrentRing.size()); + + List autoRepairHistories = getAutoRepairHistoryForLocalGroup(repairType); + Set autoRepairHistoryIds = new HashSet<>(); + + // 1. Remove any node that is not part of group based on goissip info + if (autoRepairHistories != null) + { + for (AutoRepairHistory nodeHistory : autoRepairHistories) + { + autoRepairHistoryIds.add(nodeHistory.hostId); + // clear delete_hosts if the node's delete hosts is not growing for more than two hours + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + if (nodeHistory.deleteHosts.size() > 0 + && config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds() < TimeUnit.MILLISECONDS.toSeconds( + System.currentTimeMillis() - nodeHistory.deleteHostsUpdateTime + )) + { + clearDeleteHosts(repairType, nodeHistory.hostId); + logger.info("Delete hosts for {} for repair type {} has not been updated for more than {} seconds. Delete hosts has been cleared. Delete hosts before clear {}" + , nodeHistory.hostId, repairType, config.getAutoRepairHistoryClearDeleteHostsBufferInterval(), nodeHistory.deleteHosts); + } + else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) + { + if (nodeHistory.deleteHosts.size() > Math.max(2, hostIdsInCurrentRing.size() * 0.5)) + { + // More than half of the groups thinks the record should be deleted + logger.info("{} think {} is orphan node, will delete auto repair history for repair type {}.", nodeHistory.deleteHosts, nodeHistory.hostId, repairType); + deleteAutoRepairHistory(repairType, nodeHistory.hostId); + } + else + { + // I think this host should be deleted + logger.info("I({}) think {} is not part of ring, vote to delete it for repair type {}.", myId, nodeHistory.hostId, repairType); + addHostIdToDeleteHosts(repairType, myId, nodeHistory.hostId); + } + } + } + } + + // 2. Add node to auto repair history table if a node is in gossip info + for (UUID hostId : hostIdsInCurrentRing) + { + if (!autoRepairHistoryIds.contains(hostId)) + { + logger.info("{} for repair type {} doesn't exist in the auto repair history table, insert a new record.", repairType, hostId); + insertNewRepairHistory(repairType, hostId, System.currentTimeMillis(), System.currentTimeMillis()); + } + } + + //get current repair status + CurrentRepairStatus currentRepairStatus = getCurrentRepairStatus(repairType, autoRepairHistories); + if (currentRepairStatus != null) + { + logger.info("Latest repair status {}", currentRepairStatus); + //check if I am forced to run repair + for (AutoRepairHistory history : currentRepairStatus.historiesWithoutOnGoingRepair) + { + if (history.forceRepair && history.hostId.equals(myId)) + { + return MY_TURN_FORCE_REPAIR; + } + } + } + + int parallelRepairNumber = getMaxNumberOfNodeRunAutoRepairInGroup(repairType, + autoRepairHistories == null ? 0 : autoRepairHistories.size()); + logger.info("Will run repairs concurrently on {} node(s)", parallelRepairNumber); + + if (currentRepairStatus == null || parallelRepairNumber > currentRepairStatus.hostIdsWithOnGoingRepair.size()) + { + // more repairs can be run, I might be the new one + + if (autoRepairHistories != null) + { + logger.info("Auto repair history table has {} records", autoRepairHistories.size()); + } + else + { + // try to fetch again + autoRepairHistories = getAutoRepairHistoryForLocalGroup(repairType); + currentRepairStatus = getCurrentRepairStatus(repairType, autoRepairHistories); + if (autoRepairHistories == null || currentRepairStatus == null) + { + logger.error("No record found"); + return NOT_MY_TURN; + } + } + + // get the longest unrepaired node from the nodes which are not running repair + AutoRepairHistory defaultNodeToBeRepaired = getHostWithLongestUnrepairTime(currentRepairStatus.historiesWithoutOnGoingRepair); + //check who is next, which is helpful for debugging + logger.info("Next node to be repaired for repair type {} by default: {}", repairType, defaultNodeToBeRepaired); + UUID priorityHostId = null; + if (currentRepairStatus.priority != null) + { + for (UUID priorityID : currentRepairStatus.priority) + { + // remove ids doesn't belong to this ring + if (!hostIdsInCurrentRing.contains(priorityID)) + { + logger.info("{} is not part of the current ring, will be removed from priority list.", priorityID); + removePriorityStatus(repairType, priorityID); + } + else + { + priorityHostId = priorityID; + break; + } + } + } + + if (priorityHostId != null && !myId.equals(priorityHostId)) + { + logger.info("Priority list is not empty and I'm not the first node in the list, not my turn." + + "First node in priority list is {}", ClusterMetadata.current().directory.addresses.get(NodeId.fromUUID(priorityHostId))); + return NOT_MY_TURN; + } + + if (myId.equals(priorityHostId)) + { + //I have a priority for repair hence its my turn now + return MY_TURN_DUE_TO_PRIORITY; + } + + if (defaultNodeToBeRepaired.hostId.equals(myId)) + return MY_TURN; + } + else if (currentRepairStatus.hostIdsWithOnGoingForceRepair.contains(myId)) + { + return MY_TURN_FORCE_REPAIR; + } + // for some reason I was not done with the repair hence resume (maybe node restart in-between, etc.) + return currentRepairStatus.hostIdsWithOnGoingRepair.contains(myId) ? MY_TURN : NOT_MY_TURN; + } + catch (Exception e) + { + logger.error("Exception while deciding node's turn:", e); + } + return NOT_MY_TURN; + } + + static void deleteAutoRepairHistory(RepairType repairType, UUID hostId) + { + //delete the given hostId from current local group + delStatementRepairHistory.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostId))), Dispatcher.RequestTime.forImmediateExecution()); + } + + static void updateStartAutoRepairHistory(RepairType repairType, UUID myId, long timestamp, RepairTurn turn) + { + recordStartRepairHistoryStatement.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(timestamp), + ByteBufferUtil.bytes(turn.name()), + ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(myId) + )), Dispatcher.RequestTime.forImmediateExecution()); + } + + static void updateFinishAutoRepairHistory(RepairType repairType, UUID myId, long timestamp) + { + recordFinishRepairHistoryStatement.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(timestamp), + ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(myId) + )), Dispatcher.RequestTime.forImmediateExecution()); + // Do not remove beblow log, the log is used by dtest + logger.info("Auto repair finished for {}", myId); + } + + public static void insertNewRepairHistory(RepairType repairType, UUID hostId, long startTime, long finishTime) + { + try + { + Keyspace autoRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); + ConsistencyLevel cl = autoRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? + ConsistencyLevel.LOCAL_SERIAL : null; + + UntypedResultSet resultSet; + ResultMessage.Rows resultMessage = (ResultMessage.Rows) insertNewRepairHistoryStatement.execute( + QueryState.forInternalCalls(), QueryOptions.create(internalQueryCL, Lists.newArrayList( + ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostId), + ByteBufferUtil.bytes(startTime), + ByteBufferUtil.bytes(finishTime), + ByteBufferUtil.bytes(System.currentTimeMillis()) + ), false, -1, null, cl, ProtocolVersion.CURRENT, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME), + Dispatcher.RequestTime.forImmediateExecution()); + resultSet = UntypedResultSet.create(resultMessage.result); + boolean applied = resultSet.one().getBoolean(ModificationStatement.CAS_RESULT_COLUMN.toString()); + if (applied) + { + logger.info("Successfully inserted a new auto repair history record for host id: {}", hostId); + } + else + { + logger.info("Record exists, no need to insert again for host id: {}", hostId); + } + } + catch (Exception e) + { + logger.error("Exception in inserting new repair history:", e); + } + } + + public static void insertNewRepairHistory(RepairType repairType, long startTime, long finishTime) + { + UUID hostId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); + insertNewRepairHistory(repairType, hostId, startTime, finishTime); + } + + public static void addHostIdToDeleteHosts(RepairType repairType, UUID myID, UUID hostToBeDeleted) + { + SetSerializer serializer = SetSerializer.getInstance(UUIDSerializer.instance, UTF8Type.instance.comparatorSet); + addHostIDToDeleteHostsStatement.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(serializer.serialize(new HashSet<>(Arrays.asList(myID))), + ByteBufferUtil.bytes(System.currentTimeMillis()), + ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostToBeDeleted) + )), Dispatcher.RequestTime.forImmediateExecution()); + } + + public static void addPriorityHosts(RepairType repairType, Set hosts) + { + Set hostIds = new HashSet<>(); + for (InetAddressAndPort host : hosts) + { + //find hostId from IP address + UUID hostId = ClusterMetadata.current().directory.hostId(ClusterMetadata.current().directory.peerId(host)); + //UUID hostId = StorageService.instance.getTokenMetadata().getHostId(host); + hostIds.add(hostId); + if (hostId != null) + { + logger.info("Add host {} to the priority list", hostId); + } + } + if (hostIds.size() > 0) + { + SetSerializer serializer = SetSerializer.getInstance(UUIDSerializer.instance, UTF8Type.instance.comparatorSet); + addPriorityHost.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(serializer.serialize(hostIds), + ByteBufferUtil.bytes(repairType.toString()))), + Dispatcher.RequestTime.forImmediateExecution()); + } + } + + static void removePriorityStatus(RepairType repairType, UUID hostId) + { + logger.info("Remove host {} from priority list", hostId); + delStatementPriorityStatus.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList(ByteBufferUtil.bytes(hostId), + ByteBufferUtil.bytes(repairType.toString()))), + Dispatcher.RequestTime.forImmediateExecution()); + } + + public static Set getPriorityHostIds(RepairType repairType) + { + UntypedResultSet repairPriorityResult; + + ResultMessage.Rows repairPriorityRows = selectStatementRepairPriority.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()))), Dispatcher.RequestTime.forImmediateExecution()); + repairPriorityResult = UntypedResultSet.create(repairPriorityRows.result); + + Set priorities = null; + if (repairPriorityResult.size() > 0) + { + // there should be only one row + UntypedResultSet.Row row = repairPriorityResult.one(); + priorities = row.getSet(COL_REPAIR_PRIORITY, UUIDType.instance); + } + if (priorities != null) + { + return priorities; + } + return Collections.emptySet(); + } + + public static Set getPriorityHosts(RepairType repairType) + { + Set hosts = new HashSet<>(); + for (UUID hostId : getPriorityHostIds(repairType)) + { + hosts.add(ClusterMetadata.current().directory.addresses.get(NodeId.fromUUID(hostId)).broadcastAddress); + //hosts.add(StorageService.instance.getTokenMetadata().getEndpointForHostId(hostId)); + } + return hosts; + } + + public static boolean checkNodeContainsKeyspaceReplica(Keyspace ks) + { + AbstractReplicationStrategy replicationStrategy = ks.getReplicationStrategy(); + boolean ksReplicaOnNode = true; + if (replicationStrategy instanceof NetworkTopologyStrategy) + { + Set datacenters = ((NetworkTopologyStrategy) replicationStrategy).getDatacenters(); + String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort()); + if (!datacenters.contains(localDC)) + { + ksReplicaOnNode = false; + } + } + if (replicationStrategy instanceof LocalStrategy) + { + ksReplicaOnNode = false; + } + return ksReplicaOnNode; + } + + + public static boolean tableMaxRepairTimeExceeded(RepairType repairType, long startTime) + { + long tableRepairTimeSoFar = TimeUnit.MILLISECONDS.toSeconds + (System.currentTimeMillis() - startTime); + return AutoRepairService.instance.getAutoRepairConfig().getAutoRepairTableMaxRepairTime(repairType).toSeconds() < + tableRepairTimeSoFar; + } + + public static boolean keyspaceMaxRepairTimeExceeded(RepairType repairType, long startTime, int numOfTablesToBeRepaired) + { + long keyspaceRepairTimeSoFar = TimeUnit.MILLISECONDS.toSeconds((System.currentTimeMillis() - startTime)); + return (long) AutoRepairService.instance.getAutoRepairConfig().getAutoRepairTableMaxRepairTime(repairType).toSeconds() * + numOfTablesToBeRepaired < keyspaceRepairTimeSoFar; + } + + public static List getAllMVs(RepairType repairType, Keyspace keyspace, TableMetadata tableMetadata) + { + List allMvs = new ArrayList<>(); + if (AutoRepairService.instance.getAutoRepairConfig().getMVRepairEnabled(repairType) && keyspace.getMetadata().views != null) + { + Iterator views = keyspace.getMetadata().views.forTable(tableMetadata.id).iterator(); + while (views.hasNext()) + { + String viewName = views.next().name(); + logger.info("Adding MV to the list {}.{}.{}", keyspace.getName(), tableMetadata.name, viewName); + allMvs.add(viewName); + } + } + return allMvs; + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java new file mode 100644 index 000000000000..18231fff2af0 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.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.repair.autorepair; + + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.apache.cassandra.service.AutoRepairService; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.utils.Pair; + +public class DefaultAutoRepairTokenSplitter implements IAutoRepairTokenRangeSplitter +{ + private static final Logger logger = LoggerFactory.getLogger(DefaultAutoRepairTokenSplitter.class); + + + @Override + public List> getRange(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, String tableName) + { + List> range = new ArrayList<>(); + + Collection> tokens = StorageService.instance.getPrimaryRanges(keyspaceName); + if (!primaryRangeOnly) + { + // if we need to repair non-primary token ranges, then change the tokens accrodingly + tokens = StorageService.instance.getLocalReplicas(keyspaceName).ranges(); + } + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + int numberOfSubranges = config.getRepairSubRangeNum(repairType); + for (Range token : tokens) + { + Murmur3Partitioner.LongToken l = (Murmur3Partitioner.LongToken) (token.left); + Murmur3Partitioner.LongToken r = (Murmur3Partitioner.LongToken) (token.right); + // Token.TokenFactory factory = ClusterMetadata.current().partitioner.getTokenFactory(); + + Token parentStartToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + l.getTokenValue()); + Token parentEndToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + r.getTokenValue()); + logger.debug("Parent Token Left side {}, right side {}", parentStartToken.toString(), + parentEndToken.toString()); + + long left = (Long) l.getTokenValue(); + long right = (Long) r.getTokenValue(); + long repairTokenWidth = (right - left) / numberOfSubranges; + for (int i = 0; i < numberOfSubranges; i++) + { + long curLeft = left + (i * repairTokenWidth); + long curRight = curLeft + repairTokenWidth; + + if ((i + 1) == numberOfSubranges) + { + curRight = right; + } + + Token childStartToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + curLeft); + Token childEndToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + curRight); + logger.debug("Current Token Left side {}, right side {}", childStartToken + .toString(), childEndToken.toString()); + range.add(Pair.create(childStartToken, childEndToken)); + } + } + return range; + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java new file mode 100644 index 000000000000..2b69898a360d --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.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.repair.autorepair; + + +import java.util.List; + +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.utils.Pair; + +public interface IAutoRepairTokenRangeSplitter +{ + // split the token range you wish to repair into multiple subranges + // the autorepair framework will repair the list of returned subrange in a sequence + List> getRange(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, String tableName); +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/schema/AutoRepairParams.java b/src/java/org/apache/cassandra/schema/AutoRepairParams.java new file mode 100644 index 000000000000..50f482caf1b7 --- /dev/null +++ b/src/java/org/apache/cassandra/schema/AutoRepairParams.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.schema; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableMap; +import org.apache.commons.lang3.StringUtils; + +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; + +import static java.lang.String.format; + +public final class AutoRepairParams +{ + public enum Option + { + ENABLED; + + @Override + public String toString() + { + return name().toLowerCase(); + } + } + + public static final Map> DEFAULT_OPTIONS = + ImmutableMap.of(AutoRepairConfig.RepairType.full, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), + AutoRepairConfig.RepairType.incremental, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true))); + + public final AutoRepairConfig.RepairType type; + + private Map> options = DEFAULT_OPTIONS; + + AutoRepairParams(AutoRepairConfig.RepairType type) + { + this.type = type; + } + + public static AutoRepairParams create(AutoRepairConfig.RepairType repairType, Map options) + { + Map> optionsMap = new HashMap<>(); + for (Map.Entry> entry : DEFAULT_OPTIONS.entrySet()) + { + optionsMap.put(entry.getKey(), new HashMap<>(entry.getValue())); + } + for (Map.Entry entry : options.entrySet()) + { + if (!Option.ENABLED.toString().equals(entry.getKey().toLowerCase())) + { + throw new ConfigurationException(format("Unknown property '%s'", entry.getKey())); + } + optionsMap.get(repairType).put(entry.getKey(), entry.getValue()); + } + AutoRepairParams repairParams = new AutoRepairParams(repairType); + repairParams.options = optionsMap; + return repairParams; + } + + public boolean repairEnabled() + { + String enabled = options.get(type).get(Option.ENABLED.toString()); + return enabled == null + ? Boolean.parseBoolean(DEFAULT_OPTIONS.get(type).get(Option.ENABLED.toString())) + : Boolean.parseBoolean(enabled); + } + + public void validate() + { + String enabled = options.get(type).get(Option.ENABLED.toString()); + if (enabled != null && !isValidBoolean(enabled)) + { + throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be a boolean", + enabled, + Option.ENABLED)); + } + } + + public static boolean isValidBoolean(String value) + { + return StringUtils.equalsIgnoreCase(value, "true") || StringUtils.equalsIgnoreCase(value, "false"); + } + + public Map options() + { + return options.get(type); + } + + public static AutoRepairParams fromMap(AutoRepairConfig.RepairType repairType, Map map) + { + return create(repairType, map); + } + + public Map asMap() + { + return options.get(type); + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("options", options) + .toString(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof AutoRepairParams)) + return false; + + AutoRepairParams cp = (AutoRepairParams) o; + + return options.equals(cp.options); + } + + @Override + public int hashCode() + { + return Objects.hash(options); + } +} diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index a6877cc6318c..49007bdb9c99 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -44,6 +44,7 @@ import org.apache.cassandra.db.partitions.*; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; import org.apache.cassandra.schema.ColumnMetadata.ClusteringOrder; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; @@ -128,6 +129,8 @@ private SchemaKeyspace() + "additional_write_policy text," + "cdc boolean," + "read_repair text," + + "automated_repair_full frozen>," + + "automated_repair_incremental frozen>," + "PRIMARY KEY ((keyspace_name), table_name))"); private static final TableMetadata Columns = @@ -212,6 +215,8 @@ private SchemaKeyspace() + "additional_write_policy text," + "cdc boolean," + "read_repair text," + + "automated_repair_full frozen>," + + "automated_repair_incremental frozen>," + "PRIMARY KEY ((keyspace_name), view_name))"); private static final TableMetadata Indexes = @@ -559,7 +564,10 @@ private static void addTableParamsToRowBuilder(TableParams params, Row.SimpleBui .add("compaction", params.compaction.asMap()) .add("compression", params.compression.asMap()) .add("read_repair", params.readRepair.toString()) - .add("extensions", params.extensions); + .add("extensions", params.extensions) + .add("automated_repair_full", params.automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) + .add("automated_repair_incremental", params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); + // Only add CDC-enabled flag to schema if it's enabled on the node. This is to work around RTE's post-8099 if a 3.8+ // node sends table schema to a < 3.8 versioned node with an unknown column. @@ -1042,7 +1050,9 @@ static TableParams createTableParamsFromRow(UntypedResultSet.Row row) SpeculativeRetryPolicy.fromString(row.getString("additional_write_policy")) : SpeculativeRetryPolicy.fromString("99PERCENTILE")) .cdc(row.has("cdc") && row.getBoolean("cdc")) - .readRepair(getReadRepairStrategy(row)); + .readRepair(getReadRepairStrategy(row)) + .automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, row.getFrozenTextMap("automated_repair_full"))) + .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, row.getFrozenTextMap("automated_repair_incremental"))); // allow_auto_snapshot column was introduced in 4.2 if (row.has("allow_auto_snapshot")) diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index 6903179525b7..751051984a80 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; @@ -34,6 +35,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.service.reads.PercentileSpeculativeRetryPolicy; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; @@ -69,7 +71,10 @@ public enum Option ADDITIONAL_WRITE_POLICY, CRC_CHECK_CHANCE, CDC, - READ_REPAIR; + READ_REPAIR, + AUTOMATED_REPAIR_FULL, + AUTOMATED_REPAIR_INCREMENTAL, + ; @Override public String toString() @@ -98,6 +103,8 @@ public String toString() public final boolean cdc; public final ReadRepairStrategy readRepair; + public final Map automatedRepair; + private TableParams(Builder builder) { comment = builder.comment; @@ -121,6 +128,13 @@ private TableParams(Builder builder) extensions = builder.extensions; cdc = builder.cdc; readRepair = builder.readRepair; + automatedRepair = new HashMap<>() + { + { + put(AutoRepairConfig.RepairType.full, builder.automatedRepairFull); + put(AutoRepairConfig.RepairType.incremental, builder.automatedRepairIncremental); + } + }; } public static Builder builder() @@ -148,7 +162,10 @@ public static Builder builder(TableParams params) .additionalWritePolicy(params.additionalWritePolicy) .extensions(params.extensions) .cdc(params.cdc) - .readRepair(params.readRepair); + .readRepair(params.readRepair) + .automatedRepairFull(params.automatedRepair.get(AutoRepairConfig.RepairType.full)) + .automatedRepairIncremental(params.automatedRepair.get(AutoRepairConfig.RepairType.incremental)) + ; } public Builder unbuild() @@ -162,7 +179,7 @@ public void validate() compression.validate(); double minBloomFilterFpChanceValue = BloomCalculations.minSupportedBloomFilterFpChance(); - if (bloomFilterFpChance <= minBloomFilterFpChanceValue || bloomFilterFpChance > 1) + if (bloomFilterFpChance <= minBloomFilterFpChanceValue || bloomFilterFpChance > 1) { fail("%s must be larger than %s and less than or equal to 1.0 (got %s)", BLOOM_FILTER_FP_CHANCE, @@ -203,6 +220,11 @@ public void validate() if (cdc && memtable.factory().writesShouldSkipCommitLog()) fail("CDC cannot work if writes skip the commit log. Check your memtable configuration."); + + for (Map.Entry entry : automatedRepair.entrySet()) + { + entry.getValue().validate(); + } } private static void fail(String format, Object... args) @@ -239,7 +261,8 @@ public boolean equals(Object o) && memtable.equals(p.memtable) && extensions.equals(p.extensions) && cdc == p.cdc - && readRepair == p.readRepair; + && readRepair == p.readRepair + && automatedRepair == p.automatedRepair; } @Override @@ -263,7 +286,8 @@ public int hashCode() memtable, extensions, cdc, - readRepair); + readRepair, + automatedRepair); } @Override @@ -289,6 +313,8 @@ public String toString() .add(EXTENSIONS.toString(), extensions) .add(CDC.toString(), cdc) .add(READ_REPAIR.toString(), readRepair) + .add(Option.AUTOMATED_REPAIR_FULL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.full)) + .add(Option.AUTOMATED_REPAIR_INCREMENTAL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.incremental)) .toString(); } @@ -340,7 +366,11 @@ public void appendCqlTo(CqlBuilder builder, boolean isView) .newLine() .append("AND read_repair = ").appendWithSingleQuotes(readRepair.toString()) .newLine() - .append("AND speculative_retry = ").appendWithSingleQuotes(speculativeRetry.toString()); + .append("AND speculative_retry = ").appendWithSingleQuotes(speculativeRetry.toString()) + .newLine() + .append("AND automated_repair_full = ").append(automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) + .newLine() + .append("AND automated_repair_incremental = ").append(automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); } public static final class Builder @@ -365,6 +395,9 @@ public static final class Builder private boolean cdc; private ReadRepairStrategy readRepair = ReadRepairStrategy.BLOCKING; + private AutoRepairParams automatedRepairFull = new AutoRepairParams(AutoRepairConfig.RepairType.full); + private AutoRepairParams automatedRepairIncremental = new AutoRepairParams(AutoRepairConfig.RepairType.incremental); + public Builder() { } @@ -487,6 +520,18 @@ public Builder extensions(Map val) extensions = ImmutableMap.copyOf(val); return this; } + + public Builder automatedRepairFull(AutoRepairParams val) + { + automatedRepairFull = val; + return this; + } + + public Builder automatedRepairIncremental(AutoRepairParams val) + { + automatedRepairIncremental = val; + return this; + } } public static class Serializer implements MetadataSerializer diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java index 9682c8e9594d..ed3d14752add 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairService.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java @@ -51,6 +51,26 @@ import com.google.common.collect.Lists; import com.google.common.collect.Multimap; +import org.apache.cassandra.concurrent.ExecutorPlus; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.repair.Scheduler; +import org.apache.cassandra.locator.AbstractReplicationStrategy; +import org.apache.cassandra.locator.EndpointsByRange; +import org.apache.cassandra.locator.EndpointsForRange; +import org.apache.cassandra.service.disk.usage.DiskUsageMonitor; +import org.apache.cassandra.utils.ExecutorUtils; +import org.apache.cassandra.repair.state.CoordinatorState; +import org.apache.cassandra.repair.state.ParticipateState; +import org.apache.cassandra.repair.state.ValidationState; +import org.apache.cassandra.utils.Simulate; +import org.apache.cassandra.locator.EndpointsForToken; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.utils.TimeUUID; +import org.apache.cassandra.utils.concurrent.CountDownLatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -656,6 +676,9 @@ public boolean verifyCompactionsPendingThreshold(TimeUUID parentRepairSession, P public Future prepareForRepair(TimeUUID parentRepairSession, InetAddressAndPort coordinator, Set endpoints, RepairOption options, boolean isForcedRepair, List columnFamilyStores) { + if (!verifyDiskHeadroomThreshold(parentRepairSession, options.getPreviewKind(), options.isIncremental())) + failRepair(parentRepairSession, "Rejecting incoming repair, disk usage above threshold"); // failRepair throws exception + if (!verifyCompactionsPendingThreshold(parentRepairSession, options.getPreviewKind())) failRepair(parentRepairSession, "Rejecting incoming repair, pending compactions above threshold"); // failRepair throws exception @@ -713,6 +736,24 @@ public Future prepareForRepair(TimeUUID parentRepairSession, InetAddressAndPo return promise; } + public static boolean verifyDiskHeadroomThreshold(TimeUUID parentRepairSession, PreviewKind previewKind, boolean isIncremental) + { + if (!isIncremental) // disk headroom is required for anti-compaction which is only performed by incremental repair + return true; + + double diskUsage = DiskUsageMonitor.instance.getDiskUsage(); + double rejectRatio = ActiveRepairService.instance().getIncrementalRepairDiskHeadroomRejectRatio(); + + if (diskUsage + rejectRatio > 1) + { + logger.error("[{}] Rejecting incoming repair, disk usage ({}%) above threshold ({}%)", + previewKind.logPrefix(parentRepairSession), String.format("%.2f", diskUsage * 100), String.format("%.2f", (1 - rejectRatio) * 100)); + return false; + } + + return true; + } + private void sendPrepareWithRetries(TimeUUID parentRepairSession, AtomicInteger pending, Set failedNodes, @@ -1075,6 +1116,16 @@ public void setRepairPendingCompactionRejectThreshold(int value) DatabaseDescriptor.setRepairPendingCompactionRejectThreshold(value); } + public double getIncrementalRepairDiskHeadroomRejectRatio() + { + return DatabaseDescriptor.getIncrementalRepairDiskHeadroomRejectRatio(); + } + + public void setIncrementalRepairDiskHeadroomRejectRatio(double value) + { + DatabaseDescriptor.setIncrementalRepairDiskHeadroomRejectRatio(value); + } + /** * Remove any parent repair sessions matching predicate */ diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java new file mode 100644 index 000000000000..b07d770fb2de --- /dev/null +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -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. + */ +package org.apache.cassandra.service; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.utils.MBeanWrapper; + +import java.util.Set; + +import com.google.common.annotations.VisibleForTesting; + +public class AutoRepairService implements AutoRepairServiceMBean +{ + public static final String MBEAN_NAME = "org.apache.cassandra.db:type=AutoRepairService"; + + @VisibleForTesting + protected AutoRepairConfig config; + + public static final AutoRepairService instance = new AutoRepairService(); + + @VisibleForTesting + protected AutoRepairService() + { + } + + public static void setup() + { + instance.config = DatabaseDescriptor.getAutoRepairConfig(); + } + + static + { + MBeanWrapper.instance.registerMBean(instance, MBEAN_NAME); + } + + @Override + public AutoRepairConfig getAutoRepairConfig() + { + return config; + } + + @Override + public void setAutoRepairEnabled(RepairType repairType, boolean enabled) + { + config.setAutoRepairEnabled(repairType, enabled); + } + + @Override + public void setRepairThreads(RepairType repairType, int repairThreads) + { + config.setRepairThreads(repairType, repairThreads); + } + + @Override + public void setRepairPriorityForHosts(RepairType repairType, Set hosts) + { + AutoRepairUtils.addPriorityHosts(repairType, hosts); + } + + @Override + public Set getRepairHostPriority(RepairType repairType) { + return AutoRepairUtils.getPriorityHosts(repairType); + } + + @Override + public void setForceRepairForHosts(RepairType repairType, Set hosts) + { + AutoRepairUtils.setForceRepair(repairType, hosts); + } + + @Override + public void setRepairSubRangeNum(RepairType repairType, int repairSubRanges) + { + config.setRepairSubRangeNum(repairType, repairSubRanges); + } + + @Override + public void setRepairMinInterval(RepairType repairType, String minRepairInterval) + { + config.setRepairMinInterval(repairType, minRepairInterval); + } + + public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) + { + config.setAutoRepairHistoryClearDeleteHostsBufferInterval(duration); + } + + @Override + public void setRepairSSTableCountHigherThreshold(RepairType repairType, int sstableHigherThreshold) + { + config.setRepairSSTableCountHigherThreshold(repairType, sstableHigherThreshold); + } + + @Override + public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime) + { + config.setAutoRepairTableMaxRepairTime(repairType, autoRepairTableMaxRepairTime); + } + + @Override + public void setIgnoreDCs(RepairType repairType, Set ignoreDCs) + { + config.setIgnoreDCs(repairType, ignoreDCs); + } + + @Override + public void setPrimaryTokenRangeOnly(RepairType repairType, boolean primaryTokenRangeOnly) + { + config.setRepairPrimaryTokenRangeOnly(repairType, primaryTokenRangeOnly); + } + + @Override + public void setParallelRepairPercentageInGroup(RepairType repairType, int percentageInGroup) + { + config.setParallelRepairPercentageInGroup(repairType, percentageInGroup); + } + + @Override + public void setParallelRepairCountInGroup(RepairType repairType, int countInGroup) + { + config.setParallelRepairCountInGroup(repairType, countInGroup); + } + + public void setMVRepairEnabled(RepairType repairType, boolean enabled) + { + config.setMVRepairEnabled(repairType, enabled); + } +} diff --git a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java new file mode 100644 index 000000000000..5dda1b157517 --- /dev/null +++ b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.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.service; + +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; + +import java.util.Set; + +public interface AutoRepairServiceMBean +{ + /** + * Enable or disable auto-repair for a given repair type + */ + public void setAutoRepairEnabled(RepairType repairType, boolean enabled); + + public void setRepairThreads(RepairType repairType, int repairThreads); + + public void setRepairPriorityForHosts(RepairType repairType, Set host); + + public void setForceRepairForHosts(RepairType repairType, Set host); + + public Set getRepairHostPriority(RepairType repairType); + + public void setRepairSubRangeNum(RepairType repairType, int repairSubRangeNum); + + public void setRepairMinInterval(RepairType repairType, String minRepairInterval); + + public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration); + + public void setRepairSSTableCountHigherThreshold(RepairType repairType, int ssTableHigherThreshold); + + public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime); + public void setIgnoreDCs(RepairType repairType, Set ignorDCs); + + public void setPrimaryTokenRangeOnly(RepairType repairType, boolean primaryTokenRangeOnly); + + public void setParallelRepairPercentageInGroup(RepairType repairType, int percentageInGroup); + public void setParallelRepairCountInGroup(RepairType repairType, int countInGroup); + + public void setMVRepairEnabled(RepairType repairType, boolean enabled); + + public AutoRepairConfig getAutoRepairConfig(); +} diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index ec66cf2fe9de..35197c3e1866 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -405,6 +405,8 @@ protected void setup() AuditLogManager.instance.initialize(); + StorageService.instance.doAutoRepairSetup(); + // schedule periodic background compaction task submission. this is simply a backstop against compactions stalling // due to scheduling errors or race conditions ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(ColumnFamilyStore.getBackgroundCompactionTaskSubmitter(), 5, 1, TimeUnit.MINUTES); diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index ab8ddfe9bbb1..c84fcd5d822c 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -74,6 +74,8 @@ import com.google.common.collect.Sets; import com.google.common.util.concurrent.RateLimiter; import com.google.common.util.concurrent.Uninterruptibles; +import org.apache.cassandra.repair.autorepair.AutoRepairKeyspace; +import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -463,7 +465,7 @@ public enum Mode { STARTING, NORMAL, JOINING, JOINING_FAILED, LEAVING, DECOMMISS private volatile int totalCFs, remainingCFs; - private static final AtomicInteger nextRepairCommand = new AtomicInteger(); + public static final AtomicInteger nextRepairCommand = new AtomicInteger(); private final List lifecycleSubscribers = new CopyOnWriteArrayList<>(); @@ -1127,6 +1129,17 @@ public void doAuthSetup(boolean async) } } + public void doAutoRepairSetup() + { + if (DatabaseDescriptor.getAutoRepairConfig().isAutoRepairSchedulingEnabled()) + { + logger.info("Enable auto-repair scheduling"); + AutoRepair.instance.setup(); + } + logger.info("AutoRepair setup complete!"); + } + + public boolean isAuthSetupComplete() { return authSetupComplete; diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 5c75985bda2e..f9ccb037fc8e 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -93,13 +93,17 @@ import org.apache.cassandra.locator.DynamicEndpointSnitchMBean; import org.apache.cassandra.locator.EndpointSnitchInfoMBean; import org.apache.cassandra.metrics.CIDRAuthorizerMetrics; +import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.StorageMetrics; import org.apache.cassandra.metrics.TableMetrics; import org.apache.cassandra.metrics.ThreadPoolMetrics; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.MessagingServiceMBean; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.service.ActiveRepairServiceMBean; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.AutoRepairServiceMBean; import org.apache.cassandra.service.CacheService; import org.apache.cassandra.service.CacheServiceMBean; import org.apache.cassandra.tcm.CMSOperationsMBean; @@ -171,6 +175,8 @@ public class NodeProbe implements AutoCloseable protected CIDRGroupsMappingManagerMBean cmbProxy; protected PermissionsCacheMBean pcProxy; protected RolesCacheMBean rcProxy; + protected AutoRepairServiceMBean autoRepairProxy; + protected Output output; private boolean failed; @@ -311,6 +317,9 @@ protected void connect() throws IOException name = new ObjectName(CIDRFilteringMetricsTable.MBEAN_NAME); cfmProxy = JMX.newMBeanProxy(mbeanServerConn, name, CIDRFilteringMetricsTableMBean.class); + + name = new ObjectName(AutoRepairService.MBEAN_NAME); + autoRepairProxy = JMX.newMBeanProxy(mbeanServerConn, name, AutoRepairServiceMBean.class); } catch (MalformedObjectNameException e) { @@ -2405,6 +2414,82 @@ public void abortBootstrap(String nodeId, String endpoint) { ssProxy.abortBootstrap(nodeId, endpoint); } + + public AutoRepairConfig getAutoRepairConfig() { + return autoRepairProxy.getAutoRepairConfig(); + } + + public void setAutoRepairEnabled(AutoRepairConfig.RepairType repairType, boolean enabled) + { + autoRepairProxy.setAutoRepairEnabled(repairType, enabled); + } + + public void setRepairThreads(AutoRepairConfig.RepairType repairType, int repairThreads) + { + autoRepairProxy.setRepairThreads(repairType, repairThreads); + } + + public void setRepairPriorityForHosts(AutoRepairConfig.RepairType repairType, Set hosts) + { + autoRepairProxy.setRepairPriorityForHosts(repairType, hosts); + } + + public Set getRepairPriorityForHosts(AutoRepairConfig.RepairType repairType) + { + return autoRepairProxy.getRepairHostPriority(repairType); + } + + public void setForceRepairForHosts(AutoRepairConfig.RepairType repairType, Set hosts){ + autoRepairProxy.setForceRepairForHosts(repairType, hosts); + } + + public void setRepairSubRangeNum(AutoRepairConfig.RepairType repairType, int repairSubRanges) + { + autoRepairProxy.setRepairSubRangeNum(repairType, repairSubRanges); + } + + public void setRepairMinInterval(AutoRepairConfig.RepairType repairType, String minRepairInterval) + { + autoRepairProxy.setRepairMinInterval(repairType, minRepairInterval); + } + + public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) + { + autoRepairProxy.setAutoRepairHistoryClearDeleteHostsBufferDuration(duration); + } + + public void setRepairSSTableCountHigherThreshold(AutoRepairConfig.RepairType repairType, int ssTableHigherThreshold) + { + autoRepairProxy.setRepairSSTableCountHigherThreshold(repairType, ssTableHigherThreshold); + } + + public void setAutoRepairTableMaxRepairTime(AutoRepairConfig.RepairType repairType, String autoRepairTableMaxRepairTime) + { + autoRepairProxy.setAutoRepairTableMaxRepairTime(repairType, autoRepairTableMaxRepairTime); + } + + public void setAutoRepairIgnoreDCs(AutoRepairConfig.RepairType repairType, Set ignoreDCs) + { + autoRepairProxy.setIgnoreDCs(repairType, ignoreDCs); + } + + public void setParallelRepairPercentageInGroup(AutoRepairConfig.RepairType repairType, int percentageInGroup) { + autoRepairProxy.setParallelRepairPercentageInGroup(repairType, percentageInGroup); + } + + public void setParallelRepairCountInGroup(AutoRepairConfig.RepairType repairType, int countInGroup) { + autoRepairProxy.setParallelRepairCountInGroup(repairType, countInGroup); + } + + public void setPrimaryTokenRangeOnly(AutoRepairConfig.RepairType repairType, boolean primaryTokenRangeOnly) + { + autoRepairProxy.setPrimaryTokenRangeOnly(repairType, primaryTokenRangeOnly); + } + + public void setMVRepairEnabled(AutoRepairConfig.RepairType repairType, boolean enabled) + { + autoRepairProxy.setMVRepairEnabled(repairType, enabled); + } } class ColumnFamilyStoreMBeanIterator implements Iterator> diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index bd1e302ba066..707a9e0c64f1 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -134,6 +134,7 @@ public int execute(String... args) GcStats.class, GetAuditLog.class, GetAuthCacheConfig.class, + GetAutoRepairConfig.class, GetBatchlogReplayTrottle.class, GetCIDRGroupsOfIP.class, GetColumnIndexSize.class, @@ -196,6 +197,7 @@ public int execute(String... args) Ring.class, Scrub.class, SetAuthCacheConfig.class, + SetAutoRepairConfig.class, SetBatchlogReplayThrottle.class, SetCacheCapacity.class, SetCacheKeysToSave.class, diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java new file mode 100644 index 000000000000..49ccf0f2aa24 --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.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.tools.nodetool; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import io.airlift.airline.Command; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.NodeTool.NodeToolCmd; + +import java.io.PrintStream; + +@Command(name = "getautorepairconfig", description = "Print autorepair configurations") +public class GetAutoRepairConfig extends NodeToolCmd +{ + @VisibleForTesting + protected static PrintStream out = System.out; + + @Override + public void execute(NodeProbe probe) + { + AutoRepairConfig config = probe.getAutoRepairConfig(); + if (config == null || !config.isAutoRepairSchedulingEnabled()) + { + out.println("Auto-repair is not enabled"); + return; + } + + StringBuilder sb = new StringBuilder(); + sb.append("repair scheduler configuration:"); + sb.append("\n\trepair eligibility check interval: " + config.getRepairCheckInterval()); + sb.append("\n\tTTL for repair history for dead nodes: " + config.getAutoRepairHistoryClearDeleteHostsBufferInterval()); + for (RepairType repairType : RepairType.values()) + { + sb.append(formatRepairTypeConfig(probe, repairType, config)); + } + + out.println(sb); + } + + private String formatRepairTypeConfig(NodeProbe probe, RepairType repairType, AutoRepairConfig config) + { + StringBuilder sb = new StringBuilder(); + sb.append("\nconfiguration for repair type: " + repairType); + sb.append("\n\tenabled: " + config.isAutoRepairEnabled(repairType)); + sb.append("\n\tminimum repair interval: " + config.getRepairMinInterval(repairType)); + sb.append("\n\trepair threads: " + config.getRepairThreads(repairType)); + sb.append("\n\tnumber of repair subranges: " + config.getRepairSubRangeNum(repairType)); + sb.append("\n\tpriority hosts: " + Joiner.on(',').skipNulls().join(probe.getRepairPriorityForHosts(repairType))); + sb.append("\n\tsstable count higher threshold: " + config.getRepairSSTableCountHigherThreshold(repairType)); + sb.append("\n\ttable max repair time in sec: " + config.getAutoRepairTableMaxRepairTime(repairType)); + sb.append("\n\tignore datacenters: " + Joiner.on(',').skipNulls().join(config.getIgnoreDCs(repairType))); + sb.append("\n\trepair primary token-range: " + config.getRepairPrimaryTokenRangeOnly(repairType)); + sb.append("\n\tnumber of parallel repairs within group: " + config.getParallelRepairCountInGroup(repairType)); + sb.append("\n\tpercentage of parallel repairs within group: " + config.getParallelRepairPercentageInGroup(repairType)); + sb.append("\n\tmv repair enabled: " + config.getMVRepairEnabled(repairType)); + + return sb.toString(); + } +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java new file mode 100644 index 000000000000..bd00809dcf22 --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Splitter; +import io.airlift.airline.Arguments; +import io.airlift.airline.Command; +import io.airlift.airline.Option; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.NodeTool.NodeToolCmd; + +import java.io.PrintStream; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static com.google.common.base.Preconditions.checkArgument; + +@Command(name = "setautorepairconfig", description = "sets the autorepair configuration") +public class SetAutoRepairConfig extends NodeToolCmd +{ + @VisibleForTesting + @Arguments(title = " ", usage = " ", + description = "autorepair param and value.\nPossible autorepair parameters are as following: " + + "[number_of_repair_threads|number_of_subranges|min_repair_interval|sstable_upper_threshold" + + "|enabled|table_max_repair_time|priority_hosts|forcerepair_hosts|ignore_dcs" + + "|history_clear_delete_hosts_buffer_interval|repair_primary_token_range_only|parallel_repair_count_in_group|parallel_repair_percentage_in_group|mv_repair_enabled]", + required = true) + protected List args = new ArrayList<>(); + + @VisibleForTesting + @Option(title = "repair type", name = { "-t", "--repair-type" }, description = "Repair type") + protected RepairType repairType; + + @VisibleForTesting + protected PrintStream out = System.out; + + @Override + public void execute(NodeProbe probe) + { + checkArgument(repairType != null, "--repair-type is required."); + checkArgument(args.size() == 2, "setautorepairconfig requires param-type, and value args."); + String paramType = args.get(0); + String paramVal = args.get(1); + + if (!probe.getAutoRepairConfig().isAutoRepairSchedulingEnabled()) + { + out.println("Auto-repair is not enabled"); + return; + } + + if (paramType.equals("history_clear_delete_hosts_buffer_interval")) + { + probe.setAutoRepairHistoryClearDeleteHostsBufferDuration(paramVal); + return; + } + + // options below require --repair-type option + checkArgument(repairType != null, "--repair-type is required for this parameter."); + Set hosts; + switch (paramType) + { + case "enabled": + probe.setAutoRepairEnabled(repairType, Boolean.parseBoolean(paramVal)); + break; + case "number_of_repair_threads": + probe.setRepairThreads(repairType, Integer.parseInt(paramVal)); + break; + case "number_of_subranges": + probe.setRepairSubRangeNum(repairType, Integer.parseInt(paramVal)); + break; + case "min_repair_interval": + probe.setRepairMinInterval(repairType, paramVal); + break; + case "sstable_upper_threshold": + probe.setRepairSSTableCountHigherThreshold(repairType, Integer.parseInt(paramVal)); + break; + case "table_max_repair_time": + probe.setAutoRepairTableMaxRepairTime(repairType, paramVal); + break; + case "priority_hosts": + hosts = validateLocalGroupHosts(probe, repairType, paramVal); + if (!hosts.isEmpty()) + { + probe.setRepairPriorityForHosts(repairType, hosts); + } + break; + case "forcerepair_hosts": + hosts = validateLocalGroupHosts(probe, repairType, paramVal); + if (!hosts.isEmpty()) + { + probe.setForceRepairForHosts(repairType, hosts); + } + break; + case "ignore_dcs": + Set ignoreDCs = new HashSet<>(); + for (String dc : Splitter.on(',').split(paramVal)) + { + ignoreDCs.add(dc); + } + probe.setAutoRepairIgnoreDCs(repairType, ignoreDCs); + break; + case "repair_primary_token_range_only": + probe.setPrimaryTokenRangeOnly(repairType, Boolean.parseBoolean(paramVal)); + break; + case "parallel_repair_count_in_group": + probe.setParallelRepairCountInGroup(repairType, Integer.parseInt(paramVal)); + break; + case "parallel_repair_percentage_in_group": + probe.setParallelRepairPercentageInGroup(repairType, Integer.parseInt(paramVal)); + break; + case "mv_repair_enabled": + probe.setMVRepairEnabled(repairType, Boolean.parseBoolean(paramVal)); + break; + default: + throw new IllegalArgumentException("Unknown parameter: " + paramType); + } + } + + private Set validateLocalGroupHosts(NodeProbe probe, RepairType repairType, String paramVal) { + Set hosts = new HashSet<>(); + for (String host : Splitter.on(',').split(paramVal)) + { + try + { + hosts.add(InetAddressAndPort.getByName(host)); + } + catch (UnknownHostException e) + { + out.println("invalid ip address: " + host); + } + } + + return hosts; + } +} diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index d52cfeb30b29..576f7faaf6aa 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -66,6 +66,14 @@ import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.vdurmont.semver4j.Semver; + +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.FileInputStreamPlus; +import org.apache.cassandra.io.util.FileOutputStreamPlus; +import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter; +import org.apache.cassandra.utils.concurrent.*; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -718,6 +726,12 @@ public static AbstractCryptoProvider newCryptoProvider(String className, Map !Modifier.isStatic(f.getModifiers())) - .filter(isDurationSpec.or(isDataRateSpec).or(isDataStorageSpec)).count(), 0); + .filter(f -> !Modifier.isStatic(f.getModifiers())) + .filter(isDurationSpec.or(isDataRateSpec).or(isDataStorageSpec)).count(), 0); } @Test @@ -113,12 +116,12 @@ public void updateInPlace() { Config config = new Config(); Map map = ImmutableMap.builder().put("storage_port", 123) - .put("commitlog_sync", Config.CommitLogSync.batch) - .put("seed_provider.class_name", "org.apache.cassandra.locator.SimpleSeedProvider") - .put("client_encryption_options.cipher_suites", Collections.singletonList("FakeCipher")) - .put("client_encryption_options.optional", false) - .put("client_encryption_options.enabled", true) - .build(); + .put("commitlog_sync", Config.CommitLogSync.batch) + .put("seed_provider.class_name", "org.apache.cassandra.locator.SimpleSeedProvider") + .put("client_encryption_options.cipher_suites", Collections.singletonList("FakeCipher")) + .put("client_encryption_options.optional", false) + .put("client_encryption_options.enabled", true) + .build(); Config updated = YamlConfigurationLoader.updateFromMap(map, true, config); assert updated == config : "Config pointers do not match"; assertThat(config.storage_port).isEqualTo(123); @@ -271,6 +274,12 @@ public void fromMapTest() Map encryptionOptions = ImmutableMap.of("cipher_suites", Collections.singletonList("FakeCipher"), "optional", false, "enabled", true); + Map autoRepairConfig = ImmutableMap.of("enabled", true, + "global_settings", ImmutableMap.of("repair_dc_groups", + ImmutableSet.of("all the groups")), + "repair_type_overrides", ImmutableMap.of( + "full", ImmutableMap.of("repair_dc_groups", + ImmutableSet.of("none of the groups")))); Map map = new ImmutableMap.Builder() .put("storage_port", storagePort) .put("commitlog_sync", commitLogSync) @@ -279,6 +288,7 @@ public void fromMapTest() .put("internode_socket_send_buffer_size", "5B") .put("internode_socket_receive_buffer_size", "5B") .put("commitlog_sync_group_window_in_ms", "42") + .put("auto_repair", autoRepairConfig) .build(); Config config = YamlConfigurationLoader.fromMap(map, Config.class); @@ -289,6 +299,9 @@ public void fromMapTest() assertEquals(true, config.client_encryption_options.enabled); // Check a nested object assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_send_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_receive_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) + assertEquals(true, config.auto_repair.enabled); + assertEquals(6 * 60 * 60L, config.auto_repair.getAutoRepairTableMaxRepairTime(AutoRepairConfig.RepairType.incremental)); + config.auto_repair.setMVRepairEnabled(AutoRepairConfig.RepairType.incremental, false); } @Test @@ -485,4 +498,4 @@ public static Config load(String path) } return new YamlConfigurationLoader().loadConfig(url); } -} +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java new file mode 100644 index 000000000000..dea0c9a827c3 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -0,0 +1,453 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.EnumMap; +import java.util.Objects; +import java.util.Set; + +import com.google.common.collect.ImmutableSet; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.Options; +import org.apache.cassandra.utils.FBUtilities; + +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; + +@RunWith(Parameterized.class) +public class AutoRepairConfigTest extends CQLTester +{ + private AutoRepairConfig config; + + private Set testSet = ImmutableSet.of("dc1"); + + @Parameterized.Parameter + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameters + public static Object[] repairTypes() + { + return AutoRepairConfig.RepairType.values(); + } + + @Before + public void setUp() + { + config = new AutoRepairConfig(true); + config.repair_type_overrides = null; + } + + @Test + public void autoRepairConfigDefaultsAreNotNull() + { + AutoRepairConfig config = new AutoRepairConfig(); + assertNotNull(config.global_settings); + } + + @Test + public void autoRepairConfigRepairTypesAreNotNull() + { + AutoRepairConfig config = new AutoRepairConfig(); + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + assertNotNull(config.repair_type_overrides.get(repairType)); + } + } + + @Test + public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsEnabled() + { + config.global_settings.enabled = true; + + assertTrue(config.isAutoRepairEnabled(repairType)); + } + + @Test + public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledGlobally() + { + config = new AutoRepairConfig(false); + config.global_settings.enabled = true; + assertFalse(config.isAutoRepairEnabled(repairType)); + } + + + @Test + public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledForRepairType() + { + config.global_settings.enabled = true; + config.repair_type_overrides = new EnumMap<>(AutoRepairConfig.RepairType.class); + config.repair_type_overrides.put(repairType, new Options()); + config.repair_type_overrides.get(repairType).enabled = false; + assertFalse(config.isAutoRepairEnabled(repairType)); + } + + @Test + public void testSetAutoRepairEnabledNoMVOrCDC() + { + DatabaseDescriptor.setCDCEnabled(false); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + config.setAutoRepairEnabled(repairType, true); + + assertTrue(config.repair_type_overrides.get(repairType).enabled); + } + + @Test + public void testSetAutoRepairEnabledWithMV() + { + DatabaseDescriptor.setCDCEnabled(false); + DatabaseDescriptor.setMaterializedViewsEnabled(true); + + try + { + config.setAutoRepairEnabled(repairType, true); + + if (repairType == AutoRepairConfig.RepairType.incremental) + { + assertFalse(config.repair_type_overrides.get(repairType).enabled); // IR should not be allowed with MV + assertNotEquals(AutoRepairConfig.RepairType.incremental, repairType); // should receive exception + } + else + { + assertTrue(config.repair_type_overrides.get(repairType).enabled); + } + } + catch (ConfigurationException e) + { + // should throw only if repairType is incremental + assertEquals(AutoRepairConfig.RepairType.incremental, repairType); + } + } + + @Test + public void testSetAutoRepairEnabledWithCDC() + { + DatabaseDescriptor.setCDCEnabled(true); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + + try + { + config.setAutoRepairEnabled(repairType, true); + + + if (repairType == AutoRepairConfig.RepairType.incremental) + { + assertFalse(config.repair_type_overrides.get(repairType).enabled); // IR should not be allowed with CDC + assertNotEquals(AutoRepairConfig.RepairType.incremental, repairType); // should receive exception + } + else + { + assertTrue(config.repair_type_overrides.get(repairType).enabled); + } + } + catch (ConfigurationException e) + { + // should throw only if repairType is incremental + assertEquals(AutoRepairConfig.RepairType.incremental, repairType); + } + } + + + @Test + public void testSetRepairByKeyspace() + { + config.setRepairByKeyspace(repairType, true); + + assertTrue(config.repair_type_overrides.get(repairType).repair_by_keyspace); + } + + @Test + public void testGetRepairByKeyspace() + { + config.global_settings.repair_by_keyspace = true; + + boolean result = config.getRepairByKeyspace(repairType); + + assertTrue(result); + } + + @Test + public void testSetRepairThreads() + { + config.setRepairThreads(repairType, 5); + + assert config.repair_type_overrides.get(repairType).number_of_repair_threads == 5; + } + + @Test + public void testGetRepairThreads() + { + config.global_settings.number_of_repair_threads = 5; + + int result = config.getRepairThreads(repairType); + + assertEquals(5, result); + } + + @Test + public void testGetRepairSubRangeNum() + { + config.global_settings.number_of_subranges = 5; + + int result = config.getRepairSubRangeNum(repairType); + + assertEquals(5, result); + } + + @Test + public void testSetRepairSubRangeNum() + { + config.setRepairSubRangeNum(repairType, 5); + + assert config.repair_type_overrides.get(repairType).number_of_subranges == 5; + } + + @Test + public void testGetRepairMinFrequencyInHours() + { + config.global_settings.min_repair_interval = new DurationSpec.IntSecondsBound("5s"); + + DurationSpec.IntSecondsBound result = config.getRepairMinInterval(repairType); + + assertEquals(5, result.toSeconds()); + } + + @Test + public void testSetRepairMinFrequencyInHours() + { + config.setRepairMinInterval(repairType, "5s"); + + assert config.repair_type_overrides.get(repairType).min_repair_interval.toSeconds() == 5; + } + + @Test + public void testGetAutoRepairHistoryClearDeleteHostsBufferInSec() + { + config.history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound("5s"); + + int result = config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds(); + + assertEquals(5, result); + } + + @Test + public void testSetAutoRepairHistoryClearDeleteHostsBufferInSec() + { + config.setAutoRepairHistoryClearDeleteHostsBufferInterval("5s"); + + assert Objects.equals(config.history_clear_delete_hosts_buffer_interval, new DurationSpec.IntSecondsBound("5s")); + } + + @Test + public void testGetRepairSSTableCountHigherThreshold() + { + config.global_settings.sstable_upper_threshold = 5; + + int result = config.getRepairSSTableCountHigherThreshold(repairType); + + assertEquals(5, result); + } + + @Test + public void testSetRepairSSTableCountHigherThreshold() + { + config.setRepairSSTableCountHigherThreshold(repairType, 5); + + assert config.repair_type_overrides.get(repairType).sstable_upper_threshold == 5; + } + + @Test + public void testGetAutoRepairTableMaxRepairTimeInSec() + { + config.global_settings.table_max_repair_time = new DurationSpec.IntSecondsBound("5s"); + + DurationSpec.IntSecondsBound result = config.getAutoRepairTableMaxRepairTime(repairType); + + assertEquals(5, result.toSeconds()); + } + + @Test + public void testSetAutoRepairTableMaxRepairTimeInSec() + { + config.setAutoRepairTableMaxRepairTime(repairType, "5s"); + + assert config.repair_type_overrides.get(repairType).table_max_repair_time.toSeconds() == 5; + } + + @Test + public void testGetIgnoreDCs() + { + config.global_settings.ignore_dcs = testSet; + + Set result = config.getIgnoreDCs(repairType); + + assertEquals(testSet, result); + } + + @Test + public void testSetIgnoreDCs() + { + config.setIgnoreDCs(repairType, testSet); + + assertEquals(config.repair_type_overrides.get(repairType).ignore_dcs, testSet); + } + + @Test + public void testGetRepairPrimaryTokenRangeOnly() + { + config.global_settings.repair_primary_token_range_only = true; + + boolean result = config.getRepairPrimaryTokenRangeOnly(repairType); + + assertTrue(result); + } + + @Test + public void testSetRepairPrimaryTokenRangeOnly() + { + config.setRepairPrimaryTokenRangeOnly(repairType, true); + + assertTrue(config.repair_type_overrides.get(repairType).repair_primary_token_range_only); + } + + @Test + public void testGetParallelRepairPercentageInGroup() + { + config.global_settings.parallel_repair_percentage_in_group = 5; + + int result = config.getParallelRepairPercentageInGroup(repairType); + + assertEquals(5, result); + } + + @Test + public void testSetParallelRepairPercentageInGroup() + { + config.setParallelRepairPercentageInGroup(repairType, 5); + + assert config.repair_type_overrides.get(repairType).parallel_repair_percentage_in_group == 5; + } + + @Test + public void testGetParallelRepairCountInGroup() + { + config.global_settings.parallel_repair_count_in_group = 5; + + int result = config.getParallelRepairCountInGroup(repairType); + + assertEquals(5, result); + } + + @Test + public void testSetParallelRepairCountInGroup() + { + config.setParallelRepairCountInGroup(repairType, 5); + + assert config.repair_type_overrides.get(repairType).parallel_repair_count_in_group == 5; + } + + @Test + public void testGetMVRepairEnabled() + { + config.global_settings.mv_repair_enabled = true; + + boolean result = config.getMVRepairEnabled(repairType); + + assertTrue(result); + } + + @Test + public void testSetMVRepairEnabled() + { + config.setMVRepairEnabled(repairType, true); + + assertTrue(config.repair_type_overrides.get(repairType).mv_repair_enabled); + } + + @Test + public void testSetForceRepairNewNode() + { + config.setForceRepairNewNode(repairType, true); + + assertTrue(config.repair_type_overrides.get(repairType).force_repair_new_node); + } + + @Test + public void testGetForceRepairNewNode() + { + config.global_settings.force_repair_new_node = true; + + boolean result = config.getForceRepairNewNode(repairType); + + assertTrue(result); + } + + @Test + public void testIsAutoRepairSchedulingEnabledDefault() + { + config = new AutoRepairConfig(); + + boolean result = config.isAutoRepairSchedulingEnabled(); + + assertFalse(result); + } + + @Test + public void testIsAutoRepairSchedulingEnabledTrue() + { + boolean result = config.isAutoRepairSchedulingEnabled(); + + assertTrue(result); + } + + @Test + public void testGetDefaultOptionsMVRepairIsEnabledByDefault() + { + Options defaultOptions = Options.getDefaultOptions(); + + assertTrue(defaultOptions.mv_repair_enabled); + } + + @Test + public void testGetDefaultOptionsTokenRangeSplitter() + { + Options defaultOptions = Options.getDefaultOptions(); + + assertEquals(DefaultAutoRepairTokenSplitter.class.getName(),defaultOptions.token_range_splitter); + assertEquals(DefaultAutoRepairTokenSplitter.class.getName(), FBUtilities.newAutoRepairTokenRangeSplitter(defaultOptions.token_range_splitter).getClass().getName()); + } + + @Test(expected = ConfigurationException.class) + public void testInvalidTokenRangeSplitter() + { + assertEquals(DefaultAutoRepairTokenSplitter.class.getName(), FBUtilities.newAutoRepairTokenRangeSplitter("invalid-class").getClass().getName()); + } + +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java new file mode 100644 index 000000000000..579b996a12f5 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.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.repair.autorepair; + +import java.util.Optional; +import java.util.Set; + +import com.google.common.collect.ImmutableSet; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.TableMetadata; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; + +public class AutoRepairKeyspaceTest +{ + private static final Set tables = ImmutableSet.of( + AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + AutoRepairKeyspace.AUTO_REPAIR_PRIORITY + ); + + @BeforeClass + public static void setupDatabaseDescriptor() + { + DatabaseDescriptor.daemonInitialization(); + } + + + @Test + public void testMetadataCanParseSchemas() throws Exception + { + setAutoRepairEnabled(true); + KeyspaceMetadata keyspaceMetadata = AutoRepairKeyspace.metadata(); + + assert keyspaceMetadata.tables.size() == tables.size() : "Expected " + tables.size() + " tables, got " + keyspaceMetadata.tables.size(); + + for (String table : tables) + { + Optional tableMetadata = keyspaceMetadata.tables.get(table); + + assert tableMetadata.isPresent() : "Table " + table + " not found in metadata"; + } + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java new file mode 100644 index 000000000000..17b175be1eec --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -0,0 +1,581 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; + +import org.apache.cassandra.cql3.statements.schema.TableAttributes; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.repair.RepairCoordinator; +import org.apache.cassandra.schema.AutoRepairParams; +import org.apache.cassandra.schema.TableParams; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.Pair; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import junit.framework.Assert; +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.concurrent.ScheduledExecutorPlus; +import org.apache.cassandra.config.DatabaseDescriptor; +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.marshal.IntegerType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.metrics.AutoRepairMetricsManager; +import org.apache.cassandra.metrics.AutoRepairMetrics; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.progress.ProgressEvent; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.Util.token; +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(Parameterized.class) +public class AutoRepairParameterizedTest extends CQLTester +{ + private static final String KEYSPACE = "ks"; + private static final String TABLE = "tbl"; + private static final String TABLE_DISABLED_AUTO_REPAIR = "tbl_disabled_auto_repair"; + private static final String MV = "mv"; + private static TableMetadata cfm; + private static TableMetadata cfmDisabledAutoRepair; + private static Keyspace keyspace; + private static int timeFuncCalls; + @Mock + ScheduledExecutorPlus mockExecutor; + @Mock + ProgressEvent progressEvent; + @Mock + AutoRepairState autoRepairState; + @Mock + RepairCoordinator repairRunnable; + private static AutoRepairConfig defaultConfig; + + + @Parameterized.Parameter() + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameters(name = "repairType={0}") + public static Collection repairTypes() + { + return Arrays.asList(AutoRepairConfig.RepairType.values()); + } + + @BeforeClass + public static void setupClass() throws Exception + { + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + setAutoRepairEnabled(true); + requireNetwork(); + AutoRepairUtils.setup(); + + + cfm = TableMetadata.builder(KEYSPACE, TABLE) + .addPartitionKeyColumn("k", UTF8Type.instance) + .addStaticColumn("s", UTF8Type.instance) + .addClusteringColumn("i", IntegerType.instance) + .addRegularColumn("v", UTF8Type.instance) + .params(TableParams.builder().automatedRepairFull(AutoRepairParams.create(AutoRepairConfig.RepairType.full, ImmutableMap.of(AutoRepairParams.Option.ENABLED.toString(), Boolean.toString(true)))). + automatedRepairIncremental(AutoRepairParams.create(AutoRepairConfig.RepairType.incremental, ImmutableMap.of(AutoRepairParams.Option.ENABLED.toString(), Boolean.toString(true)))).build()) + .build(); + + cfmDisabledAutoRepair = TableMetadata.builder(KEYSPACE, TABLE_DISABLED_AUTO_REPAIR) + .addPartitionKeyColumn("k", UTF8Type.instance) + .addStaticColumn("s", UTF8Type.instance) + .addClusteringColumn("i", IntegerType.instance) + .addRegularColumn("v", UTF8Type.instance) + .params(TableParams.builder().automatedRepairFull(AutoRepairParams.create(AutoRepairConfig.RepairType.full, ImmutableMap.of(AutoRepairParams.Option.ENABLED.toString(), Boolean.toString(false)))). + automatedRepairIncremental(AutoRepairParams.create(AutoRepairConfig.RepairType.incremental, ImmutableMap.of(AutoRepairParams.Option.ENABLED.toString(), Boolean.toString(false)))).build()) + .build(); + + SchemaLoader.prepareServer(); + SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), cfm, cfmDisabledAutoRepair); + cfm = Schema.instance.getTableMetadata(KEYSPACE, TABLE); + cfmDisabledAutoRepair = Schema.instance.getTableMetadata(KEYSPACE, TABLE_DISABLED_AUTO_REPAIR); + keyspace = Keyspace.open(KEYSPACE); + DatabaseDescriptor.setMaterializedViewsEnabled(true); + QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + + "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); + + defaultConfig = new AutoRepairConfig(true); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + DatabaseDescriptor.setCDCEnabled(false); + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + defaultConfig.setAutoRepairEnabled(repairType, true); + defaultConfig.setMVRepairEnabled(repairType, false); + } + } + + @Before + public void setup() + { + MockitoAnnotations.initMocks(this); + + Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE).truncateBlocking(); + Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE).disableAutoCompaction(); + + Keyspace.open(KEYSPACE).getColumnFamilyStore(MV).truncateBlocking(); + Keyspace.open(KEYSPACE).getColumnFamilyStore(MV).disableAutoCompaction(); + + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(AutoRepairKeyspace.AUTO_REPAIR_PRIORITY).truncateBlocking(); + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(AutoRepairKeyspace.AUTO_REPAIR_HISTORY).truncateBlocking(); + + + AutoRepair.instance = new AutoRepair(); + executeCQL(); + + timeFuncCalls = 0; + AutoRepair.timeFunc = System::currentTimeMillis; + resetCounters(); + resetConfig(); + } + + + private void resetCounters() + { + AutoRepairMetrics metrics = AutoRepairMetricsManager.getMetrics(repairType); + Metrics.removeMatching((name, metric) -> name.startsWith("repairTurn")); + metrics.repairTurnMyTurn = Metrics.counter(String.format("repairTurnMyTurn-%s", repairType)); + metrics.repairTurnMyTurnForceRepair = Metrics.counter(String.format("repairTurnMyTurnForceRepair-%s", repairType)); + metrics.repairTurnMyTurnDueToPriority = Metrics.counter(String.format("repairTurnMyTurnDueToPriority-%s", repairType)); + } + + private void resetConfig() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.repair_type_overrides = defaultConfig.repair_type_overrides; + config.global_settings = defaultConfig.global_settings; + config.history_clear_delete_hosts_buffer_interval = defaultConfig.history_clear_delete_hosts_buffer_interval; + config.setRepairSubRangeNum(repairType, 1); + } + + private void executeCQL() + { + QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, s) VALUES ('k', 's')"); + QueryProcessor.executeInternal("SELECT s FROM ks.tbl WHERE k='k'"); + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME) + .getColumnFamilyStore(AutoRepairKeyspace.AUTO_REPAIR_PRIORITY) + .forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + } + + + @Test + public void testRepairAsync() + { + AutoRepair.instance.repairExecutors.put(repairType, mockExecutor); + + AutoRepair.instance.repairAsync(repairType, 60); + + verify(mockExecutor, Mockito.times(1)).submit(any(Runnable.class)); + } + + @Test + public void testRepairTurn() + { + UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); + Assert.assertTrue("Expected my turn for the repair", AutoRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); + } + + @Test + public void testRepair() + { + AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repair(repairType, 0); + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + long lastRepairTime = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + //if repair was done then lastRepairTime should be non-zero + Assert.assertTrue(String.format("Expected lastRepairTime > 0, actual value lastRepairTime %d", + lastRepairTime), lastRepairTime > 0); + } + + @Test + public void testTooFrequentRepairs() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + //in the first round let repair run + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repair(repairType, 0); + long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + int consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); + Assert.assertNotSame(String.format("Expected total repaired tables > 0, actual value %s ", consideredTables), + consideredTables, 0); + + //if repair was done in last 24 hours then it should not trigger another repair + config.setRepairMinInterval(repairType, "24h"); + AutoRepair.instance.repair(repairType, 0); + long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + Assert.assertEquals(String.format("Expected repair time to be same, actual value lastRepairTime1 %d, " + + "lastRepairTime2 %d", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); + consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); + Assert.assertEquals("Expected total repaired tables = 0, actual value: " + consideredTables, + consideredTables, 0); + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + } + + @Test + public void testNonFrequentRepairs() + { + Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + long prevCount = state.getTotalMVTablesConsideredForRepair(); + AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repair(repairType, 0); + long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + Assert.assertTrue(String.format("Expected lastRepairTime1 > 0, actual value lastRepairTime1 %d", + lastRepairTime1), lastRepairTime1 > 0); + UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); + Assert.assertTrue("Expected my turn for the repair", + AutoRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); + AutoRepair.instance.repair(repairType, 0); + long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + Assert.assertNotSame(String.format("Expected repair time to be same, actual value lastRepairTime1 %d, " + + "lastRepairTime2 ", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); + assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); + assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + } + + @Test + public void testGetPriorityHosts() + { + Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + long prevCount = state.getTotalMVTablesConsideredForRepair(); + AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); + Assert.assertSame(String.format("Priority host count is not same, actual value %d, expected value %d", + AutoRepairUtils.getPriorityHosts(repairType).size(), 0), AutoRepairUtils.getPriorityHosts(repairType).size(), 0); + UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); + Assert.assertTrue("Expected my turn for the repair", AutoRepairUtils.myTurnToRunRepair(repairType, myId) != + NOT_MY_TURN); + AutoRepair.instance.repair(repairType, 0); + AutoRepairUtils.addPriorityHosts(repairType, Sets.newHashSet(FBUtilities.getBroadcastAddressAndPort())); + AutoRepair.instance.repair(repairType, 0); + Assert.assertSame(String.format("Priority host count is not same actual value %d, expected value %d", + AutoRepairUtils.getPriorityHosts(repairType).size(), 0), AutoRepairUtils.getPriorityHosts(repairType).size(), 0); + assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); + assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + } + + @Test + public void testCheckAutoRepairStartStop() throws Throwable + { + Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + long prevCount = state.getTotalMVTablesConsideredForRepair(); + config.setRepairMinInterval(repairType, "0s"); + config.setAutoRepairEnabled(repairType, false); + long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + AutoRepair.instance.repair(repairType, 0); + long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + //Since repair has not happened, both the last repair times should be same + Assert.assertEquals(String.format("Expected lastRepairTime1 %d, and lastRepairTime2 %d to be same", + lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); + + config.setAutoRepairEnabled(repairType, true); + AutoRepair.instance.repair(repairType, 0); + //since repair is done now, so lastRepairTime1/lastRepairTime2 and lastRepairTime3 should not be same + long lastRepairTime3 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + Assert.assertNotSame(String.format("Expected lastRepairTime1 %d, and lastRepairTime3 %d to be not same", + lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime3); + assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); + assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + } + + @Test + public void testRepairPrimaryRangesByDefault() + { + Assert.assertTrue("Expected primary range repair only", + AutoRepairService.instance.getAutoRepairConfig().getRepairPrimaryTokenRangeOnly(repairType)); + } + + @Test + public void testGetAllMVs() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setMVRepairEnabled(repairType, false); + assertFalse(config.getMVRepairEnabled(repairType)); + assertEquals(0, AutoRepairUtils.getAllMVs(repairType, keyspace, cfm).size()); + + config.setMVRepairEnabled(repairType, true); + + assertTrue(config.getMVRepairEnabled(repairType)); + assertEquals(Arrays.asList(MV), AutoRepairUtils.getAllMVs(repairType, keyspace, cfm)); + config.setMVRepairEnabled(repairType, false); + } + + + @Test + public void testMVRepair() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setMVRepairEnabled(repairType, true); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(0); + AutoRepair.instance.repair(repairType, 0); + assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + + config.setMVRepairEnabled(repairType, false); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(0); + AutoRepair.instance.repair(repairType, 0); + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + + config.setMVRepairEnabled(repairType, true); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(0); + AutoRepair.instance.repair(repairType, 0); + assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + + config.setMVRepairEnabled(repairType, false); + } + + @Test + public void testSkipRepairSSTableCountHigherThreshold() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + ColumnFamilyStore cfsBaseTable = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE); + ColumnFamilyStore cfsMVTable = Keyspace.open(KEYSPACE).getColumnFamilyStore(MV); + Set preBaseTable = cfsBaseTable.getLiveSSTables(); + Set preMVTable = cfsBaseTable.getLiveSSTables(); + + for (int i = 0; i < 10; i++) + { + QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (k, i, v) VALUES('k1', %d, 'v1')", KEYSPACE, TABLE, i)); + cfsBaseTable.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + cfsMVTable.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + } + + Set postBaseTable = cfsBaseTable.getLiveSSTables(); + Set diffBaseTable = new HashSet<>(postBaseTable); + diffBaseTable.removeAll(preBaseTable); + assert diffBaseTable.size() == 10; + + Set postMVTable = cfsBaseTable.getLiveSSTables(); + Set diffMVTable = new HashSet<>(postMVTable); + diffMVTable.removeAll(preMVTable); + assert diffMVTable.size() == 10; + + int beforeCount = config.getRepairSSTableCountHigherThreshold(repairType); + config.setMVRepairEnabled(repairType, true); + config.setRepairSSTableCountHigherThreshold(repairType, 9); + assertEquals(0, state.getRepairSkippedTablesCount()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + state.setLastRepairTime(0); + AutoRepair.instance.repair(repairType, 0); + assertEquals(1, state.getTotalMVTablesConsideredForRepair()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + // skipping one time for the base table and another time for MV table + assertEquals(2, state.getRepairSkippedTablesCount()); + assertEquals(2, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + + // set it to higher value, and this time, the tables should not be skipped + config.setRepairSSTableCountHigherThreshold(repairType, 11); + config.setRepairSSTableCountHigherThreshold(repairType, beforeCount); + state.setLastRepairTime(0); + AutoRepair.instance.repair(repairType, 0); + assertEquals(1, state.getTotalMVTablesConsideredForRepair()); + assertEquals(0, state.getRepairSkippedTablesCount()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + + config.setMVRepairEnabled(repairType, false); + config.setRepairSSTableCountHigherThreshold(repairType, beforeCount); + } + + @Test + public void testGetRepairState() + { + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getRepairKeyspaceCount()); + + AutoRepairState state = AutoRepair.instance.getRepairState(repairType); + state.setRepairKeyspaceCount(100); + + assertEquals(100L, AutoRepair.instance.getRepairState(repairType).getRepairKeyspaceCount()); + } + + @Test + public void testMetrics() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setMVRepairEnabled(repairType, true); + config.setRepairMinInterval(repairType, "0s"); + config.setAutoRepairTableMaxRepairTime(repairType, "0s"); + AutoRepair.timeFunc = () -> { + timeFuncCalls++; + return timeFuncCalls * 1000L; + }; + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(1000L); + + AutoRepair.instance.repair(repairType, 0); + + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).nodeRepairTimeInSec.getValue() > 0); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).clusterRepairTimeInSec.getValue() > 0); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).repairTurnMyTurn.getCount()); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue() > 0); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue().intValue()); + + config.setAutoRepairTableMaxRepairTime(repairType, String.valueOf(Integer.MAX_VALUE-1) + 's'); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) + .thenReturn(repairRunnable); + when(autoRepairState.getRepairFailedTablesCount()).thenReturn(10); + when(autoRepairState.getLongestUnrepairedSec()).thenReturn(10); + + AutoRepair.instance.repair(repairType, 0); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).failedTablesCount.getValue() > 0); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue() > 0); + } + + @Test + public void testRepairWaitsForRepairToFinishBeforeSchedullingNewSession() throws Exception + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setMVRepairEnabled(repairType, false); + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) + .thenReturn(repairRunnable); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + when(autoRepairState.getLastRepairTime()).thenReturn((long) 0); + AtomicInteger resetWaitConditionCalls = new AtomicInteger(); + AtomicInteger waitForRepairCompletedCalls = new AtomicInteger(); + doAnswer(invocation -> { + resetWaitConditionCalls.getAndIncrement(); + assertEquals("waitForRepairToComplete was called before resetWaitCondition", + resetWaitConditionCalls.get(), waitForRepairCompletedCalls.get() + 1); + return null; + }).when(autoRepairState).resetWaitCondition(); + doAnswer(invocation -> { + waitForRepairCompletedCalls.getAndIncrement(); + assertEquals("resetWaitCondition was not called before waitForRepairToComplete", + resetWaitConditionCalls.get(), waitForRepairCompletedCalls.get()); + return null; + }).when(autoRepairState).waitForRepairToComplete(); + + AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType, 0); + } + + @Test + public void testDisabledAutoRepairForATableThroughTableLevelConfiguration() + { + Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); + Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); + + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setRepairMinInterval(repairType, "0s"); + int disabledTablesRepairCountBefore = AutoRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); + AutoRepair.instance.repair(repairType, 0); + int consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); + Assert.assertNotSame(String.format("Expected total repaired tables > 0, actual value %s ", consideredTables), + consideredTables, 0); + int disabledTablesRepairCountAfter = AutoRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); + Assert.assertTrue(String.format("A table %s should be skipped from auto repair, expected value: %d, actual value %d ", TABLE_DISABLED_AUTO_REPAIR, disabledTablesRepairCountBefore + 1, disabledTablesRepairCountAfter), + disabledTablesRepairCountBefore < disabledTablesRepairCountAfter); + } + + @Test + public void testTokenRangesNoSplit() + { + Collection> tokens = StorageService.instance.getPrimaryRanges(KEYSPACE); + assertEquals(1, tokens.size()); + List> expectedToken = new ArrayList<>(); + expectedToken.addAll(tokens); + + List> ranges = new DefaultAutoRepairTokenSplitter().getRange(repairType, true, KEYSPACE, TABLE); + assertEquals(1, ranges.size()); + assertEquals(expectedToken.get(0).left, ranges.get(0).left); + assertEquals(expectedToken.get(0).right, ranges.get(0).right); + } + + @Test + public void testTokenRangesSplit() + { + Collection> tokens = StorageService.instance.getPrimaryRanges(KEYSPACE); + assertEquals(1, tokens.size()); + List> expectedToken = new ArrayList<>(); + expectedToken.addAll(tokens); + + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setRepairSubRangeNum(repairType, 4); + List> ranges = new DefaultAutoRepairTokenSplitter().getRange(repairType, true, KEYSPACE, TABLE); + assertEquals(4, ranges.size()); + } + + @Test + public void testTableAttribute() + { + assertTrue(TableAttributes.validKeywords().contains("automated_repair_full")); + assertTrue(TableAttributes.validKeywords().contains("automated_repair_incremental")); + } + + @Test + public void testDefaultAutomatedRepair() + { + Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); + Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java new file mode 100644 index 000000000000..a0e5bdc45294 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.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.repair.autorepair; + +import org.junit.Test; + +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +public class AutoRepairStateFactoryTest +{ + @Test + public void testGetRepairState() { + AutoRepairState state = RepairType.getAutoRepairState(RepairType.full); + + assert state instanceof FullRepairState; + + state = RepairType.getAutoRepairState(RepairType.incremental); + + assert state instanceof IncrementalRepairState; + } + + @Test + public void testGetRepairStateSupportsAllRepairTypes() { + for (RepairType repairType : RepairType.values()) { + try { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + assertNotNull(state); + } catch (IllegalArgumentException e) { + assertNull(e); + } + } + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java new file mode 100644 index 000000000000..972b6339954f --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java @@ -0,0 +1,382 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +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.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.utils.concurrent.Condition; +import org.apache.cassandra.utils.progress.ProgressEvent; +import org.apache.cassandra.utils.progress.ProgressEventType; +import org.mockito.Mock; + +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.assertTrue; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +@RunWith(Parameterized.class) +public class AutoRepairStateTest extends CQLTester +{ + private static final String testTable = "test"; + + @Parameterized.Parameter + public RepairType repairType; + + @Mock + ProgressEvent progressEvent; + + @Parameterized.Parameters + public static Collection repairTypes() + { + return Arrays.asList(RepairType.values()); + } + + @Before + public void setUp() { + initMocks(this); + createTable(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int)", KEYSPACE, testTable)); + } + + @Test + public void testGetRepairRunnable() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + AutoRepairService.setup(); + + Runnable runnable = state.getRepairRunnable(KEYSPACE, ImmutableList.of(testTable), ImmutableSet.of(), false); + + assertNotNull(runnable); + } + + @Test + public void testProgressError() throws InterruptedException + { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + when(progressEvent.getType()).thenReturn(ProgressEventType.ERROR); + + state.progress("test", progressEvent); + + assertFalse(state.success); + assertTrue(state.condition.await(0, TimeUnit.MILLISECONDS)); + } + + @Test + public void testProgress_progress() throws InterruptedException + { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + when(progressEvent.getType()).thenReturn(ProgressEventType.PROGRESS); + + state.progress("test", progressEvent); + + assertTrue(state.success); + assertFalse(state.condition.await(0, TimeUnit.MILLISECONDS)); + } + + + @Test + public void testProgress_complete() throws InterruptedException + { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + when(progressEvent.getType()).thenReturn(ProgressEventType.COMPLETE); + + state.progress("test", progressEvent); + + assertTrue(state.success); + assertTrue(state.condition.await(1, TimeUnit.MILLISECONDS)); + } + + @Test + public void testWaitForRepairToComplete() throws Exception + { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.condition.signalAll(); + Condition finishedCondition = Condition.newOneTimeCondition(); + Callable waitForRepairToComplete = () -> { + state.waitForRepairToComplete(); + finishedCondition.signalAll(); + return null; + }; + + waitForRepairToComplete.call(); + + assertTrue(finishedCondition.await(1, TimeUnit.MILLISECONDS)); + } + + @Test + public void testGetLastRepairTime() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.lastRepairTimeInMs = 1; + + assertEquals(1, state.getLastRepairTime()); + } + + @Test + public void testSetTotalTablesConsideredForRepair() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + + state.setTotalTablesConsideredForRepair(1); + + assertEquals(1, state.totalTablesConsideredForRepair); + } + + @Test + public void testGetTotalTablesConsideredForRepair() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.totalTablesConsideredForRepair = 1; + + assertEquals(1, state.getTotalTablesConsideredForRepair()); + } + + @Test + public void testSetLastRepairTimeInMs() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + + state.setLastRepairTime(1); + + assertEquals(1, state.lastRepairTimeInMs); + } + + @Test + public void testGetClusterRepairTimeInSec() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.clusterRepairTimeInSec = 1; + + assertEquals(1, state.getClusterRepairTimeInSec()); + } + + @Test + public void testGetNodeRepairTimeInSec() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.nodeRepairTimeInSec = 1; + + assertEquals(1, state.getNodeRepairTimeInSec()); + } + + @Test + public void testSetRepairInProgress() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + + state.setRepairInProgress(true); + + assertTrue(state.repairInProgress); + } + + @Test + public void testIsRepairInProgress() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.repairInProgress = true; + + assertTrue(state.isRepairInProgress()); + } + + @Test + public void testSetRepairSkippedTablesCount() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + + state.setRepairSkippedTablesCount(1); + + assertEquals(1, state.repairTableSkipCount); + } + + @Test + public void testGetRepairSkippedTablesCount() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.repairTableSkipCount = 1; + + assertEquals(1, state.getRepairSkippedTablesCount()); + } + + @Test + public void testGetLongestUnrepairedSecNull() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.longestUnrepairedNode = null; + + try + { + assertEquals(0, state.getLongestUnrepairedSec()); + } catch (Exception e) { + assertNull(e); + } + } + + @Test + public void testGetLongestUnrepairedSec() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.longestUnrepairedNode = new AutoRepairHistory(UUID.randomUUID(), "", 0, 1000, + null, 0, false); + AutoRepairState.timeFunc = () -> 2000L; + + try + { + assertEquals(1, state.getLongestUnrepairedSec()); + } catch (Exception e) { + assertNull(e); + } + } + + @Test + public void testSetRepairFailedTablesCount() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + + state.setRepairFailedTablesCount(1); + + assertEquals(1, state.repairTableFailureCount); + } + + @Test + public void testGetRepairFailedTablesCount() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.repairTableFailureCount = 1; + + assertEquals(1, state.getRepairFailedTablesCount()); + } + + @Test + public void testSetTotalMVTablesConsideredForRepair() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + + state.setTotalMVTablesConsideredForRepair(1); + + assertEquals(1, state.totalMVTablesConsideredForRepair); + } + + @Test + public void testGetTotalMVTablesConsideredForRepair() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.totalMVTablesConsideredForRepair = 1; + + assertEquals(1, state.getTotalMVTablesConsideredForRepair()); + } + + @Test + public void testSetNodeRepairTimeInSec() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + + state.setNodeRepairTimeInSec(1); + + assertEquals(1, state.nodeRepairTimeInSec); + } + + @Test + public void testSetClusterRepairTimeInSec() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + + state.setClusterRepairTimeInSec(1); + + assertEquals(1, state.clusterRepairTimeInSec); + } + + @Test + public void testSetRepairKeyspaceCount() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + + state.setRepairKeyspaceCount(1); + + assertEquals(1, state.repairKeyspaceCount); + } + + @Test + public void testSetRepairTableSuccessCount() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + + state.setRepairTableSuccessCount(1); + + assertEquals(1, state.repairTableSuccessCount); + } + + @Test + public void testGetRepairKeyspaceCount() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.repairKeyspaceCount = 1; + + assertEquals(1, state.getRepairKeyspaceCount()); + } + + @Test + public void testGetRepairTableSuccessCount() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.repairTableSuccessCount = 1; + + assertEquals(1, state.getRepairTableSuccessCount()); + } + + @Test + public void testSetLongestUnrepairedNode() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + AutoRepairHistory history = new AutoRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); + + state.setLongestUnrepairedNode(history); + + assertEquals(history, state.longestUnrepairedNode); + } + + @Test + public void isSuccess() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.success = true; + + assertTrue(state.isSuccess()); + + state.success = false; + + assertFalse(state.isSuccess()); + } + + @Test + public void testWaitForRepairToCompleteDoesNotSetSuccessWhenProgressReceivesError() throws InterruptedException + { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + when(progressEvent.getType()).thenReturn(ProgressEventType.ERROR); + + state.progress("test", progressEvent); + assertFalse(state.success); + + state.waitForRepairToComplete(); + assertFalse(state.success); + } + + @Test + public void testResetWaitCondition() + { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.condition.signalAll(); + assertTrue(state.condition.isSignalled()); + + state.resetWaitCondition(); + + assertFalse(state.condition.isSignalled()); + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java new file mode 100644 index 000000000000..b16e0ea5b491 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.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.repair.autorepair; + +import java.util.HashMap; +import java.util.Map; + +import org.junit.BeforeClass; +import org.junit.Test; + +import junit.framework.Assert; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.ReplicationParams; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.schema.SchemaTestUtil; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.junit.Assert.assertEquals; + +public class AutoRepairTest extends CQLTester +{ + @BeforeClass + public static void setupClass() throws Exception + { + setAutoRepairEnabled(true); + requireNetwork(); + } + + @Test + public void testSetup() + { + AutoRepair instance = new AutoRepair(); + + instance.setup(); + + assertEquals(RepairType.values().length, instance.repairExecutors.size()); + for (RepairType repairType : instance.repairExecutors.keySet()) + { + assertEquals(String.format("Expected 1 task in queue for %s", repairType), + 1, instance.repairExecutors.get(repairType).getPendingTaskCount() + + instance.repairExecutors.get(repairType).getActiveTaskCount()); + } + } + + @Test(expected = ConfigurationException.class) + public void testSetupFailsWhenIREnabledWithCDC() + { + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + DatabaseDescriptor.setCDCEnabled(true); + + AutoRepair instance = new AutoRepair(); + instance.setup(); + } + + @Test(expected = ConfigurationException.class) + public void testSetupFailsWhenIREnabledWithMV() + { + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + DatabaseDescriptor.setMaterializedViewsEnabled(true); + + AutoRepair instance = new AutoRepair(); + instance.setup(); + } + + @Test + public void testCheckNTSreplicationNodeInsideOutsideDC() + { + String ksname1 = "ks_nts1"; + String ksname2 = "ks_nts2"; + Map configOptions1 = new HashMap<>(); + configOptions1.put("datacenter1", "3"); + configOptions1.put(ReplicationParams.CLASS, "NetworkTopologyStrategy"); + SchemaTestUtil.addOrUpdateKeyspace(KeyspaceMetadata.create(ksname1, KeyspaceParams.create(false, configOptions1)), false); + Map configOptions2 = new HashMap<>(); + configOptions2.put("datacenter2", "3"); + configOptions2.put(ReplicationParams.CLASS, "NetworkTopologyStrategy"); + SchemaTestUtil.addOrUpdateKeyspace(KeyspaceMetadata.create(ksname2, KeyspaceParams.create(false, configOptions2)), false); + + for (Keyspace ks : Keyspace.all()) + { + if (ks.getName().equals(ksname1)) + { + // case 1 : + // node reside in "datacenter1" + // keyspace has replica in "datacenter1" + Assert.assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + } + else if (ks.getName().equals(ksname2)) + { + // case 2 : + // node reside in "datacenter1" + // keyspace has replica in "datacenter2" + Assert.assertFalse(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + } + } + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java new file mode 100644 index 000000000000..0ea495fea3b1 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -0,0 +1,478 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.UUID; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.locator.IEndpointSnitch; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.CurrentRepairStatus; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.tcm.membership.NodeAddresses; +import org.apache.cassandra.utils.FBUtilities; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_DELETE_HOSTS; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_FORCE_REPAIR; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_FINISH_TS; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_PRIORITY; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_START_TS; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_TURN; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.when; + +public class AutoRepairUtilsTest extends CQLTester +{ + static RepairType repairType = RepairType.incremental; + static UUID hostId; + + static InetAddressAndPort localEndpoint; + + @Mock + static IEndpointSnitch snitchMock; + + static IEndpointSnitch defaultSnitch; + + + @BeforeClass + public static void setupClass() throws Exception + { + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + setAutoRepairEnabled(true); + requireNetwork(); + defaultSnitch = DatabaseDescriptor.getEndpointSnitch(); + localEndpoint = FBUtilities.getBroadcastAddressAndPort(); + hostId = StorageService.instance.getHostIdForEndpoint(localEndpoint); + AutoRepairUtils.setup(); + SchemaLoader.prepareServer(); + SchemaLoader.createKeyspace("ks", KeyspaceParams.create(false, + ImmutableMap.of("class", "NetworkTopologyStrategy", "datacenter1", "1")), + TableMetadata.builder("ks", "tbl") + .addPartitionKeyColumn("k", UTF8Type.instance) + .build()); + } + + @Before + public void setup() + { + MockitoAnnotations.initMocks(this); + DatabaseDescriptor.setEndpointSnitch(defaultSnitch); + QueryProcessor.executeInternal(String.format( + "TRUNCATE %s.%s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY)); + QueryProcessor.executeInternal(String.format( + "TRUNCATE %s.%s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY)); + } + + @Test + public void testSetForceRepair() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, force_repair) VALUES ('%s', %s, false)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + + AutoRepairUtils.setForceRepair(repairType, ImmutableSet.of(localEndpoint)); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT force_repair FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(1, result.size()); + assertTrue(result.one().getBoolean(COL_FORCE_REPAIR)); + } + + @Test + public void testSetForceRepairNewNode() + { + AutoRepairUtils.setForceRepairNewNode(repairType); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT force_repair FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(1, result.size()); + assertTrue(result.one().getBoolean(COL_FORCE_REPAIR)); + } + + + @Test + public void testClearDeleteHosts() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, delete_hosts, delete_hosts_update_time) VALUES ('%s', %s, { %s }, toTimestamp(now()))", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId, hostId)); + + AutoRepairUtils.clearDeleteHosts(repairType, hostId); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT delete_hosts FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(1, result.size()); + Set deleteHosts = result.one().getSet(COL_DELETE_HOSTS, UUIDType.instance); + assertNull(deleteHosts); + } + + @Test + public void testGetAutoRepairHistoryForLocalGroup() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, force_repair) VALUES ('%s', %s, false)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + + List history = AutoRepairUtils.getAutoRepairHistoryForLocalGroup(repairType); + + assertNotNull(history); + assertEquals(1, history.size()); + assertEquals(hostId, history.get(0).hostId); + } + + @Test + public void testGetAutoRepairHistoryForLocalGroup_empty_history() + { + List history = AutoRepairUtils.getAutoRepairHistoryForLocalGroup(repairType); + + assertNull(history); + } + + @Test + public void testGetCurrentRepairStatus() + { + UUID forceRepair = UUID.randomUUID(); + UUID regularRepair = UUID.randomUUID(); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, force_repair, repair_start_ts) VALUES ('%s', %s, true, toTimestamp(now()))", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), forceRepair)); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, repair_start_ts) VALUES ('%s', %s, toTimestamp(now()))", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), regularRepair)); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, + repairType.toString(), regularRepair)); + + CurrentRepairStatus status = AutoRepairUtils.getCurrentRepairStatus(repairType); + + assertNotNull(status); + assertEquals(1, status.historiesWithoutOnGoingRepair.size()); + assertEquals(hostId, status.historiesWithoutOnGoingRepair.get(0).hostId); + assertEquals(1, status.hostIdsWithOnGoingRepair.size()); + assertTrue(status.hostIdsWithOnGoingRepair.contains(regularRepair)); + assertEquals(1, status.hostIdsWithOnGoingForceRepair.size()); + assertTrue(status.hostIdsWithOnGoingForceRepair.contains(forceRepair)); + assertEquals(1, status.priority.size()); + assertTrue(status.priority.contains(regularRepair)); + } + + @Test + public void testGetHostIdsInCurrentRing() + { + TreeSet hosts = AutoRepairUtils.getHostIdsInCurrentRing(repairType); + + assertNotNull(hosts); + assertEquals(1, hosts.size()); + assertTrue(hosts.contains(hostId)); + } + + @Test + public void testGetHostIdsInCurrentRing_multiple_nodes() + { + InetAddressAndPort ignoredEndpoint = localEndpoint.withPort(localEndpoint.getPort() + 1); + InetAddressAndPort deadEndpoint = localEndpoint.withPort(localEndpoint.getPort() + 2); + DatabaseDescriptor.getAutoRepairConfig().setIgnoreDCs(repairType, ImmutableSet.of("dc2")); + DatabaseDescriptor.setEndpointSnitch(snitchMock); + when(snitchMock.getDatacenter(localEndpoint)).thenReturn("dc1"); + when(snitchMock.getDatacenter(ignoredEndpoint)).thenReturn("dc2"); + when(snitchMock.getDatacenter(deadEndpoint)).thenReturn("dc1"); + + TreeSet hosts = AutoRepairUtils.getHostIdsInCurrentRing(repairType, ImmutableSet.of(new NodeAddresses(localEndpoint), new NodeAddresses(ignoredEndpoint), new NodeAddresses(deadEndpoint))); + + assertNotNull(hosts); + assertEquals(1, hosts.size()); + assertTrue(hosts.contains(hostId)); + } + + @Test + public void testGetHostWithLongestUnrepairTime() + { + UUID otherHostId = UUID.randomUUID(); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id, repair_finish_ts) VALUES ('%s', %s, toTimestamp(now()))", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), otherHostId)); + + AutoRepairHistory history = AutoRepairUtils.getHostWithLongestUnrepairTime(repairType); + + assertEquals(hostId, history.hostId); + } + + @Test + public void testGetMaxNumberOfNodeRunAutoRepairInGroup_0_group_size() + { + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCountInGroup(repairType, 2); + + int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepairInGroup(repairType, 0); + + assertEquals(2, count); + } + + + @Test + public void testGetMaxNumberOfNodeRunAutoRepairInGroup_percentage() + { + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCountInGroup(repairType, 2); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairPercentageInGroup(repairType, 50); + + + int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepairInGroup(repairType, 10); + + assertEquals(5, count); + } + + @Test + public void testDeleteAutoRepairHistory() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + + AutoRepairUtils.deleteAutoRepairHistory(repairType, hostId); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT * FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(0, result.size()); + } + + @Test + public void testUpdateStartAutoRepairHistory() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + + AutoRepairUtils.updateStartAutoRepairHistory(repairType, hostId, 123, AutoRepairUtils.RepairTurn.MY_TURN); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT repair_start_ts, repair_turn FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(1, result.size()); + UntypedResultSet.Row row = result.one(); + assertEquals(123, row.getLong(COL_REPAIR_START_TS, 0)); + assertEquals(AutoRepairUtils.RepairTurn.MY_TURN.toString(), row.getString(COL_REPAIR_TURN)); + } + + @Test + public void testUpdateFinishAutoRepairHistory() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + + AutoRepairUtils.updateFinishAutoRepairHistory(repairType, hostId, 123); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT repair_finish_ts FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), hostId)); + assertNotNull(result); + assertEquals(1, result.size()); + assertEquals(123, result.one().getLong(COL_REPAIR_FINISH_TS, 0)); + } + + @Test + public void testAddHostIdToDeleteHosts() + { + UUID otherHostId = UUID.randomUUID(); + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), otherHostId)); + + AutoRepairUtils.addHostIdToDeleteHosts(repairType, hostId, otherHostId); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT * FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + repairType.toString(), otherHostId)); + assertNotNull(result); + assertEquals(1, result.size()); + Set deleteHosts = result.one().getSet(COL_DELETE_HOSTS, UUIDType.instance); + assertNotNull(deleteHosts); + assertEquals(1, deleteHosts.size()); + assertTrue(deleteHosts.contains(hostId)); + } + + @Test + public void testAddPriorityHost() + { + AutoRepairUtils.addPriorityHosts(repairType, ImmutableSet.of(localEndpoint)); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT * FROM %s.%s WHERE repair_type = '%s'", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, + repairType.toString())); + assertNotNull(result); + assertEquals(1, result.size()); + Set repairPriority = result.one().getSet(COL_REPAIR_PRIORITY, UUIDType.instance); + assertNotNull(repairPriority); + assertEquals(1, repairPriority.size()); + assertTrue(repairPriority.contains(hostId)); + } + + @Test + public void testRemovePriorityStatus() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, + repairType.toString(), hostId)); + + AutoRepairUtils.removePriorityStatus(repairType, hostId); + + UntypedResultSet result = QueryProcessor.executeInternal(String.format( + "SELECT * FROM %s.%s WHERE repair_type = '%s'", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, + repairType.toString())); + assertNotNull(result); + assertEquals(1, result.size()); + Set repairPriority = result.one().getSet(COL_REPAIR_PRIORITY, UUIDType.instance); + assertNull(repairPriority); + } + + @Test + public void testGetPriorityHosts() + { + QueryProcessor.executeInternal(String.format( + "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, + repairType.toString(), hostId)); + + Set hosts = AutoRepairUtils.getPriorityHosts(repairType); + + assertNotNull(hosts); + assertEquals(1, hosts.size()); + assertTrue(hosts.contains(localEndpoint)); + } + + @Test + public void testCheckNodeContainsKeyspaceReplica() + { + Keyspace ks = Keyspace.open("ks"); + + assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + } + + @Test + public void testTableMaxRepairTimeExceeded() + { + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairTableMaxRepairTime(repairType, "0s"); + + assertTrue(AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, 0)); + } + + @Test + public void testKeyspaceMaxRepairTimeExceeded() + { + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairTableMaxRepairTime(repairType, "0s"); + + assertTrue(AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, 0, 1)); + } + + @Test + public void testGetLastRepairFinishTime() + { + AutoRepairHistory history = new AutoRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); + + assertEquals(0, history.getLastRepairFinishTime()); + + history.lastRepairFinishTime = 100; + + assertEquals(100, history.getLastRepairFinishTime()); + } + + @Test + public void testMyTurnToRunRepairShouldReturnMyTurnWhenRepairOngoing() + { + UUID myID = UUID.randomUUID(); + UUID otherID = UUID.randomUUID(); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCountInGroup(repairType, 5); + long currentMillis = System.currentTimeMillis(); + // finish time less than start time means that repair is ongoing + AutoRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); + // finish time is larger than start time means that repair for other node is finished + AutoRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); + + assertEquals(AutoRepairUtils.RepairTurn.MY_TURN, AutoRepairUtils.myTurnToRunRepair(repairType, myID)); + } +} diff --git a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java index 23cfa80bfc24..7eecb54a056a 100644 --- a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java +++ b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java @@ -36,6 +36,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; + +import org.apache.cassandra.service.disk.usage.DiskUsageMonitor; +import org.apache.cassandra.utils.TimeUUID; +import org.apache.cassandra.utils.concurrent.Condition; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -71,6 +75,7 @@ import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.concurrent.Condition; import org.apache.cassandra.utils.concurrent.Refs; +import org.mockito.Mock; import static org.apache.cassandra.ServerTestUtils.*; import static org.apache.cassandra.config.CassandraRelevantProperties.ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION; @@ -80,12 +85,15 @@ import static org.apache.cassandra.repair.messages.RepairOption.INCREMENTAL_KEY; import static org.apache.cassandra.repair.messages.RepairOption.RANGES_KEY; import static org.apache.cassandra.service.ActiveRepairService.UNREPAIRED_SSTABLE; +import static org.apache.cassandra.service.ActiveRepairService.instance; import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID; import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; public class ActiveRepairServiceTest { @@ -97,6 +105,8 @@ public class ActiveRepairServiceTest public String cfname; public ColumnFamilyStore store; public static InetAddressAndPort LOCAL, REMOTE; + @Mock + public DiskUsageMonitor diskUsageMonitor; @BeforeClass public static void defineSchema() throws ConfigurationException, UnknownHostException @@ -121,6 +131,7 @@ public void prepare() throws Exception NodeId remote = Register.register(new NodeAddresses(REMOTE)); UnsafeJoin.unsafeJoin(local, Collections.singleton(DatabaseDescriptor.getPartitioner().getRandomToken())); UnsafeJoin.unsafeJoin(remote, Collections.singleton(DatabaseDescriptor.getPartitioner().getMinimumToken())); + initMocks(this); } @Test @@ -488,7 +499,9 @@ public void testRepairSessionSpaceInMiB() activeRepairService.setRepairSessionSpaceInMiB(0); fail("Should have received an IllegalArgumentException for depth of 0"); } - catch (IllegalArgumentException ignored) { } + catch (IllegalArgumentException ignored) + { + } Assert.assertEquals(10, activeRepairService.getRepairSessionSpaceInMiB()); } @@ -498,6 +511,40 @@ public void testRepairSessionSpaceInMiB() } } + public void testVerifyDiskHeadroomThresholdFullRepair() + { + Assert.assertTrue(ActiveRepairService.verifyDiskHeadroomThreshold(TimeUUID.maxAtUnixMillis(0), PreviewKind.NONE, false)); + } + + @Test + public void testVerifyDiskHeadroomThresholdDiskFull() + { + DiskUsageMonitor.instance = diskUsageMonitor; + when(diskUsageMonitor.getDiskUsage()).thenReturn(1.0); + DatabaseDescriptor.setIncrementalRepairDiskHeadroomRejectRatio(1.0); + + Assert.assertFalse(ActiveRepairService.verifyDiskHeadroomThreshold(TimeUUID.maxAtUnixMillis(0), PreviewKind.NONE, true)); + } + + @Test + public void testVerifyDiskHeadroomThresholdSufficientDisk() + { + DiskUsageMonitor.instance = diskUsageMonitor; + when(diskUsageMonitor.getDiskUsage()).thenReturn(0.0); + DatabaseDescriptor.setIncrementalRepairDiskHeadroomRejectRatio(0.0); + + Assert.assertTrue(ActiveRepairService.verifyDiskHeadroomThreshold(TimeUUID.maxAtUnixMillis(0), PreviewKind.NONE, true)); + } + + @Test(expected = RuntimeException.class) + public void testPrepareForRepairThrowsExceptionForInsufficientDisk() + { + DiskUsageMonitor.instance = diskUsageMonitor; + when(diskUsageMonitor.getDiskUsage()).thenReturn(1.5); + + instance().prepareForRepair(TimeUUID.maxAtUnixMillis(0), null, null, opts(INCREMENTAL_KEY, b2s(true)), false, null); + } + private static class Task implements Runnable { private final Condition blocked; From c849deb6038d352885ccc97ab9fc8e957a2d3d42 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 23 Jul 2024 14:28:07 -0700 Subject: [PATCH 002/115] cr --- .../cassandra/repair/autorepair/AutoRepair.java | 15 ++++++++++----- .../repair/autorepair/AutoRepairState.java | 3 ++- .../repair/autorepair/AutoRepairUtils.java | 15 ++++++++------- .../apache/cassandra/schema/AutoRepairParams.java | 11 +++++++---- .../org/apache/cassandra/schema/TableParams.java | 4 ++-- .../autorepair/AutoRepairParameterizedTest.java | 3 +-- .../repair/autorepair/AutoRepairTest.java | 3 +-- 7 files changed, 31 insertions(+), 23 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index f6a08e73b16d..1750d4c698df 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -26,7 +26,6 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import java.util.concurrent.FutureTask; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; @@ -40,7 +39,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.concurrent.NamedThreadFactory; import org.apache.cassandra.concurrent.ScheduledExecutorPlus; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; @@ -54,6 +52,7 @@ import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn; +import org.apache.cassandra.utils.concurrent.Future; import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN; @@ -74,6 +73,9 @@ public class AutoRepair @VisibleForTesting protected final Map repairExecutors; + + protected final Map repairRunnableExecutors; + @VisibleForTesting protected final Map repairStates; @@ -85,10 +87,12 @@ protected AutoRepair() { AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); repairExecutors = new EnumMap<>(AutoRepairConfig.RepairType.class); + repairRunnableExecutors = new EnumMap<>(AutoRepairConfig.RepairType.class); repairStates = new EnumMap<>(AutoRepairConfig.RepairType.class); for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { repairExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-Repair-" + repairType, Thread.NORM_PRIORITY)); + repairRunnableExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-RepairRunnable-" + repairType, Thread.NORM_PRIORITY)); repairStates.put(repairType, AutoRepairConfig.RepairType.getAutoRepairState(repairType)); tokenRangeSplitters.put(repairType, FBUtilities.newAutoRepairTokenRangeSplitter(config.getTokenRangeSplitter(repairType))); } @@ -298,7 +302,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : ImmutableList.of(tableName), ranges, primaryRangeOnly); repairState.resetWaitCondition(); - new Thread(NamedThreadFactory.createAnonymousThread(new FutureTask<>(task, null))).start(); + Future f = repairRunnableExecutors.get(repairType).submit(task); try { repairState.waitForRepairToComplete(); @@ -318,10 +322,11 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) else { repairSuccess = false; + boolean cancellationStatus = f.cancel(true); //in future we can add retry, etc. logger.info("Repair failed for range {}-{} for {}.{} total subranges: {}," + - "processed subranges: {}", childStartToken, childEndToken, - keyspaceName, config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, totalSubRanges, totalProcessedSubRanges); + "processed subranges: {}, cancellationStatus: {}", childStartToken, childEndToken, + keyspaceName, config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, totalSubRanges, totalProcessedSubRanges, cancellationStatus); } ranges.clear(); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index 523a48545d59..3e4169a9b7e3 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -50,6 +50,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition; // AutoRepairState represents the state of automated repair for a given repair type. @@ -115,7 +116,7 @@ protected RepairCoordinator getRepairRunnable(String keyspace, RepairOption opti public void progress(String tag, ProgressEvent event) { ProgressEventType type = event.getType(); - String message = String.format("[%s] %s", format.format(System.currentTimeMillis()), event.getMessage()); + String message = String.format("[%s] %s", format.format(currentTimeMillis()), event.getMessage()); if (type == ProgressEventType.ERROR) { logger.error("Repair failure for {} repair: {}", repairType.toString(), message); diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index cac3e6bf8d4c..2d513750da2e 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -75,6 +75,7 @@ import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; +import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; /** * This class serves as a utility class for AutoRepair. It contains various helper APIs @@ -338,7 +339,7 @@ public static void setForceRepairNewNode(RepairType repairType) // this function will be called when a node bootstrap finished UUID hostId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); // insert the data first - insertNewRepairHistory(repairType, System.currentTimeMillis(), System.currentTimeMillis()); + insertNewRepairHistory(repairType, currentTimeMillis(), currentTimeMillis()); setForceRepair(repairType, hostId); } @@ -477,7 +478,7 @@ public static RepairTurn myTurnToRunRepair(RepairType repairType, UUID myId) AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); if (nodeHistory.deleteHosts.size() > 0 && config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds() < TimeUnit.MILLISECONDS.toSeconds( - System.currentTimeMillis() - nodeHistory.deleteHostsUpdateTime + currentTimeMillis() - nodeHistory.deleteHostsUpdateTime )) { clearDeleteHosts(repairType, nodeHistory.hostId); @@ -508,7 +509,7 @@ else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) if (!autoRepairHistoryIds.contains(hostId)) { logger.info("{} for repair type {} doesn't exist in the auto repair history table, insert a new record.", repairType, hostId); - insertNewRepairHistory(repairType, hostId, System.currentTimeMillis(), System.currentTimeMillis()); + insertNewRepairHistory(repairType, hostId, currentTimeMillis(), currentTimeMillis()); } } @@ -651,7 +652,7 @@ public static void insertNewRepairHistory(RepairType repairType, UUID hostId, lo ByteBufferUtil.bytes(hostId), ByteBufferUtil.bytes(startTime), ByteBufferUtil.bytes(finishTime), - ByteBufferUtil.bytes(System.currentTimeMillis()) + ByteBufferUtil.bytes(currentTimeMillis()) ), false, -1, null, cl, ProtocolVersion.CURRENT, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME), Dispatcher.RequestTime.forImmediateExecution()); resultSet = UntypedResultSet.create(resultMessage.result); @@ -683,7 +684,7 @@ public static void addHostIdToDeleteHosts(RepairType repairType, UUID myID, UUID addHostIDToDeleteHostsStatement.execute(QueryState.forInternalCalls(), QueryOptions.forInternalCalls(internalQueryCL, Lists.newArrayList(serializer.serialize(new HashSet<>(Arrays.asList(myID))), - ByteBufferUtil.bytes(System.currentTimeMillis()), + ByteBufferUtil.bytes(currentTimeMillis()), ByteBufferUtil.bytes(repairType.toString()), ByteBufferUtil.bytes(hostToBeDeleted) )), Dispatcher.RequestTime.forImmediateExecution()); @@ -781,14 +782,14 @@ public static boolean checkNodeContainsKeyspaceReplica(Keyspace ks) public static boolean tableMaxRepairTimeExceeded(RepairType repairType, long startTime) { long tableRepairTimeSoFar = TimeUnit.MILLISECONDS.toSeconds - (System.currentTimeMillis() - startTime); + (currentTimeMillis() - startTime); return AutoRepairService.instance.getAutoRepairConfig().getAutoRepairTableMaxRepairTime(repairType).toSeconds() < tableRepairTimeSoFar; } public static boolean keyspaceMaxRepairTimeExceeded(RepairType repairType, long startTime, int numOfTablesToBeRepaired) { - long keyspaceRepairTimeSoFar = TimeUnit.MILLISECONDS.toSeconds((System.currentTimeMillis() - startTime)); + long keyspaceRepairTimeSoFar = TimeUnit.MILLISECONDS.toSeconds((currentTimeMillis() - startTime)); return (long) AutoRepairService.instance.getAutoRepairConfig().getAutoRepairTableMaxRepairTime(repairType).toSeconds() * numOfTablesToBeRepaired < keyspaceRepairTimeSoFar; } diff --git a/src/java/org/apache/cassandra/schema/AutoRepairParams.java b/src/java/org/apache/cassandra/schema/AutoRepairParams.java index 50f482caf1b7..ae97e562635f 100644 --- a/src/java/org/apache/cassandra/schema/AutoRepairParams.java +++ b/src/java/org/apache/cassandra/schema/AutoRepairParams.java @@ -63,13 +63,16 @@ public static AutoRepairParams create(AutoRepairConfig.RepairType repairType, Ma { optionsMap.put(entry.getKey(), new HashMap<>(entry.getValue())); } - for (Map.Entry entry : options.entrySet()) + if (options != null) { - if (!Option.ENABLED.toString().equals(entry.getKey().toLowerCase())) + for (Map.Entry entry : options.entrySet()) { - throw new ConfigurationException(format("Unknown property '%s'", entry.getKey())); + if (!Option.ENABLED.toString().equals(entry.getKey().toLowerCase())) + { + throw new ConfigurationException(format("Unknown property '%s'", entry.getKey())); + } + optionsMap.get(repairType).put(entry.getKey(), entry.getValue()); } - optionsMap.get(repairType).put(entry.getKey(), entry.getValue()); } AutoRepairParams repairParams = new AutoRepairParams(repairType); repairParams.options = optionsMap; diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index 751051984a80..3bf45fc1c8cb 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -19,7 +19,7 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.util.HashMap; +import java.util.EnumMap; import java.util.Map; import java.util.Map.Entry; @@ -128,7 +128,7 @@ private TableParams(Builder builder) extensions = builder.extensions; cdc = builder.cdc; readRepair = builder.readRepair; - automatedRepair = new HashMap<>() + automatedRepair = new EnumMap(AutoRepairConfig.RepairType.class) { { put(AutoRepairConfig.RepairType.full, builder.automatedRepairFull); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 17b175be1eec..2b8915600a23 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -39,13 +39,13 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.Pair; +import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import junit.framework.Assert; import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.concurrent.ScheduledExecutorPlus; import org.apache.cassandra.config.DatabaseDescriptor; @@ -71,7 +71,6 @@ import org.mockito.MockitoAnnotations; import static org.apache.cassandra.Util.setAutoRepairEnabled; -import static org.apache.cassandra.Util.token; import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index b16e0ea5b491..c1837aa6a2cc 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -24,8 +24,7 @@ import org.junit.BeforeClass; import org.junit.Test; -import junit.framework.Assert; - +import org.junit.Assert; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.Keyspace; From ca1891f1d8b916b05ffce9d869ab4c940e905b21 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Sat, 27 Jul 2024 09:43:23 -0700 Subject: [PATCH 003/115] chop the repair API --- .../repair/autorepair/AutoRepair.java | 150 ++++++++++-------- 1 file changed, 83 insertions(+), 67 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 1750d4c698df..1225434d0106 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -140,10 +140,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) logger.debug("Auto-repair is disabled for repair type {}", repairType); return; } - - AutoRepairState repairState = repairStates.get(repairType); - try { String localDC = DatabaseDescriptor.getLocalDataCenter(); @@ -169,19 +166,9 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) boolean primaryRangeOnly = config.getRepairPrimaryTokenRangeOnly(repairType) && turn != MY_TURN_FORCE_REPAIR; repairState.setTotalTablesConsideredForRepair(0); - if (repairState.getLastRepairTime() != 0) + if (tooSoonToRunRepair(repairType, repairState, config)) { - /** check if it is too soon to run repair. one of the reason we - * should not run frequent repair is because repair triggers - * memtable flush - */ - long timeElapsedSinceLastRepair = TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - repairState.getLastRepairTime()); - if (timeElapsedSinceLastRepair < config.getRepairMinInterval(repairType).toSeconds()) - { - logger.info("Too soon to run repair, last repair was done {} seconds ago", - timeElapsedSinceLastRepair); - return; - } + return; } long startTime = timeFunc.get(); @@ -197,34 +184,16 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) repairState.setTotalMVTablesConsideredForRepair(0); for (Keyspace keyspace : Keyspace.all()) { - Tables tables = keyspace.getMetadata().tables; - Iterator iter = tables.iterator(); - String keyspaceName = keyspace.getName(); if (!AutoRepairUtils.checkNodeContainsKeyspaceReplica(keyspace)) { continue; } repairState.setRepairKeyspaceCount(repairState.getRepairKeyspaceCount() + 1); - List tablesToBeRepaired = new ArrayList<>(); - while (iter.hasNext()) - { - repairState.setTotalTablesConsideredForRepair(repairState.getTotalTablesConsideredForRepair() + 1); - TableMetadata tableMetadata = iter.next(); - String tableName = tableMetadata.name; - tablesToBeRepaired.add(tableName); - - // See if we should repair MVs as well that are associated with this given table - List mvs = AutoRepairUtils.getAllMVs(repairType, keyspace, tableMetadata); - if (!mvs.isEmpty()) - { - tablesToBeRepaired.addAll(mvs); - repairState.setTotalMVTablesConsideredForRepair(repairState.getTotalMVTablesConsideredForRepair() + mvs.size()); - } - } - + List tablesToBeRepaired = retrieveTablesToBeRepaired(keyspace, repairType, repairState); for (String tableName : tablesToBeRepaired) { + String keyspaceName = keyspace.getName(); try { ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName); @@ -356,38 +325,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) } } } - - //if it was due to priority then remove it now - if (turn == MY_TURN_DUE_TO_PRIORITY) - { - logger.info("Remove current host from priority list"); - AutoRepairUtils.removePriorityStatus(repairType, myId); - } - - repairState.setNodeRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - startTime)); - long timeInHours = TimeUnit.SECONDS.toHours(repairState.getNodeRepairTimeInSec()); - logger.info("Local {} repair time {} hour(s), stats: repairKeyspaceCount {}, " + - "repairTableSuccessCount {}, repairTableFailureCount {}, " + - "repairTableSkipCount {}", repairType, timeInHours, repairState.getRepairKeyspaceCount(), - repairState.getRepairTableSuccessCount(), repairState.getRepairFailedTablesCount(), - repairState.getRepairSkippedTablesCount()); - if (repairState.getLastRepairTime() != 0) - { - repairState.setClusterRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - - repairState.getLastRepairTime())); - logger.info("Cluster repair time for repair type {}: {} day(s)", repairType, - TimeUnit.SECONDS.toDays(repairState.getClusterRepairTimeInSec())); - } - repairState.setLastRepairTime(timeFunc.get()); - if (timeInHours == 0 && millisToWait > 0) - { - //If repair finished quickly, happens for an empty instance, in such case - //wait for a minute so that the JMX metrics can detect the repairInProgress - logger.info("Wait for {} milliseconds for repair type {}.", millisToWait, repairType); - Thread.sleep(millisToWait); - } - repairState.setRepairInProgress(false); - AutoRepairUtils.updateFinishAutoRepairHistory(repairType, myId, timeFunc.get()); + cleanupAndUpdateStats(turn, repairType, repairState, myId, startTime, millisToWait); } else { @@ -400,6 +338,84 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) } } + private boolean tooSoonToRunRepair(AutoRepairConfig.RepairType repairType, AutoRepairState repairState, AutoRepairConfig config) + { + if (repairState.getLastRepairTime() != 0) + { + /** check if it is too soon to run repair. one of the reason we + * should not run frequent repair is that repair triggers + * memtable flush + */ + long timeElapsedSinceLastRepair = TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - repairState.getLastRepairTime()); + if (timeElapsedSinceLastRepair < config.getRepairMinInterval(repairType).toSeconds()) + { + logger.info("Too soon to run repair, last repair was done {} seconds ago", + timeElapsedSinceLastRepair); + return true; + } + } + return false; + } + + private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairConfig.RepairType repairType, AutoRepairState repairState) + { + Tables tables = keyspace.getMetadata().tables; + List tablesToBeRepaired = new ArrayList<>(); + Iterator iter = tables.iterator(); + while (iter.hasNext()) + { + repairState.setTotalTablesConsideredForRepair(repairState.getTotalTablesConsideredForRepair() + 1); + TableMetadata tableMetadata = iter.next(); + String tableName = tableMetadata.name; + tablesToBeRepaired.add(tableName); + + // See if we should repair MVs as well that are associated with this given table + List mvs = AutoRepairUtils.getAllMVs(repairType, keyspace, tableMetadata); + if (!mvs.isEmpty()) + { + tablesToBeRepaired.addAll(mvs); + repairState.setTotalMVTablesConsideredForRepair(repairState.getTotalMVTablesConsideredForRepair() + mvs.size()); + } + } + return tablesToBeRepaired; + } + + private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, UUID myId, + long startTime, long millisToWait) throws InterruptedException + { + //if it was due to priority then remove it now + if (turn == MY_TURN_DUE_TO_PRIORITY) + { + logger.info("Remove current host from priority list"); + AutoRepairUtils.removePriorityStatus(repairType, myId); + } + + repairState.setNodeRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - startTime)); + long timeInHours = TimeUnit.SECONDS.toHours(repairState.getNodeRepairTimeInSec()); + logger.info("Local {} repair time {} hour(s), stats: repairKeyspaceCount {}, " + + "repairTableSuccessCount {}, repairTableFailureCount {}, " + + "repairTableSkipCount {}", repairType, timeInHours, repairState.getRepairKeyspaceCount(), + repairState.getRepairTableSuccessCount(), repairState.getRepairFailedTablesCount(), + repairState.getRepairSkippedTablesCount()); + if (repairState.getLastRepairTime() != 0) + { + repairState.setClusterRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - + repairState.getLastRepairTime())); + logger.info("Cluster repair time for repair type {}: {} day(s)", repairType, + TimeUnit.SECONDS.toDays(repairState.getClusterRepairTimeInSec())); + } + repairState.setLastRepairTime(timeFunc.get()); + if (timeInHours == 0 && millisToWait > 0) + { + //If repair finished quickly, happens for an empty instance, in such case + //wait for a minute so that the JMX metrics can detect the repairInProgress + logger.info("Wait for {} milliseconds for repair type {}.", millisToWait, repairType); + Thread.sleep(millisToWait); + } + repairState.setRepairInProgress(false); + AutoRepairUtils.updateFinishAutoRepairHistory(repairType, myId, timeFunc.get()); + } + public AutoRepairState getRepairState(AutoRepairConfig.RepairType repairType) { return repairStates.get(repairType); From 3ad85dd4d18a9375b40f2d4629ec28446e28bd0c Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Fri, 16 Aug 2024 14:54:06 -0700 Subject: [PATCH 004/115] Add a test case that ensures LocalStrategy.class is ignored from being repaired --- .../cassandra/repair/autorepair/AutoRepairUtilsTest.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java index 0ea495fea3b1..b96488437e8e 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -62,6 +62,7 @@ import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_START_TS; import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_TURN; 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.assertTrue; @@ -79,7 +80,6 @@ public class AutoRepairUtilsTest extends CQLTester static IEndpointSnitch defaultSnitch; - @BeforeClass public static void setupClass() throws Exception { @@ -475,4 +475,11 @@ public void testMyTurnToRunRepairShouldReturnMyTurnWhenRepairOngoing() assertEquals(AutoRepairUtils.RepairTurn.MY_TURN, AutoRepairUtils.myTurnToRunRepair(repairType, myID)); } + + @Test + public void testLocalStrategyAndNetworkKeyspace() + { + assertFalse(AutoRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open("system"))); + assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open(KEYSPACE))); + } } From 060e3907879c3bf15a17d160dcbc95b483370dde Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Fri, 16 Aug 2024 15:14:20 -0700 Subject: [PATCH 005/115] Shuffle keyspaces and tables when running auto-repair --- .../repair/autorepair/AutoRepair.java | 14 +++++++++- .../AutoRepairParameterizedTest.java | 26 +++++++++++++++++++ 2 files changed, 39 insertions(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 1225434d0106..f2436292c481 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -27,6 +27,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import java.util.function.Supplier; import com.google.common.annotations.VisibleForTesting; @@ -79,6 +80,9 @@ public class AutoRepair @VisibleForTesting protected final Map repairStates; + @VisibleForTesting + protected static Consumer> shuffleFunc = java.util.Collections::shuffle; + protected final Map tokenRangeSplitters = new EnumMap<>(AutoRepairConfig.RepairType.class); @@ -182,7 +186,14 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) repairState.setRepairSkippedTablesCount(0); repairState.setRepairInProgress(true); repairState.setTotalMVTablesConsideredForRepair(0); - for (Keyspace keyspace : Keyspace.all()) + + List keyspaces = new ArrayList<>(); + Keyspace.all().forEach(keyspaces::add); + // Auto-repair is likely to be run on multiple nodes independently, we want to avoid running multiple repair + // sessions on overlapping datasets at the same time. Shuffling keyspaces reduces the likelihood of this happening. + shuffleFunc.accept(keyspaces); + + for (Keyspace keyspace : keyspaces) { if (!AutoRepairUtils.checkNodeContainsKeyspaceReplica(keyspace)) { @@ -191,6 +202,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) repairState.setRepairKeyspaceCount(repairState.getRepairKeyspaceCount() + 1); List tablesToBeRepaired = retrieveTablesToBeRepaired(keyspace, repairType, repairState); + shuffleFunc.accept(tablesToBeRepaired); for (String tableName : tablesToBeRepaired) { String keyspaceName = keyspace.getName(); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 2b8915600a23..ca66e2ba5091 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -182,6 +182,8 @@ public void setup() AutoRepair.timeFunc = System::currentTimeMillis; resetCounters(); resetConfig(); + + AutoRepair.shuffleFunc = java.util.Collections::shuffle; } @@ -577,4 +579,28 @@ public void testDefaultAutomatedRepair() Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); } + + @Test + public void testRepairShufflesKeyspacesAndTables() + { + AtomicInteger shuffleKeyspacesCall = new AtomicInteger(); + AtomicInteger shuffleTablesCall = new AtomicInteger(); + AutoRepair.shuffleFunc = (List list) -> { + assertTrue(list.get(0) instanceof Keyspace || list.get(0) instanceof String); + if (list.get(0) instanceof Keyspace) + { + shuffleKeyspacesCall.getAndIncrement(); + assertFalse(list.isEmpty()); + } + else if (list.get(0) instanceof String) + { + shuffleTablesCall.getAndIncrement(); + } + }; + + AutoRepair.instance.repair(repairType, 0); + + assertEquals(1, shuffleKeyspacesCall.get()); + assertEquals(4, shuffleTablesCall.get()); + } } From 3dcebe66c27e52456fdef06c43aba75f89ee0821 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Sat, 17 Aug 2024 10:06:52 -0700 Subject: [PATCH 006/115] Prevent node restart from scheduling a premature auto-repair cycle --- .../repair/autorepair/AutoRepair.java | 33 ++++++------ .../repair/autorepair/AutoRepairUtils.java | 28 ++++++++++ .../AutoRepairParameterizedTest.java | 51 ++++++++++++------- .../autorepair/AutoRepairUtilsTest.java | 20 ++++++++ 4 files changed, 100 insertions(+), 32 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index f2436292c481..c8da85c48a3e 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -169,8 +169,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) // When doing force repair, we want to repair without -pr. boolean primaryRangeOnly = config.getRepairPrimaryTokenRangeOnly(repairType) && turn != MY_TURN_FORCE_REPAIR; - repairState.setTotalTablesConsideredForRepair(0); - if (tooSoonToRunRepair(repairType, repairState, config)) + if (tooSoonToRunRepair(repairType, repairState, config, myId)) { return; } @@ -185,6 +184,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) repairState.setRepairFailedTablesCount(0); repairState.setRepairSkippedTablesCount(0); repairState.setRepairInProgress(true); + repairState.setTotalTablesConsideredForRepair(0); repairState.setTotalMVTablesConsideredForRepair(0); List keyspaces = new ArrayList<>(); @@ -350,21 +350,24 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) } } - private boolean tooSoonToRunRepair(AutoRepairConfig.RepairType repairType, AutoRepairState repairState, AutoRepairConfig config) + private boolean tooSoonToRunRepair(AutoRepairConfig.RepairType repairType, AutoRepairState repairState, AutoRepairConfig config, UUID myId) { - if (repairState.getLastRepairTime() != 0) + if (repairState.getLastRepairTime() == 0) { - /** check if it is too soon to run repair. one of the reason we - * should not run frequent repair is that repair triggers - * memtable flush - */ - long timeElapsedSinceLastRepair = TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - repairState.getLastRepairTime()); - if (timeElapsedSinceLastRepair < config.getRepairMinInterval(repairType).toSeconds()) - { - logger.info("Too soon to run repair, last repair was done {} seconds ago", - timeElapsedSinceLastRepair); - return true; - } + // the node has either just boooted or has not run repair before, + // we should check for the node's repair history in the DB + repairState.setLastRepairTime(AutoRepairUtils.getLastRepairTimeForNode(repairType, myId)); + } + /** check if it is too soon to run repair. one of the reason we + * should not run frequent repair is that repair triggers + * memtable flush + */ + long timeElapsedSinceLastRepair = TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - repairState.getLastRepairTime()); + if (timeElapsedSinceLastRepair < config.getRepairMinInterval(repairType).toSeconds()) + { + logger.info("Too soon to run repair, last repair was done {} seconds ago", + timeElapsedSinceLastRepair); + return true; } return false; } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index 2d513750da2e..a22faafd1b83 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -143,10 +143,15 @@ public class AutoRepairUtils , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_FORCE_REPAIR, COL_REPAIR_TYPE, COL_HOST_ID); + final static String SELECT_LAST_REPAIR_TIME_FOR_NODE = String.format( + "SELECT %s FROM %s.%s WHERE %s = ? AND %s = ?", COL_REPAIR_FINISH_TS, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, + AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID); + static ModificationStatement delStatementRepairHistory; static SelectStatement selectStatementRepairHistory; static ModificationStatement delStatementPriorityStatus; static SelectStatement selectStatementRepairPriority; + static SelectStatement selectLastRepairTimeForNode; static ModificationStatement addPriorityHost; static ModificationStatement insertNewRepairHistoryStatement; static ModificationStatement recordStartRepairHistoryStatement; @@ -170,6 +175,10 @@ public static void setup() .forInternalCalls()); selectStatementRepairPriority = (SelectStatement) QueryProcessor.getStatement(SELECT_REPAIR_PRIORITY, ClientState .forInternalCalls()); + + selectLastRepairTimeForNode = (SelectStatement) QueryProcessor.getStatement(SELECT_LAST_REPAIR_TIME_FOR_NODE, ClientState + .forInternalCalls()); + delStatementPriorityStatus = (ModificationStatement) QueryProcessor.getStatement(DEL_REPAIR_PRIORITY, ClientState .forInternalCalls()); addPriorityHost = (ModificationStatement) QueryProcessor.getStatement(ADD_PRIORITY_HOST, ClientState @@ -370,6 +379,25 @@ public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType) return getCurrentRepairStatus(repairType, autoRepairHistories); } + public static long getLastRepairTimeForNode(RepairType repairType, UUID hostId) + { + ResultMessage.Rows rows = selectLastRepairTimeForNode.execute(QueryState.forInternalCalls(), + QueryOptions.forInternalCalls(internalQueryCL, + Lists.newArrayList( + ByteBufferUtil.bytes(repairType.toString()), + ByteBufferUtil.bytes(hostId))), + Dispatcher.RequestTime.forImmediateExecution()); + + UntypedResultSet repairTime = UntypedResultSet.create(rows.result); + + if (repairTime.isEmpty()) + { + return 0; + } + + return repairTime.one().getLong(COL_REPAIR_FINISH_TS); + } + public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType, List autoRepairHistories) { if (autoRepairHistories != null) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index ca66e2ba5091..6100a57a1b85 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -150,14 +150,8 @@ public static void setupClass() throws Exception QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); - defaultConfig = new AutoRepairConfig(true); DatabaseDescriptor.setMaterializedViewsEnabled(false); DatabaseDescriptor.setCDCEnabled(false); - for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) - { - defaultConfig.setAutoRepairEnabled(repairType, true); - defaultConfig.setMVRepairEnabled(repairType, false); - } } @Before @@ -198,6 +192,15 @@ private void resetCounters() private void resetConfig() { + // prepare a fresh default config + defaultConfig = new AutoRepairConfig(true); + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + defaultConfig.setAutoRepairEnabled(repairType, true); + defaultConfig.setMVRepairEnabled(repairType, false); + } + + // reset the AutoRepairService config to default AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.repair_type_overrides = defaultConfig.repair_type_overrides; config.global_settings = defaultConfig.global_settings; @@ -263,9 +266,6 @@ public void testTooFrequentRepairs() long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertEquals(String.format("Expected repair time to be same, actual value lastRepairTime1 %d, " + "lastRepairTime2 %d", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); - consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); - Assert.assertEquals("Expected total repaired tables = 0, actual value: " + consideredTables, - consideredTables, 0); assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); } @@ -367,24 +367,23 @@ public void testMVRepair() { AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setMVRepairEnabled(repairType, true); - AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(0); + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); AutoRepair.instance.repair(repairType, 0); assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); config.setMVRepairEnabled(repairType, false); - AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(0); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); AutoRepair.instance.repair(repairType, 0); assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); config.setMVRepairEnabled(repairType, true); - AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(0); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); AutoRepair.instance.repair(repairType, 0); assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); - - config.setMVRepairEnabled(repairType, false); } @Test @@ -396,6 +395,7 @@ public void testSkipRepairSSTableCountHigherThreshold() ColumnFamilyStore cfsMVTable = Keyspace.open(KEYSPACE).getColumnFamilyStore(MV); Set preBaseTable = cfsBaseTable.getLiveSSTables(); Set preMVTable = cfsBaseTable.getLiveSSTables(); + config.setRepairMinInterval(repairType, "0s"); for (int i = 0; i < 10; i++) { @@ -436,9 +436,6 @@ public void testSkipRepairSSTableCountHigherThreshold() assertEquals(0, state.getRepairSkippedTablesCount()); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); - - config.setMVRepairEnabled(repairType, false); - config.setRepairSSTableCountHigherThreshold(repairType, beforeCount); } @Test @@ -598,9 +595,29 @@ else if (list.get(0) instanceof String) } }; + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setRepairMinInterval(repairType, "0s"); AutoRepair.instance.repair(repairType, 0); assertEquals(1, shuffleKeyspacesCall.get()); assertEquals(4, shuffleTablesCall.get()); } + + @Test + public void testRepairTakesLastRepairTimeFromDB() + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setMVRepairEnabled(repairType, true); + long lastRepairTime = System.currentTimeMillis() - 1000; + AutoRepairUtils.insertNewRepairHistory(repairType, 0, lastRepairTime); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(0); + config.setRepairMinInterval(repairType, "1h"); + + AutoRepair.instance.repair(repairType, 0); + + // repair scheduler should not attempt to run repair as last repair time in DB is current time - 1s + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair()); + // repair scheduler should load the repair time from the DB + assertEquals(lastRepairTime, AutoRepair.instance.repairStates.get(repairType).getLastRepairTime()); + } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java index b96488437e8e..a00019577019 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -482,4 +482,24 @@ public void testLocalStrategyAndNetworkKeyspace() assertFalse(AutoRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open("system"))); assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open(KEYSPACE))); } + + @Test + public void testGetLastRepairTimeForNode() + { + UUID myID = UUID.randomUUID(); + UUID otherID = UUID.randomUUID(); + long currentMillis = System.currentTimeMillis(); + AutoRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); + AutoRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); + + assertEquals(currentMillis - 100, AutoRepairUtils.getLastRepairTimeForNode(repairType, myID)); + } + + @Test + public void testGetLastRepairTimeForNodeWhenHistoryIsEmpty() + { + UUID myID = UUID.randomUUID(); + + assertEquals(0, AutoRepairUtils.getLastRepairTimeForNode(repairType, myID)); + } } From dd54c74b15085a0667b8684bd1c598f3272c9835 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Sat, 17 Aug 2024 10:26:17 -0700 Subject: [PATCH 007/115] Implement SucceededTokenRangesCount and FailedTokenRangesCount metrics for auto-repair --- .../cassandra/metrics/AutoRepairMetrics.java | 15 ++++- .../repair/autorepair/AutoRepair.java | 28 ++++----- .../repair/autorepair/AutoRepairState.java | 46 +++++++-------- .../AutoRepairParameterizedTest.java | 6 +- .../autorepair/AutoRepairStateTest.java | 59 +++++++++---------- 5 files changed, 79 insertions(+), 75 deletions(-) diff --git a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java index cd46c462fb66..426034ffcd4b 100644 --- a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java +++ b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java @@ -36,7 +36,8 @@ public class AutoRepairMetrics public Gauge clusterRepairTimeInSec; public Gauge skippedTablesCount; public Gauge longestUnrepairedSec; - public Gauge failedTablesCount; + public Gauge succeededTokenRangesCount; + public Gauge failedTokenRangesCount; public Counter repairTurnMyTurn; public Counter repairTurnMyTurnDueToPriority; public Counter repairTurnMyTurnForceRepair; @@ -87,11 +88,19 @@ public Integer getValue() } }); - failedTablesCount = Metrics.register(factory.createMetricName("FailedTablesCount"), new Gauge() + succeededTokenRangesCount = Metrics.register(factory.createMetricName("SucceededTokenRangesCount"), new Gauge() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).getRepairFailedTablesCount(); + return AutoRepair.instance.getRepairState(repairType).getSucceededTokenRangesCount(); + } + }); + + failedTokenRangesCount = Metrics.register(factory.createMetricName("FailedTokenRangesCount"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getFailedTokenRangesCount(); } }); diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index c8da85c48a3e..41161d01c57e 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -180,13 +180,14 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) AutoRepairUtils.updateStartAutoRepairHistory(repairType, myId, timeFunc.get(), turn); repairState.setRepairKeyspaceCount(0); - repairState.setRepairTableSuccessCount(0); - repairState.setRepairFailedTablesCount(0); repairState.setRepairSkippedTablesCount(0); repairState.setRepairInProgress(true); repairState.setTotalTablesConsideredForRepair(0); repairState.setTotalMVTablesConsideredForRepair(0); + int failedTokenRanges = 0; + int succeededTokenRanges = 0; + List keyspaces = new ArrayList<>(); Keyspace.all().forEach(keyspaces::add); // Auto-repair is likely to be run on multiple nodes independently, we want to avoid running multiple repair @@ -235,7 +236,6 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) logger.info("Repair table {}.{}", keyspaceName, tableName); } long tableStartTime = timeFunc.get(); - boolean repairSuccess = true; Set> ranges = new HashSet<>(); List> subRangesToBeRepaired = tokenRangeSplitters.get(repairType).getRange(repairType, primaryRangeOnly, keyspaceName, tableName); int totalSubRanges = subRangesToBeRepaired.size(); @@ -299,28 +299,20 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) logger.info("Repair completed for range {}-{} for {}.{}, total subranges: {}," + "processed subranges: {}", childStartToken, childEndToken, keyspaceName, config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, totalSubRanges, totalProcessedSubRanges); + succeededTokenRanges += ranges.size(); } else { - repairSuccess = false; boolean cancellationStatus = f.cancel(true); //in future we can add retry, etc. logger.info("Repair failed for range {}-{} for {}.{} total subranges: {}," + "processed subranges: {}, cancellationStatus: {}", childStartToken, childEndToken, keyspaceName, config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, totalSubRanges, totalProcessedSubRanges, cancellationStatus); + failedTokenRanges += ranges.size(); } ranges.clear(); } } - int touchedTables = config.getRepairByKeyspace(repairType) ? tablesToBeRepaired.size() : 1; - if (repairSuccess) - { - repairState.setRepairTableSuccessCount(repairState.getRepairTableSuccessCount() + touchedTables); - } - else - { - repairState.setRepairFailedTablesCount(repairState.getRepairFailedTablesCount() + touchedTables); - } if (config.getRepairByKeyspace(repairType)) { logger.info("Repair completed for keyspace {}, tables: {}", keyspaceName, tablesToBeRepaired); @@ -337,7 +329,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) } } } - cleanupAndUpdateStats(turn, repairType, repairState, myId, startTime, millisToWait); + cleanupAndUpdateStats(turn, repairType, repairState, myId, startTime, millisToWait, failedTokenRanges, succeededTokenRanges); } else { @@ -396,7 +388,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon } private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, UUID myId, - long startTime, long millisToWait) throws InterruptedException + long startTime, long millisToWait, int failedTokenRanges, int succeededTokenRanges) throws InterruptedException { //if it was due to priority then remove it now if (turn == MY_TURN_DUE_TO_PRIORITY) @@ -405,12 +397,14 @@ private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType AutoRepairUtils.removePriorityStatus(repairType, myId); } + repairState.setFailedTokenRangesCount(failedTokenRanges); + repairState.setSucceededTokenRangesCount(succeededTokenRanges); repairState.setNodeRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - startTime)); long timeInHours = TimeUnit.SECONDS.toHours(repairState.getNodeRepairTimeInSec()); logger.info("Local {} repair time {} hour(s), stats: repairKeyspaceCount {}, " + - "repairTableSuccessCount {}, repairTableFailureCount {}, " + + "repairTokenRangesSuccessCount {}, repairTokenRangesFailureCount {}, " + "repairTableSkipCount {}", repairType, timeInHours, repairState.getRepairKeyspaceCount(), - repairState.getRepairTableSuccessCount(), repairState.getRepairFailedTablesCount(), + repairState.getSucceededTokenRangesCount(), repairState.getFailedTokenRangesCount(), repairState.getRepairSkippedTablesCount()); if (repairState.getLastRepairTime() != 0) { diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index 3e4169a9b7e3..d51ebde50ec9 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -76,16 +76,16 @@ public abstract class AutoRepairState implements ProgressListener @VisibleForTesting protected int repairKeyspaceCount = 0; @VisibleForTesting - protected int repairTableSuccessCount = 0; - @VisibleForTesting protected int repairTableSkipCount = 0; @VisibleForTesting - protected int repairTableFailureCount = 0; - @VisibleForTesting protected int totalMVTablesConsideredForRepair = 0; - @VisibleForTesting protected int totalDisabledTablesRepairCount = 0; + + @VisibleForTesting + protected int failedTokenRangesCount = 0; + @VisibleForTesting + protected int succeededTokenRangesCount = 0; @VisibleForTesting protected AutoRepairHistory longestUnrepairedNode; @VisibleForTesting @@ -203,16 +203,6 @@ public int getLongestUnrepairedSec() return (int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - longestUnrepairedNode.getLastRepairFinishTime()); } - public void setRepairFailedTablesCount(int count) - { - repairTableFailureCount = count; - } - - public int getRepairFailedTablesCount() - { - return repairTableFailureCount; - } - public void setTotalMVTablesConsideredForRepair(int count) { totalMVTablesConsideredForRepair = count; @@ -238,24 +228,34 @@ public void setRepairKeyspaceCount(int count) repairKeyspaceCount = count; } - public void setRepairTableSuccessCount(int count) + public int getRepairKeyspaceCount() + { + return repairKeyspaceCount; + } + + public void setLongestUnrepairedNode(AutoRepairHistory longestUnrepairedNode) { - repairTableSuccessCount = count; + this.longestUnrepairedNode = longestUnrepairedNode; } - public int getRepairKeyspaceCount() + public void setFailedTokenRangesCount(int count) { - return repairKeyspaceCount; + failedTokenRangesCount = count; } - public int getRepairTableSuccessCount() + public int getFailedTokenRangesCount() { - return repairTableSuccessCount; + return failedTokenRangesCount; } - public void setLongestUnrepairedNode(AutoRepairHistory longestUnrepairedNode) + public void setSucceededTokenRangesCount(int count) { - this.longestUnrepairedNode = longestUnrepairedNode; + succeededTokenRangesCount = count; + } + + public int getSucceededTokenRangesCount() + { + return succeededTokenRangesCount; } public boolean isSuccess() diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 6100a57a1b85..5a94a1499efe 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -475,12 +475,14 @@ public void testMetrics() AutoRepair.instance.repairStates.put(repairType, autoRepairState); when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) .thenReturn(repairRunnable); - when(autoRepairState.getRepairFailedTablesCount()).thenReturn(10); + when(autoRepairState.getFailedTokenRangesCount()).thenReturn(10); + when(autoRepairState.getSucceededTokenRangesCount()).thenReturn(11); when(autoRepairState.getLongestUnrepairedSec()).thenReturn(10); AutoRepair.instance.repair(repairType, 0); assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); - assertTrue(AutoRepairMetricsManager.getMetrics(repairType).failedTablesCount.getValue() > 0); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).failedTokenRangesCount.getValue() > 0); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).succeededTokenRangesCount.getValue() > 0); assertTrue(AutoRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue() > 0); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java index 972b6339954f..19aa0a8b439c 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java @@ -247,23 +247,6 @@ public void testGetLongestUnrepairedSec() { } } - @Test - public void testSetRepairFailedTablesCount() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); - - state.setRepairFailedTablesCount(1); - - assertEquals(1, state.repairTableFailureCount); - } - - @Test - public void testGetRepairFailedTablesCount() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); - state.repairTableFailureCount = 1; - - assertEquals(1, state.getRepairFailedTablesCount()); - } - @Test public void testSetTotalMVTablesConsideredForRepair() { AutoRepairState state = RepairType.getAutoRepairState(repairType); @@ -307,40 +290,56 @@ public void testSetRepairKeyspaceCount() { assertEquals(1, state.repairKeyspaceCount); } + @Test + public void testGetRepairKeyspaceCount() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.repairKeyspaceCount = 1; + + assertEquals(1, state.getRepairKeyspaceCount()); + } @Test - public void testSetRepairTableSuccessCount() { + public void testSetLongestUnrepairedNode() { AutoRepairState state = RepairType.getAutoRepairState(repairType); + AutoRepairHistory history = new AutoRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); - state.setRepairTableSuccessCount(1); + state.setLongestUnrepairedNode(history); - assertEquals(1, state.repairTableSuccessCount); + assertEquals(history, state.longestUnrepairedNode); } @Test - public void testGetRepairKeyspaceCount() { + public void testSetSucceededTokenRangesCount() { AutoRepairState state = RepairType.getAutoRepairState(repairType); - state.repairKeyspaceCount = 1; - assertEquals(1, state.getRepairKeyspaceCount()); + state.setSucceededTokenRangesCount(1); + + assertEquals(1, state.succeededTokenRangesCount); } @Test - public void testGetRepairTableSuccessCount() { + public void testGetSucceededTokenRangesCount() { AutoRepairState state = RepairType.getAutoRepairState(repairType); - state.repairTableSuccessCount = 1; + state.succeededTokenRangesCount = 1; - assertEquals(1, state.getRepairTableSuccessCount()); + assertEquals(1, state.getSucceededTokenRangesCount()); } @Test - public void testSetLongestUnrepairedNode() { + public void testSetFailedTokenRangesCount() { AutoRepairState state = RepairType.getAutoRepairState(repairType); - AutoRepairHistory history = new AutoRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); - state.setLongestUnrepairedNode(history); + state.setFailedTokenRangesCount(1); - assertEquals(history, state.longestUnrepairedNode); + assertEquals(1, state.failedTokenRangesCount); + } + + @Test + public void testGetFailedTokenRangesCount() { + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.failedTokenRangesCount = 1; + + assertEquals(1, state.getFailedTokenRangesCount()); } @Test From d1501fa71a9a5fa1c130bfcd0420b55978f1a263 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Sat, 17 Aug 2024 10:52:47 -0700 Subject: [PATCH 008/115] Add intial_scheduler_delay option to AutoRepairConfig --- .../repair/autorepair/AutoRepair.java | 5 +---- .../repair/autorepair/AutoRepairConfig.java | 15 +++++++++++++++ .../tools/nodetool/GetAutoRepairConfig.java | 1 + .../autorepair/AutoRepairConfigTest.java | 18 ++++++++++++++++++ 4 files changed, 35 insertions(+), 4 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 41161d01c57e..c7ed1d708155 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -62,9 +62,6 @@ public class AutoRepair { - // Initial delay for repair session to start after setup - final static long INITIAL_REPAIR_DELAY_SEC = 30; - private static final Logger logger = LoggerFactory.getLogger(AutoRepair.class); @VisibleForTesting @@ -114,7 +111,7 @@ public void setup() { repairExecutors.get(repairType).scheduleWithFixedDelay( () -> repair(repairType, 60000), - INITIAL_REPAIR_DELAY_SEC, + config.getInitialSchedulerDelay(repairType).toSeconds(), config.getRepairCheckInterval().toSeconds(), TimeUnit.SECONDS); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index bd20c1d3c548..8e1c321dd1c0 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -254,6 +254,17 @@ public String getTokenRangeSplitter(RepairType repairType) return applyOverrides(repairType, opt -> opt.token_range_splitter); } + public void setInitialSchedulerDelay(RepairType repairType, String initialSchedulerDelay) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).intial_scheduler_delay = new DurationSpec.IntSecondsBound(initialSchedulerDelay); + } + + public DurationSpec.IntSecondsBound getInitialSchedulerDelay(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.intial_scheduler_delay); + } + // Options configures auto-repair behavior for a given repair type. // All fields can be modified dynamically. public static class Options implements Serializable @@ -288,6 +299,7 @@ protected static Options getDefaultOptions() opts.table_max_repair_time = new DurationSpec.IntSecondsBound("6h"); opts.mv_repair_enabled = true; opts.token_range_splitter = DefaultAutoRepairTokenSplitter.class.getName(); + opts.intial_scheduler_delay = new DurationSpec.IntSecondsBound("15m"); // 15 minutes return opts; } @@ -353,6 +365,8 @@ protected static Options getDefaultOptions() // the default is DefaultAutoRepairTokenSplitter.class.getName(). The class should implement IAutoRepairTokenRangeSplitter. // The default implementation splits the tokens based on the token ranges owned by this node divided by the number of 'number_of_subranges' public volatile String token_range_splitter; + // the minimum delay in seconds after a node starts before the scheduler starts running repair + public volatile DurationSpec.IntSecondsBound intial_scheduler_delay; public String toString() { @@ -371,6 +385,7 @@ public String toString() ", table_max_repair_time=" + table_max_repair_time + ", mv_repair_enabled=" + mv_repair_enabled + ", token_range_splitter=" + token_range_splitter + + ", intial_scheduler_delay=" + intial_scheduler_delay + '}'; } } diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java index 49ccf0f2aa24..feaf4c98ae17 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java @@ -71,6 +71,7 @@ private String formatRepairTypeConfig(NodeProbe probe, RepairType repairType, Au sb.append("\n\tnumber of parallel repairs within group: " + config.getParallelRepairCountInGroup(repairType)); sb.append("\n\tpercentage of parallel repairs within group: " + config.getParallelRepairPercentageInGroup(repairType)); sb.append("\n\tmv repair enabled: " + config.getMVRepairEnabled(repairType)); + sb.append("\n\tinitial scheduler delay: " + config.getInitialSchedulerDelay(repairType)); return sb.toString(); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java index dea0c9a827c3..aa3db7835d54 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -450,4 +450,22 @@ public void testInvalidTokenRangeSplitter() assertEquals(DefaultAutoRepairTokenSplitter.class.getName(), FBUtilities.newAutoRepairTokenRangeSplitter("invalid-class").getClass().getName()); } + @Test + public void testSetInitialSchedulerDelay() + { + config.setInitialSchedulerDelay(repairType, "5s"); + + assert config.repair_type_overrides.get(repairType).intial_scheduler_delay.toSeconds() == 5; + } + + @Test + public void testGetInitialSchedulerDelay() + { + config.global_settings.intial_scheduler_delay = new DurationSpec.IntSecondsBound("5s"); + + int result = config.getInitialSchedulerDelay(repairType).toSeconds(); + + assertEquals(5, result); + } + } From 50110b651bce9ab9c605da9f2a7f68b227b10a43 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Sun, 18 Aug 2024 10:19:20 -0700 Subject: [PATCH 009/115] Implement nodetool sstablerepairedset --- .../cassandra/service/StorageService.java | 33 +++++ .../service/StorageServiceMBean.java | 7 + .../org/apache/cassandra/tools/NodeProbe.java | 10 ++ .../org/apache/cassandra/tools/NodeTool.java | 1 + .../tools/nodetool/SSTableRepairedSet.java | 112 +++++++++++++++ .../service/StorageServiceServerTest.java | 132 ++++++++++++++++++ .../cassandra/tools/JMXStandardsTest.java | 2 + .../nodetool/SSTableRepairedSetTest.java | 114 +++++++++++++++ 8 files changed, 411 insertions(+) create mode 100644 src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java create mode 100644 test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index c84fcd5d822c..b7b6a470e5ca 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -74,6 +74,8 @@ import com.google.common.collect.Sets; import com.google.common.util.concurrent.RateLimiter; import com.google.common.util.concurrent.Uninterruptibles; + +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.repair.autorepair.AutoRepairKeyspace; import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.commons.lang3.StringUtils; @@ -5683,4 +5685,35 @@ public void setEnforceNativeDeadlineForHints(boolean value) DatabaseDescriptor.setEnforceNativeDeadlineForHints(value); } + public List getTablesForKeyspace(String keyspace) { + return Keyspace.open(keyspace).getColumnFamilyStores().stream().map(cfs -> cfs.name).collect(Collectors.toList()); + } + + public List mutateSSTableRepairedState(boolean repaired, boolean preview, String keyspace, List tableNames) throws InvalidRequestException + { + Map tables = Keyspace.open(keyspace).getColumnFamilyStores() + .stream().collect(Collectors.toMap(c -> c.name, c -> c)); + for (String tableName : tableNames) { + if (!tables.containsKey(tableName)) + throw new InvalidRequestException("Table " + tableName + " does not exist in keyspace " + keyspace); + } + + // only select SSTables that are unrepaired when repaired is true and vice versa + Predicate predicate = sst -> repaired != sst.isRepaired(); + + // mutate SSTables + long repairedAt = !repaired ? 0 : currentTimeMillis(); + List sstablesTouched = new ArrayList<>(); + for (String tableName : tableNames) { + ColumnFamilyStore table = tables.get(tableName); + Set result = table.runWithCompactionsDisabled(() -> { + Set sstables = table.getLiveSSTables().stream().filter(predicate).collect(Collectors.toSet()); + if (!preview) + table.getCompactionStrategyManager().mutateRepaired(sstables, repairedAt, null, false); + return sstables; + }, predicate, true, false, true); + sstablesTouched.addAll(result.stream().map(sst -> sst.descriptor.baseFilename()).collect(Collectors.toList())); + } + return sstablesTouched; + } } diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index f23b3b454382..445e4946d1ef 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -36,6 +36,7 @@ import org.apache.cassandra.db.ColumnFamilyStoreMBean; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.utils.BreaksJMX; +import org.apache.cassandra.exceptions.InvalidRequestException; public interface StorageServiceMBean extends NotificationEmitter { @@ -1294,4 +1295,10 @@ public void enableAuditLog(String loggerName, String includedKeyspaces, String e boolean getEnforceNativeDeadlineForHints(); void setEnforceNativeDeadlineForHints(boolean value); + + /** Gets the names of all tables for the given keyspace */ + public List getTablesForKeyspace(String keyspace); + + /** Mutates the repaired state of all SSTables for the given SSTables */ + public List mutateSSTableRepairedState(boolean repaired, boolean preview, String keyspace, List tables) throws InvalidRequestException; } diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index f9ccb037fc8e..622f53618e69 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -82,6 +82,7 @@ import org.apache.cassandra.db.compaction.CompactionManagerMBean; import org.apache.cassandra.db.virtual.CIDRFilteringMetricsTable; import org.apache.cassandra.db.virtual.CIDRFilteringMetricsTableMBean; +import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.fql.FullQueryLoggerOptions; import org.apache.cassandra.fql.FullQueryLoggerOptionsCompositeData; import org.apache.cassandra.gms.FailureDetector; @@ -2490,6 +2491,15 @@ public void setMVRepairEnabled(AutoRepairConfig.RepairType repairType, boolean e { autoRepairProxy.setMVRepairEnabled(repairType, enabled); } + + public List mutateSSTableRepairedState(boolean repair, boolean preview, String keyspace, List tables) throws InvalidRequestException + { + return ssProxy.mutateSSTableRepairedState(repair, preview, keyspace, tables); + } + + public List getTablesForKeyspace(String keyspace) { + return ssProxy.getTablesForKeyspace(keyspace); + } } class ColumnFamilyStoreMBeanIterator implements Iterator> diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index 707a9e0c64f1..02f71468f7d2 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -218,6 +218,7 @@ public int execute(String... args) SetTraceProbability.class, Sjk.class, Snapshot.class, + SSTableRepairedSet.class, Status.class, StatusAutoCompaction.class, StatusBackup.class, diff --git a/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java b/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java new file mode 100644 index 000000000000..32ee09f93590 --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.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.tools.nodetool; + +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.List; + + +import io.airlift.airline.Arguments; +import io.airlift.airline.Command; +import io.airlift.airline.Option; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.NodeTool; + +@Command(name = "sstablerepairedset", description = "Set the repaired state of SSTables for given keyspace/tables") +public class SSTableRepairedSet extends NodeTool.NodeToolCmd +{ + @Arguments(usage = " []", description = "The keyspace optionally followed by one or more tables", required = true) + protected List args = new ArrayList<>(); + + @Option(title = "really-set", + name = { "--really-set" }, + description = "Really set the repaired state of SSTables. If not set, only print SSTables that would be affected.") + protected boolean reallySet = false; + + @Option(title = "is-repaired", + name = { "--is-repaired" }, + description = "Set SSTables to repaired state.") + protected boolean isRepaired = false; + + @Option(title = "is-unrepaired", + name = { "--is-unrepaired" }, + description = "Set SSTables to unrepaired state.") + protected boolean isUnrepaired = false; + + @Override + public void execute(NodeProbe probe) + { + PrintStream out = probe.output().out; + + String message; + if (reallySet) + { + message = "Mutating repaired state of SSTables for"; + } + else + { + message = "Previewing repaired state mutation of SSTables for"; + } + + if (args.isEmpty()) + { + out.println("At least a keyspace name must be provided."); + return; + } + String keyspace = args.get(0); + + List tables; + if (args.size() > 1) + { + tables = args.subList(1, args.size()); + message += " tables " + String.join(", ", tables) + " in"; + } + else + { + tables = probe.getTablesForKeyspace(keyspace); + message += " all tables in"; + } + message += " keyspace " + keyspace; + + if (isRepaired == isUnrepaired) + { + out.println("Exactly one of --is-repaired or --is-unrepaired must be provided."); + return; + } + message += " to " + (isRepaired ? "repaired" : "unrepaired"); + out.println(message); + + try + { + List mutatedSSTables = probe.mutateSSTableRepairedState(isRepaired, !reallySet, keyspace, new ArrayList<>(tables)); + if (!reallySet) + out.println("The following SSTables would be mutated:"); + else + out.println("The following SSTables were mutated:"); + for (String sstable : mutatedSSTables) + out.println(sstable); + } + catch (InvalidRequestException e) + { + out.println(e.getMessage()); + } + } +} diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java index 87534135801a..b33af747884d 100644 --- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java +++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.net.UnknownHostException; + import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -29,6 +30,17 @@ import java.util.UUID; import com.google.common.collect.Sets; +import java.util.*; +import java.util.stream.Collectors; + +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.SystemKeyspace; + import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -46,6 +58,11 @@ import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.AbstractNetworkTopologySnitch; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.gms.ApplicationState; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.gms.VersionedValue; import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.WithPartitioner; @@ -66,10 +83,17 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.GOSSIP_DISABLE_THREAD_VALIDATION; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + public class StorageServiceServerTest { + public static final String keyspace = "test_keyspace"; + public static ColumnFamilyStore table1; + public static ColumnFamilyStore table2; + static final String DC1 = "DC1"; static final String DC2 = "DC2"; static final String RACK = "rack1"; @@ -119,6 +143,21 @@ private Location location(InetAddressAndPort endpoint) id5 = InetAddressAndPort.getByName("127.0.0.5"); registerNodes(); ServerTestUtils.markCMS(); + + SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), + SchemaLoader.standardCFMD(keyspace, "table1").build(), + SchemaLoader.standardCFMD(keyspace, "table2").build()); + table1 = Keyspace.open(keyspace).getColumnFamilyStore("table1"); + assert table1 != null; + table2 = Keyspace.open(keyspace).getColumnFamilyStore("table2"); + assert table2 != null; + } + + @Before + public void clearData() + { + table1.truncateBlocking(); + table2.truncateBlocking(); } private static void registerNodes() @@ -614,4 +653,97 @@ public void testAuditLogEnableLoggerTransitions() throws Exception assertTrue(AuditLogManager.instance.isEnabled()); StorageService.instance.disableAuditLog(); } + + @Test + public void testGetTablesForKeyspace() + { + List result = StorageService.instance.getTablesForKeyspace(keyspace); + + assertEquals(Arrays.asList(table1.name, table2.name), result.stream().sorted().collect(Collectors.toList())); + } + + @Test + public void testGetTablesForKeyspaceNotFound() + { + String missingKeyspace = "MISSING_KEYSPACE"; + try + { + StorageService.instance.getTablesForKeyspace(missingKeyspace); + fail("Expected an AssertionError to be thrown"); + } + catch (AssertionError e) + { + assertEquals("Unknown keyspace " + missingKeyspace, e.getMessage()); + } + } + + @Test + public void testMutateSSTableRepairedStateTableNotFound() + { + try + { + StorageService.instance.mutateSSTableRepairedState(true, false, keyspace, Arrays.asList("MISSING_TABLE")); + fail("Expected an InvalidRequestException to be thrown"); + } + catch (InvalidRequestException e) + { + // Test passed + } + } + + @Test + public void testMutateSSTableRepairedStateTablePreview() + { + SchemaLoader.insertData(keyspace, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + assertEquals(1, table1.getLiveSSTables().size()); + + List result = StorageService.instance.mutateSSTableRepairedState(true, true, keyspace, Arrays.asList(table1.name)); + + assertEquals(1, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFilename())); + }); + } + + @Test + public void testMutateSSTableRepairedStateTableRepaired() + { + SchemaLoader.insertData(keyspace, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + SchemaLoader.insertData(keyspace, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + assertEquals(2, table1.getLiveSSTables().size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + }); + + List result = StorageService.instance.mutateSSTableRepairedState(true, false, keyspace, Arrays.asList(table1.name)); + + assertEquals(2, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertTrue(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFilename())); + }); + } + + @Test + public void testMutateSSTableRepairedStateTableUnrepaired() throws Exception + { + SchemaLoader.insertData(keyspace, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + SchemaLoader.insertData(keyspace, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + table1.getCompactionStrategyManager().mutateRepaired(table1.getLiveSSTables(), 1, null, false); + assertEquals(2, table1.getLiveSSTables().stream().filter(SSTableReader::isRepaired).count()); + + List result = StorageService.instance.mutateSSTableRepairedState(false, false, keyspace, Arrays.asList(table1.name)); + + assertEquals(2, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFilename())); + }); + } } diff --git a/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java b/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java index 87b9ff93fded..62b6b9b5ec16 100644 --- a/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java +++ b/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java @@ -50,6 +50,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.utils.BreaksJMX; import org.assertj.core.api.Assertions; import org.reflections.Reflections; @@ -98,6 +99,7 @@ public class JMXStandardsTest .add(IllegalStateException.class) .add(ClassNotFoundException.class) .add(OpenDataException.class) + .add(InvalidRequestException.class) .build(); /** * This list is a set of types under java.* and javax.*, but are too vague that could cause issues; this does not diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java new file mode 100644 index 000000000000..0a2ead81b778 --- /dev/null +++ b/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.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.tools.nodetool; + +import java.io.OutputStream; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Arrays; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.Output; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class SSTableRepairedSetTest +{ + @Mock + private NodeProbe probe; + + private SSTableRepairedSet cmd; + + @Before + public void setUp() + { + MockitoAnnotations.initMocks(this); + PrintStream noopStream = new PrintStream(new OutputStream() + { + @Override + public void write(int b) + { + } + }); + when(probe.output()).thenReturn(new Output(noopStream, noopStream)); + cmd = new SSTableRepairedSet(); + } + + @Test + public void testNoKeyspace() + { + cmd.execute(probe); + verify(probe, never()).mutateSSTableRepairedState(anyBoolean(), anyBoolean(), anyString(), anyList()); + } + + @Test + public void testBothRepairedAndUnrepaired() + { + cmd.args = Arrays.asList("keyspace"); + cmd.execute(probe); + verify(probe, never()).mutateSSTableRepairedState(anyBoolean(), anyBoolean(), anyString(), anyList()); + } + + @Test + public void testNeitherRepairedNorUnrepaired() + { + cmd.args = Arrays.asList("keyspace"); + cmd.execute(probe); + verify(probe, never()).mutateSSTableRepairedState(anyBoolean(), anyBoolean(), anyString(), anyList()); + } + + @Test + public void testRepairedPreview() + { + cmd.args = Arrays.asList("keyspace"); + cmd.isRepaired = true; + cmd.execute(probe); + verify(probe).mutateSSTableRepairedState(true, true, "keyspace", new ArrayList<>()); + } + + @Test + public void testUnrepairedReallySet() + { + cmd.args = Arrays.asList("keyspace"); + cmd.isUnrepaired = true; + cmd.reallySet = true; + cmd.execute(probe); + verify(probe).mutateSSTableRepairedState(false, false, "keyspace", new ArrayList<>()); + } + + @Test + public void testExecuteWithTableNames() + { + cmd.args = Arrays.asList("keyspace", "table1", "table2"); + cmd.isRepaired = true; + cmd.reallySet = true; + cmd.execute(probe); + verify(probe).mutateSSTableRepairedState(true, false, "keyspace", Arrays.asList("table1", "table2")); + } +} From cfc6bdea5b05f19e3dc3c14a742af23d1df28e7e Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Sun, 25 Aug 2024 14:07:59 -0700 Subject: [PATCH 010/115] Change SkippedTablesCount metric to SkippedTokenRangesCount --- .../cassandra/metrics/AutoRepairMetrics.java | 7 +++--- .../repair/autorepair/AutoRepair.java | 22 +++++++++-------- .../repair/autorepair/AutoRepairConfig.java | 10 ++++---- .../repair/autorepair/AutoRepairState.java | 24 +++++++++---------- .../autorepair/AutoRepairConfigTest.java | 4 ++-- .../AutoRepairParameterizedTest.java | 19 +++++++-------- .../autorepair/AutoRepairStateTest.java | 12 +++++----- 7 files changed, 49 insertions(+), 49 deletions(-) diff --git a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java index 426034ffcd4b..d765899631b7 100644 --- a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java +++ b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java @@ -34,10 +34,11 @@ public class AutoRepairMetrics public Gauge repairsInProgress; public Gauge nodeRepairTimeInSec; public Gauge clusterRepairTimeInSec; - public Gauge skippedTablesCount; public Gauge longestUnrepairedSec; public Gauge succeededTokenRangesCount; public Gauge failedTokenRangesCount; + public Gauge skippedTokenRangesCount; + public Counter repairTurnMyTurn; public Counter repairTurnMyTurnDueToPriority; public Counter repairTurnMyTurnForceRepair; @@ -72,11 +73,11 @@ public Integer getValue() } }); - skippedTablesCount = Metrics.register(factory.createMetricName("SkippedTablesCount"), new Gauge() + skippedTokenRangesCount = Metrics.register(factory.createMetricName("SkippedTokenRangesCount"), new Gauge() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).getRepairSkippedTablesCount(); + return AutoRepair.instance.getRepairState(repairType).getSkippedTokenRangesCount(); } }); diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index c7ed1d708155..937370720a32 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -177,13 +177,13 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) AutoRepairUtils.updateStartAutoRepairHistory(repairType, myId, timeFunc.get(), turn); repairState.setRepairKeyspaceCount(0); - repairState.setRepairSkippedTablesCount(0); repairState.setRepairInProgress(true); repairState.setTotalTablesConsideredForRepair(0); repairState.setTotalMVTablesConsideredForRepair(0); int failedTokenRanges = 0; int succeededTokenRanges = 0; + int skippedTokenRanges = 0; List keyspaces = new ArrayList<>(); Keyspace.all().forEach(keyspaces::add); @@ -206,6 +206,9 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) String keyspaceName = keyspace.getName(); try { + List> subRangesToBeRepaired = tokenRangeSplitters.get(repairType).getRange(repairType, primaryRangeOnly, keyspaceName, tableName); + int totalSubRanges = subRangesToBeRepaired.size(); + ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName); if (!columnFamilyStore.metadata().params.automatedRepair.get(repairType).repairEnabled()) { @@ -220,7 +223,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) { logger.info("Too many SSTables for repair, not doing repair on table {}.{} " + "totalSSTables {}", keyspaceName, tableName, columnFamilyStore.getLiveSSTables().size()); - repairState.setRepairSkippedTablesCount(repairState.getRepairSkippedTablesCount() + 1); + skippedTokenRanges += totalSubRanges; continue; } @@ -234,8 +237,6 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) } long tableStartTime = timeFunc.get(); Set> ranges = new HashSet<>(); - List> subRangesToBeRepaired = tokenRangeSplitters.get(repairType).getRange(repairType, primaryRangeOnly, keyspaceName, tableName); - int totalSubRanges = subRangesToBeRepaired.size(); int totalProcessedSubRanges = 0; for (Pair token : subRangesToBeRepaired) { @@ -250,7 +251,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) { if (AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, tableStartTime, tablesToBeRepaired.size())) { - repairState.setRepairSkippedTablesCount(repairState.getRepairSkippedTablesCount() + tablesToBeRepaired.size()); + skippedTokenRanges += totalSubRanges - totalProcessedSubRanges; logger.info("Keyspace took too much time to repair hence skipping it {}", keyspaceName); break; @@ -260,7 +261,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) { if (AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, tableStartTime)) { - repairState.setRepairSkippedTablesCount(repairState.getRepairSkippedTablesCount() + 1); + skippedTokenRanges += totalSubRanges - totalProcessedSubRanges; logger.info("Table took too much time to repair hence skipping it {}.{}", keyspaceName, tableName); break; @@ -326,7 +327,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) } } } - cleanupAndUpdateStats(turn, repairType, repairState, myId, startTime, millisToWait, failedTokenRanges, succeededTokenRanges); + cleanupAndUpdateStats(turn, repairType, repairState, myId, startTime, millisToWait, failedTokenRanges, succeededTokenRanges, skippedTokenRanges); } else { @@ -385,7 +386,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon } private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, UUID myId, - long startTime, long millisToWait, int failedTokenRanges, int succeededTokenRanges) throws InterruptedException + long startTime, long millisToWait, int failedTokenRanges, int succeededTokenRanges, int skippedTokenRanges) throws InterruptedException { //if it was due to priority then remove it now if (turn == MY_TURN_DUE_TO_PRIORITY) @@ -396,13 +397,14 @@ private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType repairState.setFailedTokenRangesCount(failedTokenRanges); repairState.setSucceededTokenRangesCount(succeededTokenRanges); + repairState.setSkippedTokenRangesCount(skippedTokenRanges); repairState.setNodeRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - startTime)); long timeInHours = TimeUnit.SECONDS.toHours(repairState.getNodeRepairTimeInSec()); logger.info("Local {} repair time {} hour(s), stats: repairKeyspaceCount {}, " + "repairTokenRangesSuccessCount {}, repairTokenRangesFailureCount {}, " + - "repairTableSkipCount {}", repairType, timeInHours, repairState.getRepairKeyspaceCount(), + "repairTokenRangesSkipCount {}", repairType, timeInHours, repairState.getRepairKeyspaceCount(), repairState.getSucceededTokenRangesCount(), repairState.getFailedTokenRangesCount(), - repairState.getRepairSkippedTablesCount()); + repairState.getSkippedTokenRangesCount()); if (repairState.getLastRepairTime() != 0) { repairState.setClusterRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index 8e1c321dd1c0..5cbb4f9f5bb6 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -257,12 +257,12 @@ public String getTokenRangeSplitter(RepairType repairType) public void setInitialSchedulerDelay(RepairType repairType, String initialSchedulerDelay) { ensureOverrides(repairType); - repair_type_overrides.get(repairType).intial_scheduler_delay = new DurationSpec.IntSecondsBound(initialSchedulerDelay); + repair_type_overrides.get(repairType).initial_scheduler_delay = new DurationSpec.IntSecondsBound(initialSchedulerDelay); } public DurationSpec.IntSecondsBound getInitialSchedulerDelay(RepairType repairType) { - return applyOverrides(repairType, opt -> opt.intial_scheduler_delay); + return applyOverrides(repairType, opt -> opt.initial_scheduler_delay); } // Options configures auto-repair behavior for a given repair type. @@ -299,7 +299,7 @@ protected static Options getDefaultOptions() opts.table_max_repair_time = new DurationSpec.IntSecondsBound("6h"); opts.mv_repair_enabled = true; opts.token_range_splitter = DefaultAutoRepairTokenSplitter.class.getName(); - opts.intial_scheduler_delay = new DurationSpec.IntSecondsBound("15m"); // 15 minutes + opts.initial_scheduler_delay = new DurationSpec.IntSecondsBound("15m"); // 15 minutes return opts; } @@ -366,7 +366,7 @@ protected static Options getDefaultOptions() // The default implementation splits the tokens based on the token ranges owned by this node divided by the number of 'number_of_subranges' public volatile String token_range_splitter; // the minimum delay in seconds after a node starts before the scheduler starts running repair - public volatile DurationSpec.IntSecondsBound intial_scheduler_delay; + public volatile DurationSpec.IntSecondsBound initial_scheduler_delay; public String toString() { @@ -385,7 +385,7 @@ public String toString() ", table_max_repair_time=" + table_max_repair_time + ", mv_repair_enabled=" + mv_repair_enabled + ", token_range_splitter=" + token_range_splitter + - ", intial_scheduler_delay=" + intial_scheduler_delay + + ", intial_scheduler_delay=" + initial_scheduler_delay + '}'; } } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index d51ebde50ec9..d7a473cab1df 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -76,8 +76,6 @@ public abstract class AutoRepairState implements ProgressListener @VisibleForTesting protected int repairKeyspaceCount = 0; @VisibleForTesting - protected int repairTableSkipCount = 0; - @VisibleForTesting protected int totalMVTablesConsideredForRepair = 0; @VisibleForTesting protected int totalDisabledTablesRepairCount = 0; @@ -87,6 +85,8 @@ public abstract class AutoRepairState implements ProgressListener @VisibleForTesting protected int succeededTokenRangesCount = 0; @VisibleForTesting + protected int skippedTokenRangesCount = 0; + @VisibleForTesting protected AutoRepairHistory longestUnrepairedNode; @VisibleForTesting protected Condition condition = newOneTimeCondition(); @@ -184,16 +184,6 @@ public boolean isRepairInProgress() return repairInProgress; } - public void setRepairSkippedTablesCount(int count) - { - repairTableSkipCount = count; - } - - public int getRepairSkippedTablesCount() - { - return repairTableSkipCount; - } - public int getLongestUnrepairedSec() { if (longestUnrepairedNode == null) @@ -258,6 +248,16 @@ public int getSucceededTokenRangesCount() return succeededTokenRangesCount; } + public void setSkippedTokenRangesCount(int count) + { + skippedTokenRangesCount = count; + } + + public int getSkippedTokenRangesCount() + { + return skippedTokenRangesCount; + } + public boolean isSuccess() { return success; diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java index aa3db7835d54..923e35b73796 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -455,13 +455,13 @@ public void testSetInitialSchedulerDelay() { config.setInitialSchedulerDelay(repairType, "5s"); - assert config.repair_type_overrides.get(repairType).intial_scheduler_delay.toSeconds() == 5; + assert config.repair_type_overrides.get(repairType).initial_scheduler_delay.toSeconds() == 5; } @Test public void testGetInitialSchedulerDelay() { - config.global_settings.intial_scheduler_delay = new DurationSpec.IntSecondsBound("5s"); + config.global_settings.initial_scheduler_delay = new DurationSpec.IntSecondsBound("5s"); int result = config.getInitialSchedulerDelay(repairType).toSeconds(); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 5a94a1499efe..4bbc77443e51 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -65,7 +65,6 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.progress.ProgressEvent; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; @@ -97,8 +96,6 @@ public class AutoRepairParameterizedTest extends CQLTester @Mock ScheduledExecutorPlus mockExecutor; @Mock - ProgressEvent progressEvent; - @Mock AutoRepairState autoRepairState; @Mock RepairCoordinator repairRunnable; @@ -417,15 +414,15 @@ public void testSkipRepairSSTableCountHigherThreshold() int beforeCount = config.getRepairSSTableCountHigherThreshold(repairType); config.setMVRepairEnabled(repairType, true); config.setRepairSSTableCountHigherThreshold(repairType, 9); - assertEquals(0, state.getRepairSkippedTablesCount()); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + assertEquals(0, state.getSkippedTokenRangesCount()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); state.setLastRepairTime(0); AutoRepair.instance.repair(repairType, 0); assertEquals(1, state.getTotalMVTablesConsideredForRepair()); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); // skipping one time for the base table and another time for MV table - assertEquals(2, state.getRepairSkippedTablesCount()); - assertEquals(2, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + assertEquals(2, state.getSkippedTokenRangesCount()); + assertEquals(2, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); // set it to higher value, and this time, the tables should not be skipped config.setRepairSSTableCountHigherThreshold(repairType, 11); @@ -433,9 +430,9 @@ public void testSkipRepairSSTableCountHigherThreshold() state.setLastRepairTime(0); AutoRepair.instance.repair(repairType, 0); assertEquals(1, state.getTotalMVTablesConsideredForRepair()); - assertEquals(0, state.getRepairSkippedTablesCount()); + assertEquals(0, state.getSkippedTokenRangesCount()); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); } @Test @@ -468,7 +465,7 @@ public void testMetrics() assertTrue(AutoRepairMetricsManager.getMetrics(repairType).nodeRepairTimeInSec.getValue() > 0); assertTrue(AutoRepairMetricsManager.getMetrics(repairType).clusterRepairTimeInSec.getValue() > 0); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).repairTurnMyTurn.getCount()); - assertTrue(AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue() > 0); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue() > 0); assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue().intValue()); config.setAutoRepairTableMaxRepairTime(repairType, String.valueOf(Integer.MAX_VALUE-1) + 's'); @@ -480,7 +477,7 @@ public void testMetrics() when(autoRepairState.getLongestUnrepairedSec()).thenReturn(10); AutoRepair.instance.repair(repairType, 0); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); assertTrue(AutoRepairMetricsManager.getMetrics(repairType).failedTokenRangesCount.getValue() > 0); assertTrue(AutoRepairMetricsManager.getMetrics(repairType).succeededTokenRangesCount.getValue() > 0); assertTrue(AutoRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue() > 0); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java index 19aa0a8b439c..340a21f97bfa 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java @@ -203,20 +203,20 @@ public void testIsRepairInProgress() { } @Test - public void testSetRepairSkippedTablesCount() { + public void testSetSkippedTokenRangesCount() { AutoRepairState state = RepairType.getAutoRepairState(repairType); - state.setRepairSkippedTablesCount(1); + state.setSkippedTokenRangesCount(1); - assertEquals(1, state.repairTableSkipCount); + assertEquals(1, state.skippedTokenRangesCount); } @Test - public void testGetRepairSkippedTablesCount() { + public void testGetSkippedTokenRangesCount() { AutoRepairState state = RepairType.getAutoRepairState(repairType); - state.repairTableSkipCount = 1; + state.skippedTokenRangesCount = 1; - assertEquals(1, state.getRepairSkippedTablesCount()); + assertEquals(1, state.getSkippedTokenRangesCount()); } @Test From 027171852e312055692f516df73542d2dc0aa714 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Mon, 26 Aug 2024 15:12:41 -0700 Subject: [PATCH 011/115] config description --- .../pages/managing/operating/metrics.adoc | 21 +++++----- .../repair/autorepair/AutoRepairConfig.java | 42 +++++++++++-------- 2 files changed, 35 insertions(+), 28 deletions(-) diff --git a/doc/modules/cassandra/pages/managing/operating/metrics.adoc b/doc/modules/cassandra/pages/managing/operating/metrics.adoc index bce50953e01e..ab4eb7c37231 100644 --- a/doc/modules/cassandra/pages/managing/operating/metrics.adoc +++ b/doc/modules/cassandra/pages/managing/operating/metrics.adoc @@ -1085,9 +1085,9 @@ Metrics specifc to automated repair. Reported name format: *Metric Name*:: -`org.apache.cassandra.metrics.AutoRepairMetrics.` +`org.apache.cassandra.metrics.AutoRepair.` *JMX MBean*:: -`org.apache.cassandra.metrics:type=AutoRepairMetrics name=` +`org.apache.cassandra.metrics:type=AutoRepair name= repairType=` [cols=",,",options="header",] |=== @@ -1096,19 +1096,20 @@ Reported name format: on the node |NodeRepairTimeInSec |Gauge |Time taken to repair -the node +the node in seconds |ClusterRepairTimeInSec |Gauge |Time taken to repair -the entire Cassandra cluster - -|SkippedTablesCount |Gauge |Number of tables skipped -on the node +the entire Cassandra cluster in seconds |LongestUnrepairedSec |Gauge |Time since the last repair -ran on the node +ran on the node in seconds + +|SucceededTokenRangesCount |Gauge |Number of token ranges successfully repaired on the node -|FailedTablesCount |Gauge |Number of tables encountered -failure during repair on the node +|FailedTokenRangesCount |Gauge |Number of token ranges failed to repair on the node + +|SkippedTokenRangesCount |Gauge |Number of token ranges skipped +on the node |TotalMVTablesConsideredForRepair |Gauge |Number of materialized views considered on the node diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index 5cbb4f9f5bb6..9522d35bcdc8 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -37,10 +37,16 @@ public class AutoRepairConfig implements Serializable { // enable/disable auto repair globally, overrides all other settings. Cannot be modified dynamically. + // if it is set to false, then no repair will be scheduled, including full and incremental repairs by this framework. + // if it is set to true, then this repair scheduler will consult another config available for each RepairType, and based on that config, it will schedule repairs. public final Boolean enabled; - // the interval in seconds between checks for eligible repair operations. Cannot be modified dynamically. + // the interval between successive checks for repair scheduler to check if either the ongoing repair is completed or if + // none is going, then check if it's time to schedule or wait public final DurationSpec.IntSecondsBound repair_check_interval = new DurationSpec.IntSecondsBound("5m"); - // configures how long repair history is kept for a replaced node + // when any nodes leave the ring then the repair schedule needs to adjust the order, etc. + // the repair scheduler keeps the deleted hosts information in its persisted metadata for the defined interval in this config. + // This information is useful so the scheduler is absolutely sure that the node is indeed removed from the ring, and then it can adjust the repair schedule accordingly. + // So, the duration in this config determinses for how long deleted host's information is kept in the scheduler's metadata. public volatile DurationSpec.IntSecondsBound history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound("2h"); // global_settings overides Options.defaultOptions for all repair types public volatile Options global_settings; @@ -287,7 +293,7 @@ protected static Options getDefaultOptions() opts.enabled = false; opts.repair_by_keyspace = false; - opts.number_of_subranges = 1; + opts.number_of_subranges = 16; opts.number_of_repair_threads = 1; opts.parallel_repair_count_in_group = 1; opts.parallel_repair_percentage_in_group = 0; @@ -316,32 +322,32 @@ protected static Options getDefaultOptions() // If you do not use v-nodes or the number of v-nodes is pretty small, say 8, setting this value to a higher number, say 16, will be useful to repair on a smaller range, and the chance of succeeding is higher. public volatile Integer number_of_subranges; // the number of repair threads to run for a given invoked Repair Job. - // Once the scheduler schedules one Job, then howmany threads to use inside that job will be controlled through this parameter. + // Once the scheduler schedules one repair session, then howmany threads to use inside that job will be controlled through this parameter. // This is similar to -j for repair options for the nodetool repair command. public volatile Integer number_of_repair_threads; // the number of repair sessions that can run in parallel in a single group // The number of nodes running repair parallelly. If parallelrepaircount is set, it will choose the larger value of the two. The default is 3. // This configuration controls how many nodes would run repair in parallel. - // The value “3” means, at any given point in time, at most 3 nodes would be running repair in parallel. + // The value “3” means, at any given point in time, at most 3 nodes would be running repair in parallel. These selected nodes can be from any datacenters. // If one or more node(s) finish repair, then the framework automatically picks up the next candidate and ensures the maximum number of nodes running repair do not exceed “3”. public volatile Integer parallel_repair_count_in_group; // the number of repair sessions that can run in parallel in a single groupas a percentage // of the total number of nodes in the group [0,100] // The percentage of nodes in the cluster that run repair parallelly. If parallelrepaircount is set, it will choose the larger value of the two. - //The problem with a fixed number of nodes (the above property) is that in a large-scale environment, + // The problem with a fixed number of nodes (the above property) is that in a large-scale environment, // the nodes keep getting added/removed due to elasticity, so if we have a fixed number, then manual interventions would increase because, on a continuous basis,operators would have to adjust to meet the SLA. - //The default is 3%, which means that 3% of the nodes in the Cassandra cluster would be repaired in parallel. + // The default is 3%, which means that 3% of the nodes in the Cassandra cluster would be repaired in parallel. // So now, if a fleet, an operator won't have to worry about changing the repair frequency, etc., as overall repair time will continue to remain the same even if nodes are added or removed due to elasticity. // Extremely fewer manual interventions as it will rarely violate the repair SLA for customers public volatile Integer parallel_repair_percentage_in_group; // the upper threshold of SSTables allowed to participate in a single repair session // Threshold to skip a table if it has too many sstables. The default is 10000. This means, if a table on a node has 10000 or more SSTables, then that table will be skipped. - // This is to avoid penalizing good neighbors with an outlier. + // This is to avoid penalizing good tables (neighbors) with an outlier. public volatile Integer sstable_upper_threshold; - // the minimum time in hours between repairs of the same token range - // The minimum number of hours to run one repair cycle is 24 hours. The default is 24 hours. - // This means that if auto repair finishes one round on one cluster within 24 hours, it won’t start a new round. - // This is applicable for extremely tiny clusters, say 3 nodes. + // the minimum time in hours between repairing the same node again. This is useful for extremely tiny clusters, say 5 nodes, which finishes + // repair quicly. + // The default is 24 hours. This means that if the scheduler finishes one round on all the nodes in < 24 hours. On a given node it won’t start a new repair round + // until the last repair conducted on a given node is < 24 hours. public volatile DurationSpec.IntSecondsBound min_repair_interval; // specifies a denylist of datacenters to repair // This is useful if you want to completely avoid running repairs in one or more data centers. By default, it is empty, i.e., the framework will repair nodes in all the datacenters. @@ -350,22 +356,22 @@ protected static Options getDefaultOptions() // It is the same as -pr in nodetool repair options. public volatile Boolean repair_primary_token_range_only; // configures whether to force immediate repair on new nodes + // default it is set to 'false'; this is useful if you want to repair new nodes immediately after they join the ring. public volatile Boolean force_repair_new_node; - // the maximum time in seconds that a repair session can run for a single table - // Max time for repairing one table, if exceeded, skip the table. The default is 6 * 60 * 60, which is 6 hours. + // the maximum time that a repair session can run for a single table + // Max time for repairing one table on a given node, if exceeded, skip the table. The default is 6 hours. // Let's say there is a Cassandra cluster in that there are 10 tables belonging to 10 different customers. // Out of these 10 tables, 1 table is humongous. Repairing this 1 table, say, takes 5 days, in the worst case, but others could finish in just 1 hour. - // Then we would penalize 9 customers just because of one bad actor, and those 9 customers would ping an operator telling them they are violating SLA even if I am a neighbor, and it would require a lot of back-and-forth manual interventions, etc. + // Then we would penalize 9 customers just because of one bad actor, and those 9 customers would ping an operator and would require a lot of back-and-forth manual interventions, etc. // So, the idea here is to penalize the outliers instead of good candidates. This can easily be configured with a higher value if we want to disable the functionality. - // Please note the repair will still run in parallel on other nodes, this is to address outliers on a given node. public volatile DurationSpec.IntSecondsBound table_max_repair_time; - // the default is 'true'. MVs are mutated at LOCAL_ONE consistency level in Cassandra. + // the default is 'true'. // This flag determines whether the auto-repair framework needs to run anti-entropy, a.k.a, repair on the MV table or not. public volatile Boolean mv_repair_enabled; // the default is DefaultAutoRepairTokenSplitter.class.getName(). The class should implement IAutoRepairTokenRangeSplitter. // The default implementation splits the tokens based on the token ranges owned by this node divided by the number of 'number_of_subranges' public volatile String token_range_splitter; - // the minimum delay in seconds after a node starts before the scheduler starts running repair + // the minimum delay after a node starts before the scheduler starts running repair public volatile DurationSpec.IntSecondsBound initial_scheduler_delay; public String toString() From 99acef9698cf77892ef5ec1f3e4f08df469e7da6 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Wed, 28 Aug 2024 20:45:16 -0700 Subject: [PATCH 012/115] Implement repair retries for auto-repair scheduler --- .../repair/autorepair/AutoRepair.java | 62 +++- .../repair/autorepair/AutoRepairConfig.java | 25 ++ .../repair/autorepair/AutoRepairUtils.java | 2 +- .../cassandra/service/AutoRepairService.java | 12 + .../service/AutoRepairServiceMBean.java | 2 + .../cassandra/service/StorageService.java | 4 +- .../org/apache/cassandra/tools/NodeProbe.java | 10 + .../tools/nodetool/GetAutoRepairConfig.java | 2 + .../tools/nodetool/SetAutoRepairConfig.java | 31 +- .../AutoRepairParameterizedTest.java | 57 ++++ .../service/AutoRepairServiceTest.java | 203 ++++++++++++ .../service/StorageServiceServerTest.java | 6 +- .../nodetool/SetAutoRepairConfigTest.java | 312 ++++++++++++++++++ 13 files changed, 695 insertions(+), 33 deletions(-) create mode 100644 test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java create mode 100644 test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 937370720a32..2fa1db89c386 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -27,11 +27,13 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Supplier; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.Uninterruptibles; import org.apache.cassandra.repair.RepairCoordinator; import org.apache.cassandra.service.StorageService; @@ -80,6 +82,9 @@ public class AutoRepair @VisibleForTesting protected static Consumer> shuffleFunc = java.util.Collections::shuffle; + @VisibleForTesting + protected static BiConsumer sleepFunc = Uninterruptibles::sleepUninterruptibly; + protected final Map tokenRangeSplitters = new EnumMap<>(AutoRepairConfig.RepairType.class); @@ -277,20 +282,38 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) if ((totalProcessedSubRanges % config.getRepairThreads(repairType) == 0) || (totalProcessedSubRanges == totalSubRanges)) { - RepairCoordinator task = repairState.getRepairRunnable(keyspaceName, - config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : ImmutableList.of(tableName), - ranges, primaryRangeOnly); - repairState.resetWaitCondition(); - Future f = repairRunnableExecutors.get(repairType).submit(task); - try + int retryCount = 0; + Future f = null; + while (retryCount <= config.getRepairMaxRetries()) { - repairState.waitForRepairToComplete(); + RepairCoordinator task = repairState.getRepairRunnable(keyspaceName, + config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : ImmutableList.of(tableName), + ranges, primaryRangeOnly); + repairState.resetWaitCondition(); + f = repairRunnableExecutors.get(repairType).submit(task); + try + { + repairState.waitForRepairToComplete(); + } + catch (InterruptedException e) + { + logger.error("Exception in cond await:", e); + } + if (repairState.isSuccess()) + { + break; + } + else if (retryCount < config.getRepairMaxRetries()) + { + boolean cancellationStatus = f.cancel(true); + logger.warn("Repair failed for range {}-{} for {}.{} with cancellationStatus: {} retrying after {} seconds...", + childStartToken, childEndToken, + keyspaceName, config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, + cancellationStatus, config.getRepairRetryBackoff().toSeconds()); + sleepFunc.accept(config.getRepairRetryBackoff().toSeconds(), TimeUnit.SECONDS); + } + retryCount++; } - catch (InterruptedException e) - { - logger.error("Exception in cond await:", e); - } - //check repair status if (repairState.isSuccess()) { @@ -301,11 +324,15 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) } else { - boolean cancellationStatus = f.cancel(true); - //in future we can add retry, etc. - logger.info("Repair failed for range {}-{} for {}.{} total subranges: {}," + - "processed subranges: {}, cancellationStatus: {}", childStartToken, childEndToken, - keyspaceName, config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, totalSubRanges, totalProcessedSubRanges, cancellationStatus); + boolean cancellationStatus = true; + if (f != null) + { + cancellationStatus = f.cancel(true); + } + //in the future we can add retry, etc. + logger.error("Repair failed for range {}-{} for {}.{} after {} retries, total subranges: {}," + + "processed subranges: {}, cancellationStatus: {}", childStartToken.toString(), childEndToken.toString(), keyspaceName, + config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, retryCount, totalSubRanges, totalProcessedSubRanges, cancellationStatus); failedTokenRanges += ranges.size(); } ranges.clear(); @@ -395,6 +422,7 @@ private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType AutoRepairUtils.removePriorityStatus(repairType, myId); } + logger.info("TEST123"); repairState.setFailedTokenRangesCount(failedTokenRanges); repairState.setSucceededTokenRangesCount(succeededTokenRanges); repairState.setSkippedTokenRangesCount(skippedTokenRanges); diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index 9522d35bcdc8..e3e7d34658e9 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -48,6 +48,11 @@ public class AutoRepairConfig implements Serializable // This information is useful so the scheduler is absolutely sure that the node is indeed removed from the ring, and then it can adjust the repair schedule accordingly. // So, the duration in this config determinses for how long deleted host's information is kept in the scheduler's metadata. public volatile DurationSpec.IntSecondsBound history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound("2h"); + // the maximum number of retries for a repair session. + public volatile Integer repair_max_retries = 3; + // the backoff time in seconds for retrying a repair session. + public volatile DurationSpec.LongSecondsBound repair_retry_backoff = new DurationSpec.LongSecondsBound("60s"); + // global_settings overides Options.defaultOptions for all repair types public volatile Options global_settings; @@ -108,6 +113,26 @@ public void setAutoRepairHistoryClearDeleteHostsBufferInterval(String duration) history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound(duration); } + public int getRepairMaxRetries() + { + return repair_max_retries; + } + + public void setRepairMaxRetries(int maxRetries) + { + repair_max_retries = maxRetries; + } + + public DurationSpec.LongSecondsBound getRepairRetryBackoff() + { + return repair_retry_backoff; + } + + public void setRepairRetryBackoff(String interval) + { + repair_retry_backoff = new DurationSpec.LongSecondsBound(interval); + } + public boolean isAutoRepairEnabled(RepairType repairType) { return enabled && applyOverrides(repairType, opt -> opt.enabled); diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index a22faafd1b83..02f9d04fee6e 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -732,7 +732,7 @@ public static void addPriorityHosts(RepairType repairType, Set 0) + if (!hostIds.isEmpty()) { SetSerializer serializer = SetSerializer.getInstance(UUIDSerializer.instance, UTF8Type.instance.comparatorSet); addPriorityHost.execute(QueryState.forInternalCalls(), diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index b07d770fb2de..60073b20d4b9 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -104,6 +104,18 @@ public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) config.setAutoRepairHistoryClearDeleteHostsBufferInterval(duration); } + @Override + public void setAutoRepairMaxRetriesCount(int retries) + { + config.setRepairMaxRetries(retries); + } + + @Override + public void setAutoRepairRetryBackoff(String interval) + { + config.setRepairRetryBackoff(interval); + } + @Override public void setRepairSSTableCountHigherThreshold(RepairType repairType, int sstableHigherThreshold) { diff --git a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java index 5dda1b157517..d01c6d61db1d 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java +++ b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java @@ -44,6 +44,8 @@ public interface AutoRepairServiceMBean public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration); + public void setAutoRepairMaxRetriesCount(int retries); + public void setAutoRepairRetryBackoff(String interval); public void setRepairSSTableCountHigherThreshold(RepairType repairType, int ssTableHigherThreshold); public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime); diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index b7b6a470e5ca..2b4d2c7c7f86 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -5711,8 +5711,8 @@ public List mutateSSTableRepairedState(boolean repaired, boolean preview if (!preview) table.getCompactionStrategyManager().mutateRepaired(sstables, repairedAt, null, false); return sstables; - }, predicate, true, false, true); - sstablesTouched.addAll(result.stream().map(sst -> sst.descriptor.baseFilename()).collect(Collectors.toList())); + }, predicate, OperationType.ANTICOMPACTION, true, false, true); + sstablesTouched.addAll(result.stream().map(sst -> sst.descriptor.baseFile().name()).collect(Collectors.toList())); } return sstablesTouched; } diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 622f53618e69..c2345d5a8239 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -2459,6 +2459,16 @@ public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) autoRepairProxy.setAutoRepairHistoryClearDeleteHostsBufferDuration(duration); } + public void setAutoRepairMaxRetriesCount(int retries) + { + autoRepairProxy.setAutoRepairMaxRetriesCount(retries); + } + + public void setAutoRepairRetryBackoff(String interval) + { + autoRepairProxy.setAutoRepairRetryBackoff(interval); + } + public void setRepairSSTableCountHigherThreshold(AutoRepairConfig.RepairType repairType, int ssTableHigherThreshold) { autoRepairProxy.setRepairSSTableCountHigherThreshold(repairType, ssTableHigherThreshold); diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java index feaf4c98ae17..cfb005542efc 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java @@ -47,6 +47,8 @@ public void execute(NodeProbe probe) sb.append("repair scheduler configuration:"); sb.append("\n\trepair eligibility check interval: " + config.getRepairCheckInterval()); sb.append("\n\tTTL for repair history for dead nodes: " + config.getAutoRepairHistoryClearDeleteHostsBufferInterval()); + sb.append("\n\tmax retries for repair: " + config.getRepairMaxRetries()); + sb.append("\n\tretry backoff: " + config.getRepairRetryBackoff()); for (RepairType repairType : RepairType.values()) { sb.append(formatRepairTypeConfig(probe, repairType, config)); diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java index bd00809dcf22..bf2d290ca54d 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java @@ -44,7 +44,8 @@ public class SetAutoRepairConfig extends NodeToolCmd description = "autorepair param and value.\nPossible autorepair parameters are as following: " + "[number_of_repair_threads|number_of_subranges|min_repair_interval|sstable_upper_threshold" + "|enabled|table_max_repair_time|priority_hosts|forcerepair_hosts|ignore_dcs" + - "|history_clear_delete_hosts_buffer_interval|repair_primary_token_range_only|parallel_repair_count_in_group|parallel_repair_percentage_in_group|mv_repair_enabled]", + "|history_clear_delete_hosts_buffer_interval|repair_primary_token_range_only" + + "|parallel_repair_count|parallel_repair_percentage|mv_repair_enabled|repair_max_retries|repair_retry_backoff]", required = true) protected List args = new ArrayList<>(); @@ -58,7 +59,6 @@ public class SetAutoRepairConfig extends NodeToolCmd @Override public void execute(NodeProbe probe) { - checkArgument(repairType != null, "--repair-type is required."); checkArgument(args.size() == 2, "setautorepairconfig requires param-type, and value args."); String paramType = args.get(0); String paramVal = args.get(1); @@ -69,10 +69,21 @@ public void execute(NodeProbe probe) return; } - if (paramType.equals("history_clear_delete_hosts_buffer_interval")) + // options that do not require --repair-type option + switch (paramType) { - probe.setAutoRepairHistoryClearDeleteHostsBufferDuration(paramVal); - return; + case "history_clear_delete_hosts_buffer_interval": + probe.setAutoRepairHistoryClearDeleteHostsBufferDuration(paramVal); + return; + case "repair_max_retries": + probe.setAutoRepairMaxRetriesCount(Integer.parseInt(paramVal)); + return; + case "repair_retry_backoff": + probe.setAutoRepairRetryBackoff(paramVal); + return; + default: + // proceed to options that require --repair-type option + break; } // options below require --repair-type option @@ -99,14 +110,14 @@ public void execute(NodeProbe probe) probe.setAutoRepairTableMaxRepairTime(repairType, paramVal); break; case "priority_hosts": - hosts = validateLocalGroupHosts(probe, repairType, paramVal); + hosts = validateLocalGroupHosts(paramVal); if (!hosts.isEmpty()) { probe.setRepairPriorityForHosts(repairType, hosts); } break; case "forcerepair_hosts": - hosts = validateLocalGroupHosts(probe, repairType, paramVal); + hosts = validateLocalGroupHosts(paramVal); if (!hosts.isEmpty()) { probe.setForceRepairForHosts(repairType, hosts); @@ -123,10 +134,10 @@ public void execute(NodeProbe probe) case "repair_primary_token_range_only": probe.setPrimaryTokenRangeOnly(repairType, Boolean.parseBoolean(paramVal)); break; - case "parallel_repair_count_in_group": + case "parallel_repair_count": probe.setParallelRepairCountInGroup(repairType, Integer.parseInt(paramVal)); break; - case "parallel_repair_percentage_in_group": + case "parallel_repair_percentage": probe.setParallelRepairPercentageInGroup(repairType, Integer.parseInt(paramVal)); break; case "mv_repair_enabled": @@ -137,7 +148,7 @@ public void execute(NodeProbe probe) } } - private Set validateLocalGroupHosts(NodeProbe probe, RepairType repairType, String paramVal) { + private Set validateLocalGroupHosts(String paramVal) { Set hosts = new HashSet<>(); for (String host : Splitter.on(',').split(paramVal)) { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 4bbc77443e51..7f7238a6f1e2 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Set; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import com.google.common.collect.ImmutableMap; @@ -619,4 +620,60 @@ public void testRepairTakesLastRepairTimeFromDB() // repair scheduler should load the repair time from the DB assertEquals(lastRepairTime, AutoRepair.instance.repairStates.get(repairType).getLastRepairTime()); } + + @Test + public void testRepairMaxRetries() + { + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + when(autoRepairState.isSuccess()).thenReturn(false); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + AtomicInteger sleepCalls = new AtomicInteger(); + AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + sleepCalls.getAndIncrement(); + assertEquals(TimeUnit.SECONDS, unit); + assertEquals(config.getRepairRetryBackoff().toSeconds(), (long) duration); + }; + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + + AutoRepair.instance.repair(repairType, 0); + + //system_auth.role_permissions,system_auth.network_permissions,system_auth.role_members,system_auth.roles, + // system_auth.resource_role_permissons_index,system_traces.sessions,system_traces.events,ks.tbl, + // system_distributed.auto_repair_priority,system_distributed.repair_history,system_distributed.auto_repair_history, + // system_distributed.view_build_status,system_distributed.parent_repair_history,system_distributed.partition_denylist + int exptedTablesGoingThroughRepair = 14; + assertEquals(config.getRepairMaxRetries()*exptedTablesGoingThroughRepair, sleepCalls.get()); + verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(0); + verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); + verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(exptedTablesGoingThroughRepair); + } + + @Test + public void testRepairSuccessAfterRetry() + { + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + AtomicInteger sleepCalls = new AtomicInteger(); + AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + sleepCalls.getAndIncrement(); + assertEquals(TimeUnit.SECONDS, unit); + assertEquals(config.getRepairRetryBackoff().toSeconds(), (long) duration); + }; + when(autoRepairState.isSuccess()).then((invocationOnMock) -> { + if (sleepCalls.get() == 0) { + return false; + } + return true; + }); + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + AutoRepair.instance.repair(repairType, 0); + + assertEquals(1, sleepCalls.get()); + verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(14); + verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); + verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(0); + } } diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java new file mode 100644 index 000000000000..11df262f9500 --- /dev/null +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.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.service; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Set; +import java.util.UUID; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.collect.ImmutableSet; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Suite; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairKeyspace; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.junit.Assert.assertEquals; + +@RunWith(Suite.class) +@Suite.SuiteClasses({ AutoRepairServiceTest.BasicTests.class, AutoRepairServiceTest.SetterTests.class }) +public class AutoRepairServiceTest +{ + public static class BasicTests + { + private static AutoRepairService autoRepairService; + private static AutoRepairConfig config; + + @Before + public void setUp() + { + config = new AutoRepairConfig(); + autoRepairService = new AutoRepairService(); + autoRepairService.config = config; + } + + + @Test + public void testSetup() + { + AutoRepairService.instance.config = null; + + AutoRepairService.setup(); + + assertEquals(DatabaseDescriptor.getAutoRepairConfig(), AutoRepairService.instance.config); + } + + @Test + public void testGetAutoRepairConfigReturnsConfig() + { + assertEquals(config, autoRepairService.getAutoRepairConfig()); + } + + @Test + public void testsetAutoRepairHistoryClearDeleteHostsBufferInSecV2() + { + autoRepairService.setAutoRepairHistoryClearDeleteHostsBufferDuration("100s"); + + assertEquals(100, config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds()); + } + + + @Test + public void testsetAutoRepairMaxRetriesCount() + { + autoRepairService.setAutoRepairMaxRetriesCount(101); + + assertEquals(101, config.getRepairMaxRetries()); + } + + + @Test + public void testsetAutoRepairRetryBackoffInSec() + { + autoRepairService.setAutoRepairRetryBackoff("102s"); + + assertEquals(102, config.getRepairRetryBackoff().toSeconds()); + } + } + + @RunWith(Parameterized.class) + public static class SetterTests extends CQLTester + { + private static final AutoRepairConfig config = new AutoRepairConfig(true); + + @Parameterized.Parameter + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameter(1) + public T arg; + + @Parameterized.Parameter(2) + public BiConsumer setter; + + @Parameterized.Parameter(3) + public Function getter; + + @Parameterized.Parameters(name = "{index}: repairType={0}, arg={1}") + public static Collection testCases() + { + return Stream.of( + forEachRepairType(true, AutoRepairService.instance::setAutoRepairEnabled, config::isAutoRepairEnabled), + forEachRepairType(100, AutoRepairService.instance::setRepairThreads, config::getRepairThreads), + forEachRepairType(200, AutoRepairService.instance::setRepairSubRangeNum, config::getRepairSubRangeNum), + forEachRepairType(400, AutoRepairService.instance::setRepairSSTableCountHigherThreshold, config::getRepairSSTableCountHigherThreshold), + forEachRepairType(ImmutableSet.of("dc1", "dc2"), AutoRepairService.instance::setIgnoreDCs, config::getIgnoreDCs), + forEachRepairType(true, AutoRepairService.instance::setPrimaryTokenRangeOnly, config::getRepairPrimaryTokenRangeOnly), + forEachRepairType(600, AutoRepairService.instance::setParallelRepairPercentageInGroup, config::getParallelRepairPercentageInGroup), + forEachRepairType(700, AutoRepairService.instance::setParallelRepairCountInGroup, config::getParallelRepairCountInGroup), + forEachRepairType(true, AutoRepairService.instance::setMVRepairEnabled, config::getMVRepairEnabled), + forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setRepairPriorityForHosts, AutoRepairUtils::getPriorityHosts), + forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setForceRepairForHosts, SetterTests::isLocalHostForceRepair) + ).flatMap(Function.identity()).collect(Collectors.toList()); + } + + private static Set isLocalHostForceRepair(AutoRepairConfig.RepairType type) + { + UUID hostId = StorageService.instance.getHostIdForEndpoint(InetAddressAndPort.getLocalHost()); + UntypedResultSet resultSet = QueryProcessor.executeInternal(String.format( + "SELECT force_repair FROM %s.%s WHERE host_id = %s and repair_type = '%s'", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, hostId, type)); + + if (!resultSet.isEmpty() && resultSet.one().getBoolean("force_repair")) + { + return ImmutableSet.of(InetAddressAndPort.getLocalHost()); + } + return ImmutableSet.of(); + } + + private static Stream forEachRepairType(T arg, BiConsumer setter, Function getter) + { + Object[][] testCases = new Object[AutoRepairConfig.RepairType.values().length][4]; + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + testCases[repairType.ordinal()] = new Object[]{ repairType, arg, setter, getter }; + } + + return Arrays.stream(testCases); + } + + @BeforeClass + public static void setup() throws Exception + { + setAutoRepairEnabled(true); + requireNetwork(); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + DatabaseDescriptor.setCDCEnabled(false); + AutoRepairUtils.setup(); + AutoRepairService.instance.config = config; + } + + @Before + public void prepare() + { + QueryProcessor.executeInternal(String.format( + "TRUNCATE %s.%s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY)); + QueryProcessor.executeInternal(String.format( + "TRUNCATE %s.%s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY)); + } + + @Test + public void testSetters() + { + setter.accept(repairType, arg); + assertEquals(arg, getter.apply(repairType)); + } + } +} diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java index b33af747884d..76d56ebadc54 100644 --- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java +++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java @@ -703,7 +703,7 @@ public void testMutateSSTableRepairedStateTablePreview() assertEquals(1, result.size()); table1.getLiveSSTables().forEach(sstable -> { assertFalse(sstable.isRepaired()); - assertTrue(result.contains(sstable.descriptor.baseFilename())); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); }); } @@ -724,7 +724,7 @@ public void testMutateSSTableRepairedStateTableRepaired() assertEquals(2, result.size()); table1.getLiveSSTables().forEach(sstable -> { assertTrue(sstable.isRepaired()); - assertTrue(result.contains(sstable.descriptor.baseFilename())); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); }); } @@ -743,7 +743,7 @@ public void testMutateSSTableRepairedStateTableUnrepaired() throws Exception assertEquals(2, result.size()); table1.getLiveSSTables().forEach(sstable -> { assertFalse(sstable.isRepaired()); - assertTrue(result.contains(sstable.descriptor.baseFilename())); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); }); } } diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java new file mode 100644 index 000000000000..2a69cefcace9 --- /dev/null +++ b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.PrintStream; +import java.util.Arrays; +import java.util.Collection; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Suite; + +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.tools.NodeProbe; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.junit.Assert.fail; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +@RunWith(Suite.class) +@Suite.SuiteClasses({ SetAutoRepairConfigTest.NoParamTests.class, SetAutoRepairConfigTest.RepairTypeParamTests.class, + SetAutoRepairConfigTest.RepairTypeAndArgsParamsTests.class }) +public class SetAutoRepairConfigTest +{ + protected static AutoRepairConfig config; + + protected static SetAutoRepairConfig cmd; + + public static void before(NodeProbe probeMock, PrintStream outMock) + { + config = new AutoRepairConfig(true); + when(probeMock.getAutoRepairConfig()).thenReturn(config); + cmd = new SetAutoRepairConfig(); + cmd.out = outMock; + } + + public static class NoParamTests + { + @Mock + private static NodeProbe probe; + + @Mock + private static PrintStream out; + + @Before + public void setUp() + { + MockitoAnnotations.initMocks(this); + before(probe, out); + } + + @Test + public void testHistoryDeleteHostsClearBufferInSec() + { + cmd.args = ImmutableList.of("history_clear_delete_hosts_buffer_interval", "1s"); + + cmd.execute(probe); + + verify(probe, times(1)).setAutoRepairHistoryClearDeleteHostsBufferDuration("1s"); + + // test scenario when auto repair is disabled + when(probe.getAutoRepairConfig()).thenReturn(new AutoRepairConfig(false)); + + cmd.execute(probe); + + // test new calls are not made when auto repair is disabled + verify(probe, times(1)).setAutoRepairHistoryClearDeleteHostsBufferDuration("1s"); + } + + @Test + public void testRepairMaxRetries() + { + cmd.args = ImmutableList.of("repair_max_retries", "2"); + + cmd.execute(probe); + + verify(probe, times(1)).setAutoRepairMaxRetriesCount(2); + } + + + @Test + public void testRetryBackoffInSec() + { + cmd.args = ImmutableList.of("repair_retry_backoff", "3s"); + + cmd.execute(probe); + + verify(probe, times(1)).setAutoRepairRetryBackoff("3s"); + } + } + + @RunWith(Parameterized.class) + public static class RepairTypeParamTests + { + @Mock + private static NodeProbe probe; + + @Mock + private static PrintStream out; + + @Parameterized.Parameter + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameters(name = "repairType={0}") + public static Object[] data() + { + return AutoRepairConfig.RepairType.values(); + } + + private static InetAddressAndPort localEndpoint; + private static InetAddressAndPort otherEndpoint; + + @Before + public void setUp() throws Exception + { + MockitoAnnotations.initMocks(this); + before(probe, out); + localEndpoint = InetAddressAndPort.getByName("127.0.0.1:7000"); + otherEndpoint = localEndpoint.withPort(localEndpoint.getPort() + 1); + } + + @Test(expected = IllegalArgumentException.class) + public void testNoArgs() + { + cmd.repairType = repairType; + cmd.execute(probe); + } + + @Test + public void testRepairSchedulingDisabled() + { + when(probe.getAutoRepairConfig()).thenReturn(new AutoRepairConfig(false)); + cmd.repairType = repairType; + cmd.args = ImmutableList.of("threads", "1"); + + cmd.execute(probe); + + verify(out, times(1)).println("Auto-repair is not enabled"); + verify(probe, times(0)).setRepairThreads(repairType, 1); + } + + @Test + public void testRepairTypeDisabled() + { + config.setAutoRepairEnabled(repairType, false); + cmd.repairType = repairType; + cmd.args = ImmutableList.of("number_of_repair_threads", "1"); + + cmd.execute(probe); + + verify(probe, times(1)).setRepairThreads(repairType, 1); + } + + + @Test + public void testV2FlagMissing() + { + cmd.repairType = repairType; + cmd.args = ImmutableList.of("threads", "1"); + + try + { + cmd.execute(probe); + + fail("expected IllegalArgumentException"); + } + catch (IllegalArgumentException e) + { + // expected + } + + verify(probe, times(0)).setRepairThreads(repairType, 0); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidParamType() + { + cmd.repairType = repairType; + cmd.args = ImmutableList.of("unknown_type", "1"); + + cmd.execute(probe); + } + + + @Test + public void testPriorityHosts() + { + cmd.repairType = repairType; + cmd.args = ImmutableList.of("priority_hosts", String.join(",", localEndpoint.toString().substring(1), otherEndpoint.toString().substring(1))); + + cmd.execute(probe); + + verify(probe, times(1)).setRepairPriorityForHosts(repairType, ImmutableSet.of(localEndpoint, otherEndpoint)); + } + + @Test + public void testForceRepairHosts() + { + cmd.repairType = repairType; + cmd.args = ImmutableList.of("forcerepair_hosts", String.join(",", localEndpoint.toString().substring(1), otherEndpoint.toString().substring(1))); + + cmd.execute(probe); + + verify(probe, times(1)).setForceRepairForHosts(repairType, ImmutableSet.of(localEndpoint, otherEndpoint)); + } + } + + @RunWith(Parameterized.class) + public static class RepairTypeAndArgsParamsTests + { + @Parameterized.Parameter + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameter(1) + public String paramType; + + @Parameterized.Parameter(2) + public String paramVal; + + @Parameterized.Parameter(3) + public Consumer verifyFunc; + + @Parameterized.Parameters(name = "repairType={0},paramType={1}") + public static Collection testCases() + { + return Stream.of( + forEachRepairType("enabled", "true", (type) -> verify(probe, times(1)).setAutoRepairEnabled(type, true)), + forEachRepairType("number_of_repair_threads", "1", (type) -> verify(probe, times(1)).setRepairThreads(type, 1)), + forEachRepairType("number_of_subranges", "2", (type) -> verify(probe, times(1)).setRepairSubRangeNum(type, 2)), + forEachRepairType("min_repair_interval", "3h", (type) -> verify(probe, times(1)).setRepairMinInterval(type, "3h")), + forEachRepairType("sstable_upper_threshold", "4", (type) -> verify(probe, times(1)).setRepairSSTableCountHigherThreshold(type, 4)), + forEachRepairType("table_max_repair_time", "5s", (type) -> verify(probe, times(1)).setAutoRepairTableMaxRepairTime(type, "5s")), + forEachRepairType("repair_primary_token_range_only", "true", (type) -> verify(probe, times(1)).setPrimaryTokenRangeOnly(type, true)), + forEachRepairType("parallel_repair_count", "6", (type) -> verify(probe, times(1)).setParallelRepairCountInGroup(type, 6)), + forEachRepairType("parallel_repair_percentage", "7", (type) -> verify(probe, times(1)).setParallelRepairPercentageInGroup(type, 7)), + forEachRepairType("mv_repair_enabled", "true", (type) -> verify(probe, times(1)).setMVRepairEnabled(type, true)), + forEachRepairType("ignore_dcs", "dc1,dc2", (type) -> verify(probe, times(1)).setAutoRepairIgnoreDCs(type, ImmutableSet.of("dc1", "dc2"))) + ).flatMap(Function.identity()).collect(Collectors.toList()); + } + + private static Stream forEachRepairType(String paramType, String paramVal, Consumer verifyFunc) + { + Object[][] testCases = new Object[AutoRepairConfig.RepairType.values().length][4]; + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + testCases[repairType.ordinal()] = new Object[]{ repairType, paramType, paramVal, verifyFunc }; + } + + return Arrays.stream(testCases); + } + + @Mock + private static NodeProbe probe; + + @Mock + private static PrintStream out; + + @Before + public void setUp() + { + MockitoAnnotations.initMocks(this); + before(probe, out); + } + + @Test + public void test() + { + cmd.repairType = repairType; + cmd.args = ImmutableList.of(paramType, paramVal); + + cmd.execute(probe); + + verifyFunc.accept(repairType); + + // test scenario when auto repair is disabled + when(probe.getAutoRepairConfig()).thenReturn(new AutoRepairConfig(false)); + + cmd.execute(probe); + + // test new calls are not made when auto repair is disabled + verifyFunc.accept(repairType); + } + } +} From b071c59d71f824872f6d524c085cf0e43772abe8 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Fri, 30 Aug 2024 09:35:48 -0700 Subject: [PATCH 013/115] Extend nodetool sstablerepairedset to affect all keyspaces when no keyspace is provided in args --- .../repair/autorepair/AutoRepair.java | 1 - .../tools/nodetool/SSTableRepairedSet.java | 72 +++++++++---------- .../nodetool/SSTableRepairedSetTest.java | 55 +++++++------- 3 files changed, 61 insertions(+), 67 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 2fa1db89c386..6f9673dfdb96 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -422,7 +422,6 @@ private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType AutoRepairUtils.removePriorityStatus(repairType, myId); } - logger.info("TEST123"); repairState.setFailedTokenRangesCount(failedTokenRanges); repairState.setSucceededTokenRangesCount(succeededTokenRanges); repairState.setSkippedTokenRangesCount(skippedTokenRanges); diff --git a/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java b/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java index 32ee09f93590..f22f3ca0df40 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java +++ b/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java @@ -33,7 +33,7 @@ @Command(name = "sstablerepairedset", description = "Set the repaired state of SSTables for given keyspace/tables") public class SSTableRepairedSet extends NodeTool.NodeToolCmd { - @Arguments(usage = " []", description = "The keyspace optionally followed by one or more tables", required = true) + @Arguments(usage = "[ ]", description = "Optional keyspace followed by zero or more tables") protected List args = new ArrayList<>(); @Option(title = "really-set", @@ -56,57 +56,49 @@ public void execute(NodeProbe probe) { PrintStream out = probe.output().out; + if (isRepaired == isUnrepaired) + { + out.println("Exactly one of --is-repaired or --is-unrepaired must be provided."); + return; + } + String message; if (reallySet) - { message = "Mutating repaired state of SSTables for"; - } else - { message = "Previewing repaired state mutation of SSTables for"; - } - if (args.isEmpty()) - { - out.println("At least a keyspace name must be provided."); - return; - } - String keyspace = args.get(0); + List keyspaces = parseOptionalKeyspace(args, probe, KeyspaceSet.NON_LOCAL_STRATEGY); + List tables = List.of(parseOptionalTables(args)); - List tables; - if (args.size() > 1) - { - tables = args.subList(1, args.size()); - message += " tables " + String.join(", ", tables) + " in"; - } + if (args.isEmpty()) + message += " all keyspaces"; else - { - tables = probe.getTablesForKeyspace(keyspace); - message += " all tables in"; - } - message += " keyspace " + keyspace; - - if (isRepaired == isUnrepaired) - { - out.println("Exactly one of --is-repaired or --is-unrepaired must be provided."); - return; - } + message += tables.isEmpty() ? " all tables" : " tables " + String.join(", ", tables) + + " in keyspace " + keyspaces.get(0); message += " to " + (isRepaired ? "repaired" : "unrepaired"); out.println(message); - try + List sstableList = new ArrayList<>(); + for (String keyspace : keyspaces) { - List mutatedSSTables = probe.mutateSSTableRepairedState(isRepaired, !reallySet, keyspace, new ArrayList<>(tables)); - if (!reallySet) - out.println("The following SSTables would be mutated:"); - else - out.println("The following SSTables were mutated:"); - for (String sstable : mutatedSSTables) - out.println(sstable); - } - catch (InvalidRequestException e) - { - out.println(e.getMessage()); + try + { + sstableList.addAll(probe.mutateSSTableRepairedState(isRepaired, !reallySet, keyspace, + tables.isEmpty() + ? probe.getTablesForKeyspace(keyspace) // mutate all tables + : tables)); // mutate specific tables + } + catch (InvalidRequestException e) + { + out.println(e.getMessage()); + } } + if (!reallySet) + out.println("The following SSTables would be mutated:"); + else + out.println("The following SSTables were mutated:"); + for (String sstable : sstableList) + out.println(sstable); } } diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java index 0a2ead81b778..139cde3d839d 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java @@ -22,6 +22,7 @@ import java.io.PrintStream; import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import org.junit.Before; import org.junit.Test; @@ -35,6 +36,7 @@ import static org.mockito.ArgumentMatchers.anyList; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -46,56 +48,57 @@ public class SSTableRepairedSetTest private SSTableRepairedSet cmd; @Before - public void setUp() - { + public void setUp() { MockitoAnnotations.initMocks(this); - PrintStream noopStream = new PrintStream(new OutputStream() - { - @Override - public void write(int b) - { - } - }); + PrintStream noopStream = new PrintStream(new OutputStream() {@Override public void write(int b) {}}); when(probe.output()).thenReturn(new Output(noopStream, noopStream)); cmd = new SSTableRepairedSet(); } @Test - public void testNoKeyspace() - { + public void testNoKeyspace() { + when(probe.getNonLocalStrategyKeyspaces()).thenReturn(new ArrayList<>(List.of("ks1", "ks2"))); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(List.of("ks1", "ks2"))); + when(probe.getTablesForKeyspace("ks1")).thenReturn(new ArrayList<>(List.of("table1", "table2"))); + when(probe.getTablesForKeyspace("ks2")).thenReturn(new ArrayList<>(List.of("table3", "table4"))); + cmd.isRepaired = true; + cmd.reallySet = true; + cmd.execute(probe); - verify(probe, never()).mutateSSTableRepairedState(anyBoolean(), anyBoolean(), anyString(), anyList()); + + verify(probe, times(1)).mutateSSTableRepairedState(true, false, "ks1", List.of("table1", "table2")); + verify(probe, times(1)).mutateSSTableRepairedState(true, false, "ks2", List.of("table3", "table4")); } @Test - public void testBothRepairedAndUnrepaired() - { - cmd.args = Arrays.asList("keyspace"); + public void testBothRepairedAndUnrepaired() { + cmd.args = List.of("keyspace"); + cmd.isRepaired = true; + cmd.isUnrepaired = true; cmd.execute(probe); verify(probe, never()).mutateSSTableRepairedState(anyBoolean(), anyBoolean(), anyString(), anyList()); } @Test - public void testNeitherRepairedNorUnrepaired() - { - cmd.args = Arrays.asList("keyspace"); + public void testNeitherRepairedNorUnrepaired() { + cmd.args = List.of("keyspace"); cmd.execute(probe); verify(probe, never()).mutateSSTableRepairedState(anyBoolean(), anyBoolean(), anyString(), anyList()); } @Test - public void testRepairedPreview() - { - cmd.args = Arrays.asList("keyspace"); + public void testRepairedPreview() { + cmd.args = List.of("keyspace"); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(List.of("keyspace"))); cmd.isRepaired = true; cmd.execute(probe); verify(probe).mutateSSTableRepairedState(true, true, "keyspace", new ArrayList<>()); } @Test - public void testUnrepairedReallySet() - { - cmd.args = Arrays.asList("keyspace"); + public void testUnrepairedReallySet() { + cmd.args = List.of("keyspace"); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(List.of("keyspace"))); cmd.isUnrepaired = true; cmd.reallySet = true; cmd.execute(probe); @@ -103,9 +106,9 @@ public void testUnrepairedReallySet() } @Test - public void testExecuteWithTableNames() - { + public void testExecuteWithTableNames() { cmd.args = Arrays.asList("keyspace", "table1", "table2"); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(List.of("keyspace"))); cmd.isRepaired = true; cmd.reallySet = true; cmd.execute(probe); From 4e4485faf65e9bc7161147adf4add72043e3ccb6 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Fri, 30 Aug 2024 15:16:12 -0700 Subject: [PATCH 014/115] Make repair session timeout configurable --- .../repair/autorepair/AutoRepair.java | 2 +- .../repair/autorepair/AutoRepairConfig.java | 21 +++++++++++++++---- .../repair/autorepair/AutoRepairState.java | 5 +++-- .../cassandra/service/AutoRepairService.java | 5 +++++ .../service/AutoRepairServiceMBean.java | 2 ++ .../org/apache/cassandra/tools/NodeProbe.java | 5 +++++ .../tools/nodetool/GetAutoRepairConfig.java | 1 + .../tools/nodetool/SetAutoRepairConfig.java | 9 ++++++-- .../autorepair/AutoRepairConfigTest.java | 8 +++++++ .../AutoRepairParameterizedTest.java | 4 +++- .../autorepair/AutoRepairStateTest.java | 5 +++-- 11 files changed, 55 insertions(+), 12 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 6f9673dfdb96..fbe94d8f7365 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -293,7 +293,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) f = repairRunnableExecutors.get(repairType).submit(task); try { - repairState.waitForRepairToComplete(); + repairState.waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); } catch (InterruptedException e) { diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index e3e7d34658e9..6e187255df77 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -296,6 +296,17 @@ public DurationSpec.IntSecondsBound getInitialSchedulerDelay(RepairType repairTy return applyOverrides(repairType, opt -> opt.initial_scheduler_delay); } + public DurationSpec.IntSecondsBound getRepairSessionTimeout(RepairType repairType) + { + return applyOverrides(repairType, opt -> opt.repair_session_timeout); + } + + public void setRepairSessionTimeout(RepairType repairType, String repairSessionTimeout) + { + ensureOverrides(repairType); + repair_type_overrides.get(repairType).repair_session_timeout = new DurationSpec.IntSecondsBound(repairSessionTimeout); + } + // Options configures auto-repair behavior for a given repair type. // All fields can be modified dynamically. public static class Options implements Serializable @@ -331,6 +342,7 @@ protected static Options getDefaultOptions() opts.mv_repair_enabled = true; opts.token_range_splitter = DefaultAutoRepairTokenSplitter.class.getName(); opts.initial_scheduler_delay = new DurationSpec.IntSecondsBound("15m"); // 15 minutes + opts.repair_session_timeout = new DurationSpec.IntSecondsBound("3h"); // 3 hours return opts; } @@ -398,6 +410,9 @@ protected static Options getDefaultOptions() public volatile String token_range_splitter; // the minimum delay after a node starts before the scheduler starts running repair public volatile DurationSpec.IntSecondsBound initial_scheduler_delay; + // repair session timeout - this is applicable for each repair session + // the major issue with Repair is a session sometimes hangs; so this timeout is useful to unblock such problems + public volatile DurationSpec.IntSecondsBound repair_session_timeout; public String toString() { @@ -417,6 +432,7 @@ public String toString() ", mv_repair_enabled=" + mv_repair_enabled + ", token_range_splitter=" + token_range_splitter + ", intial_scheduler_delay=" + initial_scheduler_delay + + ", repair_session_timeout=" + repair_session_timeout + '}'; } } @@ -446,9 +462,6 @@ protected void ensureOverrides(RepairType repairType) repair_type_overrides = new EnumMap<>(RepairType.class); } - if (repair_type_overrides.get(repairType) == null) - { - repair_type_overrides.put(repairType, new Options()); - } + repair_type_overrides.computeIfAbsent(repairType, k -> new Options()); } } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index d7a473cab1df..e15fe6f7a0a8 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -20,6 +20,7 @@ import com.google.common.annotations.VisibleForTesting; +import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.view.TableViews; @@ -135,10 +136,10 @@ public void progress(String tag, ProgressEvent event) } } - public void waitForRepairToComplete() throws InterruptedException + public void waitForRepairToComplete(DurationSpec.IntSecondsBound repairSessionTimeout) throws InterruptedException { //if for some reason we don't hear back on repair progress for sometime - if (!condition.await(12, TimeUnit.HOURS)) + if (!condition.await(repairSessionTimeout.toSeconds(), TimeUnit.SECONDS)) { success = false; } diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index 60073b20d4b9..cbef9889fb27 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -156,4 +156,9 @@ public void setMVRepairEnabled(RepairType repairType, boolean enabled) { config.setMVRepairEnabled(repairType, enabled); } + + public void setRepairSessionTimeout(RepairType repairType, String timeout) + { + config.setRepairSessionTimeout(repairType, timeout); + } } diff --git a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java index d01c6d61db1d..4ebf118e023e 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java +++ b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java @@ -59,4 +59,6 @@ public interface AutoRepairServiceMBean public void setMVRepairEnabled(RepairType repairType, boolean enabled); public AutoRepairConfig getAutoRepairConfig(); + + public void setRepairSessionTimeout(RepairType repairType, String timeout); } diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index c2345d5a8239..b5b85d500303 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -2510,6 +2510,11 @@ public List mutateSSTableRepairedState(boolean repair, boolean preview, public List getTablesForKeyspace(String keyspace) { return ssProxy.getTablesForKeyspace(keyspace); } + + public void setRepairSessionTimeout(AutoRepairConfig.RepairType repairType, String timeout) + { + autoRepairProxy.setRepairSessionTimeout(repairType, timeout); + } } class ColumnFamilyStoreMBeanIterator implements Iterator> diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java index cfb005542efc..30dd1ff889c4 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java @@ -74,6 +74,7 @@ private String formatRepairTypeConfig(NodeProbe probe, RepairType repairType, Au sb.append("\n\tpercentage of parallel repairs within group: " + config.getParallelRepairPercentageInGroup(repairType)); sb.append("\n\tmv repair enabled: " + config.getMVRepairEnabled(repairType)); sb.append("\n\tinitial scheduler delay: " + config.getInitialSchedulerDelay(repairType)); + sb.append("\n\trepair setssion timeout: " + config.getRepairSessionTimeout(repairType)); return sb.toString(); } diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java index bf2d290ca54d..b467d9b82998 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java @@ -19,6 +19,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Splitter; + import io.airlift.airline.Arguments; import io.airlift.airline.Command; import io.airlift.airline.Option; @@ -45,7 +46,7 @@ public class SetAutoRepairConfig extends NodeToolCmd "[number_of_repair_threads|number_of_subranges|min_repair_interval|sstable_upper_threshold" + "|enabled|table_max_repair_time|priority_hosts|forcerepair_hosts|ignore_dcs" + "|history_clear_delete_hosts_buffer_interval|repair_primary_token_range_only" + - "|parallel_repair_count|parallel_repair_percentage|mv_repair_enabled|repair_max_retries|repair_retry_backoff]", + "|parallel_repair_count|parallel_repair_percentage|mv_repair_enabled|repair_max_retries|repair_retry_backoff|repair_session_timeout]", required = true) protected List args = new ArrayList<>(); @@ -143,12 +144,16 @@ public void execute(NodeProbe probe) case "mv_repair_enabled": probe.setMVRepairEnabled(repairType, Boolean.parseBoolean(paramVal)); break; + case "repair_session_timeout": + probe.setRepairSessionTimeout(repairType, paramVal); + break; default: throw new IllegalArgumentException("Unknown parameter: " + paramType); } } - private Set validateLocalGroupHosts(String paramVal) { + private Set validateLocalGroupHosts(String paramVal) + { Set hosts = new HashSet<>(); for (String host : Splitter.on(',').split(paramVal)) { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java index 923e35b73796..6d25dbeb20b4 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -468,4 +468,12 @@ public void testGetInitialSchedulerDelay() assertEquals(5, result); } + @Test + public void testSetRepairSessionTimeout() + { + config.setRepairSessionTimeout(repairType, "1h"); + + assert config.repair_type_overrides.get(repairType).repair_session_timeout.toSeconds() == 3600; + } + } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 7f7238a6f1e2..8c0ac1b7dc56 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -31,6 +31,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; +import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.statements.schema.TableAttributes; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; @@ -489,6 +490,7 @@ public void testRepairWaitsForRepairToFinishBeforeSchedullingNewSession() throws { AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setMVRepairEnabled(repairType, false); + config.setRepairRetryBackoff("0s"); when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) .thenReturn(repairRunnable); AutoRepair.instance.repairStates.put(repairType, autoRepairState); @@ -506,7 +508,7 @@ public void testRepairWaitsForRepairToFinishBeforeSchedullingNewSession() throws assertEquals("resetWaitCondition was not called before waitForRepairToComplete", resetWaitConditionCalls.get(), waitForRepairCompletedCalls.get()); return null; - }).when(autoRepairState).waitForRepairToComplete(); + }).when(autoRepairState).waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); AutoRepair.instance.repair(repairType, 0); AutoRepair.instance.repair(repairType, 0); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java index 340a21f97bfa..657b11310f64 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java @@ -31,6 +31,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; @@ -125,7 +126,7 @@ public void testWaitForRepairToComplete() throws Exception state.condition.signalAll(); Condition finishedCondition = Condition.newOneTimeCondition(); Callable waitForRepairToComplete = () -> { - state.waitForRepairToComplete(); + state.waitForRepairToComplete(new DurationSpec.IntSecondsBound("12h")); finishedCondition.signalAll(); return null; }; @@ -363,7 +364,7 @@ public void testWaitForRepairToCompleteDoesNotSetSuccessWhenProgressReceivesErro state.progress("test", progressEvent); assertFalse(state.success); - state.waitForRepairToComplete(); + state.waitForRepairToComplete(new DurationSpec.IntSecondsBound("12h")); assertFalse(state.success); } From f51a1efaabebf9d0b4e993a2dc08cd0aa5d1a28e Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Thu, 5 Sep 2024 18:31:09 -0700 Subject: [PATCH 015/115] Comments from Joshua M --- .../repair/autorepair/AutoRepairConfig.java | 29 +++++++++---------- .../autorepair/AutoRepairConfigTest.java | 10 +++---- 2 files changed, 19 insertions(+), 20 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index 6e187255df77..97868a594f25 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -238,24 +238,24 @@ public void setRepairPrimaryTokenRangeOnly(RepairType repairType, boolean primar public int getParallelRepairPercentageInGroup(RepairType repairType) { - return applyOverrides(repairType, opt -> opt.parallel_repair_percentage_in_group); + return applyOverrides(repairType, opt -> opt.parallel_repair_percentage); } public void setParallelRepairPercentageInGroup(RepairType repairType, int percentageInGroup) { ensureOverrides(repairType); - repair_type_overrides.get(repairType).parallel_repair_percentage_in_group = percentageInGroup; + repair_type_overrides.get(repairType).parallel_repair_percentage = percentageInGroup; } public int getParallelRepairCountInGroup(RepairType repairType) { - return applyOverrides(repairType, opt -> opt.parallel_repair_count_in_group); + return applyOverrides(repairType, opt -> opt.parallel_repair_count); } public void setParallelRepairCountInGroup(RepairType repairType, int countInGroup) { ensureOverrides(repairType); - repair_type_overrides.get(repairType).parallel_repair_count_in_group = countInGroup; + repair_type_overrides.get(repairType).parallel_repair_count = countInGroup; } public boolean getMVRepairEnabled(RepairType repairType) @@ -331,15 +331,15 @@ protected static Options getDefaultOptions() opts.repair_by_keyspace = false; opts.number_of_subranges = 16; opts.number_of_repair_threads = 1; - opts.parallel_repair_count_in_group = 1; - opts.parallel_repair_percentage_in_group = 0; + opts.parallel_repair_count = 3; + opts.parallel_repair_percentage = 3; opts.sstable_upper_threshold = 10000; opts.min_repair_interval = new DurationSpec.IntSecondsBound("24h"); opts.ignore_dcs = new HashSet<>(); opts.repair_primary_token_range_only = true; opts.force_repair_new_node = false; opts.table_max_repair_time = new DurationSpec.IntSecondsBound("6h"); - opts.mv_repair_enabled = true; + opts.mv_repair_enabled = false; opts.token_range_splitter = DefaultAutoRepairTokenSplitter.class.getName(); opts.initial_scheduler_delay = new DurationSpec.IntSecondsBound("15m"); // 15 minutes opts.repair_session_timeout = new DurationSpec.IntSecondsBound("3h"); // 3 hours @@ -362,13 +362,12 @@ protected static Options getDefaultOptions() // Once the scheduler schedules one repair session, then howmany threads to use inside that job will be controlled through this parameter. // This is similar to -j for repair options for the nodetool repair command. public volatile Integer number_of_repair_threads; - // the number of repair sessions that can run in parallel in a single group - // The number of nodes running repair parallelly. If parallelrepaircount is set, it will choose the larger value of the two. The default is 3. + // The number of nodes running repair parallelly. If parallel_repair_count is set, it will choose the larger value of the two. The default is 3. // This configuration controls how many nodes would run repair in parallel. // The value “3” means, at any given point in time, at most 3 nodes would be running repair in parallel. These selected nodes can be from any datacenters. // If one or more node(s) finish repair, then the framework automatically picks up the next candidate and ensures the maximum number of nodes running repair do not exceed “3”. - public volatile Integer parallel_repair_count_in_group; - // the number of repair sessions that can run in parallel in a single groupas a percentage + public volatile Integer parallel_repair_count; + // the number of repair nodes that can run in parallel // of the total number of nodes in the group [0,100] // The percentage of nodes in the cluster that run repair parallelly. If parallelrepaircount is set, it will choose the larger value of the two. // The problem with a fixed number of nodes (the above property) is that in a large-scale environment, @@ -376,7 +375,7 @@ protected static Options getDefaultOptions() // The default is 3%, which means that 3% of the nodes in the Cassandra cluster would be repaired in parallel. // So now, if a fleet, an operator won't have to worry about changing the repair frequency, etc., as overall repair time will continue to remain the same even if nodes are added or removed due to elasticity. // Extremely fewer manual interventions as it will rarely violate the repair SLA for customers - public volatile Integer parallel_repair_percentage_in_group; + public volatile Integer parallel_repair_percentage; // the upper threshold of SSTables allowed to participate in a single repair session // Threshold to skip a table if it has too many sstables. The default is 10000. This means, if a table on a node has 10000 or more SSTables, then that table will be skipped. // This is to avoid penalizing good tables (neighbors) with an outlier. @@ -421,8 +420,8 @@ public String toString() ", repair_by_keyspace=" + repair_by_keyspace + ", number_of_subranges=" + number_of_subranges + ", number_of_repair_threads=" + number_of_repair_threads + - ", parallel_repair_count_in_group=" + parallel_repair_count_in_group + - ", parallel_repair_percentage_in_group=" + parallel_repair_percentage_in_group + + ", parallel_repair_count_in_group=" + parallel_repair_count + + ", parallel_repair_percentage_in_group=" + parallel_repair_percentage + ", sstable_upper_threshold=" + sstable_upper_threshold + ", min_repair_interval=" + min_repair_interval + ", ignore_dcs=" + ignore_dcs + @@ -432,7 +431,7 @@ public String toString() ", mv_repair_enabled=" + mv_repair_enabled + ", token_range_splitter=" + token_range_splitter + ", intial_scheduler_delay=" + initial_scheduler_delay + - ", repair_session_timeout=" + repair_session_timeout + + ", repair_session_timeout=" + repair_session_timeout + '}'; } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java index 6d25dbeb20b4..07eb9070fe85 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -340,7 +340,7 @@ public void testSetRepairPrimaryTokenRangeOnly() @Test public void testGetParallelRepairPercentageInGroup() { - config.global_settings.parallel_repair_percentage_in_group = 5; + config.global_settings.parallel_repair_percentage = 5; int result = config.getParallelRepairPercentageInGroup(repairType); @@ -352,13 +352,13 @@ public void testSetParallelRepairPercentageInGroup() { config.setParallelRepairPercentageInGroup(repairType, 5); - assert config.repair_type_overrides.get(repairType).parallel_repair_percentage_in_group == 5; + assert config.repair_type_overrides.get(repairType).parallel_repair_percentage == 5; } @Test public void testGetParallelRepairCountInGroup() { - config.global_settings.parallel_repair_count_in_group = 5; + config.global_settings.parallel_repair_count = 5; int result = config.getParallelRepairCountInGroup(repairType); @@ -370,7 +370,7 @@ public void testSetParallelRepairCountInGroup() { config.setParallelRepairCountInGroup(repairType, 5); - assert config.repair_type_overrides.get(repairType).parallel_repair_count_in_group == 5; + assert config.repair_type_overrides.get(repairType).parallel_repair_count == 5; } @Test @@ -432,7 +432,7 @@ public void testGetDefaultOptionsMVRepairIsEnabledByDefault() { Options defaultOptions = Options.getDefaultOptions(); - assertTrue(defaultOptions.mv_repair_enabled); + assertFalse(defaultOptions.mv_repair_enabled); } @Test From 095d3211af7547f64a90610fb0be7244831c1c53 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Sat, 7 Sep 2024 16:32:20 -0700 Subject: [PATCH 016/115] AutoRepair scheduler InJvm dtest --- .../repair/autorepair/AutoRepair.java | 4 +- .../tools/nodetool/SSTableRepairedSet.java | 3 +- .../test/repair/AutoRepairSchedulerTest.java | 128 ++++++++++++++++++ .../config/DatabaseDescriptorRefTest.java | 9 ++ .../config/YamlConfigurationLoaderTest.java | 10 +- .../AutoRepairParameterizedTest.java | 1 - .../nodetool/SSTableRepairedSetTest.java | 27 ++-- 7 files changed, 159 insertions(+), 23 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index fbe94d8f7365..1a3c5071db36 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -115,7 +115,7 @@ public void setup() for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { repairExecutors.get(repairType).scheduleWithFixedDelay( - () -> repair(repairType, 60000), + () -> repair(repairType, 5000), config.getInitialSchedulerDelay(repairType).toSeconds(), config.getRepairCheckInterval().toSeconds(), TimeUnit.SECONDS); @@ -443,7 +443,7 @@ private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType if (timeInHours == 0 && millisToWait > 0) { //If repair finished quickly, happens for an empty instance, in such case - //wait for a minute so that the JMX metrics can detect the repairInProgress + //wait for some duration so that the JMX metrics can detect the repairInProgress logger.info("Wait for {} milliseconds for repair type {}.", millisToWait, repairType); Thread.sleep(millisToWait); } diff --git a/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java b/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java index f22f3ca0df40..e1773fe47b7f 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java +++ b/src/java/org/apache/cassandra/tools/nodetool/SSTableRepairedSet.java @@ -20,6 +20,7 @@ import java.io.PrintStream; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; @@ -69,7 +70,7 @@ public void execute(NodeProbe probe) message = "Previewing repaired state mutation of SSTables for"; List keyspaces = parseOptionalKeyspace(args, probe, KeyspaceSet.NON_LOCAL_STRATEGY); - List tables = List.of(parseOptionalTables(args)); + List tables = new ArrayList<>(Arrays.asList(parseOptionalTables(args))); if (args.isEmpty()) message += " all keyspaces"; diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java new file mode 100644 index 000000000000..49b4300dc50b --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.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.distributed.test.repair; + +import java.io.IOException; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.Assert; +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 org.apache.cassandra.repair.autorepair.AutoRepair; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairKeyspace; + +import static org.apache.cassandra.schema.SchemaConstants.DISTRIBUTED_KEYSPACE_NAME; +import static org.junit.Assert.assertEquals; + +public class AutoRepairSchedulerTest extends TestBaseImpl +{ + + private static Cluster cluster; + static SimpleDateFormat sdf; + + @BeforeClass + public static void init() throws IOException + { + // Define the expected date format pattern + String pattern = "EEE MMM dd HH:mm:ss z yyyy"; + // Create SimpleDateFormat object with the given pattern + sdf = new SimpleDateFormat(pattern); + sdf.setLenient(false); + cluster = Cluster.build(3).withConfig(config -> config + .set("auto_repair", + ImmutableMap.of( + "repair_type_overrides", + ImmutableMap.of(AutoRepairConfig.RepairType.full.toString(), + ImmutableMap.of( + "initial_scheduler_delay", "5s", + "enabled", "true", + "parallel_repair_count", "1", + "parallel_repair_percentage", "0", + "min_repair_interval", "1s"), + AutoRepairConfig.RepairType.incremental.toString(), + ImmutableMap.of( + "initial_scheduler_delay", "5s", + "enabled", "true", + "parallel_repair_count", "1", + "parallel_repair_percentage", "0", + "min_repair_interval", "1s")))) + .set("auto_repair.enabled", "true") + .set("auto_repair.repair_check_interval", "10s")).start(); + + cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};"); + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck text, v1 int, v2 int, PRIMARY KEY (pk, ck)) WITH read_repair='NONE'")); + } + + @Test + public void testScheduler() throws ParseException + { + // ensure there was no history of previous repair runs through the scheduler + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s", DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY), ConsistencyLevel.QUORUM); + assertEquals(0, rows.length); + + cluster.forEach(i -> i.runOnInstance(() -> { + try + { + AutoRepair.instance.setup(); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + })); + // wait for a couple of minutes for repair to go through on all three nodes + Uninterruptibles.sleepUninterruptibly(2, TimeUnit.MINUTES); + + validate(AutoRepairConfig.RepairType.full.toString()); + validate(AutoRepairConfig.RepairType.incremental.toString()); + } + + private void validate(String repairType) throws ParseException + { + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s where repair_type='%s'", DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, repairType), ConsistencyLevel.QUORUM); + assertEquals(3, rows.length); + for (int node = 0; node < rows.length; node++) + { + Object[] row = rows[node]; + // repair_type + Assert.assertEquals(repairType, row[0].toString()); + // host_id + Assert.assertEquals(String.format("00000000-0000-4000-8000-%012d", node + 1), row[1].toString()); + // ensure there is a legit repair_start_ts and repair_finish_ts + sdf.parse(row[2].toString()); + sdf.parse(row[3].toString()); + // the reason why the repair was scheduled + Assert.assertEquals("MY_TURN", row[4].toString()); + for (Object col : row) + { + System.out.println("Data:" + col); + } + System.out.println("====================================="); + } + } +} diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java index dea0d867eefe..490f2534baf2 100644 --- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java +++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java @@ -103,6 +103,14 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.config.ConfigurationLoader", "org.apache.cassandra.config.Config$CorruptedTombstoneStrategy", "org.apache.cassandra.config.Config$BatchlogEndpointStrategy", + "org.apache.cassandra.repair.autorepair.AutoRepairConfig", + "org.apache.cassandra.repair.autorepair.AutoRepairConfig$Options", + "org.apache.cassandra.repair.autorepair.DefaultAutoRepairTokenSplitter", + "org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter", + "org.apache.cassandra.repair.autorepair.FullRepairState", + "org.apache.cassandra.repair.autorepair.IncrementalRepairState", + "org.apache.cassandra.repair.autorepair.AutoRepairConfig$RepairType", + "org.apache.cassandra.repair.autorepair.AutoRepairState", "org.apache.cassandra.config.DatabaseDescriptor$ByteUnit", "org.apache.cassandra.config.DataRateSpec", "org.apache.cassandra.config.DataRateSpec$DataRateUnit", @@ -289,6 +297,7 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.utils.concurrent.RefCounted", "org.apache.cassandra.utils.concurrent.SelfRefCounted", "org.apache.cassandra.utils.concurrent.Transactional", + "org.apache.cassandra.utils.progress.ProgressListener", "org.apache.cassandra.utils.concurrent.UncheckedInterruptedException", "org.apache.cassandra.utils.StorageCompatibilityMode" }; diff --git a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java index 5f70585bb687..a99f998f2c21 100644 --- a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java +++ b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java @@ -275,11 +275,11 @@ public void fromMapTest() "optional", false, "enabled", true); Map autoRepairConfig = ImmutableMap.of("enabled", true, - "global_settings", ImmutableMap.of("repair_dc_groups", - ImmutableSet.of("all the groups")), + "global_settings", ImmutableMap.of("number_of_repair_threads", + 1), "repair_type_overrides", ImmutableMap.of( - "full", ImmutableMap.of("repair_dc_groups", - ImmutableSet.of("none of the groups")))); + "full", ImmutableMap.of("number_of_repair_threads", + 2))); Map map = new ImmutableMap.Builder() .put("storage_port", storagePort) .put("commitlog_sync", commitLogSync) @@ -300,7 +300,7 @@ public void fromMapTest() assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_send_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_receive_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) assertEquals(true, config.auto_repair.enabled); - assertEquals(6 * 60 * 60L, config.auto_repair.getAutoRepairTableMaxRepairTime(AutoRepairConfig.RepairType.incremental)); + assertEquals(new DurationSpec.IntSecondsBound("6h"), config.auto_repair.getAutoRepairTableMaxRepairTime(AutoRepairConfig.RepairType.incremental)); config.auto_repair.setMVRepairEnabled(AutoRepairConfig.RepairType.incremental, false); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 8c0ac1b7dc56..2120a3540035 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -31,7 +31,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; -import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.statements.schema.TableAttributes; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java index 139cde3d839d..bc168c0752d7 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/SSTableRepairedSetTest.java @@ -22,7 +22,6 @@ import java.io.PrintStream; import java.util.ArrayList; import java.util.Arrays; -import java.util.List; import org.junit.Before; import org.junit.Test; @@ -57,22 +56,22 @@ public void setUp() { @Test public void testNoKeyspace() { - when(probe.getNonLocalStrategyKeyspaces()).thenReturn(new ArrayList<>(List.of("ks1", "ks2"))); - when(probe.getKeyspaces()).thenReturn(new ArrayList<>(List.of("ks1", "ks2"))); - when(probe.getTablesForKeyspace("ks1")).thenReturn(new ArrayList<>(List.of("table1", "table2"))); - when(probe.getTablesForKeyspace("ks2")).thenReturn(new ArrayList<>(List.of("table3", "table4"))); + when(probe.getNonLocalStrategyKeyspaces()).thenReturn(new ArrayList<>(Arrays.asList("ks1", "ks2"))); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(Arrays.asList("ks1", "ks2"))); + when(probe.getTablesForKeyspace("ks1")).thenReturn(new ArrayList<>(Arrays.asList("table1", "table2"))); + when(probe.getTablesForKeyspace("ks2")).thenReturn(new ArrayList<>(Arrays.asList("table3", "table4"))); cmd.isRepaired = true; cmd.reallySet = true; cmd.execute(probe); - verify(probe, times(1)).mutateSSTableRepairedState(true, false, "ks1", List.of("table1", "table2")); - verify(probe, times(1)).mutateSSTableRepairedState(true, false, "ks2", List.of("table3", "table4")); + verify(probe, times(1)).mutateSSTableRepairedState(true, false, "ks1", Arrays.asList("table1", "table2")); + verify(probe, times(1)).mutateSSTableRepairedState(true, false, "ks2", Arrays.asList("table3", "table4")); } @Test public void testBothRepairedAndUnrepaired() { - cmd.args = List.of("keyspace"); + cmd.args = Arrays.asList("keyspace"); cmd.isRepaired = true; cmd.isUnrepaired = true; cmd.execute(probe); @@ -81,15 +80,15 @@ public void testBothRepairedAndUnrepaired() { @Test public void testNeitherRepairedNorUnrepaired() { - cmd.args = List.of("keyspace"); + cmd.args = Arrays.asList("keyspace"); cmd.execute(probe); verify(probe, never()).mutateSSTableRepairedState(anyBoolean(), anyBoolean(), anyString(), anyList()); } @Test public void testRepairedPreview() { - cmd.args = List.of("keyspace"); - when(probe.getKeyspaces()).thenReturn(new ArrayList<>(List.of("keyspace"))); + cmd.args = Arrays.asList("keyspace"); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(Arrays.asList("keyspace"))); cmd.isRepaired = true; cmd.execute(probe); verify(probe).mutateSSTableRepairedState(true, true, "keyspace", new ArrayList<>()); @@ -97,8 +96,8 @@ public void testRepairedPreview() { @Test public void testUnrepairedReallySet() { - cmd.args = List.of("keyspace"); - when(probe.getKeyspaces()).thenReturn(new ArrayList<>(List.of("keyspace"))); + cmd.args = Arrays.asList("keyspace"); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(Arrays.asList("keyspace"))); cmd.isUnrepaired = true; cmd.reallySet = true; cmd.execute(probe); @@ -108,7 +107,7 @@ public void testUnrepairedReallySet() { @Test public void testExecuteWithTableNames() { cmd.args = Arrays.asList("keyspace", "table1", "table2"); - when(probe.getKeyspaces()).thenReturn(new ArrayList<>(List.of("keyspace"))); + when(probe.getKeyspaces()).thenReturn(new ArrayList<>(Arrays.asList("keyspace"))); cmd.isRepaired = true; cmd.reallySet = true; cmd.execute(probe); From 0ac15d9d5d0b986c9ec118293cb7270c64b25a18 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Mon, 23 Sep 2024 18:19:36 -0700 Subject: [PATCH 017/115] Increase CircleCI parallelism 1-->4 --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 864919b8f418..553e0e32a8a7 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -5160,7 +5160,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra From b39b58f5c1b2cc36e1a72d5dc155f8685d2687e2 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 24 Sep 2024 14:07:04 -0700 Subject: [PATCH 018/115] Fix a test case failure org.apache.cassandra.schema.SchemaKeyspaceTest::testConversionsInverses --- src/java/org/apache/cassandra/schema/TableParams.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index 3bf45fc1c8cb..6578cbe75dd1 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -262,7 +262,7 @@ public boolean equals(Object o) && extensions.equals(p.extensions) && cdc == p.cdc && readRepair == p.readRepair - && automatedRepair == p.automatedRepair; + && automatedRepair.equals(p.automatedRepair); } @Override From f69d7d750f68bf3f27a73a877fdb46ed90dc5b52 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 24 Sep 2024 14:29:08 -0700 Subject: [PATCH 019/115] Trigger automatic CircleCI jobs --- .circleci/config.yml | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 553e0e32a8a7..456bd04063e6 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -9477,18 +9477,15 @@ workflows: java11_separate_tests: jobs: - start_j11_build: - type: approval - j11_build: requires: - start_j11_build - start_j11_unit_tests: - type: approval - j11_unit_tests: requires: - start_j11_unit_tests - j11_build - start_j11_jvm_dtests: - type: approval - j11_jvm_dtests: requires: - start_j11_jvm_dtests @@ -9518,13 +9515,11 @@ workflows: - start_j11_simulator_dtests - j11_build - start_j11_cqlshlib_tests: - type: approval - j11_cqlshlib_tests: requires: - start_j11_cqlshlib_tests - j11_build - start_j11_cqlshlib_cython_tests: - type: approval - j11_cqlshlib_cython_tests: requires: - start_j11_cqlshlib_cython_tests @@ -9656,13 +9651,11 @@ workflows: - start_jvm_upgrade_dtests - j11_dtest_jars_build - start_j11_dtests: - type: approval - j11_dtests: requires: - start_j11_dtests - j11_build - start_j11_dtests_vnode: - type: approval - j11_dtests_vnode: requires: - start_j11_dtests_vnode @@ -9692,13 +9685,11 @@ workflows: - start_j17_dtests_latest - j11_build - start_j11_dtests_large: - type: approval - j11_dtests_large: requires: - start_j11_dtests_large - j11_build - start_j11_dtests_large_vnode: - type: approval - j11_dtests_large_vnode: requires: - start_j11_dtests_large_vnode @@ -9780,7 +9771,6 @@ workflows: java11_pre-commit_tests: jobs: - start_pre-commit_tests: - type: approval - j11_build: requires: - start_pre-commit_tests @@ -9916,7 +9906,6 @@ workflows: requires: - j11_build - start_j11_dtests_large: - type: approval - j11_dtests_large: requires: - start_j11_dtests_large From 461f18ae34c81e15f6cd3bc21410c9f571befde5 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 24 Sep 2024 14:31:09 -0700 Subject: [PATCH 020/115] CircleCI ask for a manual approval --- .circleci/config.yml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/.circleci/config.yml b/.circleci/config.yml index 456bd04063e6..553e0e32a8a7 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -9477,15 +9477,18 @@ workflows: java11_separate_tests: jobs: - start_j11_build: + type: approval - j11_build: requires: - start_j11_build - start_j11_unit_tests: + type: approval - j11_unit_tests: requires: - start_j11_unit_tests - j11_build - start_j11_jvm_dtests: + type: approval - j11_jvm_dtests: requires: - start_j11_jvm_dtests @@ -9515,11 +9518,13 @@ workflows: - start_j11_simulator_dtests - j11_build - start_j11_cqlshlib_tests: + type: approval - j11_cqlshlib_tests: requires: - start_j11_cqlshlib_tests - j11_build - start_j11_cqlshlib_cython_tests: + type: approval - j11_cqlshlib_cython_tests: requires: - start_j11_cqlshlib_cython_tests @@ -9651,11 +9656,13 @@ workflows: - start_jvm_upgrade_dtests - j11_dtest_jars_build - start_j11_dtests: + type: approval - j11_dtests: requires: - start_j11_dtests - j11_build - start_j11_dtests_vnode: + type: approval - j11_dtests_vnode: requires: - start_j11_dtests_vnode @@ -9685,11 +9692,13 @@ workflows: - start_j17_dtests_latest - j11_build - start_j11_dtests_large: + type: approval - j11_dtests_large: requires: - start_j11_dtests_large - j11_build - start_j11_dtests_large_vnode: + type: approval - j11_dtests_large_vnode: requires: - start_j11_dtests_large_vnode @@ -9771,6 +9780,7 @@ workflows: java11_pre-commit_tests: jobs: - start_pre-commit_tests: + type: approval - j11_build: requires: - start_pre-commit_tests @@ -9906,6 +9916,7 @@ workflows: requires: - j11_build - start_j11_dtests_large: + type: approval - j11_dtests_large: requires: - start_j11_dtests_large From 4b58140d6a419adef33202a25c6b716b0ea0b9bc Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Tue, 24 Sep 2024 20:27:33 -0700 Subject: [PATCH 021/115] Fix the JMXStandardsTest::interfaces --- .../apache/cassandra/repair/autorepair/AutoRepairConfig.java | 2 +- .../apache/cassandra/config/YamlConfigurationLoaderTest.java | 1 - test/unit/org/apache/cassandra/tools/JMXStandardsTest.java | 5 +++++ 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index 97868a594f25..5c839063c036 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -56,7 +56,7 @@ public class AutoRepairConfig implements Serializable // global_settings overides Options.defaultOptions for all repair types public volatile Options global_settings; - public enum RepairType + public enum RepairType implements Serializable { full, incremental; diff --git a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java index a99f998f2c21..141c762ba461 100644 --- a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java +++ b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java @@ -31,7 +31,6 @@ import java.util.function.Predicate; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import org.junit.Test; import org.apache.cassandra.distributed.shared.WithProperties; diff --git a/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java b/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java index 62b6b9b5ec16..df0a2c1be2ea 100644 --- a/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java +++ b/test/unit/org/apache/cassandra/tools/JMXStandardsTest.java @@ -51,6 +51,8 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.utils.BreaksJMX; import org.assertj.core.api.Assertions; import org.reflections.Reflections; @@ -100,6 +102,9 @@ public class JMXStandardsTest .add(ClassNotFoundException.class) .add(OpenDataException.class) .add(InvalidRequestException.class) + .add(AutoRepairConfig.RepairType.class) + .add(InetAddressAndPort.class) + .add(AutoRepairConfig.class) .build(); /** * This list is a set of types under java.* and javax.*, but are too vague that could cause issues; this does not From 91911b4861a650ddbfd7c771f4409caa9560cc31 Mon Sep 17 00:00:00 2001 From: Jaydeepkumar Chovatia Date: Tue, 24 Sep 2024 21:00:25 -0700 Subject: [PATCH 022/115] Circle AutoStart J8 tests From 938d8d3e31ac811ba17161846101f0f03711cb6d Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Wed, 25 Sep 2024 21:15:17 -0700 Subject: [PATCH 023/115] Allow IR to run with MVs and CDC if write path replay is disabled --- .../config/CassandraRelevantProperties.java | 11 ++ .../db/streaming/CassandraStreamReceiver.java | 7 +- .../repair/autorepair/AutoRepair.java | 16 +-- .../repair/autorepair/AutoRepairConfig.java | 6 - .../cassandra/service/AutoRepairService.java | 18 +++ .../cassandra/streaming/StreamOperation.java | 4 +- .../streaming/CassandraSreamReceiverTest.java | 134 ++++++++++++++++++ .../autorepair/AutoRepairConfigTest.java | 57 -------- .../AutoRepairParameterizedTest.java | 58 ++++++++ .../repair/autorepair/AutoRepairTest.java | 16 ++- .../service/AutoRepairServiceTest.java | 63 +++++++- 11 files changed, 308 insertions(+), 82 deletions(-) create mode 100644 test/unit/org/apache/cassandra/db/streaming/CassandraSreamReceiverTest.java diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index 9862f227822d..3babc18fad81 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -624,6 +624,17 @@ public enum CassandraRelevantProperties /** Gossiper compute expiration timeout. Default value 3 days. */ VERY_LONG_TIME_MS("cassandra.very_long_time_ms", "259200000"), WAIT_FOR_TRACING_EVENTS_TIMEOUT_SECS("cassandra.wait_for_tracing_events_timeout_secs", "0"), + /** + * If set to true, mutations streamed during anti-entropy repair will be replayed via the regular write path for associated views. + */ + STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR("cassandra.streaming.requires_view_build_during_repair", "true"), + + /** + * If set to true, streamed mutations via the regular write path for CDC. + * Deprecate this property in trunk (or 5.0) as a new config has been added to control this https://issues.apache.org/jira/browse/CASSANDRA-17666 + */ + STREAMING_REQUIRES_CDC_REPLAY("cassandra.streaming.requires_cdc_replay", "true"), + ; static diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java index 50f87c799ece..2d80bace7d15 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java @@ -25,6 +25,11 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.io.sstable.SSTable; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -190,7 +195,7 @@ private boolean cdcRequiresWriteCommitLog(ColumnFamilyStore cfs) * For CDC-enabled tables and write path for CDC is enabled, we want to ensure that the mutations are * run through the CommitLog, so they can be archived by the CDC process on discard. */ - private boolean requiresWritePath(ColumnFamilyStore cfs) + public boolean requiresWritePath(ColumnFamilyStore cfs) { return cdcRequiresWriteCommitLog(cfs) || cfs.streamToMemtable() diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 1a3c5071db36..cef97595cffc 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -48,7 +48,6 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Tables; @@ -106,14 +105,15 @@ protected AutoRepair() public void setup() { - verifyIsSafeToEnable(); - AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); AutoRepairService.setup(); AutoRepairUtils.setup(); for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { + if (config.isAutoRepairEnabled(repairType)) + AutoRepairService.instance.checkCanRun(repairType); + repairExecutors.get(repairType).scheduleWithFixedDelay( () -> repair(repairType, 5000), config.getInitialSchedulerDelay(repairType).toSeconds(), @@ -122,15 +122,6 @@ public void setup() } } - @VisibleForTesting - protected void verifyIsSafeToEnable() - { - AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); - if (config.isAutoRepairEnabled(AutoRepairConfig.RepairType.incremental) && - (DatabaseDescriptor.getMaterializedViewsEnabled() || DatabaseDescriptor.isCDCEnabled())) - throw new ConfigurationException("Cannot enable incremental repair with materialized views or CDC enabled"); - } - // repairAsync runs a repair session of the given type asynchronously. public void repairAsync(AutoRepairConfig.RepairType repairType, long millisToWait) { @@ -146,6 +137,7 @@ public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) logger.debug("Auto-repair is disabled for repair type {}", repairType); return; } + AutoRepairService.instance.checkCanRun(repairType); AutoRepairState repairState = repairStates.get(repairType); try { diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index 5c839063c036..b43262e9a4e4 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -30,9 +30,7 @@ import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.DurationSpec; -import org.apache.cassandra.exceptions.ConfigurationException; public class AutoRepairConfig implements Serializable { @@ -140,10 +138,6 @@ public boolean isAutoRepairEnabled(RepairType repairType) public void setAutoRepairEnabled(RepairType repairType, boolean enabled) { - if (enabled && repairType == RepairType.incremental && - (DatabaseDescriptor.getMaterializedViewsEnabled() || DatabaseDescriptor.isCDCEnabled())) - throw new ConfigurationException("Cannot enable incremental repair with materialized views or CDC enabled"); - ensureOverrides(repairType); repair_type_overrides.get(repairType).enabled = enabled; } diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index cbef9889fb27..3d59207a8c28 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -17,7 +17,9 @@ */ package org.apache.cassandra.service; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; @@ -52,6 +54,21 @@ public static void setup() MBeanWrapper.instance.registerMBean(instance, MBEAN_NAME); } + public void checkCanRun(RepairType repairType) + { + if (!config.isAutoRepairSchedulingEnabled()) + throw new ConfigurationException("Auto-repair scheduller is disabled."); + + if (repairType != RepairType.incremental) + return; + + if (CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.getBoolean()) + throw new ConfigurationException("Cannot run incremental repair while materialized view replay is enabled."); + + if (CassandraRelevantProperties.STREAMING_REQUIRES_CDC_REPLAY.getBoolean()) + throw new ConfigurationException("Cannot run incremental repair while CDC replay is enabled."); + } + @Override public AutoRepairConfig getAutoRepairConfig() { @@ -61,6 +78,7 @@ public AutoRepairConfig getAutoRepairConfig() @Override public void setAutoRepairEnabled(RepairType repairType, boolean enabled) { + checkCanRun(repairType); config.setAutoRepairEnabled(repairType, enabled); } diff --git a/src/java/org/apache/cassandra/streaming/StreamOperation.java b/src/java/org/apache/cassandra/streaming/StreamOperation.java index 98a4070d2b0c..8c308830d230 100644 --- a/src/java/org/apache/cassandra/streaming/StreamOperation.java +++ b/src/java/org/apache/cassandra/streaming/StreamOperation.java @@ -17,6 +17,8 @@ */ package org.apache.cassandra.streaming; +import org.apache.cassandra.config.CassandraRelevantProperties; + public enum StreamOperation { OTHER("Other", true, false), // Fallback to avoid null types when deserializing from string @@ -26,7 +28,7 @@ public enum StreamOperation BOOTSTRAP("Bootstrap", false, true), REBUILD("Rebuild", false, true), BULK_LOAD("Bulk Load", true, false), - REPAIR("Repair", true, false); + REPAIR("Repair", CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.getBoolean(), false); private final String description; private final boolean requiresViewBuild; diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraSreamReceiverTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraSreamReceiverTest.java new file mode 100644 index 000000000000..43020feac991 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraSreamReceiverTest.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming; + +import org.junit.Before; +import org.junit.BeforeClass; +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.streaming.StreamOperation; +import org.apache.cassandra.streaming.StreamSession; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.when; + +public class CassandraSreamReceiverTest extends CQLTester +{ + @Mock + private StreamSession session; + + private static final String CDC_TABLE = "cdc_table"; + private static final String MV_TABLE = "mv_table"; + private static final String CDC_MV_TABLE = "cdc_mv_table"; + private static final String NO_CDC_MV_TABLE = "no_cdc_mv_table"; + + @BeforeClass + public static void beforeClass() + { + System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); + } + + @Before + public void setup() + { + MockitoAnnotations.initMocks(this); + QueryProcessor.executeInternal(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int) WITH cdc=true", KEYSPACE, CDC_TABLE)); + QueryProcessor.executeInternal(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int) WITH cdc=false", KEYSPACE, MV_TABLE)); + QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW IF NOT EXISTS %s.mv AS SELECT * FROM %s.%s WHERE pk IS NOT NULL PRIMARY KEY (pk)", KEYSPACE, KEYSPACE, MV_TABLE)); + QueryProcessor.executeInternal(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int) WITH cdc=true", KEYSPACE, CDC_MV_TABLE)); + QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW IF NOT EXISTS %s.mv2 AS SELECT * FROM %s.%s WHERE pk IS NOT NULL PRIMARY KEY (pk)", KEYSPACE, KEYSPACE, CDC_MV_TABLE)); + QueryProcessor.executeInternal(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int) WITH cdc=false", KEYSPACE, NO_CDC_MV_TABLE)); + } + + @Test + public void testRequiresWritePathRepair() + { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CDC_MV_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.REPAIR); + CassandraStreamReceiver receiver = new CassandraStreamReceiver(cfs, session, 1); + + assertTrue(receiver.requiresWritePath(cfs)); + } + + @Test + public void testRequiresWritePathBulkLoad() + { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CDC_MV_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.BULK_LOAD); + CassandraStreamReceiver receiver = new CassandraStreamReceiver(cfs, session, 1); + + assertTrue(receiver.requiresWritePath(cfs)); + } + + @Test + public void testRequiresWritePathNoCDCOrMV() + { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(NO_CDC_MV_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.BULK_LOAD); + CassandraStreamReceiver receiver = new CassandraStreamReceiver(cfs, session, 1); + + assertFalse(receiver.requiresWritePath(cfs)); + } + + @Test + public void testRequiresWritePathRepairMVOnly() + { + // Access the private field using reflection + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(MV_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.REPAIR); + CassandraStreamReceiver receiver = new CassandraStreamReceiver(cfs, session, 1); + + assertTrue(receiver.requiresWritePath(cfs)); + } + + @Test + public void testRequiresWritePathRepairCDCWithSystemProp() + { + System.setProperty("cassandra.streaming.requires_cdc_replay", "true"); + + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CDC_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.REPAIR); + CassandraStreamReceiver receiver = new CassandraStreamReceiver(cfs, session, 1); + + assertTrue(receiver.requiresWritePath(cfs)); + + System.clearProperty("cassandra.streaming.requires_cdc_replay"); + } + + @Test + public void testDoesNotRequiresWritePathRepairCDCOnly() + { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CDC_TABLE); + when(session.streamOperation()).thenReturn(StreamOperation.BULK_LOAD); + CassandraStreamReceiver receiver1 = new CassandraStreamReceiver(cfs, session, 1); + assertFalse(receiver1.requiresWritePath(cfs)); + + System.setProperty("cassandra.streaming.requires_cdc_replay", "true"); + CassandraStreamReceiver receiver2 = new CassandraStreamReceiver(cfs, session, 1); + assertTrue(receiver2.requiresWritePath(cfs)); + + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java index 07eb9070fe85..da7154077bc2 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -38,7 +38,6 @@ 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; @@ -119,62 +118,6 @@ public void testSetAutoRepairEnabledNoMVOrCDC() assertTrue(config.repair_type_overrides.get(repairType).enabled); } - @Test - public void testSetAutoRepairEnabledWithMV() - { - DatabaseDescriptor.setCDCEnabled(false); - DatabaseDescriptor.setMaterializedViewsEnabled(true); - - try - { - config.setAutoRepairEnabled(repairType, true); - - if (repairType == AutoRepairConfig.RepairType.incremental) - { - assertFalse(config.repair_type_overrides.get(repairType).enabled); // IR should not be allowed with MV - assertNotEquals(AutoRepairConfig.RepairType.incremental, repairType); // should receive exception - } - else - { - assertTrue(config.repair_type_overrides.get(repairType).enabled); - } - } - catch (ConfigurationException e) - { - // should throw only if repairType is incremental - assertEquals(AutoRepairConfig.RepairType.incremental, repairType); - } - } - - @Test - public void testSetAutoRepairEnabledWithCDC() - { - DatabaseDescriptor.setCDCEnabled(true); - DatabaseDescriptor.setMaterializedViewsEnabled(false); - - try - { - config.setAutoRepairEnabled(repairType, true); - - - if (repairType == AutoRepairConfig.RepairType.incremental) - { - assertFalse(config.repair_type_overrides.get(repairType).enabled); // IR should not be allowed with CDC - assertNotEquals(AutoRepairConfig.RepairType.incremental, repairType); // should receive exception - } - else - { - assertTrue(config.repair_type_overrides.get(repairType).enabled); - } - } - catch (ConfigurationException e) - { - // should throw only if repairType is incremental - assertEquals(AutoRepairConfig.RepairType.incremental, repairType); - } - } - - @Test public void testSetRepairByKeyspace() { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 2120a3540035..bef7b6d2cf56 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -35,11 +35,13 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.repair.RepairCoordinator; +import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.schema.AutoRepairParams; import org.apache.cassandra.schema.TableParams; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.Pair; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -77,6 +79,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.doAnswer; @@ -155,6 +158,8 @@ public static void setupClass() throws Exception @Before public void setup() { + System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); MockitoAnnotations.initMocks(this); Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE).truncateBlocking(); @@ -178,6 +183,12 @@ public void setup() AutoRepair.shuffleFunc = java.util.Collections::shuffle; } + @After + public void tearDown() + { + System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); + System.clearProperty("cassandra.streaming.requires_cdc_replay"); + } private void resetCounters() { @@ -677,4 +688,51 @@ public void testRepairSuccessAfterRetry() verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(0); } + + @Test + public void testRepairThrowsForIRWithMVReplay() + { + AutoRepair.instance.setup(); + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "true"); + + if (repairType == AutoRepairConfig.RepairType.incremental) + { + try + { + AutoRepair.instance.repair(repairType, 0); + fail("Expected ConfigurationException"); + } + catch (ConfigurationException ignored) + { + } + } + else + { + AutoRepair.instance.repair(repairType, 0); + } + } + + + @Test + public void testRepairThrowsForIRWithCDCReplay() + { + AutoRepair.instance.setup(); + System.setProperty("cassandra.streaming.requires_cdc_replay", "true"); + + if (repairType == AutoRepairConfig.RepairType.incremental) + { + try + { + AutoRepair.instance.repair(repairType, 0); + fail("Expected ConfigurationException"); + } + catch (ConfigurationException ignored) + { + } + } + else + { + AutoRepair.instance.repair(repairType, 0); + } + } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index c1837aa6a2cc..3592186365d1 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -47,6 +48,13 @@ public static void setupClass() throws Exception requireNetwork(); } + @Before + public void setup() + { + System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); + System.clearProperty("cassandra.streaming.requires_cdc_replay"); + } + @Test public void testSetup() { @@ -64,9 +72,10 @@ public void testSetup() } @Test(expected = ConfigurationException.class) - public void testSetupFailsWhenIREnabledWithCDC() + public void testSetupFailsWhenIREnabledWithCDCReplay() { DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + System.setProperty("cassandra.streaming.requires_cdc_replay", "true"); DatabaseDescriptor.setCDCEnabled(true); AutoRepair instance = new AutoRepair(); @@ -74,11 +83,10 @@ public void testSetupFailsWhenIREnabledWithCDC() } @Test(expected = ConfigurationException.class) - public void testSetupFailsWhenIREnabledWithMV() + public void testSetupFailsWhenIREnabledWithMVReplay() { DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); - DatabaseDescriptor.setMaterializedViewsEnabled(true); - + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "true"); AutoRepair instance = new AutoRepair(); instance.setup(); } diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java index 11df262f9500..1f12ad08ab65 100644 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java @@ -28,6 +28,7 @@ import java.util.stream.Stream; import com.google.common.collect.ImmutableSet; +import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -39,6 +40,7 @@ import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.autorepair.AutoRepairConfig; @@ -54,7 +56,7 @@ @Suite.SuiteClasses({ AutoRepairServiceTest.BasicTests.class, AutoRepairServiceTest.SetterTests.class }) public class AutoRepairServiceTest { - public static class BasicTests + public static class BasicTests extends CQLTester { private static AutoRepairService autoRepairService; private static AutoRepairConfig config; @@ -62,11 +64,21 @@ public static class BasicTests @Before public void setUp() { + System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + DatabaseDescriptor.setCDCEnabled(false); config = new AutoRepairConfig(); autoRepairService = new AutoRepairService(); autoRepairService.config = config; } + @After + public void tearDown() + { + System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); + System.clearProperty("cassandra.streaming.requires_cdc_replay"); + } @Test public void testSetup() @@ -109,6 +121,53 @@ public void testsetAutoRepairRetryBackoffInSec() assertEquals(102, config.getRepairRetryBackoff().toSeconds()); } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsWithSchedulerDisabled() + { + autoRepairService.config = new AutoRepairConfig(false); + + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsForIRWithMVReplay() + { + autoRepairService.config = new AutoRepairConfig(true); + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "true"); + + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test + public void testSetAutoRepairEnabledDoesNotThrowForIRWithMVReplayDisabled() + { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setMaterializedViewsEnabled(true); + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); + + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsForIRWithCDCReplay() + { + autoRepairService.config = new AutoRepairConfig(true); + System.setProperty("cassandra.streaming.requires_cdc_replay", "true"); + + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test + public void testSetAutoRepairEnabledDoesNotThrowForIRWithCDCReplayDisabled() + { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setCDCEnabled(true); + System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); + + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } } @RunWith(Parameterized.class) @@ -196,6 +255,8 @@ public void prepare() @Test public void testSetters() { + System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); setter.accept(repairType, arg); assertEquals(arg, getter.apply(repairType)); } From 8a348d7e277d68236fbcbab302bfc8ac202416ff Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Fri, 27 Sep 2024 16:10:43 -0700 Subject: [PATCH 024/115] Add a nodetool command to get the nodes going through repair --- .../repair/autorepair/AutoRepair.java | 5 + .../repair/autorepair/AutoRepairConfig.java | 19 ++-- .../repair/autorepair/AutoRepairUtils.java | 27 ++--- .../cassandra/service/AutoRepairService.java | 28 ++++- .../service/AutoRepairServiceMBean.java | 6 ++ .../org/apache/cassandra/tools/NodeProbe.java | 5 + .../org/apache/cassandra/tools/NodeTool.java | 1 + .../tools/nodetool/AutoRepairStatus.java | 70 ++++++++++++ .../tools/nodetool/GetAutoRepairConfig.java | 4 +- .../autorepair/AutoRepairConfigTest.java | 8 +- .../AutoRepairParameterizedTest.java | 7 ++ .../autorepair/AutoRepairUtilsTest.java | 17 ++- .../service/AutoRepairServiceTest.java | 51 ++++++++- .../tools/nodetool/AutoRepairStatusTest.java | 102 ++++++++++++++++++ 14 files changed, 304 insertions(+), 46 deletions(-) create mode 100644 src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java create mode 100644 test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index cef97595cffc..73a929ea0d45 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -37,6 +37,7 @@ import org.apache.cassandra.repair.RepairCoordinator; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.utils.Pair; import org.slf4j.Logger; @@ -125,6 +126,10 @@ public void setup() // repairAsync runs a repair session of the given type asynchronously. public void repairAsync(AutoRepairConfig.RepairType repairType, long millisToWait) { + if (!AutoRepairService.instance.getAutoRepairConfig().isAutoRepairEnabled(repairType)) + { + throw new ConfigurationException("Auto-repair is disabled for repair type " + repairType); + } repairExecutors.get(repairType).submit(() -> repair(repairType, millisToWait)); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index b43262e9a4e4..09e49d3d6cfc 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -230,26 +230,26 @@ public void setRepairPrimaryTokenRangeOnly(RepairType repairType, boolean primar repair_type_overrides.get(repairType).repair_primary_token_range_only = primaryTokenRangeOnly; } - public int getParallelRepairPercentageInGroup(RepairType repairType) + public int getParallelRepairPercentage(RepairType repairType) { return applyOverrides(repairType, opt -> opt.parallel_repair_percentage); } - public void setParallelRepairPercentageInGroup(RepairType repairType, int percentageInGroup) + public void setParallelRepairPercentage(RepairType repairType, int percentage) { ensureOverrides(repairType); - repair_type_overrides.get(repairType).parallel_repair_percentage = percentageInGroup; + repair_type_overrides.get(repairType).parallel_repair_percentage = percentage; } - public int getParallelRepairCountInGroup(RepairType repairType) + public int getParallelRepairCount(RepairType repairType) { return applyOverrides(repairType, opt -> opt.parallel_repair_count); } - public void setParallelRepairCountInGroup(RepairType repairType, int countInGroup) + public void setParallelRepairCount(RepairType repairType, int count) { ensureOverrides(repairType); - repair_type_overrides.get(repairType).parallel_repair_count = countInGroup; + repair_type_overrides.get(repairType).parallel_repair_count = count; } public boolean getMVRepairEnabled(RepairType repairType) @@ -305,9 +305,6 @@ public void setRepairSessionTimeout(RepairType repairType, String repairSessionT // All fields can be modified dynamically. public static class Options implements Serializable { - // The separator separating different DCs in repair_dc_groups - public static final String DC_GROUP_SEPARATOR = "\\|"; - // defaultOptions defines the default auto-repair behavior when no overrides are defined @VisibleForTesting protected static final Options defaultOptions = getDefaultOptions(); @@ -414,8 +411,8 @@ public String toString() ", repair_by_keyspace=" + repair_by_keyspace + ", number_of_subranges=" + number_of_subranges + ", number_of_repair_threads=" + number_of_repair_threads + - ", parallel_repair_count_in_group=" + parallel_repair_count + - ", parallel_repair_percentage_in_group=" + parallel_repair_percentage + + ", parallel_repair_count=" + parallel_repair_count + + ", parallel_repair_percentage=" + parallel_repair_percentage + ", sstable_upper_threshold=" + sstable_upper_threshold + ", min_repair_interval=" + min_repair_interval + ", ignore_dcs=" + ignore_dcs + diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index 02f9d04fee6e..29ffe9f14164 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -297,7 +297,7 @@ public String toString() } @VisibleForTesting - public static List getAutoRepairHistoryByGroupID(RepairType repairType) + public static List getAutoRepairHistory(RepairType repairType) { UntypedResultSet repairHistoryResult; @@ -328,11 +328,6 @@ public static List getAutoRepairHistoryByGroupID(RepairType r return null; } - public static List getAutoRepairHistoryForLocalGroup(RepairType repairType) - { - return getAutoRepairHistoryByGroupID(repairType); - } - // A host may add itself in delete hosts for some other hosts due to restart or some temp gossip issue. If a node's record // delete_hosts is not growing for more than 2 hours, we consider it as a normal node so we clear the delete_hosts for that node public static void clearDeleteHosts(RepairType repairType, UUID hostId) @@ -375,7 +370,7 @@ public static void setForceRepair(RepairType repairType, UUID hostId) public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType) { - List autoRepairHistories = getAutoRepairHistoryForLocalGroup(repairType); + List autoRepairHistories = getAutoRepairHistory(repairType); return getCurrentRepairStatus(repairType, autoRepairHistories); } @@ -446,7 +441,7 @@ public static TreeSet getHostIdsInCurrentRing(RepairType repairType) // This function will return the host ID for the node which has not been repaired for longest time public static AutoRepairHistory getHostWithLongestUnrepairTime(RepairType repairType) { - List autoRepairHistories = getAutoRepairHistoryForLocalGroup(repairType); + List autoRepairHistories = getAutoRepairHistory(repairType); return getHostWithLongestUnrepairTime(autoRepairHistories); } @@ -469,16 +464,16 @@ private static AutoRepairHistory getHostWithLongestUnrepairTime(List hostIdsInCurrentRing = getHostIdsInCurrentRing(repairType, allNodesInRing); logger.info("Total nodes qualified for repair {}", hostIdsInCurrentRing.size()); - List autoRepairHistories = getAutoRepairHistoryForLocalGroup(repairType); + List autoRepairHistories = getAutoRepairHistory(repairType); Set autoRepairHistoryIds = new HashSet<>(); // 1. Remove any node that is not part of group based on goissip info @@ -556,7 +551,7 @@ else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) } } - int parallelRepairNumber = getMaxNumberOfNodeRunAutoRepairInGroup(repairType, + int parallelRepairNumber = getMaxNumberOfNodeRunAutoRepair(repairType, autoRepairHistories == null ? 0 : autoRepairHistories.size()); logger.info("Will run repairs concurrently on {} node(s)", parallelRepairNumber); @@ -571,7 +566,7 @@ else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) else { // try to fetch again - autoRepairHistories = getAutoRepairHistoryForLocalGroup(repairType); + autoRepairHistories = getAutoRepairHistory(repairType); currentRepairStatus = getCurrentRepairStatus(repairType, autoRepairHistories); if (autoRepairHistories == null || currentRepairStatus == null) { @@ -635,7 +630,7 @@ else if (currentRepairStatus.hostIdsWithOnGoingForceRepair.contains(myId)) static void deleteAutoRepairHistory(RepairType repairType, UUID hostId) { - //delete the given hostId from current local group + //delete the given hostId delStatementRepairHistory.execute(QueryState.forInternalCalls(), QueryOptions.forInternalCalls(internalQueryCL, Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()), diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index 3d59207a8c28..c0b305f2974d 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -26,7 +26,10 @@ import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.utils.MBeanWrapper; +import java.util.HashSet; +import java.util.List; import java.util.Set; +import java.util.UUID; import com.google.common.annotations.VisibleForTesting; @@ -161,13 +164,13 @@ public void setPrimaryTokenRangeOnly(RepairType repairType, boolean primaryToken @Override public void setParallelRepairPercentageInGroup(RepairType repairType, int percentageInGroup) { - config.setParallelRepairPercentageInGroup(repairType, percentageInGroup); + config.setParallelRepairPercentage(repairType, percentageInGroup); } @Override public void setParallelRepairCountInGroup(RepairType repairType, int countInGroup) { - config.setParallelRepairCountInGroup(repairType, countInGroup); + config.setParallelRepairCount(repairType, countInGroup); } public void setMVRepairEnabled(RepairType repairType, boolean enabled) @@ -179,4 +182,25 @@ public void setRepairSessionTimeout(RepairType repairType, String timeout) { config.setRepairSessionTimeout(repairType, timeout); } + + @Override + public Set getOnGoingRepairHostIds(RepairType rType) + { + Set hostIds = new HashSet<>(); + List histories = AutoRepairUtils.getAutoRepairHistory(rType); + if (histories == null) + { + return null; + } + AutoRepairUtils.CurrentRepairStatus currentRepairStatus = new AutoRepairUtils.CurrentRepairStatus(histories, AutoRepairUtils.getPriorityHostIds(rType)); + for (UUID id : currentRepairStatus.hostIdsWithOnGoingRepair) + { + hostIds.add(id.toString()); + } + for (UUID id : currentRepairStatus.hostIdsWithOnGoingForceRepair) + { + hostIds.add(id.toString()); + } + return hostIds; + } } diff --git a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java index 4ebf118e023e..5e62e5630043 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java +++ b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java @@ -45,15 +45,19 @@ public interface AutoRepairServiceMBean public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration); public void setAutoRepairMaxRetriesCount(int retries); + public void setAutoRepairRetryBackoff(String interval); + public void setRepairSSTableCountHigherThreshold(RepairType repairType, int ssTableHigherThreshold); public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime); + public void setIgnoreDCs(RepairType repairType, Set ignorDCs); public void setPrimaryTokenRangeOnly(RepairType repairType, boolean primaryTokenRangeOnly); public void setParallelRepairPercentageInGroup(RepairType repairType, int percentageInGroup); + public void setParallelRepairCountInGroup(RepairType repairType, int countInGroup); public void setMVRepairEnabled(RepairType repairType, boolean enabled); @@ -61,4 +65,6 @@ public interface AutoRepairServiceMBean public AutoRepairConfig getAutoRepairConfig(); public void setRepairSessionTimeout(RepairType repairType, String timeout); + + public Set getOnGoingRepairHostIds(RepairType rType); } diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index b5b85d500303..e5620ac056fb 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -2515,6 +2515,11 @@ public void setRepairSessionTimeout(AutoRepairConfig.RepairType repairType, Stri { autoRepairProxy.setRepairSessionTimeout(repairType, timeout); } + + public Set getOnGoingRepairHostIds(AutoRepairConfig.RepairType type) + { + return autoRepairProxy.getOnGoingRepairHostIds(type); + } } class ColumnFamilyStoreMBeanIterator implements Iterator> diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index 02f71468f7d2..80f8ed8f09e1 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -95,6 +95,7 @@ public int execute(String... args) { List> commands = newArrayList( AbortBootstrap.class, + AutoRepairStatus.class, Assassinate.class, CassHelp.class, CIDRFilteringStats.class, diff --git a/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java b/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java new file mode 100644 index 000000000000..b29095572a2f --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.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.tools.nodetool; + +import java.io.PrintStream; +import java.util.Set; + +import com.google.common.annotations.VisibleForTesting; + +import io.airlift.airline.Command; +import io.airlift.airline.Option; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.NodeTool; +import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; + +import static com.google.common.base.Preconditions.checkArgument; + +@Command(name = "autorepairstatus", description = "Print autorepair status") +public class AutoRepairStatus extends NodeTool.NodeToolCmd +{ + @VisibleForTesting + @Option(title = "repair type", name = { "-t", "--repair-type" }, description = "Repair type") + protected AutoRepairConfig.RepairType repairType; + + @Override + public void execute(NodeProbe probe) + { + checkArgument(repairType != null, "--repair-type is required."); + PrintStream out = probe.output().out; + TableBuilder table = new TableBuilder(); + table.add("Active Repairs"); + Set ongoingRepairHostIds = probe.getOnGoingRepairHostIds(repairType); + table.add(getSetString(ongoingRepairHostIds)); + table.printTo(out); + } + + private String getSetString(Set hostIds) + { + if (hostIds.isEmpty()) + { + return "EMPTY"; + } + StringBuilder sb = new StringBuilder(); + for (String id : hostIds) + { + sb.append(id); + sb.append(","); + } + // remove last "," + sb.setLength(Math.max(sb.length() - 1, 0)); + return sb.toString(); + } +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java index 30dd1ff889c4..c62063b13b7f 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java @@ -70,8 +70,8 @@ private String formatRepairTypeConfig(NodeProbe probe, RepairType repairType, Au sb.append("\n\ttable max repair time in sec: " + config.getAutoRepairTableMaxRepairTime(repairType)); sb.append("\n\tignore datacenters: " + Joiner.on(',').skipNulls().join(config.getIgnoreDCs(repairType))); sb.append("\n\trepair primary token-range: " + config.getRepairPrimaryTokenRangeOnly(repairType)); - sb.append("\n\tnumber of parallel repairs within group: " + config.getParallelRepairCountInGroup(repairType)); - sb.append("\n\tpercentage of parallel repairs within group: " + config.getParallelRepairPercentageInGroup(repairType)); + sb.append("\n\tnumber of parallel repairs within group: " + config.getParallelRepairCount(repairType)); + sb.append("\n\tpercentage of parallel repairs within group: " + config.getParallelRepairPercentage(repairType)); sb.append("\n\tmv repair enabled: " + config.getMVRepairEnabled(repairType)); sb.append("\n\tinitial scheduler delay: " + config.getInitialSchedulerDelay(repairType)); sb.append("\n\trepair setssion timeout: " + config.getRepairSessionTimeout(repairType)); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java index da7154077bc2..163870df6120 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -285,7 +285,7 @@ public void testGetParallelRepairPercentageInGroup() { config.global_settings.parallel_repair_percentage = 5; - int result = config.getParallelRepairPercentageInGroup(repairType); + int result = config.getParallelRepairPercentage(repairType); assertEquals(5, result); } @@ -293,7 +293,7 @@ public void testGetParallelRepairPercentageInGroup() @Test public void testSetParallelRepairPercentageInGroup() { - config.setParallelRepairPercentageInGroup(repairType, 5); + config.setParallelRepairPercentage(repairType, 5); assert config.repair_type_overrides.get(repairType).parallel_repair_percentage == 5; } @@ -303,7 +303,7 @@ public void testGetParallelRepairCountInGroup() { config.global_settings.parallel_repair_count = 5; - int result = config.getParallelRepairCountInGroup(repairType); + int result = config.getParallelRepairCount(repairType); assertEquals(5, result); } @@ -311,7 +311,7 @@ public void testGetParallelRepairCountInGroup() @Test public void testSetParallelRepairCountInGroup() { - config.setParallelRepairCountInGroup(repairType, 5); + config.setParallelRepairCount(repairType, 5); assert config.repair_type_overrides.get(repairType).parallel_repair_count == 5; } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index bef7b6d2cf56..c423b0ea3b70 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -226,6 +226,13 @@ private void executeCQL() .forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); } + @Test(expected = ConfigurationException.class) + public void testRepairAsyncWithRepairTypeDisabled() + { + AutoRepairService.instance.getAutoRepairConfig().setAutoRepairEnabled(repairType, false); + + AutoRepair.instance.repairAsync(repairType, 60); + } @Test public void testRepairAsync() diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java index a00019577019..6ef9598fc7ba 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -173,8 +173,7 @@ public void testGetAutoRepairHistoryForLocalGroup() SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); - List history = AutoRepairUtils.getAutoRepairHistoryForLocalGroup(repairType); - + List history = AutoRepairUtils.getAutoRepairHistory(repairType); assertNotNull(history); assertEquals(1, history.size()); assertEquals(hostId, history.get(0).hostId); @@ -183,7 +182,7 @@ public void testGetAutoRepairHistoryForLocalGroup() @Test public void testGetAutoRepairHistoryForLocalGroup_empty_history() { - List history = AutoRepairUtils.getAutoRepairHistoryForLocalGroup(repairType); + List history = AutoRepairUtils.getAutoRepairHistory(repairType); assertNull(history); } @@ -272,9 +271,9 @@ public void testGetHostWithLongestUnrepairTime() @Test public void testGetMaxNumberOfNodeRunAutoRepairInGroup_0_group_size() { - DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCountInGroup(repairType, 2); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 2); - int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepairInGroup(repairType, 0); + int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepair(repairType, 0); assertEquals(2, count); } @@ -283,11 +282,11 @@ public void testGetMaxNumberOfNodeRunAutoRepairInGroup_0_group_size() @Test public void testGetMaxNumberOfNodeRunAutoRepairInGroup_percentage() { - DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCountInGroup(repairType, 2); - DatabaseDescriptor.getAutoRepairConfig().setParallelRepairPercentageInGroup(repairType, 50); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 2); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairPercentage(repairType, 50); - int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepairInGroup(repairType, 10); + int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepair(repairType, 10); assertEquals(5, count); } @@ -466,7 +465,7 @@ public void testMyTurnToRunRepairShouldReturnMyTurnWhenRepairOngoing() { UUID myID = UUID.randomUUID(); UUID otherID = UUID.randomUUID(); - DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCountInGroup(repairType, 5); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 5); long currentMillis = System.currentTimeMillis(); // finish time less than start time means that repair is ongoing AutoRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java index 1f12ad08ab65..e3c9c111db61 100644 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java @@ -50,6 +50,7 @@ import org.apache.cassandra.utils.FBUtilities; import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; import static org.junit.Assert.assertEquals; @RunWith(Suite.class) @@ -170,6 +171,52 @@ public void testSetAutoRepairEnabledDoesNotThrowForIRWithCDCReplayDisabled() } } + @RunWith(Parameterized.class) + public static class RepairTypeTests extends CQLTester + { + @Parameterized.Parameter() + public AutoRepairConfig.RepairType repairType; + + private final UUID host1 = UUID.fromString("00000000-0000-0000-0000-000000000001"); + private final UUID host2 = UUID.fromString("00000000-0000-0000-0000-000000000002"); + + private AutoRepairService instance; + + @Parameterized.Parameters(name = "repairType={0}") + public static Collection repairTypes() + { + return Arrays.asList(AutoRepairConfig.RepairType.values()); + } + + + @BeforeClass + public static void setupClass() throws Exception + { + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + setAutoRepairEnabled(true); + requireNetwork(); + } + + @Before + public void setUpTest() + { + AutoRepairUtils.setup(); + instance = new AutoRepairService(); + } + + @Test + public void testGetOnGoingRepairHostIds() + { + long now = System.currentTimeMillis(); + AutoRepairUtils.insertNewRepairHistory(repairType, host1, now, now - 1000000); + AutoRepairUtils.insertNewRepairHistory(repairType, host2, now, now - 1000000); + + Set hosts = instance.getOnGoingRepairHostIds(repairType); + + assertEquals(ImmutableSet.of(host1.toString(), host2.toString()), hosts); + } + } + @RunWith(Parameterized.class) public static class SetterTests extends CQLTester { @@ -197,8 +244,8 @@ public static Collection testCases() forEachRepairType(400, AutoRepairService.instance::setRepairSSTableCountHigherThreshold, config::getRepairSSTableCountHigherThreshold), forEachRepairType(ImmutableSet.of("dc1", "dc2"), AutoRepairService.instance::setIgnoreDCs, config::getIgnoreDCs), forEachRepairType(true, AutoRepairService.instance::setPrimaryTokenRangeOnly, config::getRepairPrimaryTokenRangeOnly), - forEachRepairType(600, AutoRepairService.instance::setParallelRepairPercentageInGroup, config::getParallelRepairPercentageInGroup), - forEachRepairType(700, AutoRepairService.instance::setParallelRepairCountInGroup, config::getParallelRepairCountInGroup), + forEachRepairType(600, AutoRepairService.instance::setParallelRepairPercentageInGroup, config::getParallelRepairPercentage), + forEachRepairType(700, AutoRepairService.instance::setParallelRepairCountInGroup, config::getParallelRepairCount), forEachRepairType(true, AutoRepairService.instance::setMVRepairEnabled, config::getMVRepairEnabled), forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setRepairPriorityForHosts, AutoRepairUtils::getPriorityHosts), forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setForceRepairForHosts, SetterTests::isLocalHostForceRepair) diff --git a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java new file mode 100644 index 000000000000..87ab1a2f02c7 --- /dev/null +++ b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.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.tools.nodetool; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.Arrays; +import java.util.Collection; + +import com.google.common.collect.ImmutableSet; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.Output; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.when; + +@RunWith(Parameterized.class) +public class AutoRepairStatusTest +{ + @Mock + private static NodeProbe probe; + + private ByteArrayOutputStream cmdOutput; + + @Mock + private static AutoRepairConfig config; + + private static AutoRepairStatus cmd; + + @Parameterized.Parameter() + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameters(name = "repairType={0}") + public static Collection repairTypes() + { + return Arrays.asList(AutoRepairConfig.RepairType.values()); + } + + @Before + public void setUp() + { + MockitoAnnotations.initMocks(this); + cmdOutput = new ByteArrayOutputStream(); + PrintStream out = new PrintStream(cmdOutput); + when(probe.output()).thenReturn(new Output(out, out)); + when(probe.getAutoRepairConfig()).thenReturn(config); + cmd = new AutoRepairStatus(); + } + + @Test(expected = IllegalArgumentException.class) + public void testExecuteWithoutRepairType() + { + cmd.repairType = null; + cmd.execute(probe); + } + + @Test + public void testExecuteWithNoNodes() + { + cmd.repairType = repairType; + + cmd.execute(probe); + assertEquals("Active Repairs\n" + + "EMPTY \n", cmdOutput.toString()); + } + + @Test + public void testExecute() + { + when(probe.getOnGoingRepairHostIds(repairType)).thenReturn(ImmutableSet.of("host1", "host2", "host3", "host4")); + cmd.repairType = repairType; + + cmd.execute(probe); + + assertEquals("Active Repairs \n" + + "host1,host2,host3,host4\n", cmdOutput.toString()); + } +} From 2c200a80ff02c7f4ceba16d0152c79d9df1c6493 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Fri, 27 Sep 2024 16:15:30 -0700 Subject: [PATCH 025/115] Adjust a couple of default configuration values --- .../apache/cassandra/repair/autorepair/AutoRepairConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index 09e49d3d6cfc..ac9394d9cfe1 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -49,7 +49,7 @@ public class AutoRepairConfig implements Serializable // the maximum number of retries for a repair session. public volatile Integer repair_max_retries = 3; // the backoff time in seconds for retrying a repair session. - public volatile DurationSpec.LongSecondsBound repair_retry_backoff = new DurationSpec.LongSecondsBound("60s"); + public volatile DurationSpec.LongSecondsBound repair_retry_backoff = new DurationSpec.LongSecondsBound("30s"); // global_settings overides Options.defaultOptions for all repair types public volatile Options global_settings; @@ -332,7 +332,7 @@ protected static Options getDefaultOptions() opts.table_max_repair_time = new DurationSpec.IntSecondsBound("6h"); opts.mv_repair_enabled = false; opts.token_range_splitter = DefaultAutoRepairTokenSplitter.class.getName(); - opts.initial_scheduler_delay = new DurationSpec.IntSecondsBound("15m"); // 15 minutes + opts.initial_scheduler_delay = new DurationSpec.IntSecondsBound("5m"); // 5 minutes opts.repair_session_timeout = new DurationSpec.IntSecondsBound("3h"); // 3 hours return opts; From 1648097677e340103d0afd31c29440b67d0b0177 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Fri, 27 Sep 2024 16:31:43 -0700 Subject: [PATCH 026/115] Rename EMPTY --> NONE for Nodetool autorepairstatus command --- .../org/apache/cassandra/tools/nodetool/AutoRepairStatus.java | 2 +- .../apache/cassandra/tools/nodetool/AutoRepairStatusTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java b/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java index b29095572a2f..882f2a4d3cda 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java +++ b/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java @@ -55,7 +55,7 @@ private String getSetString(Set hostIds) { if (hostIds.isEmpty()) { - return "EMPTY"; + return "NONE"; } StringBuilder sb = new StringBuilder(); for (String id : hostIds) diff --git a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java index 87ab1a2f02c7..a92f9ec960fe 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java @@ -85,7 +85,7 @@ public void testExecuteWithNoNodes() cmd.execute(probe); assertEquals("Active Repairs\n" + - "EMPTY \n", cmdOutput.toString()); + "NONE \n", cmdOutput.toString()); } @Test From 0881ee777a2772681029ce0ffc86e0c09447c5c1 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Fri, 27 Sep 2024 17:08:45 -0700 Subject: [PATCH 027/115] Minor code refactoring to simplify the repair API by removing the SLEEP_IF_REPAIR_FINISHES_QUICKLY --- .../repair/autorepair/AutoRepair.java | 22 ++++--- .../autorepair/AutoRepairConfigTest.java | 1 + .../autorepair/AutoRepairKeyspaceTest.java | 2 + .../AutoRepairParameterizedTest.java | 62 ++++++++++--------- .../autorepair/AutoRepairStateTest.java | 1 + .../repair/autorepair/AutoRepairTest.java | 2 + .../autorepair/AutoRepairUtilsTest.java | 2 + 7 files changed, 53 insertions(+), 39 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 73a929ea0d45..1df4ad668e96 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -37,6 +37,7 @@ import org.apache.cassandra.repair.RepairCoordinator; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.utils.Pair; @@ -71,6 +72,9 @@ public class AutoRepair public static AutoRepair instance = new AutoRepair(); + // Sleep for 5 seconds if repair finishes quickly to flush JMX metrics; it happens only for Cassandra nodes with tiny amount of data. + public static DurationSpec.IntSecondsBound SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("5s"); + @VisibleForTesting protected final Map repairExecutors; @@ -116,7 +120,7 @@ public void setup() AutoRepairService.instance.checkCanRun(repairType); repairExecutors.get(repairType).scheduleWithFixedDelay( - () -> repair(repairType, 5000), + () -> repair(repairType), config.getInitialSchedulerDelay(repairType).toSeconds(), config.getRepairCheckInterval().toSeconds(), TimeUnit.SECONDS); @@ -124,17 +128,17 @@ public void setup() } // repairAsync runs a repair session of the given type asynchronously. - public void repairAsync(AutoRepairConfig.RepairType repairType, long millisToWait) + public void repairAsync(AutoRepairConfig.RepairType repairType) { if (!AutoRepairService.instance.getAutoRepairConfig().isAutoRepairEnabled(repairType)) { throw new ConfigurationException("Auto-repair is disabled for repair type " + repairType); } - repairExecutors.get(repairType).submit(() -> repair(repairType, millisToWait)); + repairExecutors.get(repairType).submit(() -> repair(repairType)); } // repair runs a repair session of the given type synchronously. - public void repair(AutoRepairConfig.RepairType repairType, long millisToWait) + public void repair(AutoRepairConfig.RepairType repairType) { AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); if (!config.isAutoRepairEnabled(repairType)) @@ -351,7 +355,7 @@ else if (retryCount < config.getRepairMaxRetries()) } } } - cleanupAndUpdateStats(turn, repairType, repairState, myId, startTime, millisToWait, failedTokenRanges, succeededTokenRanges, skippedTokenRanges); + cleanupAndUpdateStats(turn, repairType, repairState, myId, startTime, failedTokenRanges, succeededTokenRanges, skippedTokenRanges); } else { @@ -410,7 +414,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon } private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, UUID myId, - long startTime, long millisToWait, int failedTokenRanges, int succeededTokenRanges, int skippedTokenRanges) throws InterruptedException + long startTime, int failedTokenRanges, int succeededTokenRanges, int skippedTokenRanges) throws InterruptedException { //if it was due to priority then remove it now if (turn == MY_TURN_DUE_TO_PRIORITY) @@ -437,12 +441,12 @@ private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType TimeUnit.SECONDS.toDays(repairState.getClusterRepairTimeInSec())); } repairState.setLastRepairTime(timeFunc.get()); - if (timeInHours == 0 && millisToWait > 0) + if (timeInHours == 0 && SLEEP_IF_REPAIR_FINISHES_QUICKLY.toSeconds() > 0) { //If repair finished quickly, happens for an empty instance, in such case //wait for some duration so that the JMX metrics can detect the repairInProgress - logger.info("Wait for {} milliseconds for repair type {}.", millisToWait, repairType); - Thread.sleep(millisToWait); + logger.info("Wait for {} for repair type {}.", SLEEP_IF_REPAIR_FINISHES_QUICKLY, repairType); + Thread.sleep(SLEEP_IF_REPAIR_FINISHES_QUICKLY.toMilliseconds()); } repairState.setRepairInProgress(false); AutoRepairUtils.updateFinishAutoRepairHistory(repairType, myId, timeFunc.get()); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java index 163870df6120..c42959eec32b 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -62,6 +62,7 @@ public void setUp() { config = new AutoRepairConfig(true); config.repair_type_overrides = null; + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); } @Test diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java index 579b996a12f5..43b8e4a5f4e8 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java @@ -27,6 +27,7 @@ import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.TableMetadata; @@ -43,6 +44,7 @@ public class AutoRepairKeyspaceTest public static void setupDatabaseDescriptor() { DatabaseDescriptor.daemonInitialization(); + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index c423b0ea3b70..ef38b7f0894e 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -31,6 +31,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; +import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.statements.schema.TableAttributes; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; @@ -118,6 +119,7 @@ public static Collection repairTypes() @BeforeClass public static void setupClass() throws Exception { + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); setAutoRepairEnabled(true); requireNetwork(); @@ -231,7 +233,7 @@ public void testRepairAsyncWithRepairTypeDisabled() { AutoRepairService.instance.getAutoRepairConfig().setAutoRepairEnabled(repairType, false); - AutoRepair.instance.repairAsync(repairType, 60); + AutoRepair.instance.repairAsync(repairType); } @Test @@ -239,7 +241,7 @@ public void testRepairAsync() { AutoRepair.instance.repairExecutors.put(repairType, mockExecutor); - AutoRepair.instance.repairAsync(repairType, 60); + AutoRepair.instance.repairAsync(repairType); verify(mockExecutor, Mockito.times(1)).submit(any(Runnable.class)); } @@ -255,7 +257,7 @@ public void testRepairTurn() public void testRepair() { AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); long lastRepairTime = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); @@ -270,7 +272,7 @@ public void testTooFrequentRepairs() AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); //in the first round let repair run config.setRepairMinInterval(repairType, "0s"); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); int consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); Assert.assertNotSame(String.format("Expected total repaired tables > 0, actual value %s ", consideredTables), @@ -278,7 +280,7 @@ public void testTooFrequentRepairs() //if repair was done in last 24 hours then it should not trigger another repair config.setRepairMinInterval(repairType, "24h"); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertEquals(String.format("Expected repair time to be same, actual value lastRepairTime1 %d, " + "lastRepairTime2 %d", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); @@ -293,14 +295,14 @@ public void testNonFrequentRepairs() AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); long prevCount = state.getTotalMVTablesConsideredForRepair(); AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertTrue(String.format("Expected lastRepairTime1 > 0, actual value lastRepairTime1 %d", lastRepairTime1), lastRepairTime1 > 0); UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); Assert.assertTrue("Expected my turn for the repair", AutoRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertNotSame(String.format("Expected repair time to be same, actual value lastRepairTime1 %d, " + "lastRepairTime2 ", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); @@ -320,9 +322,9 @@ public void testGetPriorityHosts() UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); Assert.assertTrue("Expected my turn for the repair", AutoRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); AutoRepairUtils.addPriorityHosts(repairType, Sets.newHashSet(FBUtilities.getBroadcastAddressAndPort())); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); Assert.assertSame(String.format("Priority host count is not same actual value %d, expected value %d", AutoRepairUtils.getPriorityHosts(repairType).size(), 0), AutoRepairUtils.getPriorityHosts(repairType).size(), 0); assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); @@ -339,14 +341,14 @@ public void testCheckAutoRepairStartStop() throws Throwable config.setRepairMinInterval(repairType, "0s"); config.setAutoRepairEnabled(repairType, false); long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); //Since repair has not happened, both the last repair times should be same Assert.assertEquals(String.format("Expected lastRepairTime1 %d, and lastRepairTime2 %d to be same", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); config.setAutoRepairEnabled(repairType, true); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); //since repair is done now, so lastRepairTime1/lastRepairTime2 and lastRepairTime3 should not be same long lastRepairTime3 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertNotSame(String.format("Expected lastRepairTime1 %d, and lastRepairTime3 %d to be not same", @@ -385,19 +387,19 @@ public void testMVRepair() config.setMVRepairEnabled(repairType, true); config.setRepairMinInterval(repairType, "0s"); AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); config.setMVRepairEnabled(repairType, false); AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); config.setMVRepairEnabled(repairType, true); AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); } @@ -436,7 +438,7 @@ public void testSkipRepairSSTableCountHigherThreshold() assertEquals(0, state.getSkippedTokenRangesCount()); assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); state.setLastRepairTime(0); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); assertEquals(1, state.getTotalMVTablesConsideredForRepair()); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); // skipping one time for the base table and another time for MV table @@ -447,7 +449,7 @@ public void testSkipRepairSSTableCountHigherThreshold() config.setRepairSSTableCountHigherThreshold(repairType, 11); config.setRepairSSTableCountHigherThreshold(repairType, beforeCount); state.setLastRepairTime(0); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); assertEquals(1, state.getTotalMVTablesConsideredForRepair()); assertEquals(0, state.getSkippedTokenRangesCount()); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); @@ -478,7 +480,7 @@ public void testMetrics() }; AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(1000L); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); assertTrue(AutoRepairMetricsManager.getMetrics(repairType).nodeRepairTimeInSec.getValue() > 0); @@ -495,7 +497,7 @@ public void testMetrics() when(autoRepairState.getSucceededTokenRangesCount()).thenReturn(11); when(autoRepairState.getLongestUnrepairedSec()).thenReturn(10); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); assertTrue(AutoRepairMetricsManager.getMetrics(repairType).failedTokenRangesCount.getValue() > 0); assertTrue(AutoRepairMetricsManager.getMetrics(repairType).succeededTokenRangesCount.getValue() > 0); @@ -527,9 +529,9 @@ public void testRepairWaitsForRepairToFinishBeforeSchedullingNewSession() throws return null; }).when(autoRepairState).waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); - AutoRepair.instance.repair(repairType, 0); - AutoRepair.instance.repair(repairType, 0); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); } @Test @@ -543,7 +545,7 @@ public void testDisabledAutoRepairForATableThroughTableLevelConfiguration() AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairMinInterval(repairType, "0s"); int disabledTablesRepairCountBefore = AutoRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); int consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); Assert.assertNotSame(String.format("Expected total repaired tables > 0, actual value %s ", consideredTables), consideredTables, 0); @@ -616,7 +618,7 @@ else if (list.get(0) instanceof String) AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairMinInterval(repairType, "0s"); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); assertEquals(1, shuffleKeyspacesCall.get()); assertEquals(4, shuffleTablesCall.get()); @@ -632,7 +634,7 @@ public void testRepairTakesLastRepairTimeFromDB() AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(0); config.setRepairMinInterval(repairType, "1h"); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); // repair scheduler should not attempt to run repair as last repair time in DB is current time - 1s assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair()); @@ -655,7 +657,7 @@ public void testRepairMaxRetries() config.setRepairMinInterval(repairType, "0s"); AutoRepair.instance.repairStates.put(repairType, autoRepairState); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); //system_auth.role_permissions,system_auth.network_permissions,system_auth.role_members,system_auth.roles, // system_auth.resource_role_permissons_index,system_traces.sessions,system_traces.events,ks.tbl, @@ -688,7 +690,7 @@ public void testRepairSuccessAfterRetry() }); config.setRepairMinInterval(repairType, "0s"); AutoRepair.instance.repairStates.put(repairType, autoRepairState); - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); assertEquals(1, sleepCalls.get()); verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(14); @@ -706,7 +708,7 @@ public void testRepairThrowsForIRWithMVReplay() { try { - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); fail("Expected ConfigurationException"); } catch (ConfigurationException ignored) @@ -715,7 +717,7 @@ public void testRepairThrowsForIRWithMVReplay() } else { - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); } } @@ -730,7 +732,7 @@ public void testRepairThrowsForIRWithCDCReplay() { try { - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); fail("Expected ConfigurationException"); } catch (ConfigurationException ignored) @@ -739,7 +741,7 @@ public void testRepairThrowsForIRWithCDCReplay() } else { - AutoRepair.instance.repair(repairType, 0); + AutoRepair.instance.repair(repairType); } } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java index 657b11310f64..f0974dd83c1a 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java @@ -68,6 +68,7 @@ public static Collection repairTypes() @Before public void setUp() { + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); initMocks(this); createTable(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int)", KEYSPACE, testTable)); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index 3592186365d1..6137983b3ea7 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -27,6 +27,7 @@ import org.junit.Assert; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.exceptions.ConfigurationException; @@ -51,6 +52,7 @@ public static void setupClass() throws Exception @Before public void setup() { + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); System.clearProperty("cassandra.streaming.requires_cdc_replay"); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java index 6ef9598fc7ba..5103fd74d22a 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -31,6 +31,7 @@ import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.marshal.UTF8Type; @@ -101,6 +102,7 @@ public static void setupClass() throws Exception @Before public void setup() { + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); MockitoAnnotations.initMocks(this); DatabaseDescriptor.setEndpointSnitch(defaultSnitch); QueryProcessor.executeInternal(String.format( From 9040b9cb9db5a6720e509c85757cc3416d619eb4 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Sun, 29 Sep 2024 21:53:26 -0700 Subject: [PATCH 028/115] Adjust AutoRepair on 5.0 --- .../config/CassandraRelevantProperties.java | 18 +- .../cassandra/metrics/AutoRepairMetrics.java | 1 + .../metrics/CassandraMetricsRegistry.java | 1 + .../repair/autorepair/AutoRepair.java | 1 - .../repair/autorepair/AutoRepairKeyspace.java | 4 +- .../schema/SystemDistributedKeyspace.java | 4 +- .../cassandra/service/AutoRepairService.java | 2 +- .../tcm/transformations/cms/Initialize.java | 1 + .../test/repair/AutoRepairSchedulerTest.java | 7 +- .../org/apache/cassandra/cql3/CQLTester.java | 2 + .../streaming/CassandraSreamReceiverTest.java | 11 +- .../AutoRepairParameterizedTest.java | 84 ++--- .../repair/autorepair/AutoRepairTest.java | 6 +- .../autorepair/AutoRepairUtilsTest.java | 18 +- .../service/AutoRepairServiceBasicTest.java | 126 +++++++ .../AutoRepairServiceRepairTypeTest.java | 79 +++++ .../service/AutoRepairServiceSetterTest.java | 132 ++++++++ .../service/AutoRepairServiceTest.java | 311 ------------------ 18 files changed, 404 insertions(+), 404 deletions(-) create mode 100644 test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java create mode 100644 test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java create mode 100644 test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java delete mode 100644 test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index 3babc18fad81..7598f4e7cf9f 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -507,7 +507,14 @@ public enum CassandraRelevantProperties STORAGE_HOOK("cassandra.storage_hook"), STORAGE_PORT("cassandra.storage_port"), STREAMING_HISTOGRAM_ROUND_SECONDS("cassandra.streaminghistogram.roundseconds", "60"), + + /** + * If set to true, mutations streamed during anti-entropy repair will be replayed via the regular write path for associated views. + */ + STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR("cassandra.streaming.requires_view_build_during_repair", "true"), + STREAMING_SESSION_PARALLELTRANSFERS("cassandra.streaming.session.parallelTransfers"), + STREAM_HOOK("cassandra.stream_hook"), /** Platform word size sun.arch.data.model. Examples: "32", "64", "unknown"*/ SUN_ARCH_DATA_MODEL("sun.arch.data.model"), @@ -624,17 +631,6 @@ public enum CassandraRelevantProperties /** Gossiper compute expiration timeout. Default value 3 days. */ VERY_LONG_TIME_MS("cassandra.very_long_time_ms", "259200000"), WAIT_FOR_TRACING_EVENTS_TIMEOUT_SECS("cassandra.wait_for_tracing_events_timeout_secs", "0"), - /** - * If set to true, mutations streamed during anti-entropy repair will be replayed via the regular write path for associated views. - */ - STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR("cassandra.streaming.requires_view_build_during_repair", "true"), - - /** - * If set to true, streamed mutations via the regular write path for CDC. - * Deprecate this property in trunk (or 5.0) as a new config has been added to control this https://issues.apache.org/jira/browse/CASSANDRA-17666 - */ - STREAMING_REQUIRES_CDC_REPLAY("cassandra.streaming.requires_cdc_replay", "true"), - ; static diff --git a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java index d765899631b7..dedd8f170170 100644 --- a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java +++ b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java @@ -31,6 +31,7 @@ */ public class AutoRepairMetrics { + public static final String TYPE_NAME = "autorepair"; public Gauge repairsInProgress; public Gauge nodeRepairTimeInSec; public Gauge clusterRepairTimeInSec; diff --git a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java index 8cf83f520870..919d3163e317 100644 --- a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java +++ b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java @@ -150,6 +150,7 @@ public class CassandraMetricsRegistry extends MetricRegistry .add(ThreadPoolMetrics.TYPE_NAME) .add(TrieMemtableMetricsView.TYPE_NAME) .add(UnweightedCacheMetrics.TYPE_NAME) + .add(AutoRepairMetrics.TYPE_NAME) .build(); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 1df4ad668e96..d59e4cfea643 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -50,7 +50,6 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Tables; import org.apache.cassandra.service.AutoRepairService; diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.java index 14cba17f6095..5afd2fff3275 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.java @@ -45,7 +45,7 @@ private AutoRepairKeyspace() public static final String AUTO_REPAIR_PRIORITY = "auto_repair_priority"; - private static final TableMetadata AutoRepairHistory = + public static final TableMetadata AutoRepairHistory = parse(AUTO_REPAIR_HISTORY, "Auto repair history for each node", "CREATE TABLE %s (" @@ -59,7 +59,7 @@ private AutoRepairKeyspace() + "force_repair boolean," + "PRIMARY KEY (repair_type, host_id))"); - private static final TableMetadata AutoRepairPriority = + public static final TableMetadata AutoRepairPriority = parse(AUTO_REPAIR_PRIORITY, "Auto repair priority for each group", "CREATE TABLE %s (" diff --git a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java index 08640007df1b..a6556abb7258 100644 --- a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java @@ -58,6 +58,8 @@ import static java.lang.String.format; +import static org.apache.cassandra.repair.autorepair.AutoRepairKeyspace.AutoRepairHistory; +import static org.apache.cassandra.repair.autorepair.AutoRepairKeyspace.AutoRepairPriority; import static org.apache.cassandra.utils.ByteBufferUtil.bytes; public final class SystemDistributedKeyspace @@ -171,7 +173,7 @@ public static KeyspaceMetadata metadata() { return KeyspaceMetadata.create(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, KeyspaceParams.simple(Math.max(DEFAULT_RF, DatabaseDescriptor.getDefaultKeyspaceRF())), - Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus, PartitionDenylistTable)); + Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus, PartitionDenylistTable, AutoRepairHistory, AutoRepairPriority)); } public static void startParentRepair(TimeUUID parent_id, String keyspaceName, String[] cfnames, RepairOption options) diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index c0b305f2974d..1adad42b263d 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -68,7 +68,7 @@ public void checkCanRun(RepairType repairType) if (CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.getBoolean()) throw new ConfigurationException("Cannot run incremental repair while materialized view replay is enabled."); - if (CassandraRelevantProperties.STREAMING_REQUIRES_CDC_REPLAY.getBoolean()) + if (DatabaseDescriptor.isCDCOnRepairEnabled()) throw new ConfigurationException("Cannot run incremental repair while CDC replay is enabled."); } diff --git a/src/java/org/apache/cassandra/tcm/transformations/cms/Initialize.java b/src/java/org/apache/cassandra/tcm/transformations/cms/Initialize.java index 11c2ed4d31af..635a7942260b 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/cms/Initialize.java +++ b/src/java/org/apache/cassandra/tcm/transformations/cms/Initialize.java @@ -24,6 +24,7 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.MetaStrategy; +import org.apache.cassandra.repair.autorepair.AutoRepairKeyspace; import org.apache.cassandra.schema.DistributedSchema; import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.schema.SystemDistributedKeyspace; diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java index 49b4300dc50b..5a49f669dc34 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java @@ -21,10 +21,12 @@ import java.io.IOException; import java.text.ParseException; import java.text.SimpleDateFormat; +import java.util.UUID; import java.util.concurrent.TimeUnit; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Uninterruptibles; +import org.apache.cassandra.config.DatabaseDescriptor; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -48,6 +50,8 @@ public class AutoRepairSchedulerTest extends TestBaseImpl @BeforeClass public static void init() throws IOException { + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + // Define the expected date format pattern String pattern = "EEE MMM dd HH:mm:ss z yyyy"; // Create SimpleDateFormat object with the given pattern @@ -88,6 +92,7 @@ public void testScheduler() throws ParseException cluster.forEach(i -> i.runOnInstance(() -> { try { + DatabaseDescriptor.setCDCOnRepairEnabled(false); AutoRepair.instance.setup(); } catch (Exception e) @@ -112,7 +117,7 @@ private void validate(String repairType) throws ParseException // repair_type Assert.assertEquals(repairType, row[0].toString()); // host_id - Assert.assertEquals(String.format("00000000-0000-4000-8000-%012d", node + 1), row[1].toString()); + UUID.fromString(row[1].toString()); // ensure there is a legit repair_start_ts and repair_finish_ts sdf.parse(row[2].toString()); sdf.parse(row[3].toString()); diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 54171f9a0885..4c3d130c9295 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -190,6 +190,7 @@ import org.awaitility.Awaitility; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT; +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; 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; @@ -445,6 +446,7 @@ protected static void prePrepareServer() DatabaseDescriptor.setRowCacheSizeInMiB(ROW_CACHE_SIZE_IN_MIB); StorageService.instance.registerMBeans(); StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance); + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); } @AfterClass diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraSreamReceiverTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraSreamReceiverTest.java index 43020feac991..2cc07b9c7ea6 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraSreamReceiverTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraSreamReceiverTest.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db.streaming; +import org.apache.cassandra.config.DatabaseDescriptor; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -48,7 +49,7 @@ public class CassandraSreamReceiverTest extends CQLTester @BeforeClass public static void beforeClass() { - System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); + DatabaseDescriptor.setCDCOnRepairEnabled(false); } @Before @@ -107,15 +108,11 @@ public void testRequiresWritePathRepairMVOnly() @Test public void testRequiresWritePathRepairCDCWithSystemProp() { - System.setProperty("cassandra.streaming.requires_cdc_replay", "true"); - ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CDC_TABLE); when(session.streamOperation()).thenReturn(StreamOperation.REPAIR); CassandraStreamReceiver receiver = new CassandraStreamReceiver(cfs, session, 1); - + DatabaseDescriptor.setCDCOnRepairEnabled(true); assertTrue(receiver.requiresWritePath(cfs)); - - System.clearProperty("cassandra.streaming.requires_cdc_replay"); } @Test @@ -126,8 +123,8 @@ public void testDoesNotRequiresWritePathRepairCDCOnly() CassandraStreamReceiver receiver1 = new CassandraStreamReceiver(cfs, session, 1); assertFalse(receiver1.requiresWritePath(cfs)); - System.setProperty("cassandra.streaming.requires_cdc_replay", "true"); CassandraStreamReceiver receiver2 = new CassandraStreamReceiver(cfs, session, 1); + DatabaseDescriptor.setCDCOnRepairEnabled(true); assertTrue(receiver2.requiresWritePath(cfs)); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index ef38b7f0894e..9978c36bb31f 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -28,7 +28,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import org.apache.cassandra.config.DurationSpec; @@ -37,8 +36,6 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.repair.RepairCoordinator; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.schema.AutoRepairParams; -import org.apache.cassandra.schema.TableParams; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.Pair; @@ -50,21 +47,15 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.concurrent.ScheduledExecutorPlus; import org.apache.cassandra.config.DatabaseDescriptor; 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.marshal.IntegerType; -import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.metrics.AutoRepairMetricsManager; import org.apache.cassandra.metrics.AutoRepairMetrics; -import org.apache.cassandra.schema.KeyspaceParams; -import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.AutoRepairService; @@ -119,48 +110,26 @@ public static Collection repairTypes() @BeforeClass public static void setupClass() throws Exception { - AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); setAutoRepairEnabled(true); requireNetwork(); AutoRepairUtils.setup(); - - - cfm = TableMetadata.builder(KEYSPACE, TABLE) - .addPartitionKeyColumn("k", UTF8Type.instance) - .addStaticColumn("s", UTF8Type.instance) - .addClusteringColumn("i", IntegerType.instance) - .addRegularColumn("v", UTF8Type.instance) - .params(TableParams.builder().automatedRepairFull(AutoRepairParams.create(AutoRepairConfig.RepairType.full, ImmutableMap.of(AutoRepairParams.Option.ENABLED.toString(), Boolean.toString(true)))). - automatedRepairIncremental(AutoRepairParams.create(AutoRepairConfig.RepairType.incremental, ImmutableMap.of(AutoRepairParams.Option.ENABLED.toString(), Boolean.toString(true)))).build()) - .build(); - - cfmDisabledAutoRepair = TableMetadata.builder(KEYSPACE, TABLE_DISABLED_AUTO_REPAIR) - .addPartitionKeyColumn("k", UTF8Type.instance) - .addStaticColumn("s", UTF8Type.instance) - .addClusteringColumn("i", IntegerType.instance) - .addRegularColumn("v", UTF8Type.instance) - .params(TableParams.builder().automatedRepairFull(AutoRepairParams.create(AutoRepairConfig.RepairType.full, ImmutableMap.of(AutoRepairParams.Option.ENABLED.toString(), Boolean.toString(false)))). - automatedRepairIncremental(AutoRepairParams.create(AutoRepairConfig.RepairType.incremental, ImmutableMap.of(AutoRepairParams.Option.ENABLED.toString(), Boolean.toString(false)))).build()) - .build(); - - SchemaLoader.prepareServer(); - SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), cfm, cfmDisabledAutoRepair); - cfm = Schema.instance.getTableMetadata(KEYSPACE, TABLE); - cfmDisabledAutoRepair = Schema.instance.getTableMetadata(KEYSPACE, TABLE_DISABLED_AUTO_REPAIR); - keyspace = Keyspace.open(KEYSPACE); - DatabaseDescriptor.setMaterializedViewsEnabled(true); - QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + - "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); - - DatabaseDescriptor.setMaterializedViewsEnabled(false); + StorageService.instance.doAutoRepairSetup(); DatabaseDescriptor.setCDCEnabled(false); } @Before public void setup() { - System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE)); + QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i))", KEYSPACE, TABLE)); + QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i)) WITH automated_repair_full = {'enabled': 'false'} AND automated_repair_incremental = {'enabled': 'false'}", KEYSPACE, TABLE_DISABLED_AUTO_REPAIR)); + + QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + + "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); MockitoAnnotations.initMocks(this); @@ -183,13 +152,17 @@ public void setup() resetConfig(); AutoRepair.shuffleFunc = java.util.Collections::shuffle; + + keyspace = Keyspace.open(KEYSPACE); + cfm = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE).metadata(); + cfmDisabledAutoRepair = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE_DISABLED_AUTO_REPAIR).metadata(); + DatabaseDescriptor.setCDCOnRepairEnabled(false); } @After public void tearDown() { System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); - System.clearProperty("cassandra.streaming.requires_cdc_replay"); } private void resetCounters() @@ -604,15 +577,18 @@ public void testRepairShufflesKeyspacesAndTables() AtomicInteger shuffleKeyspacesCall = new AtomicInteger(); AtomicInteger shuffleTablesCall = new AtomicInteger(); AutoRepair.shuffleFunc = (List list) -> { - assertTrue(list.get(0) instanceof Keyspace || list.get(0) instanceof String); - if (list.get(0) instanceof Keyspace) - { - shuffleKeyspacesCall.getAndIncrement(); - assertFalse(list.isEmpty()); - } - else if (list.get(0) instanceof String) + if (!list.isEmpty()) { - shuffleTablesCall.getAndIncrement(); + assertTrue(list.get(0) instanceof Keyspace || list.get(0) instanceof String); + if (list.get(0) instanceof Keyspace) + { + shuffleKeyspacesCall.getAndIncrement(); + assertFalse(list.isEmpty()); + } + else if (list.get(0) instanceof String) + { + shuffleTablesCall.getAndIncrement(); + } } }; @@ -621,7 +597,7 @@ else if (list.get(0) instanceof String) AutoRepair.instance.repair(repairType); assertEquals(1, shuffleKeyspacesCall.get()); - assertEquals(4, shuffleTablesCall.get()); + assertEquals(5, shuffleTablesCall.get()); } @Test @@ -663,7 +639,7 @@ public void testRepairMaxRetries() // system_auth.resource_role_permissons_index,system_traces.sessions,system_traces.events,ks.tbl, // system_distributed.auto_repair_priority,system_distributed.repair_history,system_distributed.auto_repair_history, // system_distributed.view_build_status,system_distributed.parent_repair_history,system_distributed.partition_denylist - int exptedTablesGoingThroughRepair = 14; + int exptedTablesGoingThroughRepair = 18; assertEquals(config.getRepairMaxRetries()*exptedTablesGoingThroughRepair, sleepCalls.get()); verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(0); verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); @@ -693,7 +669,7 @@ public void testRepairSuccessAfterRetry() AutoRepair.instance.repair(repairType); assertEquals(1, sleepCalls.get()); - verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(14); + verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(18); verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(0); } @@ -726,7 +702,7 @@ public void testRepairThrowsForIRWithMVReplay() public void testRepairThrowsForIRWithCDCReplay() { AutoRepair.instance.setup(); - System.setProperty("cassandra.streaming.requires_cdc_replay", "true"); + DatabaseDescriptor.setCDCOnRepairEnabled(true); if (repairType == AutoRepairConfig.RepairType.incremental) { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index 6137983b3ea7..eefc4556697a 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -54,7 +54,6 @@ public void setup() { AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); - System.clearProperty("cassandra.streaming.requires_cdc_replay"); } @Test @@ -76,9 +75,11 @@ public void testSetup() @Test(expected = ConfigurationException.class) public void testSetupFailsWhenIREnabledWithCDCReplay() { + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); - System.setProperty("cassandra.streaming.requires_cdc_replay", "true"); DatabaseDescriptor.setCDCEnabled(true); + DatabaseDescriptor.setCDCOnRepairEnabled(true); AutoRepair instance = new AutoRepair(); instance.setup(); @@ -89,6 +90,7 @@ public void testSetupFailsWhenIREnabledWithMVReplay() { DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); System.setProperty("cassandra.streaming.requires_view_build_during_repair", "true"); + DatabaseDescriptor.setCDCOnRepairEnabled(false); AutoRepair instance = new AutoRepair(); instance.setup(); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java index 5103fd74d22a..8cf5f5221b57 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -23,20 +23,16 @@ import java.util.TreeSet; import java.util.UUID; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; -import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.db.marshal.UUIDType; -import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; @@ -45,9 +41,7 @@ import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.QueryProcessor; -import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.SchemaConstants; -import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.membership.NodeAddresses; import org.apache.cassandra.utils.FBUtilities; @@ -90,18 +84,16 @@ public static void setupClass() throws Exception defaultSnitch = DatabaseDescriptor.getEndpointSnitch(); localEndpoint = FBUtilities.getBroadcastAddressAndPort(); hostId = StorageService.instance.getHostIdForEndpoint(localEndpoint); - AutoRepairUtils.setup(); - SchemaLoader.prepareServer(); - SchemaLoader.createKeyspace("ks", KeyspaceParams.create(false, - ImmutableMap.of("class", "NetworkTopologyStrategy", "datacenter1", "1")), - TableMetadata.builder("ks", "tbl") - .addPartitionKeyColumn("k", UTF8Type.instance) - .build()); + StorageService.instance.doAutoRepairSetup(); } @Before public void setup() { + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", "ks")); + QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i))", "ks", "tbl")); + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); MockitoAnnotations.initMocks(this); DatabaseDescriptor.setEndpointSnitch(defaultSnitch); diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java new file mode 100644 index 000000000000..826a7187219c --- /dev/null +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; + +import static org.junit.Assert.assertEquals; + +public class AutoRepairServiceBasicTest extends CQLTester { + private static AutoRepairService autoRepairService; + private static AutoRepairConfig config; + + @Before + public void setUp() { + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + DatabaseDescriptor.setCDCEnabled(false); + config = new AutoRepairConfig(); + autoRepairService = new AutoRepairService(); + autoRepairService.config = config; + } + + @After + public void tearDown() { + System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); + } + + @Test + public void testSetup() { + AutoRepairService.instance.config = null; + + AutoRepairService.setup(); + + assertEquals(DatabaseDescriptor.getAutoRepairConfig(), AutoRepairService.instance.config); + } + + @Test + public void testGetAutoRepairConfigReturnsConfig() { + assertEquals(config, autoRepairService.getAutoRepairConfig()); + } + + @Test + public void testsetAutoRepairHistoryClearDeleteHostsBufferInSecV2() { + autoRepairService.setAutoRepairHistoryClearDeleteHostsBufferDuration("100s"); + + assertEquals(100, config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds()); + } + + + @Test + public void testsetAutoRepairMaxRetriesCount() { + autoRepairService.setAutoRepairMaxRetriesCount(101); + + assertEquals(101, config.getRepairMaxRetries()); + } + + + @Test + public void testsetAutoRepairRetryBackoffInSec() { + autoRepairService.setAutoRepairRetryBackoff("102s"); + + assertEquals(102, config.getRepairRetryBackoff().toSeconds()); + } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsWithSchedulerDisabled() { + autoRepairService.config = new AutoRepairConfig(false); + + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsForIRWithMVReplay() { + autoRepairService.config = new AutoRepairConfig(true); + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "true"); + + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test + public void testSetAutoRepairEnabledDoesNotThrowForIRWithMVReplayDisabled() { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setMaterializedViewsEnabled(true); + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsForIRWithCDCReplay() { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setCDCOnRepairEnabled(true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test + public void testSetAutoRepairEnabledDoesNotThrowForIRWithCDCReplayDisabled() { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setCDCEnabled(true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } +} diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java new file mode 100644 index 000000000000..7c8645149adc --- /dev/null +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.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.service; + +import com.google.common.collect.ImmutableSet; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Set; +import java.util.UUID; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; +import static org.junit.Assert.assertEquals; + +@RunWith(Parameterized.class) +public class AutoRepairServiceRepairTypeTest extends CQLTester { + @Parameterized.Parameter() + public AutoRepairConfig.RepairType repairType; + + private final UUID host1 = UUID.fromString("00000000-0000-0000-0000-000000000001"); + private final UUID host2 = UUID.fromString("00000000-0000-0000-0000-000000000002"); + + private AutoRepairService instance; + + @Parameterized.Parameters(name = "repairType={0}") + public static Collection repairTypes() { + return Arrays.asList(AutoRepairConfig.RepairType.values()); + } + + + @BeforeClass + public static void setupClass() throws Exception { + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + setAutoRepairEnabled(true); + requireNetwork(); + } + + @Before + public void setUpTest() { + AutoRepairUtils.setup(); + instance = new AutoRepairService(); + } + + @Test + public void testGetOnGoingRepairHostIdsTest() { + long now = System.currentTimeMillis(); + AutoRepairUtils.insertNewRepairHistory(repairType, host1, now, now - 1000000); + AutoRepairUtils.insertNewRepairHistory(repairType, host2, now, now - 1000000); + + Set hosts = instance.getOnGoingRepairHostIds(repairType); + + assertEquals(ImmutableSet.of(host1.toString(), host2.toString()), hosts); + } +} diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java new file mode 100644 index 000000000000..31aae67c557e --- /dev/null +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.collect.ImmutableSet; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairKeyspace; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.schema.SchemaConstants; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Set; +import java.util.UUID; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.junit.Assert.assertEquals; + +@RunWith(Parameterized.class) +public class AutoRepairServiceSetterTest extends CQLTester { + private static final AutoRepairConfig config = new AutoRepairConfig(true); + + @Parameterized.Parameter + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameter(1) + public T arg; + + @Parameterized.Parameter(2) + public BiConsumer setter; + + @Parameterized.Parameter(3) + public Function getter; + + @Parameterized.Parameters(name = "{index}: repairType={0}, arg={1}") + public static Collection testCases() { + DatabaseDescriptor.setConfig(DatabaseDescriptor.loadConfig()); + return Stream.of( + forEachRepairType(true, AutoRepairService.instance::setAutoRepairEnabled, config::isAutoRepairEnabled), + forEachRepairType(100, AutoRepairService.instance::setRepairThreads, config::getRepairThreads), + forEachRepairType(200, AutoRepairService.instance::setRepairSubRangeNum, config::getRepairSubRangeNum), + forEachRepairType(400, AutoRepairService.instance::setRepairSSTableCountHigherThreshold, config::getRepairSSTableCountHigherThreshold), + forEachRepairType(ImmutableSet.of("dc1", "dc2"), AutoRepairService.instance::setIgnoreDCs, config::getIgnoreDCs), + forEachRepairType(true, AutoRepairService.instance::setPrimaryTokenRangeOnly, config::getRepairPrimaryTokenRangeOnly), + forEachRepairType(600, AutoRepairService.instance::setParallelRepairPercentageInGroup, config::getParallelRepairPercentage), + forEachRepairType(700, AutoRepairService.instance::setParallelRepairCountInGroup, config::getParallelRepairCount), + forEachRepairType(true, AutoRepairService.instance::setMVRepairEnabled, config::getMVRepairEnabled), + forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setRepairPriorityForHosts, AutoRepairUtils::getPriorityHosts), + forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setForceRepairForHosts, AutoRepairServiceSetterTest::isLocalHostForceRepair) + ).flatMap(Function.identity()).collect(Collectors.toList()); + } + + private static Set isLocalHostForceRepair(AutoRepairConfig.RepairType type) { + UUID hostId = StorageService.instance.getHostIdForEndpoint(InetAddressAndPort.getLocalHost()); + UntypedResultSet resultSet = QueryProcessor.executeInternal(String.format( + "SELECT force_repair FROM %s.%s WHERE host_id = %s and repair_type = '%s'", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, hostId, type)); + + if (!resultSet.isEmpty() && resultSet.one().getBoolean("force_repair")) { + return ImmutableSet.of(InetAddressAndPort.getLocalHost()); + } + return ImmutableSet.of(); + } + + private static Stream forEachRepairType(T arg, BiConsumer setter, Function getter) { + Object[][] testCases = new Object[AutoRepairConfig.RepairType.values().length][4]; + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { + testCases[repairType.ordinal()] = new Object[]{repairType, arg, setter, getter}; + } + + return Arrays.stream(testCases); + } + + @BeforeClass + public static void setup() throws Exception { + DatabaseDescriptor.daemonInitialization(); + setAutoRepairEnabled(true); + requireNetwork(); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + DatabaseDescriptor.setCDCEnabled(false); + AutoRepairUtils.setup(); + AutoRepairService.instance.config = config; + } + + @Before + public void prepare() { + QueryProcessor.executeInternal(String.format( + "TRUNCATE %s.%s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY)); + QueryProcessor.executeInternal(String.format( + "TRUNCATE %s.%s", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY)); + } + + @Test + public void testSettersTest() { + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + DatabaseDescriptor.setCDCOnRepairEnabled(false); + setter.accept(repairType, arg); + assertEquals(arg, getter.apply(repairType)); + } +} diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java deleted file mode 100644 index e3c9c111db61..000000000000 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceTest.java +++ /dev/null @@ -1,311 +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; - -import java.util.Arrays; -import java.util.Collection; -import java.util.Set; -import java.util.UUID; -import java.util.function.BiConsumer; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import com.google.common.collect.ImmutableSet; -import org.junit.After; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Suite; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.cql3.QueryProcessor; -import org.apache.cassandra.cql3.UntypedResultSet; -import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; -import org.apache.cassandra.repair.autorepair.AutoRepairKeyspace; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils; -import org.apache.cassandra.schema.SchemaConstants; -import org.apache.cassandra.utils.FBUtilities; - -import static org.apache.cassandra.Util.setAutoRepairEnabled; -import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; -import static org.junit.Assert.assertEquals; - -@RunWith(Suite.class) -@Suite.SuiteClasses({ AutoRepairServiceTest.BasicTests.class, AutoRepairServiceTest.SetterTests.class }) -public class AutoRepairServiceTest -{ - public static class BasicTests extends CQLTester - { - private static AutoRepairService autoRepairService; - private static AutoRepairConfig config; - - @Before - public void setUp() - { - System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); - DatabaseDescriptor.setMaterializedViewsEnabled(false); - DatabaseDescriptor.setCDCEnabled(false); - config = new AutoRepairConfig(); - autoRepairService = new AutoRepairService(); - autoRepairService.config = config; - } - - @After - public void tearDown() - { - System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); - System.clearProperty("cassandra.streaming.requires_cdc_replay"); - } - - @Test - public void testSetup() - { - AutoRepairService.instance.config = null; - - AutoRepairService.setup(); - - assertEquals(DatabaseDescriptor.getAutoRepairConfig(), AutoRepairService.instance.config); - } - - @Test - public void testGetAutoRepairConfigReturnsConfig() - { - assertEquals(config, autoRepairService.getAutoRepairConfig()); - } - - @Test - public void testsetAutoRepairHistoryClearDeleteHostsBufferInSecV2() - { - autoRepairService.setAutoRepairHistoryClearDeleteHostsBufferDuration("100s"); - - assertEquals(100, config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds()); - } - - - @Test - public void testsetAutoRepairMaxRetriesCount() - { - autoRepairService.setAutoRepairMaxRetriesCount(101); - - assertEquals(101, config.getRepairMaxRetries()); - } - - - @Test - public void testsetAutoRepairRetryBackoffInSec() - { - autoRepairService.setAutoRepairRetryBackoff("102s"); - - assertEquals(102, config.getRepairRetryBackoff().toSeconds()); - } - - @Test(expected = ConfigurationException.class) - public void testSetAutoRepairEnabledThrowsWithSchedulerDisabled() - { - autoRepairService.config = new AutoRepairConfig(false); - - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); - } - - @Test(expected = ConfigurationException.class) - public void testSetAutoRepairEnabledThrowsForIRWithMVReplay() - { - autoRepairService.config = new AutoRepairConfig(true); - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "true"); - - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); - } - - @Test - public void testSetAutoRepairEnabledDoesNotThrowForIRWithMVReplayDisabled() - { - autoRepairService.config = new AutoRepairConfig(true); - DatabaseDescriptor.setMaterializedViewsEnabled(true); - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); - System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); - - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); - } - - @Test(expected = ConfigurationException.class) - public void testSetAutoRepairEnabledThrowsForIRWithCDCReplay() - { - autoRepairService.config = new AutoRepairConfig(true); - System.setProperty("cassandra.streaming.requires_cdc_replay", "true"); - - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); - } - - @Test - public void testSetAutoRepairEnabledDoesNotThrowForIRWithCDCReplayDisabled() - { - autoRepairService.config = new AutoRepairConfig(true); - DatabaseDescriptor.setCDCEnabled(true); - System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); - - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); - } - } - - @RunWith(Parameterized.class) - public static class RepairTypeTests extends CQLTester - { - @Parameterized.Parameter() - public AutoRepairConfig.RepairType repairType; - - private final UUID host1 = UUID.fromString("00000000-0000-0000-0000-000000000001"); - private final UUID host2 = UUID.fromString("00000000-0000-0000-0000-000000000002"); - - private AutoRepairService instance; - - @Parameterized.Parameters(name = "repairType={0}") - public static Collection repairTypes() - { - return Arrays.asList(AutoRepairConfig.RepairType.values()); - } - - - @BeforeClass - public static void setupClass() throws Exception - { - SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); - setAutoRepairEnabled(true); - requireNetwork(); - } - - @Before - public void setUpTest() - { - AutoRepairUtils.setup(); - instance = new AutoRepairService(); - } - - @Test - public void testGetOnGoingRepairHostIds() - { - long now = System.currentTimeMillis(); - AutoRepairUtils.insertNewRepairHistory(repairType, host1, now, now - 1000000); - AutoRepairUtils.insertNewRepairHistory(repairType, host2, now, now - 1000000); - - Set hosts = instance.getOnGoingRepairHostIds(repairType); - - assertEquals(ImmutableSet.of(host1.toString(), host2.toString()), hosts); - } - } - - @RunWith(Parameterized.class) - public static class SetterTests extends CQLTester - { - private static final AutoRepairConfig config = new AutoRepairConfig(true); - - @Parameterized.Parameter - public AutoRepairConfig.RepairType repairType; - - @Parameterized.Parameter(1) - public T arg; - - @Parameterized.Parameter(2) - public BiConsumer setter; - - @Parameterized.Parameter(3) - public Function getter; - - @Parameterized.Parameters(name = "{index}: repairType={0}, arg={1}") - public static Collection testCases() - { - return Stream.of( - forEachRepairType(true, AutoRepairService.instance::setAutoRepairEnabled, config::isAutoRepairEnabled), - forEachRepairType(100, AutoRepairService.instance::setRepairThreads, config::getRepairThreads), - forEachRepairType(200, AutoRepairService.instance::setRepairSubRangeNum, config::getRepairSubRangeNum), - forEachRepairType(400, AutoRepairService.instance::setRepairSSTableCountHigherThreshold, config::getRepairSSTableCountHigherThreshold), - forEachRepairType(ImmutableSet.of("dc1", "dc2"), AutoRepairService.instance::setIgnoreDCs, config::getIgnoreDCs), - forEachRepairType(true, AutoRepairService.instance::setPrimaryTokenRangeOnly, config::getRepairPrimaryTokenRangeOnly), - forEachRepairType(600, AutoRepairService.instance::setParallelRepairPercentageInGroup, config::getParallelRepairPercentage), - forEachRepairType(700, AutoRepairService.instance::setParallelRepairCountInGroup, config::getParallelRepairCount), - forEachRepairType(true, AutoRepairService.instance::setMVRepairEnabled, config::getMVRepairEnabled), - forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setRepairPriorityForHosts, AutoRepairUtils::getPriorityHosts), - forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setForceRepairForHosts, SetterTests::isLocalHostForceRepair) - ).flatMap(Function.identity()).collect(Collectors.toList()); - } - - private static Set isLocalHostForceRepair(AutoRepairConfig.RepairType type) - { - UUID hostId = StorageService.instance.getHostIdForEndpoint(InetAddressAndPort.getLocalHost()); - UntypedResultSet resultSet = QueryProcessor.executeInternal(String.format( - "SELECT force_repair FROM %s.%s WHERE host_id = %s and repair_type = '%s'", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, hostId, type)); - - if (!resultSet.isEmpty() && resultSet.one().getBoolean("force_repair")) - { - return ImmutableSet.of(InetAddressAndPort.getLocalHost()); - } - return ImmutableSet.of(); - } - - private static Stream forEachRepairType(T arg, BiConsumer setter, Function getter) - { - Object[][] testCases = new Object[AutoRepairConfig.RepairType.values().length][4]; - for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) - { - testCases[repairType.ordinal()] = new Object[]{ repairType, arg, setter, getter }; - } - - return Arrays.stream(testCases); - } - - @BeforeClass - public static void setup() throws Exception - { - setAutoRepairEnabled(true); - requireNetwork(); - DatabaseDescriptor.setMaterializedViewsEnabled(false); - DatabaseDescriptor.setCDCEnabled(false); - AutoRepairUtils.setup(); - AutoRepairService.instance.config = config; - } - - @Before - public void prepare() - { - QueryProcessor.executeInternal(String.format( - "TRUNCATE %s.%s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY)); - QueryProcessor.executeInternal(String.format( - "TRUNCATE %s.%s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY)); - } - - @Test - public void testSetters() - { - System.setProperty("cassandra.streaming.requires_cdc_replay", "false"); - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); - setter.accept(repairType, arg); - assertEquals(arg, getter.apply(repairType)); - } - } -} From 08ec734b39e32160dd3ca5cb06dcb3b32ac41c51 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Mon, 30 Sep 2024 11:59:25 -0700 Subject: [PATCH 029/115] Fix unit test cases DescribeStatementTest & SchemaCQLHelperTest --- .../cassandra/cql3/statements/DescribeStatementTest.java | 8 ++++++-- .../unit/org/apache/cassandra/db/SchemaCQLHelperTest.java | 4 +++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java index 1bef2d82cfa2..04fc04226915 100644 --- a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java +++ b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java @@ -1074,7 +1074,9 @@ private static String tableParametersCql() " AND memtable_flush_period_in_ms = 0\n" + " AND min_index_interval = 128\n" + " AND read_repair = 'BLOCKING'\n" + - " AND speculative_retry = '99p';"; + " AND speculative_retry = '99p'\n" + + " AND automated_repair_full = {'enabled': 'true'}\n" + + " AND automated_repair_incremental = {'enabled': 'true'};"; } private static String cqlQuoted(Map map) @@ -1101,7 +1103,9 @@ private static String mvParametersCql() " AND memtable_flush_period_in_ms = 0\n" + " AND min_index_interval = 128\n" + " AND read_repair = 'BLOCKING'\n" + - " AND speculative_retry = '99p';"; + " AND speculative_retry = '99p'\n" + + " AND automated_repair_full = {'enabled': 'true'}\n" + + " AND automated_repair_incremental = {'enabled': 'true'};"; } private static String keyspaceOutput() diff --git a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java index 75db5bd3c613..dce36f16255a 100644 --- a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java +++ b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java @@ -306,7 +306,9 @@ public void testCfmOptionsCQL() " AND memtable_flush_period_in_ms = 8\n" + " AND min_index_interval = 6\n" + " AND read_repair = 'BLOCKING'\n" + - " AND speculative_retry = 'ALWAYS';" + " AND speculative_retry = 'ALWAYS'\n" + + " AND automated_repair_full = {'enabled': 'true'}\n" + + " AND automated_repair_incremental = {'enabled': 'true'};" )); } From 6d301029c73363ef67329895e021c6f8bcfbc0e1 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Mon, 30 Sep 2024 13:42:40 -0700 Subject: [PATCH 030/115] Safeguard the setup call to ensure it it done only once --- .../repair/autorepair/AutoRepair.java | 37 ++++++++++++------- .../repair/autorepair/AutoRepairTest.java | 27 ++++++++++++-- 2 files changed, 48 insertions(+), 16 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index d59e4cfea643..8f3307e8ee07 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -90,6 +90,7 @@ public class AutoRepair protected final Map tokenRangeSplitters = new EnumMap<>(AutoRepairConfig.RepairType.class); + private boolean isSetupDone = false; @VisibleForTesting protected AutoRepair() @@ -109,20 +110,30 @@ protected AutoRepair() public void setup() { - AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); - AutoRepairService.setup(); - AutoRepairUtils.setup(); - - for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + // Ensure setup is done only once; this is only for unit tests + // For production, this method should be called only once. + synchronized (this) { - if (config.isAutoRepairEnabled(repairType)) - AutoRepairService.instance.checkCanRun(repairType); - - repairExecutors.get(repairType).scheduleWithFixedDelay( - () -> repair(repairType), - config.getInitialSchedulerDelay(repairType).toSeconds(), - config.getRepairCheckInterval().toSeconds(), - TimeUnit.SECONDS); + if (isSetupDone) + { + return; + } + AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); + AutoRepairService.setup(); + AutoRepairUtils.setup(); + + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + { + if (config.isAutoRepairEnabled(repairType)) + AutoRepairService.instance.checkCanRun(repairType); + + repairExecutors.get(repairType).scheduleWithFixedDelay( + () -> repair(repairType), + config.getInitialSchedulerDelay(repairType).toSeconds(), + config.getRepairCheckInterval().toSeconds(), + TimeUnit.SECONDS); + } + isSetupDone = true; } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index eefc4556697a..c72de0ecc7e6 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -66,9 +66,30 @@ public void testSetup() assertEquals(RepairType.values().length, instance.repairExecutors.size()); for (RepairType repairType : instance.repairExecutors.keySet()) { - assertEquals(String.format("Expected 1 task in queue for %s", repairType), - 1, instance.repairExecutors.get(repairType).getPendingTaskCount() - + instance.repairExecutors.get(repairType).getActiveTaskCount()); + int expectedTasks = instance.repairExecutors.get(repairType).getPendingTaskCount() + + instance.repairExecutors.get(repairType).getActiveTaskCount(); + assertEquals(String.format("Expected 1 task in queue for %s but was %s", repairType, expectedTasks), + 1, expectedTasks); + } + } + + @Test + public void testSafeGuardSetupCall() + { + AutoRepair instance = new AutoRepair(); + + // only one should be setup, and rest should be ignored + instance.setup(); + instance.setup(); + instance.setup(); + + assertEquals(RepairType.values().length, instance.repairExecutors.size()); + for (RepairType repairType : instance.repairExecutors.keySet()) + { + int expectedTasks = instance.repairExecutors.get(repairType).getPendingTaskCount() + + instance.repairExecutors.get(repairType).getActiveTaskCount(); + assertEquals(String.format("Expected 1 task in queue for %s but was %s", repairType, expectedTasks), + 1, expectedTasks); } } From d21587039db1924f667472f5f3b1c8bf3fefed83 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Mon, 30 Sep 2024 20:54:49 -0700 Subject: [PATCH 031/115] Fix InJvm test AutoRepairSchedulerTest --- .../test/repair/AutoRepairSchedulerTest.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java index 5a49f669dc34..ca96995805f1 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java @@ -21,12 +21,13 @@ import java.io.IOException; import java.text.ParseException; import java.text.SimpleDateFormat; -import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.UUID; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Uninterruptibles; import org.apache.cassandra.config.DatabaseDescriptor; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -80,6 +81,13 @@ public static void init() throws IOException cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};"); cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck text, v1 int, v2 int, PRIMARY KEY (pk, ck)) WITH read_repair='NONE'")); + DatabaseDescriptor.setCDCOnRepairEnabled(false); + } + + @AfterClass + public static void afterClass() + { + System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); } @Test From a5cdd4f254b53afa07d2a13be3a527f3a4457f75 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 1 Oct 2024 09:35:28 -0700 Subject: [PATCH 032/115] Unit test failures AutoRepairTest.java --- .../test/repair/AutoRepairSchedulerTest.java | 1 - .../repair/autorepair/AutoRepairTest.java | 16 ++++++++++++---- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java index ca96995805f1..027a3438f0ff 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java @@ -81,7 +81,6 @@ public static void init() throws IOException cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};"); cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck text, v1 int, v2 int, PRIMARY KEY (pk, ck)) WITH read_repair='NONE'")); - DatabaseDescriptor.setCDCOnRepairEnabled(false); } @AfterClass diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index c72de0ecc7e6..1499d1b0dc56 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -21,11 +21,12 @@ import java.util.HashMap; import java.util.Map; +import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; - import org.junit.Assert; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.CQLTester; @@ -53,6 +54,15 @@ public static void setupClass() throws Exception public void setup() { AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.full, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + } + + @After + public void after() + { System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); } @@ -60,7 +70,6 @@ public void setup() public void testSetup() { AutoRepair instance = new AutoRepair(); - instance.setup(); assertEquals(RepairType.values().length, instance.repairExecutors.size()); @@ -86,8 +95,7 @@ public void testSafeGuardSetupCall() assertEquals(RepairType.values().length, instance.repairExecutors.size()); for (RepairType repairType : instance.repairExecutors.keySet()) { - int expectedTasks = instance.repairExecutors.get(repairType).getPendingTaskCount() - + instance.repairExecutors.get(repairType).getActiveTaskCount(); + int expectedTasks = instance.repairExecutors.get(repairType).getCorePoolSize(); assertEquals(String.format("Expected 1 task in queue for %s but was %s", repairType, expectedTasks), 1, expectedTasks); } From 5ed34ea74b240ec4dbac496bcaf97a8f4d7fd656 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Thu, 3 Oct 2024 14:59:04 -0700 Subject: [PATCH 033/115] 5.0 formatting --- .circleci/config.yml | 36 ++++---- .../org/apache/cassandra/config/Config.java | 5 +- .../config/YamlConfigurationLoader.java | 1 - .../statements/schema/TableAttributes.java | 1 - .../db/streaming/CassandraStreamReceiver.java | 5 +- .../repair/autorepair/AutoRepairKeyspace.java | 84 ------------------- .../repair/autorepair/AutoRepairUtils.java | 25 +++--- .../schema/SystemDistributedKeyspace.java | 33 +++++++- .../apache/cassandra/schema/TableParams.java | 3 +- .../service/ActiveRepairService.java | 19 ----- .../cassandra/service/StorageService.java | 5 +- .../tcm/transformations/cms/Initialize.java | 1 - .../apache/cassandra/utils/FBUtilities.java | 6 -- .../test/repair/AutoRepairSchedulerTest.java | 9 +- .../autorepair/AutoRepairKeyspaceTest.java | 38 +++------ .../AutoRepairParameterizedTest.java | 12 +-- .../repair/autorepair/AutoRepairTest.java | 7 +- .../autorepair/AutoRepairUtilsTest.java | 53 ++++++------ .../service/ActiveRepairServiceTest.java | 2 - .../service/AutoRepairServiceBasicTest.java | 13 +-- .../service/AutoRepairServiceSetterTest.java | 11 +-- .../service/StorageServiceServerTest.java | 7 -- 22 files changed, 135 insertions(+), 241 deletions(-) delete mode 100644 src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.java diff --git a/.circleci/config.yml b/.circleci/config.yml index 553e0e32a8a7..9da3ea0f9aea 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -220,7 +220,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -338,7 +338,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -625,7 +625,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -3788,7 +3788,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -4100,7 +4100,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - run: name: Log Environment Information @@ -4413,7 +4413,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -4530,7 +4530,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -4781,7 +4781,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -4885,7 +4885,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -5046,7 +5046,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -5278,7 +5278,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -6444,7 +6444,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -6997,7 +6997,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -7502,7 +7502,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -7843,7 +7843,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -7914,7 +7914,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - run: name: Log Environment Information @@ -8196,7 +8196,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -8833,7 +8833,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 17afdaaa3a73..7dde1bdcc594 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -354,7 +354,10 @@ public MemtableOptions() // The number of executors to use for building secondary indexes public volatile int concurrent_index_builders = 2; - public volatile double incremental_repair_disk_headroom_reject_ratio = 0.2; // at least 20% of disk must be unused to run incremental repair + + // at least 20% of disk must be unused to run incremental repair + // if you want to disable this feature (the recommendation is not to, but if you want to disable it for whatever reason) then set the ratio to 0.0 + public volatile double incremental_repair_disk_headroom_reject_ratio = 0.2; /** * @deprecated retry support removed on CASSANDRA-10992 diff --git a/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java b/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java index 9bf4e415592c..7cb48b9b3b23 100644 --- a/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java +++ b/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java @@ -455,4 +455,3 @@ public static LoaderOptions getDefaultLoaderOptions() return loaderOptions; } } - diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index 540f1036e2f5..b3ac92ef48c0 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -25,7 +25,6 @@ import org.apache.cassandra.cql3.statements.PropertyDefinitions; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.exceptions.SyntaxException; import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.schema.AutoRepairParams; import org.apache.cassandra.schema.CachingParams; diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java index 2d80bace7d15..218d830687a9 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java @@ -23,10 +23,10 @@ import java.util.List; import java.util.Set; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; -import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.io.sstable.SSTable; @@ -39,7 +39,6 @@ import org.apache.cassandra.db.Mutation; import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.filter.ColumnFilter; -import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.rows.ThrottledUnfilteredIterator; @@ -48,7 +47,6 @@ import org.apache.cassandra.dht.Bounds; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.ISSTableScanner; -import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.SSTableMultiWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.streaming.IncomingStream; @@ -195,6 +193,7 @@ private boolean cdcRequiresWriteCommitLog(ColumnFamilyStore cfs) * For CDC-enabled tables and write path for CDC is enabled, we want to ensure that the mutations are * run through the CommitLog, so they can be archived by the CDC process on discard. */ + @VisibleForTesting public boolean requiresWritePath(ColumnFamilyStore cfs) { return cdcRequiresWriteCommitLog(cfs) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.java deleted file mode 100644 index 5afd2fff3275..000000000000 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairKeyspace.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.repair.autorepair; - -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.cql3.statements.schema.CreateTableStatement; -import org.apache.cassandra.schema.KeyspaceMetadata; -import org.apache.cassandra.schema.KeyspaceParams; -import org.apache.cassandra.schema.SchemaConstants; -import org.apache.cassandra.schema.TableId; -import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.schema.Tables; - -public class AutoRepairKeyspace -{ - private AutoRepairKeyspace() - { - } - - /** - * Generation is used as a timestamp for automatic table creation on startup. - * If you make any changes to the tables below, make sure to increment the - * generation and document your change here. - */ - public static final long GENERATION = 0; - - public static final String AUTO_REPAIR_HISTORY = "auto_repair_history"; - - public static final String AUTO_REPAIR_PRIORITY = "auto_repair_priority"; - - public static final TableMetadata AutoRepairHistory = - parse(AUTO_REPAIR_HISTORY, - "Auto repair history for each node", - "CREATE TABLE %s (" - + "host_id uuid," - + "repair_type text," - + "repair_turn text," - + "repair_start_ts timestamp," - + "repair_finish_ts timestamp," - + "delete_hosts set," - + "delete_hosts_update_time timestamp," - + "force_repair boolean," - + "PRIMARY KEY (repair_type, host_id))"); - - public static final TableMetadata AutoRepairPriority = - parse(AUTO_REPAIR_PRIORITY, - "Auto repair priority for each group", - "CREATE TABLE %s (" - + "repair_type text," - + "repair_priority set," - + "PRIMARY KEY (repair_type))"); - - private static TableMetadata parse(String name, String description, String schema) - { - return CreateTableStatement.parse(String.format(schema, name), SchemaConstants.DISTRIBUTED_KEYSPACE_NAME) - .id(TableId.forSystemTable(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, name)) - .comment(description) - .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(90)) - .build(); - } - - public static KeyspaceMetadata metadata() - { - Tables tables = Tables.of(AutoRepairHistory, AutoRepairPriority); - return KeyspaceMetadata.create(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, KeyspaceParams.simple(1), tables); - } -} diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index 29ffe9f14164..c9d732725be7 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -53,6 +53,7 @@ import org.apache.cassandra.locator.NetworkTopologyStrategy; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.ViewMetadata; import org.apache.cassandra.serializers.SetSerializer; @@ -96,56 +97,56 @@ public class AutoRepairUtils final static String SELECT_REPAIR_HISTORY = String.format( "SELECT * FROM %s.%s WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE); final static String SELECT_REPAIR_PRIORITY = String.format( "SELECT * FROM %s.%s WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); final static String DEL_REPAIR_PRIORITY = String.format( "DELETE %s[?] FROM %s.%s WHERE %s = ?", COL_REPAIR_PRIORITY, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); final static String ADD_PRIORITY_HOST = String.format( "UPDATE %s.%s SET %s = %s + ? WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_TYPE); final static String INSERT_NEW_REPAIR_HISTORY = String.format( "INSERT INTO %s.%s (%s, %s, %s, %s, %s, %s) values (?, ? ,?, ?, {}, ?) IF NOT EXISTS", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID, COL_REPAIR_START_TS, COL_REPAIR_FINISH_TS, COL_DELETE_HOSTS, COL_DELETE_HOSTS_UPDATE_TIME); final static String ADD_HOST_ID_TO_DELETE_HOSTS = String.format( "UPDATE %s.%s SET %s = %s + ?, %s = ? WHERE %s = ? AND %s = ? IF EXISTS" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, COL_DELETE_HOSTS, COL_DELETE_HOSTS_UPDATE_TIME, COL_REPAIR_TYPE, COL_HOST_ID); final static String DEL_AUTO_REPAIR_HISTORY = String.format( "DELETE FROM %s.%s WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID); final static String RECORD_START_REPAIR_HISTORY = String.format( "UPDATE %s.%s SET %s= ?, repair_turn = ? WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_START_TS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_START_TS, COL_REPAIR_TYPE, COL_HOST_ID); final static String RECORD_FINISH_REPAIR_HISTORY = String.format( "UPDATE %s.%s SET %s= ?, %s=false WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_FINISH_TS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_FINISH_TS, COL_FORCE_REPAIR, COL_REPAIR_TYPE, COL_HOST_ID); final static String CLEAR_DELETE_HOSTS = String.format( "UPDATE %s.%s SET %s= {} WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, COL_REPAIR_TYPE, COL_HOST_ID); final static String SET_FORCE_REPAIR = String.format( "UPDATE %s.%s SET %s=true WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_FORCE_REPAIR, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_FORCE_REPAIR, COL_REPAIR_TYPE, COL_HOST_ID); final static String SELECT_LAST_REPAIR_TIME_FOR_NODE = String.format( "SELECT %s FROM %s.%s WHERE %s = ? AND %s = ?", COL_REPAIR_FINISH_TS, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - AutoRepairKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID); + SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID); static ModificationStatement delStatementRepairHistory; static SelectStatement selectStatementRepairHistory; diff --git a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java index a6556abb7258..1b775fb7589c 100644 --- a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java @@ -58,8 +58,6 @@ import static java.lang.String.format; -import static org.apache.cassandra.repair.autorepair.AutoRepairKeyspace.AutoRepairHistory; -import static org.apache.cassandra.repair.autorepair.AutoRepairKeyspace.AutoRepairPriority; import static org.apache.cassandra.utils.ByteBufferUtil.bytes; public final class SystemDistributedKeyspace @@ -98,7 +96,11 @@ private SystemDistributedKeyspace() public static final String PARTITION_DENYLIST_TABLE = "partition_denylist"; - public static final Set TABLE_NAMES = ImmutableSet.of(REPAIR_HISTORY, PARENT_REPAIR_HISTORY, VIEW_BUILD_STATUS, PARTITION_DENYLIST_TABLE); + public static final String AUTO_REPAIR_HISTORY = "auto_repair_history"; + + public static final String AUTO_REPAIR_PRIORITY = "auto_repair_priority"; + + public static final Set TABLE_NAMES = ImmutableSet.of(REPAIR_HISTORY, PARENT_REPAIR_HISTORY, VIEW_BUILD_STATUS, PARTITION_DENYLIST_TABLE, AUTO_REPAIR_HISTORY, AUTO_REPAIR_PRIORITY); public static final String REPAIR_HISTORY_CQL = "CREATE TABLE IF NOT EXISTS %s (" + "keyspace_name text," @@ -161,6 +163,29 @@ private SystemDistributedKeyspace() private static final TableMetadata PartitionDenylistTable = parse(PARTITION_DENYLIST_TABLE, "Partition keys which have been denied access", PARTITION_DENYLIST_CQL).build(); + public static final String AUTO_REPAIR_HISTORY_CQL = "CREATE TABLE IF NOT EXISTS %s (" + + "host_id uuid," + + "repair_type text," + + "repair_turn text," + + "repair_start_ts timestamp," + + "repair_finish_ts timestamp," + + "delete_hosts set," + + "delete_hosts_update_time timestamp," + + "force_repair boolean," + + "PRIMARY KEY (repair_type, host_id))"; + + private static final TableMetadata AutoRepairHistoryTable = + parse(AUTO_REPAIR_HISTORY, "Auto repair history for each node", AUTO_REPAIR_HISTORY_CQL).build(); + + public static final String AUTO_REPAIR_PRIORITY_CQL = "CREATE TABLE IF NOT EXISTS %s (" + + "repair_type text," + + "repair_priority set," + + "PRIMARY KEY (repair_type))"; + + private static final TableMetadata AutoRepairPriorityTable = + parse(AUTO_REPAIR_PRIORITY, "Auto repair priority for each group", AUTO_REPAIR_PRIORITY_CQL).build(); + + private static TableMetadata.Builder parse(String table, String description, String cql) { return CreateTableStatement.parse(format(cql, table), SchemaConstants.DISTRIBUTED_KEYSPACE_NAME) @@ -173,7 +198,7 @@ public static KeyspaceMetadata metadata() { return KeyspaceMetadata.create(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, KeyspaceParams.simple(Math.max(DEFAULT_RF, DatabaseDescriptor.getDefaultKeyspaceRF())), - Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus, PartitionDenylistTable, AutoRepairHistory, AutoRepairPriority)); + Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus, PartitionDenylistTable, AutoRepairHistoryTable, AutoRepairPriorityTable)); } public static void startParentRepair(TimeUUID parent_id, String keyspaceName, String[] cfnames, RepairOption options) diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index 6578cbe75dd1..39db4fb5b0f2 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -73,8 +73,7 @@ public enum Option CDC, READ_REPAIR, AUTOMATED_REPAIR_FULL, - AUTOMATED_REPAIR_INCREMENTAL, - ; + AUTOMATED_REPAIR_INCREMENTAL; @Override public String toString() diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java index ed3d14752add..d66ef8710e08 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairService.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java @@ -54,9 +54,7 @@ import org.apache.cassandra.concurrent.ExecutorPlus; import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DurationSpec; -import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.repair.Scheduler; -import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.apache.cassandra.locator.EndpointsByRange; import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.service.disk.usage.DiskUsageMonitor; @@ -65,19 +63,13 @@ import org.apache.cassandra.repair.state.ParticipateState; import org.apache.cassandra.repair.state.ValidationState; import org.apache.cassandra.utils.Simulate; -import org.apache.cassandra.locator.EndpointsForToken; -import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.utils.TimeUUID; -import org.apache.cassandra.utils.concurrent.CountDownLatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.concurrent.ExecutorPlus; -import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.dht.IPartitioner; @@ -91,8 +83,6 @@ import org.apache.cassandra.gms.IEndpointStateChangeSubscriber; import org.apache.cassandra.gms.IFailureDetectionEventListener; import org.apache.cassandra.gms.VersionedValue; -import org.apache.cassandra.locator.EndpointsByRange; -import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.metrics.RepairMetrics; import org.apache.cassandra.net.Message; @@ -103,7 +93,6 @@ import org.apache.cassandra.repair.RepairJobDesc; import org.apache.cassandra.repair.RepairParallelism; import org.apache.cassandra.repair.RepairSession; -import org.apache.cassandra.repair.Scheduler; import org.apache.cassandra.repair.SharedContext; import org.apache.cassandra.repair.consistent.CoordinatorSessions; import org.apache.cassandra.repair.consistent.LocalSessions; @@ -118,21 +107,13 @@ import org.apache.cassandra.repair.messages.RepairOption; import org.apache.cassandra.repair.messages.SyncResponse; import org.apache.cassandra.repair.messages.ValidationResponse; -import org.apache.cassandra.repair.state.CoordinatorState; -import org.apache.cassandra.repair.state.ParticipateState; -import org.apache.cassandra.repair.state.ValidationState; import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.schema.TableId; -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.tcm.ClusterMetadata; -import org.apache.cassandra.streaming.PreviewKind; -import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.MerkleTrees; import org.apache.cassandra.utils.Pair; -import org.apache.cassandra.utils.Simulate; -import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.concurrent.AsyncPromise; import org.apache.cassandra.utils.concurrent.Future; import org.apache.cassandra.utils.concurrent.FutureCombiner; diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 2b4d2c7c7f86..216d290cbc8b 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -76,7 +76,6 @@ import com.google.common.util.concurrent.Uninterruptibles; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.repair.autorepair.AutoRepairKeyspace; import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -467,12 +466,12 @@ public enum Mode { STARTING, NORMAL, JOINING, JOINING_FAILED, LEAVING, DECOMMISS private volatile int totalCFs, remainingCFs; - public static final AtomicInteger nextRepairCommand = new AtomicInteger(); - private final List lifecycleSubscribers = new CopyOnWriteArrayList<>(); private final String jmxObjectName; + public static final AtomicInteger nextRepairCommand = new AtomicInteger(); + // true when keeping strict consistency while bootstrapping public static final boolean useStrictConsistency = CONSISTENT_RANGE_MOVEMENT.getBoolean(); private boolean joinRing = JOIN_RING.getBoolean(); diff --git a/src/java/org/apache/cassandra/tcm/transformations/cms/Initialize.java b/src/java/org/apache/cassandra/tcm/transformations/cms/Initialize.java index 635a7942260b..11c2ed4d31af 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/cms/Initialize.java +++ b/src/java/org/apache/cassandra/tcm/transformations/cms/Initialize.java @@ -24,7 +24,6 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.MetaStrategy; -import org.apache.cassandra.repair.autorepair.AutoRepairKeyspace; import org.apache.cassandra.schema.DistributedSchema; import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.schema.SystemDistributedKeyspace; diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index 576f7faaf6aa..b2e24a3aab46 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -67,13 +67,8 @@ import com.google.common.collect.ImmutableList; import com.vdurmont.semver4j.Semver; -import com.fasterxml.jackson.databind.SerializationFeature; -import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.FileInputStreamPlus; -import org.apache.cassandra.io.util.FileOutputStreamPlus; import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter; -import org.apache.cassandra.utils.concurrent.*; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,7 +88,6 @@ import org.apache.cassandra.io.sstable.metadata.MetadataType; import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.io.util.DataOutputBufferFixed; -import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.security.AbstractCryptoProvider; diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java index 027a3438f0ff..1c4ebae5a34e 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java @@ -26,7 +26,9 @@ import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Uninterruptibles; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -37,7 +39,6 @@ import org.apache.cassandra.distributed.test.TestBaseImpl; import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.cassandra.repair.autorepair.AutoRepairConfig; -import org.apache.cassandra.repair.autorepair.AutoRepairKeyspace; import static org.apache.cassandra.schema.SchemaConstants.DISTRIBUTED_KEYSPACE_NAME; import static org.junit.Assert.assertEquals; @@ -51,7 +52,7 @@ public class AutoRepairSchedulerTest extends TestBaseImpl @BeforeClass public static void init() throws IOException { - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); // Define the expected date format pattern String pattern = "EEE MMM dd HH:mm:ss z yyyy"; @@ -93,7 +94,7 @@ public static void afterClass() public void testScheduler() throws ParseException { // ensure there was no history of previous repair runs through the scheduler - Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s", DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY), ConsistencyLevel.QUORUM); + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY), ConsistencyLevel.QUORUM); assertEquals(0, rows.length); cluster.forEach(i -> i.runOnInstance(() -> { @@ -116,7 +117,7 @@ public void testScheduler() throws ParseException private void validate(String repairType) throws ParseException { - Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s where repair_type='%s'", DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, repairType), ConsistencyLevel.QUORUM); + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s where repair_type='%s'", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType), ConsistencyLevel.QUORUM); assertEquals(3, rows.length); for (int node = 0; node < rows.length; node++) { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java index 43b8e4a5f4e8..1337cf3dd2d3 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java @@ -18,49 +18,37 @@ package org.apache.cassandra.repair.autorepair; -import java.util.Optional; +import java.util.HashSet; +import java.util.Iterator; import java.util.Set; -import com.google.common.collect.ImmutableSet; - +import org.apache.cassandra.schema.*; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.DurationSpec; -import org.apache.cassandra.schema.KeyspaceMetadata; -import org.apache.cassandra.schema.TableMetadata; - -import static org.apache.cassandra.Util.setAutoRepairEnabled; public class AutoRepairKeyspaceTest { - private static final Set tables = ImmutableSet.of( - AutoRepairKeyspace.AUTO_REPAIR_HISTORY, - AutoRepairKeyspace.AUTO_REPAIR_PRIORITY - ); - @BeforeClass public static void setupDatabaseDescriptor() { DatabaseDescriptor.daemonInitialization(); - AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); } - @Test - public void testMetadataCanParseSchemas() throws Exception + public void testEnsureAutoRepairTablesArePresent() { - setAutoRepairEnabled(true); - KeyspaceMetadata keyspaceMetadata = AutoRepairKeyspace.metadata(); - - assert keyspaceMetadata.tables.size() == tables.size() : "Expected " + tables.size() + " tables, got " + keyspaceMetadata.tables.size(); - - for (String table : tables) + KeyspaceMetadata keyspaceMetadata = SystemDistributedKeyspace.metadata(); + Iterator iter = keyspaceMetadata.tables.iterator(); + Set actualDistributedTablesIter = new HashSet<>(); + while (iter.hasNext()) { - Optional tableMetadata = keyspaceMetadata.tables.get(table); - - assert tableMetadata.isPresent() : "Table " + table + " not found in metadata"; + actualDistributedTablesIter.add(iter.next().name); } + + Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.AUTO_REPAIR_HISTORY)); + Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY)); } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 9978c36bb31f..988ec538b783 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -30,12 +30,14 @@ import com.google.common.collect.Sets; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.statements.schema.TableAttributes; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.repair.RepairCoordinator; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.Pair; @@ -130,7 +132,7 @@ public void setup() QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); MockitoAnnotations.initMocks(this); Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE).truncateBlocking(); @@ -139,8 +141,8 @@ public void setup() Keyspace.open(KEYSPACE).getColumnFamilyStore(MV).truncateBlocking(); Keyspace.open(KEYSPACE).getColumnFamilyStore(MV).disableAutoCompaction(); - Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(AutoRepairKeyspace.AUTO_REPAIR_PRIORITY).truncateBlocking(); - Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(AutoRepairKeyspace.AUTO_REPAIR_HISTORY).truncateBlocking(); + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY).truncateBlocking(); + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_HISTORY).truncateBlocking(); AutoRepair.instance = new AutoRepair(); @@ -197,7 +199,7 @@ private void executeCQL() QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, s) VALUES ('k', 's')"); QueryProcessor.executeInternal("SELECT s FROM ks.tbl WHERE k='k'"); Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME) - .getColumnFamilyStore(AutoRepairKeyspace.AUTO_REPAIR_PRIORITY) + .getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY) .forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); } @@ -678,7 +680,7 @@ public void testRepairSuccessAfterRetry() public void testRepairThrowsForIRWithMVReplay() { AutoRepair.instance.setup(); - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "true"); + CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(true); if (repairType == AutoRepairConfig.RepairType.incremental) { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index 1499d1b0dc56..00c45693659c 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; @@ -54,7 +55,7 @@ public static void setupClass() throws Exception public void setup() { AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); DatabaseDescriptor.setCDCOnRepairEnabled(false); DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.full, true); DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); @@ -104,7 +105,7 @@ public void testSafeGuardSetupCall() @Test(expected = ConfigurationException.class) public void testSetupFailsWhenIREnabledWithCDCReplay() { - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); DatabaseDescriptor.setCDCEnabled(true); @@ -118,7 +119,7 @@ public void testSetupFailsWhenIREnabledWithCDCReplay() public void testSetupFailsWhenIREnabledWithMVReplay() { DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "true"); + CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(true); DatabaseDescriptor.setCDCOnRepairEnabled(false); AutoRepair instance = new AutoRepair(); instance.setup(); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java index 8cf5f5221b57..9a3c551c9bf6 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -24,6 +24,7 @@ import java.util.UUID; import com.google.common.collect.ImmutableSet; +import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -99,10 +100,10 @@ public void setup() DatabaseDescriptor.setEndpointSnitch(defaultSnitch); QueryProcessor.executeInternal(String.format( "TRUNCATE %s.%s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY)); + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY)); QueryProcessor.executeInternal(String.format( "TRUNCATE %s.%s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY)); + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY)); } @Test @@ -110,14 +111,14 @@ public void testSetForceRepair() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, force_repair) VALUES ('%s', %s, false)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); AutoRepairUtils.setForceRepair(repairType, ImmutableSet.of(localEndpoint)); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT force_repair FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -131,7 +132,7 @@ public void testSetForceRepairNewNode() UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT force_repair FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -144,14 +145,14 @@ public void testClearDeleteHosts() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, delete_hosts, delete_hosts_update_time) VALUES ('%s', %s, { %s }, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId, hostId)); AutoRepairUtils.clearDeleteHosts(repairType, hostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT delete_hosts FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -164,7 +165,7 @@ public void testGetAutoRepairHistoryForLocalGroup() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, force_repair) VALUES ('%s', %s, false)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); List history = AutoRepairUtils.getAutoRepairHistory(repairType); @@ -188,19 +189,19 @@ public void testGetCurrentRepairStatus() UUID regularRepair = UUID.randomUUID(); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, force_repair, repair_start_ts) VALUES ('%s', %s, true, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), forceRepair)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, repair_start_ts) VALUES ('%s', %s, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), regularRepair)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, repairType.toString(), regularRepair)); CurrentRepairStatus status = AutoRepairUtils.getCurrentRepairStatus(repairType); @@ -250,11 +251,11 @@ public void testGetHostWithLongestUnrepairTime() UUID otherHostId = UUID.randomUUID(); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, repair_finish_ts) VALUES ('%s', %s, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), otherHostId)); AutoRepairHistory history = AutoRepairUtils.getHostWithLongestUnrepairTime(repairType); @@ -290,14 +291,14 @@ public void testDeleteAutoRepairHistory() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); AutoRepairUtils.deleteAutoRepairHistory(repairType, hostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(0, result.size()); @@ -308,14 +309,14 @@ public void testUpdateStartAutoRepairHistory() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); AutoRepairUtils.updateStartAutoRepairHistory(repairType, hostId, 123, AutoRepairUtils.RepairTurn.MY_TURN); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT repair_start_ts, repair_turn FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -329,14 +330,14 @@ public void testUpdateFinishAutoRepairHistory() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); AutoRepairUtils.updateFinishAutoRepairHistory(repairType, hostId, 123); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT repair_finish_ts FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -349,14 +350,14 @@ public void testAddHostIdToDeleteHosts() UUID otherHostId = UUID.randomUUID(); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), otherHostId)); AutoRepairUtils.addHostIdToDeleteHosts(repairType, hostId, otherHostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), otherHostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -373,7 +374,7 @@ public void testAddPriorityHost() UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s'", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, repairType.toString())); assertNotNull(result); assertEquals(1, result.size()); @@ -388,14 +389,14 @@ public void testRemovePriorityStatus() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, repairType.toString(), hostId)); AutoRepairUtils.removePriorityStatus(repairType, hostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s'", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, repairType.toString())); assertNotNull(result); assertEquals(1, result.size()); @@ -408,7 +409,7 @@ public void testGetPriorityHosts() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, repairType.toString(), hostId)); Set hosts = AutoRepairUtils.getPriorityHosts(repairType); diff --git a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java index 7eecb54a056a..ed6a1641cbfd 100644 --- a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java +++ b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java @@ -72,8 +72,6 @@ import org.apache.cassandra.tcm.transformations.Register; import org.apache.cassandra.tcm.transformations.UnsafeJoin; import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.TimeUUID; -import org.apache.cassandra.utils.concurrent.Condition; import org.apache.cassandra.utils.concurrent.Refs; import org.mockito.Mock; diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java index 826a7187219c..704d09fc7dbc 100644 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java @@ -18,7 +18,7 @@ package org.apache.cassandra.service; -import org.junit.After; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.junit.Before; import org.junit.Test; @@ -35,7 +35,7 @@ public class AutoRepairServiceBasicTest extends CQLTester { @Before public void setUp() { - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); DatabaseDescriptor.setCDCOnRepairEnabled(false); DatabaseDescriptor.setMaterializedViewsEnabled(false); DatabaseDescriptor.setCDCEnabled(false); @@ -44,11 +44,6 @@ public void setUp() { autoRepairService.config = config; } - @After - public void tearDown() { - System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); - } - @Test public void testSetup() { AutoRepairService.instance.config = null; @@ -96,7 +91,7 @@ public void testSetAutoRepairEnabledThrowsWithSchedulerDisabled() { @Test(expected = ConfigurationException.class) public void testSetAutoRepairEnabledThrowsForIRWithMVReplay() { autoRepairService.config = new AutoRepairConfig(true); - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "true"); + CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(true); autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); } @@ -105,7 +100,7 @@ public void testSetAutoRepairEnabledThrowsForIRWithMVReplay() { public void testSetAutoRepairEnabledDoesNotThrowForIRWithMVReplayDisabled() { autoRepairService.config = new AutoRepairConfig(true); DatabaseDescriptor.setMaterializedViewsEnabled(true); - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); } diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java index 31aae67c557e..271d28b926de 100644 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java @@ -19,15 +19,16 @@ package org.apache.cassandra.service; import com.google.common.collect.ImmutableSet; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.autorepair.AutoRepairConfig; -import org.apache.cassandra.repair.autorepair.AutoRepairKeyspace; import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -84,7 +85,7 @@ private static Set isLocalHostForceRepair(AutoRepairConfig.R UUID hostId = StorageService.instance.getHostIdForEndpoint(InetAddressAndPort.getLocalHost()); UntypedResultSet resultSet = QueryProcessor.executeInternal(String.format( "SELECT force_repair FROM %s.%s WHERE host_id = %s and repair_type = '%s'", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY, hostId, type)); + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, hostId, type)); if (!resultSet.isEmpty() && resultSet.one().getBoolean("force_repair")) { return ImmutableSet.of(InetAddressAndPort.getLocalHost()); @@ -116,15 +117,15 @@ public static void setup() throws Exception { public void prepare() { QueryProcessor.executeInternal(String.format( "TRUNCATE %s.%s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_HISTORY)); + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY)); QueryProcessor.executeInternal(String.format( "TRUNCATE %s.%s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, AutoRepairKeyspace.AUTO_REPAIR_PRIORITY)); + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY)); } @Test public void testSettersTest() { - System.setProperty("cassandra.streaming.requires_view_build_during_repair", "false"); + CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); DatabaseDescriptor.setCDCOnRepairEnabled(false); setter.accept(repairType, arg); assertEquals(arg, getter.apply(repairType)); diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java index 76d56ebadc54..181ef5de02d1 100644 --- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java +++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java @@ -33,13 +33,9 @@ import java.util.*; import java.util.stream.Collectors; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Multimap; - import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.SystemKeyspace; import org.junit.Assert; import org.junit.Before; @@ -60,9 +56,6 @@ import org.apache.cassandra.locator.AbstractNetworkTopologySnitch; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.gms.ApplicationState; -import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.gms.VersionedValue; import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.WithPartitioner; From ff50053495b3c91c540a101c8395959b2f3543db Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 8 Oct 2024 10:15:37 -0700 Subject: [PATCH 034/115] Fix flaky AutoRepairTest::testSetup and AutoRepairTest::testSafeGuardSetupCall --- .../cassandra/repair/autorepair/AutoRepairTest.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index 00c45693659c..e586eec9939b 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -41,6 +41,7 @@ import static org.apache.cassandra.Util.setAutoRepairEnabled; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class AutoRepairTest extends CQLTester { @@ -78,8 +79,8 @@ public void testSetup() { int expectedTasks = instance.repairExecutors.get(repairType).getPendingTaskCount() + instance.repairExecutors.get(repairType).getActiveTaskCount(); - assertEquals(String.format("Expected 1 task in queue for %s but was %s", repairType, expectedTasks), - 1, expectedTasks); + assertTrue(String.format("Expected > 0 task in queue for %s but was %s", repairType, expectedTasks), + expectedTasks > 0); } } @@ -96,9 +97,10 @@ public void testSafeGuardSetupCall() assertEquals(RepairType.values().length, instance.repairExecutors.size()); for (RepairType repairType : instance.repairExecutors.keySet()) { - int expectedTasks = instance.repairExecutors.get(repairType).getCorePoolSize(); - assertEquals(String.format("Expected 1 task in queue for %s but was %s", repairType, expectedTasks), - 1, expectedTasks); + int expectedTasks = instance.repairExecutors.get(repairType).getPendingTaskCount() + + instance.repairExecutors.get(repairType).getActiveTaskCount(); + assertTrue(String.format("Expected > 0 task in queue for %s but was %s", repairType, expectedTasks), + expectedTasks > 0); } } From 82733d797dfca1aabf5d8f290e9c7032a59accc6 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Tue, 8 Oct 2024 14:28:44 -0700 Subject: [PATCH 035/115] Fix a test failure inside pylib/cqlshlib/test/test_cqlsh_output.py --- pylib/cqlshlib/test/test_cqlsh_output.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pylib/cqlshlib/test/test_cqlsh_output.py b/pylib/cqlshlib/test/test_cqlsh_output.py index 78dc0331454a..6637129514b2 100644 --- a/pylib/cqlshlib/test/test_cqlsh_output.py +++ b/pylib/cqlshlib/test/test_cqlsh_output.py @@ -698,7 +698,9 @@ def test_describe_columnfamily_output(self): AND memtable_flush_period_in_ms = 0 AND min_index_interval = 128 AND read_repair = 'BLOCKING' - AND speculative_retry = '99p';""" % quote_name(get_keyspace())) + AND speculative_retry = '99p' + AND automated_repair_full = {'enabled': 'true'} + AND automated_repair_incremental = {'enabled': 'true'};""" % quote_name(get_keyspace())) with cqlsh_testrun(tty=True, env=self.default_env) as c: for cmdword in ('describe table', 'desc columnfamily'): From 8c6d17cd04bd718fde39aef5fb2b273370808556 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 8 Oct 2024 21:10:48 -0700 Subject: [PATCH 036/115] AutoRepair nodetool throws an exception if the framework is disabled --- .../apache/cassandra/repair/autorepair/AutoRepair.java | 1 - .../org/apache/cassandra/service/AutoRepairService.java | 2 +- src/java/org/apache/cassandra/service/StorageService.java | 1 + .../apache/cassandra/tools/nodetool/AutoRepairStatus.java | 8 ++++++++ 4 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 8f3307e8ee07..5d5bcf3568c2 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -119,7 +119,6 @@ public void setup() return; } AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); - AutoRepairService.setup(); AutoRepairUtils.setup(); for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index 1adad42b263d..d2395f1157b5 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -190,7 +190,7 @@ public Set getOnGoingRepairHostIds(RepairType rType) List histories = AutoRepairUtils.getAutoRepairHistory(rType); if (histories == null) { - return null; + return hostIds; } AutoRepairUtils.CurrentRepairStatus currentRepairStatus = new AutoRepairUtils.CurrentRepairStatus(histories, AutoRepairUtils.getPriorityHostIds(rType)); for (UUID id : currentRepairStatus.hostIdsWithOnGoingRepair) diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 216d290cbc8b..7eb10e0542b0 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -1132,6 +1132,7 @@ public void doAuthSetup(boolean async) public void doAutoRepairSetup() { + AutoRepairService.setup(); if (DatabaseDescriptor.getAutoRepairConfig().isAutoRepairSchedulingEnabled()) { logger.info("Enable auto-repair scheduling"); diff --git a/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java b/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java index 882f2a4d3cda..7b96102c6698 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java +++ b/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java @@ -44,6 +44,14 @@ public void execute(NodeProbe probe) { checkArgument(repairType != null, "--repair-type is required."); PrintStream out = probe.output().out; + + AutoRepairConfig config = probe.getAutoRepairConfig(); + if (config == null || !config.isAutoRepairSchedulingEnabled()) + { + out.println("Auto-repair is not enabled"); + return; + } + TableBuilder table = new TableBuilder(); table.add("Active Repairs"); Set ongoingRepairHostIds = probe.getOnGoingRepairHostIds(repairType); From df5b922bee4aeb9660350e706d179f7c45013e6b Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Wed, 9 Oct 2024 09:37:23 -0700 Subject: [PATCH 037/115] Run repair on bootstrapped if configured to do so --- .../cassandra/repair/autorepair/AutoRepairUtils.java | 11 +++++++++++ .../cassandra/tcm/sequences/BootstrapAndJoin.java | 3 +++ .../cassandra/tcm/sequences/BootstrapAndReplace.java | 3 +++ .../cassandra/tcm/sequences/ReplaceSameAddress.java | 4 ++++ .../cassandra/repair/autorepair/AutoRepairTest.java | 2 ++ .../tools/nodetool/AutoRepairStatusTest.java | 11 +++++++++-- 6 files changed, 32 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index c9d732725be7..2b1ca4c39cc8 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -833,4 +833,15 @@ public static List getAllMVs(RepairType repairType, Keyspace keyspace, T } return allMvs; } + + public static void runRepairOnNewlyBootstrappedNodeIfEnabled() + { + AutoRepairConfig repairConfig = DatabaseDescriptor.getAutoRepairConfig(); + if (repairConfig.isAutoRepairSchedulingEnabled()) + { + for (AutoRepairConfig.RepairType rType : AutoRepairConfig.RepairType.values()) + if (repairConfig.isAutoRepairEnabled(rType) && repairConfig.getForceRepairNewNode(rType)) + AutoRepairUtils.setForceRepairNewNode(rType); + } + } } diff --git a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java index 15182fc92b2a..d3ffb9f92cf0 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java +++ b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java @@ -39,6 +39,7 @@ import org.apache.cassandra.locator.EndpointsByReplica; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.StreamState; @@ -217,6 +218,8 @@ public SequenceState executeNext() "For more, see `nodetool help bootstrap`. {}", SystemKeyspace.getBootstrapState()); return halted(); } + // this node might have just bootstrapped; check if we should run repair immediately + AutoRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED); } else diff --git a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java index e774b77ebe9c..93251b354f2f 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java +++ b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java @@ -43,6 +43,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.EndpointsByReplica; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; @@ -216,6 +217,8 @@ public SequenceState executeNext() "For more, see `nodetool help bootstrap`. {}", SystemKeyspace.getBootstrapState()); return halted(); } + // this node might have just bootstrapped; check if we should run repair immediately + AutoRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED); } else diff --git a/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java b/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java index 2c4553a14f70..bcab9645859d 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java +++ b/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java @@ -29,6 +29,8 @@ import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.locator.EndpointsByReplica; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; @@ -85,6 +87,8 @@ public static void streamData(NodeId nodeId, ClusterMetadata metadata, boolean s "For more, see `nodetool help bootstrap`. {}", SystemKeyspace.getBootstrapState()); throw new IllegalStateException("Could not finish join for during replacement"); } + // this node might have just bootstrapped; check if we should run repair immediately + AutoRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); } if (finishJoiningRing) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index e586eec9939b..be2b6cc47d75 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -38,6 +38,7 @@ import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; import org.apache.cassandra.schema.SchemaTestUtil; +import org.apache.cassandra.service.AutoRepairService; import static org.apache.cassandra.Util.setAutoRepairEnabled; import static org.junit.Assert.assertEquals; @@ -60,6 +61,7 @@ public void setup() DatabaseDescriptor.setCDCOnRepairEnabled(false); DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.full, true); DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + AutoRepairService.setup(); } @After diff --git a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java index a92f9ec960fe..5bfe9091ff07 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java @@ -29,12 +29,14 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.Output; import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import static org.apache.cassandra.Util.setAutoRepairEnabled; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.when; @@ -61,14 +63,19 @@ public static Collection repairTypes() } @Before - public void setUp() + public void setUp() throws Exception { MockitoAnnotations.initMocks(this); cmdOutput = new ByteArrayOutputStream(); PrintStream out = new PrintStream(cmdOutput); when(probe.output()).thenReturn(new Output(out, out)); - when(probe.getAutoRepairConfig()).thenReturn(config); cmd = new AutoRepairStatus(); + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.loadConfig(); + setAutoRepairEnabled(true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.full, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + when(probe.getAutoRepairConfig()).thenReturn(DatabaseDescriptor.getAutoRepairConfig()); } @Test(expected = IllegalArgumentException.class) From 242d0827ca0f44ad6b58682bf05068606b0faacb Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Wed, 9 Oct 2024 11:00:06 -0700 Subject: [PATCH 038/115] Fix broken AutoRepairSchedulerTest.java & AutoRepairStatusTest.java --- .../distributed/test/repair/AutoRepairSchedulerTest.java | 2 ++ .../apache/cassandra/tools/nodetool/AutoRepairStatusTest.java | 3 --- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java index 1c4ebae5a34e..43c29f14e0de 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java @@ -39,6 +39,7 @@ import org.apache.cassandra.distributed.test.TestBaseImpl; import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.service.AutoRepairService; import static org.apache.cassandra.schema.SchemaConstants.DISTRIBUTED_KEYSPACE_NAME; import static org.junit.Assert.assertEquals; @@ -100,6 +101,7 @@ public void testScheduler() throws ParseException cluster.forEach(i -> i.runOnInstance(() -> { try { + AutoRepairService.instance.setup(); DatabaseDescriptor.setCDCOnRepairEnabled(false); AutoRepair.instance.setup(); } diff --git a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java index 5bfe9091ff07..16b12ee2350e 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java @@ -48,9 +48,6 @@ public class AutoRepairStatusTest private ByteArrayOutputStream cmdOutput; - @Mock - private static AutoRepairConfig config; - private static AutoRepairStatus cmd; @Parameterized.Parameter() From 819771fe118589a43111036af8e8ebd166048b9f Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Wed, 9 Oct 2024 14:47:56 -0700 Subject: [PATCH 039/115] Comments from Chris Lohfink: nix automated_ and a typo --- pylib/cqlshlib/test/test_cqlsh_output.py | 4 ++-- .../cql3/statements/schema/TableAttributes.java | 8 ++++---- .../apache/cassandra/schema/SchemaKeyspace.java | 16 ++++++++-------- .../org/apache/cassandra/schema/TableParams.java | 12 ++++++------ .../tools/nodetool/GetAutoRepairConfig.java | 2 +- .../cql3/statements/DescribeStatementTest.java | 8 ++++---- .../apache/cassandra/db/SchemaCQLHelperTest.java | 4 ++-- .../autorepair/AutoRepairParameterizedTest.java | 6 +++--- 8 files changed, 30 insertions(+), 30 deletions(-) diff --git a/pylib/cqlshlib/test/test_cqlsh_output.py b/pylib/cqlshlib/test/test_cqlsh_output.py index 6637129514b2..183df2b0a14a 100644 --- a/pylib/cqlshlib/test/test_cqlsh_output.py +++ b/pylib/cqlshlib/test/test_cqlsh_output.py @@ -699,8 +699,8 @@ def test_describe_columnfamily_output(self): AND min_index_interval = 128 AND read_repair = 'BLOCKING' AND speculative_retry = '99p' - AND automated_repair_full = {'enabled': 'true'} - AND automated_repair_incremental = {'enabled': 'true'};""" % quote_name(get_keyspace())) + AND repair_full = {'enabled': 'true'} + AND repair_incremental = {'enabled': 'true'};""" % quote_name(get_keyspace())) with cqlsh_testrun(tty=True, env=self.default_env) as c: for cmdword in ('describe table', 'desc columnfamily'): diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index b3ac92ef48c0..f47b7e4ba722 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -153,11 +153,11 @@ private TableParams build(TableParams.Builder builder) if (hasOption(READ_REPAIR)) builder.readRepair(ReadRepairStrategy.fromString(getString(READ_REPAIR))); - if (hasOption(Option.AUTOMATED_REPAIR_FULL)) - builder.automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, getMap(Option.AUTOMATED_REPAIR_FULL))); + if (hasOption(Option.REPAIR_FULL)) + builder.automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, getMap(Option.REPAIR_FULL))); - if (hasOption(Option.AUTOMATED_REPAIR_INCREMENTAL)) - builder.automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, getMap(Option.AUTOMATED_REPAIR_INCREMENTAL))); + if (hasOption(Option.REPAIR_INCREMENTAL)) + builder.automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, getMap(Option.REPAIR_INCREMENTAL))); return builder.build(); } diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index 49007bdb9c99..fe9a35f74519 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -129,8 +129,8 @@ private SchemaKeyspace() + "additional_write_policy text," + "cdc boolean," + "read_repair text," - + "automated_repair_full frozen>," - + "automated_repair_incremental frozen>," + + "repair_full frozen>," + + "repair_incremental frozen>," + "PRIMARY KEY ((keyspace_name), table_name))"); private static final TableMetadata Columns = @@ -215,8 +215,8 @@ private SchemaKeyspace() + "additional_write_policy text," + "cdc boolean," + "read_repair text," - + "automated_repair_full frozen>," - + "automated_repair_incremental frozen>," + + "repair_full frozen>," + + "repair_incremental frozen>," + "PRIMARY KEY ((keyspace_name), view_name))"); private static final TableMetadata Indexes = @@ -565,8 +565,8 @@ private static void addTableParamsToRowBuilder(TableParams params, Row.SimpleBui .add("compression", params.compression.asMap()) .add("read_repair", params.readRepair.toString()) .add("extensions", params.extensions) - .add("automated_repair_full", params.automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) - .add("automated_repair_incremental", params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); + .add("repair_full", params.automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) + .add("repair_incremental", params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); // Only add CDC-enabled flag to schema if it's enabled on the node. This is to work around RTE's post-8099 if a 3.8+ @@ -1051,8 +1051,8 @@ static TableParams createTableParamsFromRow(UntypedResultSet.Row row) SpeculativeRetryPolicy.fromString("99PERCENTILE")) .cdc(row.has("cdc") && row.getBoolean("cdc")) .readRepair(getReadRepairStrategy(row)) - .automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, row.getFrozenTextMap("automated_repair_full"))) - .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, row.getFrozenTextMap("automated_repair_incremental"))); + .automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, row.getFrozenTextMap("repair_full"))) + .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, row.getFrozenTextMap("repair_incremental"))); // allow_auto_snapshot column was introduced in 4.2 if (row.has("allow_auto_snapshot")) diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index 39db4fb5b0f2..9f698ecb9114 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -72,8 +72,8 @@ public enum Option CRC_CHECK_CHANCE, CDC, READ_REPAIR, - AUTOMATED_REPAIR_FULL, - AUTOMATED_REPAIR_INCREMENTAL; + REPAIR_FULL, + REPAIR_INCREMENTAL; @Override public String toString() @@ -312,8 +312,8 @@ public String toString() .add(EXTENSIONS.toString(), extensions) .add(CDC.toString(), cdc) .add(READ_REPAIR.toString(), readRepair) - .add(Option.AUTOMATED_REPAIR_FULL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.full)) - .add(Option.AUTOMATED_REPAIR_INCREMENTAL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.incremental)) + .add(Option.REPAIR_FULL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.full)) + .add(Option.REPAIR_INCREMENTAL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.incremental)) .toString(); } @@ -367,9 +367,9 @@ public void appendCqlTo(CqlBuilder builder, boolean isView) .newLine() .append("AND speculative_retry = ").appendWithSingleQuotes(speculativeRetry.toString()) .newLine() - .append("AND automated_repair_full = ").append(automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) + .append("AND repair_full = ").append(automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) .newLine() - .append("AND automated_repair_incremental = ").append(automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); + .append("AND repair_incremental = ").append(automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); } public static final class Builder diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java index c62063b13b7f..48f8d54de3f6 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java @@ -74,7 +74,7 @@ private String formatRepairTypeConfig(NodeProbe probe, RepairType repairType, Au sb.append("\n\tpercentage of parallel repairs within group: " + config.getParallelRepairPercentage(repairType)); sb.append("\n\tmv repair enabled: " + config.getMVRepairEnabled(repairType)); sb.append("\n\tinitial scheduler delay: " + config.getInitialSchedulerDelay(repairType)); - sb.append("\n\trepair setssion timeout: " + config.getRepairSessionTimeout(repairType)); + sb.append("\n\trepair session timeout: " + config.getRepairSessionTimeout(repairType)); return sb.toString(); } diff --git a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java index 04fc04226915..29b2484d9b30 100644 --- a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java +++ b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java @@ -1075,8 +1075,8 @@ private static String tableParametersCql() " AND min_index_interval = 128\n" + " AND read_repair = 'BLOCKING'\n" + " AND speculative_retry = '99p'\n" + - " AND automated_repair_full = {'enabled': 'true'}\n" + - " AND automated_repair_incremental = {'enabled': 'true'};"; + " AND repair_full = {'enabled': 'true'}\n" + + " AND repair_incremental = {'enabled': 'true'};"; } private static String cqlQuoted(Map map) @@ -1104,8 +1104,8 @@ private static String mvParametersCql() " AND min_index_interval = 128\n" + " AND read_repair = 'BLOCKING'\n" + " AND speculative_retry = '99p'\n" + - " AND automated_repair_full = {'enabled': 'true'}\n" + - " AND automated_repair_incremental = {'enabled': 'true'};"; + " AND repair_full = {'enabled': 'true'}\n" + + " AND repair_incremental = {'enabled': 'true'};"; } private static String keyspaceOutput() diff --git a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java index dce36f16255a..3e123b36460b 100644 --- a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java +++ b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java @@ -307,8 +307,8 @@ public void testCfmOptionsCQL() " AND min_index_interval = 6\n" + " AND read_repair = 'BLOCKING'\n" + " AND speculative_retry = 'ALWAYS'\n" + - " AND automated_repair_full = {'enabled': 'true'}\n" + - " AND automated_repair_incremental = {'enabled': 'true'};" + " AND repair_full = {'enabled': 'true'}\n" + + " AND repair_incremental = {'enabled': 'true'};" )); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 988ec538b783..fd562b5354a6 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -127,7 +127,7 @@ public void setup() SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE)); QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i))", KEYSPACE, TABLE)); - QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i)) WITH automated_repair_full = {'enabled': 'false'} AND automated_repair_incremental = {'enabled': 'false'}", KEYSPACE, TABLE_DISABLED_AUTO_REPAIR)); + QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i)) WITH repair_full = {'enabled': 'false'} AND repair_incremental = {'enabled': 'false'}", KEYSPACE, TABLE_DISABLED_AUTO_REPAIR)); QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); @@ -560,8 +560,8 @@ public void testTokenRangesSplit() @Test public void testTableAttribute() { - assertTrue(TableAttributes.validKeywords().contains("automated_repair_full")); - assertTrue(TableAttributes.validKeywords().contains("automated_repair_incremental")); + assertTrue(TableAttributes.validKeywords().contains("repair_full")); + assertTrue(TableAttributes.validKeywords().contains("repair_incremental")); } @Test From 0cf7a5780dc8712ef12c255d60c8cb03ec507657 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Wed, 9 Oct 2024 16:09:57 -0700 Subject: [PATCH 040/115] Comments from Chris Lohfink: System::currentTimeMillis --> Clock.Global::currentTimeMillis --- .../org/apache/cassandra/repair/autorepair/AutoRepair.java | 3 ++- .../apache/cassandra/repair/autorepair/AutoRepairState.java | 3 ++- .../org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java | 1 - 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 5d5bcf3568c2..51176cb97724 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -39,6 +39,7 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.Pair; import org.slf4j.Logger; @@ -67,7 +68,7 @@ public class AutoRepair private static final Logger logger = LoggerFactory.getLogger(AutoRepair.class); @VisibleForTesting - protected static Supplier timeFunc = System::currentTimeMillis; + protected static Supplier timeFunc = Clock.Global::currentTimeMillis; public static AutoRepair instance = new AutoRepair(); diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index e15fe6f7a0a8..0e00efad292e 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -36,6 +36,7 @@ import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.concurrent.Condition; import org.apache.cassandra.utils.progress.ProgressEvent; import org.apache.cassandra.utils.progress.ProgressEventType; @@ -60,7 +61,7 @@ public abstract class AutoRepairState implements ProgressListener protected static final Logger logger = LoggerFactory.getLogger(AutoRepairState.class); private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS"); @VisibleForTesting - protected static Supplier timeFunc = System::currentTimeMillis; + protected static Supplier timeFunc = Clock.Global::currentTimeMillis; @VisibleForTesting protected final RepairType repairType; diff --git a/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java b/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java index bcab9645859d..aad049ae7fa2 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java +++ b/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java @@ -29,7 +29,6 @@ import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.locator.EndpointsByReplica; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; From 5828bc984e43e27500f6b2e5eb6670baa22226b5 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Wed, 25 Sep 2024 18:05:30 -0500 Subject: [PATCH 041/115] Update IAutoRepairTokenRangeSplitter API to be more flexible Updates IAutoRepairTokenRangeSplitter API to take in a set of tables instead of a single table and return "RepairAssignments". This allows the API more control over how the repairs are issued by the AutoRepair framework, by allowing grouping of repairs by specific tables or a single repair at the keyspace level. Also updates IAutoRepairTokenRangeSplitter to be a ParameterizedClass, so custom splitters can have their own configuration. Update AutoRepair to account for these changes. Update DefaultAutoRepairTokenSplitter to account for the change, and to also evaluate on getRepairByKeyspace option. This also improves the filtering of tables that should not be repaired where previously getRepairByKeyspace would not factor this in. Move logic to split token ranges evenly into AutoRepairUtils so it can be reutilized by other splitters. --- .../cassandra/config/DatabaseDescriptor.java | 6 + .../cassandra/config/ParameterizedClass.java | 3 +- .../cassandra/metrics/AutoRepairMetrics.java | 11 +- .../repair/autorepair/AutoRepair.java | 265 +++++++++--------- .../repair/autorepair/AutoRepairConfig.java | 10 +- .../repair/autorepair/AutoRepairState.java | 14 + .../repair/autorepair/AutoRepairUtils.java | 29 +- .../DefaultAutoRepairTokenSplitter.java | 66 ++--- .../IAutoRepairTokenRangeSplitter.java | 76 ++++- .../schema/SystemDistributedKeyspace.java | 2 +- .../apache/cassandra/utils/FBUtilities.java | 29 +- .../org/apache/cassandra/cql3/CQLTester.java | 4 +- .../autorepair/AutoRepairConfigTest.java | 8 +- ...DefaultTokenSplitterParameterizedTest.java | 172 ++++++++++++ .../AutoRepairParameterizedTest.java | 47 ++-- 15 files changed, 522 insertions(+), 220 deletions(-) create mode 100644 test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 874e59f81204..2c6645ae8eed 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -5364,4 +5364,10 @@ public static void setIncrementalRepairDiskHeadroomRejectRatio(double value) { conf.incremental_repair_disk_headroom_reject_ratio = value; } + + @VisibleForTesting + public static void setPartitioner(String name) + { + partitioner = FBUtilities.newPartitioner(name); + } } diff --git a/src/java/org/apache/cassandra/config/ParameterizedClass.java b/src/java/org/apache/cassandra/config/ParameterizedClass.java index 2d0390ef4486..6f43efab8ffd 100644 --- a/src/java/org/apache/cassandra/config/ParameterizedClass.java +++ b/src/java/org/apache/cassandra/config/ParameterizedClass.java @@ -19,6 +19,7 @@ import java.lang.reflect.Constructor; import java.util.Collections; +import java.io.Serializable; import java.util.List; import java.util.Map; @@ -30,7 +31,7 @@ import static org.apache.cassandra.utils.Shared.Scope.SIMULATION; @Shared(scope = SIMULATION) -public class ParameterizedClass +public class ParameterizedClass implements Serializable { public static final String CLASS_NAME = "class_name"; public static final String PARAMETERS = "parameters"; diff --git a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java index dedd8f170170..bdb6deacf6ad 100644 --- a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java +++ b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java @@ -39,7 +39,7 @@ public class AutoRepairMetrics public Gauge succeededTokenRangesCount; public Gauge failedTokenRangesCount; public Gauge skippedTokenRangesCount; - + public Gauge skippedTablesCount; public Counter repairTurnMyTurn; public Counter repairTurnMyTurnDueToPriority; public Counter repairTurnMyTurnForceRepair; @@ -82,6 +82,15 @@ public Integer getValue() } }); + skippedTablesCount = Metrics.register(factory.createMetricName("SkippedTablesCount"), new Gauge() + { + public Integer getValue() + { + return AutoRepair.instance.getRepairState(repairType).getSkippedTablesCount(); + } + }); + + longestUnrepairedSec = Metrics.register(factory.createMetricName("LongestUnrepairedSec"), new Gauge() { public Integer getValue() diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 51176cb97724..1abb71d69cac 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -32,7 +32,7 @@ import java.util.function.Supplier; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.common.util.concurrent.Uninterruptibles; import org.apache.cassandra.repair.RepairCoordinator; @@ -40,7 +40,7 @@ import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.utils.Clock; -import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -128,10 +128,10 @@ public void setup() AutoRepairService.instance.checkCanRun(repairType); repairExecutors.get(repairType).scheduleWithFixedDelay( - () -> repair(repairType), - config.getInitialSchedulerDelay(repairType).toSeconds(), - config.getRepairCheckInterval().toSeconds(), - TimeUnit.SECONDS); + () -> repair(repairType), + config.getInitialSchedulerDelay(repairType).toSeconds(), + config.getRepairCheckInterval().toSeconds(), + TimeUnit.SECONDS); } isSetupDone = true; } @@ -197,9 +197,7 @@ public void repair(AutoRepairConfig.RepairType repairType) repairState.setTotalTablesConsideredForRepair(0); repairState.setTotalMVTablesConsideredForRepair(0); - int failedTokenRanges = 0; - int succeededTokenRanges = 0; - int skippedTokenRanges = 0; + CollectectedRepairStats collectectedRepairStats = new CollectectedRepairStats(); List keyspaces = new ArrayList<>(); Keyspace.all().forEach(keyspaces::add); @@ -215,149 +213,116 @@ public void repair(AutoRepairConfig.RepairType repairType) } repairState.setRepairKeyspaceCount(repairState.getRepairKeyspaceCount() + 1); - List tablesToBeRepaired = retrieveTablesToBeRepaired(keyspace, repairType, repairState); - shuffleFunc.accept(tablesToBeRepaired); - for (String tableName : tablesToBeRepaired) + List tablesToBeRepairedList = retrieveTablesToBeRepaired(keyspace, config, repairType, repairState, collectectedRepairStats); + shuffleFunc.accept(tablesToBeRepairedList); + String keyspaceName = keyspace.getName(); + List repairAssignments = tokenRangeSplitters.get(repairType).getRepairAssignments(repairType, primaryRangeOnly, keyspaceName, tablesToBeRepairedList); + + int totalRepairAssignments = repairAssignments.size(); + long keyspaceStartTime = timeFunc.get(); + RepairAssignment previousAssignment = null; + long tableStartTime = timeFunc.get(); + int totalProcessedAssignments = 0; + Set> ranges = new HashSet<>(); + for (RepairAssignment curRepairAssignment : repairAssignments) { - String keyspaceName = keyspace.getName(); try { - List> subRangesToBeRepaired = tokenRangeSplitters.get(repairType).getRange(repairType, primaryRangeOnly, keyspaceName, tableName); - int totalSubRanges = subRangesToBeRepaired.size(); - - ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName); - if (!columnFamilyStore.metadata().params.automatedRepair.get(repairType).repairEnabled()) + totalProcessedAssignments++; + boolean repairOneTableAtATime = !config.getRepairByKeyspace(repairType); + if (previousAssignment != null && repairOneTableAtATime && !previousAssignment.tableNames.equals(curRepairAssignment.tableNames)) { - logger.info("Repair is disabled for keyspace {} for tables: {}", keyspaceName, tableName); - repairState.setTotalDisabledTablesRepairCount(repairState.getTotalDisabledTablesRepairCount() + 1); - continue; + // In the repair assignment, all the tables are appended sequnetially. + // Check if we have a different table, and if so, we should reset the table start time. + tableStartTime = timeFunc.get(); } - // this is done to make autorepair safe as running repair on table with more sstables - // may have its own challenges - int size = columnFamilyStore.getLiveSSTables().size(); - if (size > config.getRepairSSTableCountHigherThreshold(repairType)) + previousAssignment = curRepairAssignment; + if (!config.isAutoRepairEnabled(repairType)) { - logger.info("Too many SSTables for repair, not doing repair on table {}.{} " + - "totalSSTables {}", keyspaceName, tableName, columnFamilyStore.getLiveSSTables().size()); - skippedTokenRanges += totalSubRanges; - continue; + logger.error("Auto-repair for type {} is disabled hence not running repair", repairType); + repairState.setRepairInProgress(false); + return; } - - if (config.getRepairByKeyspace(repairType)) + if (AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, keyspaceStartTime, tablesToBeRepairedList.size())) { - logger.info("Repair keyspace {} for tables: {}", keyspaceName, tablesToBeRepaired); + collectectedRepairStats.skippedTokenRanges += totalRepairAssignments - totalProcessedAssignments; + logger.info("Keyspace took too much time to repair hence skipping it {}", + keyspaceName); + break; } - else + if (repairOneTableAtATime && AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, tableStartTime)) { - logger.info("Repair table {}.{}", keyspaceName, tableName); + collectectedRepairStats.skippedTokenRanges += 1; + logger.info("Table took too much time to repair hence skipping it table name {}.{}, token range {}", + keyspaceName, curRepairAssignment.tableNames, curRepairAssignment.tokenRange); + continue; } - long tableStartTime = timeFunc.get(); - Set> ranges = new HashSet<>(); - int totalProcessedSubRanges = 0; - for (Pair token : subRangesToBeRepaired) - { - if (!config.isAutoRepairEnabled(repairType)) - { - logger.error("Auto-repair for type {} is disabled hence not running repair", repairType); - repairState.setRepairInProgress(false); - return; - } - if (config.getRepairByKeyspace(repairType)) + Range tokenRange = curRepairAssignment.getTokenRange(); + logger.debug("Current Token Left side {}, right side {}", + tokenRange.left.toString(), + tokenRange.right.toString()); + + ranges.add(curRepairAssignment.getTokenRange()); + if ((totalProcessedAssignments % config.getRepairThreads(repairType) == 0) || + (totalProcessedAssignments == totalRepairAssignments)) + { + int retryCount = 0; + Future f = null; + while (retryCount <= config.getRepairMaxRetries()) { - if (AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, tableStartTime, tablesToBeRepaired.size())) + RepairCoordinator task = repairState.getRepairRunnable(keyspaceName, + Lists.newArrayList(curRepairAssignment.getTableNames()), + ranges, primaryRangeOnly); + repairState.resetWaitCondition(); + f = repairRunnableExecutors.get(repairType).submit(task); + try { - skippedTokenRanges += totalSubRanges - totalProcessedSubRanges; - logger.info("Keyspace took too much time to repair hence skipping it {}", - keyspaceName); - break; + repairState.waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); } - } - else - { - if (AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, tableStartTime)) + catch (InterruptedException e) { - skippedTokenRanges += totalSubRanges - totalProcessedSubRanges; - logger.info("Table took too much time to repair hence skipping it {}.{}", - keyspaceName, tableName); - break; + logger.error("Exception in cond await:", e); } - } - Token childStartToken = token.left; - Token childEndToken = token.right; - logger.debug("Current Token Left side {}, right side {}", childStartToken - .toString(), childEndToken.toString()); - - ranges.add(new Range<>(childStartToken, childEndToken)); - totalProcessedSubRanges++; - if ((totalProcessedSubRanges % config.getRepairThreads(repairType) == 0) || - (totalProcessedSubRanges == totalSubRanges)) - { - int retryCount = 0; - Future f = null; - while (retryCount <= config.getRepairMaxRetries()) + if (repairState.isSuccess()) { - RepairCoordinator task = repairState.getRepairRunnable(keyspaceName, - config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : ImmutableList.of(tableName), - ranges, primaryRangeOnly); - repairState.resetWaitCondition(); - f = repairRunnableExecutors.get(repairType).submit(task); - try - { - repairState.waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); - } - catch (InterruptedException e) - { - logger.error("Exception in cond await:", e); - } - if (repairState.isSuccess()) - { - break; - } - else if (retryCount < config.getRepairMaxRetries()) - { - boolean cancellationStatus = f.cancel(true); - logger.warn("Repair failed for range {}-{} for {}.{} with cancellationStatus: {} retrying after {} seconds...", - childStartToken, childEndToken, - keyspaceName, config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, - cancellationStatus, config.getRepairRetryBackoff().toSeconds()); - sleepFunc.accept(config.getRepairRetryBackoff().toSeconds(), TimeUnit.SECONDS); - } - retryCount++; + break; } - //check repair status - if (repairState.isSuccess()) + else if (retryCount < config.getRepairMaxRetries()) { - logger.info("Repair completed for range {}-{} for {}.{}, total subranges: {}," + - "processed subranges: {}", childStartToken, childEndToken, - keyspaceName, config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, totalSubRanges, totalProcessedSubRanges); - succeededTokenRanges += ranges.size(); + boolean cancellationStatus = f.cancel(true); + logger.warn("Repair failed for range {}-{} for {} tables {} with cancellationStatus: {} retrying after {} seconds...", + tokenRange.left, tokenRange.right, + keyspaceName, curRepairAssignment.getTableNames(), + cancellationStatus, config.getRepairRetryBackoff().toSeconds()); + sleepFunc.accept(config.getRepairRetryBackoff().toSeconds(), TimeUnit.SECONDS); } - else + retryCount++; + } + //check repair status + if (repairState.isSuccess()) + { + logger.info("Repair completed for range {}-{} for {} tables {}, total assignments: {}," + + "processed assignments: {}", tokenRange.left, tokenRange.right, + keyspaceName, curRepairAssignment.getTableNames(), totalRepairAssignments, totalProcessedAssignments); + collectectedRepairStats.succeededTokenRanges += ranges.size(); + } + else + { + boolean cancellationStatus = true; + if (f != null) { - boolean cancellationStatus = true; - if (f != null) - { - cancellationStatus = f.cancel(true); - } - //in the future we can add retry, etc. - logger.error("Repair failed for range {}-{} for {}.{} after {} retries, total subranges: {}," + - "processed subranges: {}, cancellationStatus: {}", childStartToken.toString(), childEndToken.toString(), keyspaceName, - config.getRepairByKeyspace(repairType) ? tablesToBeRepaired : tableName, retryCount, totalSubRanges, totalProcessedSubRanges, cancellationStatus); - failedTokenRanges += ranges.size(); + cancellationStatus = f.cancel(true); } - ranges.clear(); + //in the future we can add retry, etc. + logger.error("Repair failed for range {}-{} for {} tables {} after {} retries, total assignments: {}," + + "processed assignments: {}, cancellationStatus: {}", tokenRange.left, tokenRange.right, keyspaceName, + curRepairAssignment.getTableNames(), retryCount, totalRepairAssignments, totalProcessedAssignments, cancellationStatus); + collectectedRepairStats.failedTokenRanges += ranges.size(); } + ranges.clear(); } - if (config.getRepairByKeyspace(repairType)) - { - logger.info("Repair completed for keyspace {}, tables: {}", keyspaceName, tablesToBeRepaired); - break; - } - else - { - logger.info("Repair completed for {}.{}", keyspaceName, tableName); - } + logger.info("Repair completed for {} tables {}, range {}", keyspaceName, curRepairAssignment.getTableNames(), curRepairAssignment.getTokenRange()); } catch (Exception e) { @@ -365,7 +330,7 @@ else if (retryCount < config.getRepairMaxRetries()) } } } - cleanupAndUpdateStats(turn, repairType, repairState, myId, startTime, failedTokenRanges, succeededTokenRanges, skippedTokenRanges); + cleanupAndUpdateStats(turn, repairType, repairState, myId, startTime, collectectedRepairStats); } else { @@ -400,7 +365,7 @@ private boolean tooSoonToRunRepair(AutoRepairConfig.RepairType repairType, AutoR return false; } - private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairConfig.RepairType repairType, AutoRepairState repairState) + private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairConfig config, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, CollectectedRepairStats collectectedRepairStats) { Tables tables = keyspace.getMetadata().tables; List tablesToBeRepaired = new ArrayList<>(); @@ -410,6 +375,27 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon repairState.setTotalTablesConsideredForRepair(repairState.getTotalTablesConsideredForRepair() + 1); TableMetadata tableMetadata = iter.next(); String tableName = tableMetadata.name; + + ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName); + if (!columnFamilyStore.metadata().params.automatedRepair.get(repairType).repairEnabled()) + { + logger.info("Repair is disabled for keyspace {} for tables: {}", keyspace.getName(), tableName); + repairState.setTotalDisabledTablesRepairCount(repairState.getTotalDisabledTablesRepairCount() + 1); + collectectedRepairStats.skippedTables++; + continue; + } + + // this is done to make autorepair safe as running repair on table with more sstables + // may have its own challenges + int totalSSTables = columnFamilyStore.getLiveSSTables().size(); + if (totalSSTables > config.getRepairSSTableCountHigherThreshold(repairType)) + { + logger.info("Too many SSTables for repair for table {}.{}" + + "totalSSTables {}", keyspace.getName(), tableName, totalSSTables); + collectectedRepairStats.skippedTables++; + continue; + } + tablesToBeRepaired.add(tableName); // See if we should repair MVs as well that are associated with this given table @@ -424,7 +410,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon } private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, UUID myId, - long startTime, int failedTokenRanges, int succeededTokenRanges, int skippedTokenRanges) throws InterruptedException + long startTime, CollectectedRepairStats collectectedRepairStats) throws InterruptedException { //if it was due to priority then remove it now if (turn == MY_TURN_DUE_TO_PRIORITY) @@ -433,16 +419,17 @@ private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType AutoRepairUtils.removePriorityStatus(repairType, myId); } - repairState.setFailedTokenRangesCount(failedTokenRanges); - repairState.setSucceededTokenRangesCount(succeededTokenRanges); - repairState.setSkippedTokenRangesCount(skippedTokenRanges); + repairState.setFailedTokenRangesCount(collectectedRepairStats.failedTokenRanges); + repairState.setSucceededTokenRangesCount(collectectedRepairStats.succeededTokenRanges); + repairState.setSkippedTokenRangesCount(collectectedRepairStats.skippedTokenRanges); + repairState.setSkippedTablesCount(collectectedRepairStats.skippedTables); repairState.setNodeRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - startTime)); long timeInHours = TimeUnit.SECONDS.toHours(repairState.getNodeRepairTimeInSec()); logger.info("Local {} repair time {} hour(s), stats: repairKeyspaceCount {}, " + "repairTokenRangesSuccessCount {}, repairTokenRangesFailureCount {}, " + - "repairTokenRangesSkipCount {}", repairType, timeInHours, repairState.getRepairKeyspaceCount(), + "repairTokenRangesSkipCount {}, repairTablesSkipCount {}", repairType, timeInHours, repairState.getRepairKeyspaceCount(), repairState.getSucceededTokenRangesCount(), repairState.getFailedTokenRangesCount(), - repairState.getSkippedTokenRangesCount()); + repairState.getSkippedTokenRangesCount(), repairState.getSkippedTablesCount()); if (repairState.getLastRepairTime() != 0) { repairState.setClusterRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - @@ -466,4 +453,12 @@ public AutoRepairState getRepairState(AutoRepairConfig.RepairType repairType) { return repairStates.get(repairType); } + + static class CollectectedRepairStats + { + int failedTokenRanges = 0; + int succeededTokenRanges = 0; + int skippedTokenRanges = 0; + int skippedTables = 0; + } } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index ac9394d9cfe1..30433a1ad7ff 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.Collections; import java.util.EnumMap; import java.util.HashSet; import java.util.Map; @@ -31,6 +32,7 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.config.ParameterizedClass; public class AutoRepairConfig implements Serializable { @@ -274,7 +276,7 @@ public boolean getForceRepairNewNode(RepairType repairType) return applyOverrides(repairType, opt -> opt.force_repair_new_node); } - public String getTokenRangeSplitter(RepairType repairType) + public ParameterizedClass getTokenRangeSplitter(RepairType repairType) { return applyOverrides(repairType, opt -> opt.token_range_splitter); } @@ -331,7 +333,7 @@ protected static Options getDefaultOptions() opts.force_repair_new_node = false; opts.table_max_repair_time = new DurationSpec.IntSecondsBound("6h"); opts.mv_repair_enabled = false; - opts.token_range_splitter = DefaultAutoRepairTokenSplitter.class.getName(); + opts.token_range_splitter = new ParameterizedClass(DefaultAutoRepairTokenSplitter.class.getName(), Collections.emptyMap()); opts.initial_scheduler_delay = new DurationSpec.IntSecondsBound("5m"); // 5 minutes opts.repair_session_timeout = new DurationSpec.IntSecondsBound("3h"); // 3 hours @@ -395,9 +397,9 @@ protected static Options getDefaultOptions() // the default is 'true'. // This flag determines whether the auto-repair framework needs to run anti-entropy, a.k.a, repair on the MV table or not. public volatile Boolean mv_repair_enabled; - // the default is DefaultAutoRepairTokenSplitter.class.getName(). The class should implement IAutoRepairTokenRangeSplitter. + // the default is DefaultAutoRepairTokenSplitter. The class should implement IAutoRepairTokenRangeSplitter. // The default implementation splits the tokens based on the token ranges owned by this node divided by the number of 'number_of_subranges' - public volatile String token_range_splitter; + public volatile ParameterizedClass token_range_splitter; // the minimum delay after a node starts before the scheduler starts running repair public volatile DurationSpec.IntSecondsBound initial_scheduler_delay; // repair session timeout - this is applicable for each repair session diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index 0e00efad292e..84701ca78fbc 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -88,6 +88,10 @@ public abstract class AutoRepairState implements ProgressListener protected int succeededTokenRangesCount = 0; @VisibleForTesting protected int skippedTokenRangesCount = 0; + + @VisibleForTesting + protected int skippedTablesCount = 0; + @VisibleForTesting protected AutoRepairHistory longestUnrepairedNode; @VisibleForTesting @@ -260,6 +264,16 @@ public int getSkippedTokenRangesCount() return skippedTokenRangesCount; } + public void setSkippedTablesCount(int count) + { + skippedTablesCount = count; + } + + public int getSkippedTablesCount() + { + return skippedTablesCount; + } + public boolean isSuccess() { return success; diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index 2b1ca4c39cc8..197cd2d6274d 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -32,6 +32,8 @@ import com.google.common.base.MoreObjects; import com.google.common.collect.Lists; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; import org.apache.cassandra.locator.LocalStrategy; import org.slf4j.Logger; @@ -777,7 +779,6 @@ public static Set getPriorityHosts(RepairType repairType) for (UUID hostId : getPriorityHostIds(repairType)) { hosts.add(ClusterMetadata.current().directory.addresses.get(NodeId.fromUUID(hostId)).broadcastAddress); - //hosts.add(StorageService.instance.getTokenMetadata().getEndpointForHostId(hostId)); } return hosts; } @@ -844,4 +845,30 @@ public static void runRepairOnNewlyBootstrappedNodeIfEnabled() AutoRepairUtils.setForceRepairNewNode(rType); } } + + public static List> splitEvenly(Range tokenRange, int numberOfSplits) + { + List> splitRanges = new ArrayList<>(); + long left = (Long) tokenRange.left.getTokenValue(); + long right = (Long) tokenRange.right.getTokenValue(); + long repairTokenWidth = (right - left) / numberOfSplits; + for (int i = 0; i < numberOfSplits; i++) + { + long curLeft = left + (i * repairTokenWidth); + long curRight = curLeft + repairTokenWidth; + + if ((i + 1) == numberOfSplits) + { + curRight = right; + } + + Token childStartToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + curLeft); + Token childEndToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + curRight); + logger.debug("Current Token Left side {}, right side {}", childStartToken + .toString(), childEndToken.toString()); + Range splitRange = new Range<>(childStartToken, childEndToken); + splitRanges.add(splitRange); + } + return splitRanges; + } } diff --git a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java index 18231fff2af0..9a884f61c581 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java @@ -20,69 +20,61 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; import org.apache.cassandra.service.AutoRepairService; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.tcm.ClusterMetadata; -import org.apache.cassandra.utils.Pair; + +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.splitEvenly; public class DefaultAutoRepairTokenSplitter implements IAutoRepairTokenRangeSplitter { - private static final Logger logger = LoggerFactory.getLogger(DefaultAutoRepairTokenSplitter.class); - - @Override - public List> getRange(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, String tableName) + public List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) { - List> range = new ArrayList<>(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + List repairAssignments = new ArrayList<>(); Collection> tokens = StorageService.instance.getPrimaryRanges(keyspaceName); if (!primaryRangeOnly) { // if we need to repair non-primary token ranges, then change the tokens accrodingly - tokens = StorageService.instance.getLocalReplicas(keyspaceName).ranges(); + tokens = StorageService.instance.getLocalReplicas(keyspaceName).onlyFull().ranges(); } - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); int numberOfSubranges = config.getRepairSubRangeNum(repairType); + + boolean byKeyspace = config.getRepairByKeyspace(repairType); + + // collect all token ranges. + List> allRanges = new ArrayList<>(); for (Range token : tokens) { - Murmur3Partitioner.LongToken l = (Murmur3Partitioner.LongToken) (token.left); - Murmur3Partitioner.LongToken r = (Murmur3Partitioner.LongToken) (token.right); - // Token.TokenFactory factory = ClusterMetadata.current().partitioner.getTokenFactory(); - - Token parentStartToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + l.getTokenValue()); - Token parentEndToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + r.getTokenValue()); - logger.debug("Parent Token Left side {}, right side {}", parentStartToken.toString(), - parentEndToken.toString()); + allRanges.addAll(splitEvenly(token, numberOfSubranges)); + } - long left = (Long) l.getTokenValue(); - long right = (Long) r.getTokenValue(); - long repairTokenWidth = (right - left) / numberOfSubranges; - for (int i = 0; i < numberOfSubranges; i++) + if (byKeyspace) + { + for (Range splitRange : allRanges) { - long curLeft = left + (i * repairTokenWidth); - long curRight = curLeft + repairTokenWidth; - - if ((i + 1) == numberOfSubranges) + // add repair assignment for each range entire keyspace's tables + repairAssignments.add(new RepairAssignment(splitRange, keyspaceName, tableNames)); + } + } + else + { + // add repair assignment per table + for (String tableName : tableNames) + { + for (Range splitRange : allRanges) { - curRight = right; + repairAssignments.add(new RepairAssignment(splitRange, keyspaceName, Collections.singletonList(tableName))); } - - Token childStartToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + curLeft); - Token childEndToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + curRight); - logger.debug("Current Token Left side {}, right side {}", childStartToken - .toString(), childEndToken.toString()); - range.add(Pair.create(childStartToken, childEndToken)); } } - return range; + return repairAssignments; } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java index 2b69898a360d..dd02c4d4f7b6 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java @@ -19,13 +19,81 @@ import java.util.List; +import java.util.Objects; +import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.utils.Pair; public interface IAutoRepairTokenRangeSplitter { - // split the token range you wish to repair into multiple subranges - // the autorepair framework will repair the list of returned subrange in a sequence - List> getRange(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, String tableName); + + /** + * Split the token range you wish to repair into multiple assignments. + * The autorepair framework will repair the list of returned subrange in a sequence. + * @param repairType The type of repair being executed + * @param primaryRangeOnly Whether to repair only this node's primary ranges or all of its ranges. + * @param keyspaceName The keyspace being repaired + * @param tableNames The tables to repair + * @return repair assignments broken up by range, keyspace and tables. + */ + List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames); + + /** + * Defines a repair assignment to be issued by the autorepair framework. + */ + class RepairAssignment + { + final Range tokenRange; + + final String keyspaceName; + + final List tableNames; + + public RepairAssignment(Range tokenRange, String keyspaceName, List tableNames) + { + this.tokenRange = tokenRange; + this.keyspaceName = keyspaceName; + this.tableNames = tableNames; + } + + public Range getTokenRange() + { + return tokenRange; + } + + public String getKeyspaceName() + { + return keyspaceName; + } + + public List getTableNames() + { + return tableNames; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RepairAssignment that = (RepairAssignment) o; + return Objects.equals(tokenRange, that.tokenRange) && Objects.equals(keyspaceName, that.keyspaceName) && Objects.equals(tableNames, that.tableNames); + } + + @Override + public int hashCode() + { + return Objects.hash(tokenRange, keyspaceName, tableNames); + } + + @Override + public String toString() + { + return "RepairAssignment{" + + "tokenRange=" + tokenRange + + ", keyspaceName='" + keyspaceName + '\'' + + ", tableNames=" + tableNames + + '}'; + } + } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java index 1b775fb7589c..42ac3aaa240b 100644 --- a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java @@ -417,4 +417,4 @@ private enum BuildStatus { UNKNOWN, STARTED, SUCCESS } -} +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index b2e24a3aab46..b2dcedde48d4 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -67,6 +67,7 @@ import com.google.common.collect.ImmutableList; import com.vdurmont.semver4j.Semver; +import org.apache.cassandra.config.ParameterizedClass; import org.apache.cassandra.io.util.File; import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter; import org.apache.commons.lang3.StringUtils; @@ -720,11 +721,31 @@ public static AbstractCryptoProvider newCryptoProvider(String className, Map tokenRangeSplitterClass = Class.forName(className); + try + { + Map parameters = parameterizedClass.parameters != null ? parameterizedClass.parameters : Collections.emptyMap(); + // first attempt to initialize with Map arguments. + return (IAutoRepairTokenRangeSplitter) tokenRangeSplitterClass.getConstructor(Map.class).newInstance(parameters); + } + catch (NoSuchMethodException nsme) + { + // fall back on no argument constructor. + return (IAutoRepairTokenRangeSplitter) tokenRangeSplitterClass.getConstructor().newInstance(); + } + } + catch (Exception ex) + { + throw new ConfigurationException("Unable to create instance of IAutoRepairTokenRangeSplitter for " + className, ex); + } } /** diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 4c3d130c9295..55d908902be4 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -3052,7 +3052,7 @@ protected static long seed() return SEED; } - protected static void setupSeed() + public static void setupSeed() { if (RANDOM != null) return; SEED = TEST_RANDOM_SEED.getLong(new DefaultRandom().nextLong()); @@ -3065,7 +3065,7 @@ public void resetSeed() RANDOM.setSeed(SEED); } - protected static void updateConfigs() + public static void updateConfigs() { if (CONFIG_GEN == null) CONFIG_GEN = new ConfigGenBuilder().build(); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java index c42959eec32b..4f1dd029ef5e 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -20,6 +20,7 @@ import java.util.EnumMap; import java.util.Objects; +import java.util.Collections; import java.util.Set; import com.google.common.collect.ImmutableSet; @@ -31,6 +32,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.config.ParameterizedClass; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.repair.autorepair.AutoRepairConfig.Options; @@ -384,14 +386,16 @@ public void testGetDefaultOptionsTokenRangeSplitter() { Options defaultOptions = Options.getDefaultOptions(); - assertEquals(DefaultAutoRepairTokenSplitter.class.getName(),defaultOptions.token_range_splitter); + ParameterizedClass expectedDefault = new ParameterizedClass(DefaultAutoRepairTokenSplitter.class.getName(), Collections.emptyMap()); + + assertEquals(expectedDefault, defaultOptions.token_range_splitter); assertEquals(DefaultAutoRepairTokenSplitter.class.getName(), FBUtilities.newAutoRepairTokenRangeSplitter(defaultOptions.token_range_splitter).getClass().getName()); } @Test(expected = ConfigurationException.class) public void testInvalidTokenRangeSplitter() { - assertEquals(DefaultAutoRepairTokenSplitter.class.getName(), FBUtilities.newAutoRepairTokenRangeSplitter("invalid-class").getClass().getName()); + FBUtilities.newAutoRepairTokenRangeSplitter(new ParameterizedClass("invalid-class", Collections.emptyMap())); } @Test diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java new file mode 100644 index 000000000000..804d0a712b48 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.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.repair.autorepair; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.ServerTestUtils; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.index.sai.disk.format.Version; +import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.tcm.ClusterMetadata; + +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; +import static org.apache.cassandra.cql3.CQLTester.Fuzzed.setupSeed; +import static org.apache.cassandra.cql3.CQLTester.Fuzzed.updateConfigs; +import static org.junit.Assert.assertEquals; + +@RunWith(Parameterized.class) +public class AutoRepairDefaultTokenSplitterParameterizedTest +{ + private static final String KEYSPACE = "ks"; + private static final String TABLE1 = "tbl1"; + private static final String TABLE2 = "tbl2"; + private static final String TABLE3 = "tbl3"; + + @Parameterized.Parameter() + public AutoRepairConfig.RepairType repairType; + + @Parameterized.Parameters(name = "repairType={0}") + public static Collection repairTypes() + { + return Arrays.asList(AutoRepairConfig.RepairType.values()); + } + + @BeforeClass + public static void setupClass() throws Exception + { + setupSeed(); + updateConfigs(); + DatabaseDescriptor.setPartitioner("org.apache.cassandra.dht.Murmur3Partitioner"); + ServerTestUtils.prepareServerNoRegister(); + + Token t1 = new Murmur3Partitioner.LongToken(0); + Token t2 = new Murmur3Partitioner.LongToken(256); + Token t3 = new Murmur3Partitioner.LongToken(1024); + Set tokens = new HashSet<>(); + tokens.add(t1); + tokens.add(t2); + tokens.add(t3); + + ServerTestUtils.registerLocal(tokens); + // Ensure that the on-disk format statics are loaded before the test run + Version.LATEST.onDiskFormat(); + StorageService.instance.doAutoRepairSetup(); + + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE)); + } + + private static void appendExpectedTokens(long left, long right, int numberOfSplits, List> expectedToken) + { + long repairTokenWidth = (right - left) / numberOfSplits; + for (int i = 0; i < numberOfSplits; i++) + { + long curLeft = left + (i * repairTokenWidth); + long curRight = curLeft + repairTokenWidth; + if ((i + 1) == numberOfSplits) + { + curRight = right; + } + Token childStartToken = ClusterMetadata.current() + .partitioner.getTokenFactory().fromString("" + curLeft); + Token childEndToken = ClusterMetadata.current() + .partitioner.getTokenFactory().fromString("" + curRight); + expectedToken.add(new Range<>(childStartToken, childEndToken)); + } + } + + @Test + public void testTokenRangesSplitByTable() + { + AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(repairType, false); + int totalTokenRanges = 3; + Collection> tokens = StorageService.instance.getPrimaryRanges(KEYSPACE); + assertEquals(totalTokenRanges, tokens.size()); + int numberOfSplits = 4; + List tables = Arrays.asList(TABLE1, TABLE2, TABLE3); + List> expectedToken = new ArrayList<>(); + for (int i = 0; i < tables.size(); i++) + { + appendExpectedTokens(1024, 0, numberOfSplits, expectedToken); + appendExpectedTokens(0, 256, numberOfSplits, expectedToken); + appendExpectedTokens(256, 1024, numberOfSplits, expectedToken); + } + + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setRepairSubRangeNum(repairType, numberOfSplits); + List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); + assertEquals(totalTokenRanges * numberOfSplits * tables.size(), assignments.size()); + assertEquals(expectedToken.size(), assignments.size()); + + int expectedTableIndex = -1; + for (int i = 0; i < totalTokenRanges * numberOfSplits * tables.size(); i++) + { + if (i % (totalTokenRanges * numberOfSplits) == 0) + { + expectedTableIndex++; + } + assertEquals(expectedToken.get(i), assignments.get(i).getTokenRange()); + assertEquals(Arrays.asList(tables.get(expectedTableIndex)), assignments.get(i).getTableNames()); + } + } + + @Test + public void testTokenRangesSplitByKeyspace() + { + AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(repairType, true); + int totalTokenRanges = 3; + Collection> tokens = StorageService.instance.getPrimaryRanges(KEYSPACE); + assertEquals(totalTokenRanges, tokens.size()); + int numberOfSplits = 4; + List tables = Arrays.asList(TABLE1, TABLE2, TABLE3); + List> expectedToken = new ArrayList<>(); + appendExpectedTokens(1024, 0, numberOfSplits, expectedToken); + appendExpectedTokens(0, 256, numberOfSplits, expectedToken); + appendExpectedTokens(256, 1024, numberOfSplits, expectedToken); + + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.setRepairSubRangeNum(repairType, numberOfSplits); + List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); + assertEquals(totalTokenRanges * numberOfSplits, assignments.size()); + assertEquals(expectedToken.size(), assignments.size()); + + for (int i = 0; i < totalTokenRanges * numberOfSplits; i++) + { + assertEquals(expectedToken.get(i), assignments.get(i).getTokenRange()); + assertEquals(tables, assignments.get(i).getTableNames()); + } + } +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index fd562b5354a6..490e7a102b94 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -37,9 +38,9 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.repair.RepairCoordinator; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.Pair; import org.junit.After; import org.junit.Assert; @@ -414,21 +415,26 @@ public void testSkipRepairSSTableCountHigherThreshold() assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); state.setLastRepairTime(0); AutoRepair.instance.repair(repairType); - assertEquals(1, state.getTotalMVTablesConsideredForRepair()); - assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); - // skipping one time for the base table and another time for MV table - assertEquals(2, state.getSkippedTokenRangesCount()); - assertEquals(2, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertEquals(0, state.getTotalMVTablesConsideredForRepair()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + // skipping both the tables - one table is due to its repair has been disabled, and another one due to high sstable count + assertEquals(0, state.getSkippedTokenRangesCount()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertEquals(2, state.getSkippedTablesCount()); + assertEquals(2, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); // set it to higher value, and this time, the tables should not be skipped - config.setRepairSSTableCountHigherThreshold(repairType, 11); config.setRepairSSTableCountHigherThreshold(repairType, beforeCount); state.setLastRepairTime(0); + state.setSkippedTablesCount(0); + state.setTotalMVTablesConsideredForRepair(0); AutoRepair.instance.repair(repairType); assertEquals(1, state.getTotalMVTablesConsideredForRepair()); - assertEquals(0, state.getSkippedTokenRangesCount()); assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertEquals(0, state.getSkippedTokenRangesCount()); assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertEquals(1, state.getSkippedTablesCount()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); } @Test @@ -534,27 +540,12 @@ public void testTokenRangesNoSplit() { Collection> tokens = StorageService.instance.getPrimaryRanges(KEYSPACE); assertEquals(1, tokens.size()); - List> expectedToken = new ArrayList<>(); - expectedToken.addAll(tokens); + List> expectedToken = new ArrayList<>(tokens); - List> ranges = new DefaultAutoRepairTokenSplitter().getRange(repairType, true, KEYSPACE, TABLE); - assertEquals(1, ranges.size()); - assertEquals(expectedToken.get(0).left, ranges.get(0).left); - assertEquals(expectedToken.get(0).right, ranges.get(0).right); - } - - @Test - public void testTokenRangesSplit() - { - Collection> tokens = StorageService.instance.getPrimaryRanges(KEYSPACE); - assertEquals(1, tokens.size()); - List> expectedToken = new ArrayList<>(); - expectedToken.addAll(tokens); - - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); - config.setRepairSubRangeNum(repairType, 4); - List> ranges = new DefaultAutoRepairTokenSplitter().getRange(repairType, true, KEYSPACE, TABLE); - assertEquals(4, ranges.size()); + List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, Collections.singletonList(TABLE)); + assertEquals(1, assignments.size()); + assertEquals(expectedToken.get(0).left, assignments.get(0).getTokenRange().left); + assertEquals(expectedToken.get(0).right, assignments.get(0).getTokenRange().right); } @Test From a69a71ace919494bcb4467541836b441445a0fcd Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Sat, 12 Oct 2024 09:45:10 -0700 Subject: [PATCH 042/115] Fix a corner-case bug NPE org.apache.cassandra.repair.autorepair.AutoRepairUtils.myTurnToRunRepair --- .../repair/autorepair/AutoRepairUtils.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index 197cd2d6274d..8849b7847aef 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -578,12 +578,8 @@ else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) } } - // get the longest unrepaired node from the nodes which are not running repair - AutoRepairHistory defaultNodeToBeRepaired = getHostWithLongestUnrepairTime(currentRepairStatus.historiesWithoutOnGoingRepair); - //check who is next, which is helpful for debugging - logger.info("Next node to be repaired for repair type {} by default: {}", repairType, defaultNodeToBeRepaired); UUID priorityHostId = null; - if (currentRepairStatus.priority != null) + if (currentRepairStatus != null && currentRepairStatus.priority != null) { for (UUID priorityID : currentRepairStatus.priority) { @@ -614,8 +610,14 @@ else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) return MY_TURN_DUE_TO_PRIORITY; } - if (defaultNodeToBeRepaired.hostId.equals(myId)) + // get the longest unrepaired node from the nodes which are not running repair + AutoRepairHistory defaultNodeToBeRepaired = getHostWithLongestUnrepairTime(currentRepairStatus.historiesWithoutOnGoingRepair); + //check who is next, which is helpful for debugging + logger.info("Next node to be repaired for repair type {} by default: {}", repairType, defaultNodeToBeRepaired); + if (defaultNodeToBeRepaired != null && defaultNodeToBeRepaired.hostId.equals(myId)) + { return MY_TURN; + } } else if (currentRepairStatus.hostIdsWithOnGoingForceRepair.contains(myId)) { From d58dc7310bc1b8676e73e0a47a3fd85ad2f496bd Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Mon, 14 Oct 2024 13:50:25 -0700 Subject: [PATCH 043/115] Update documentation for the newly added metric --- doc/modules/cassandra/pages/managing/operating/metrics.adoc | 3 +++ 1 file changed, 3 insertions(+) diff --git a/doc/modules/cassandra/pages/managing/operating/metrics.adoc b/doc/modules/cassandra/pages/managing/operating/metrics.adoc index ab4eb7c37231..2abeaefa6d39 100644 --- a/doc/modules/cassandra/pages/managing/operating/metrics.adoc +++ b/doc/modules/cassandra/pages/managing/operating/metrics.adoc @@ -1111,6 +1111,9 @@ ran on the node in seconds |SkippedTokenRangesCount |Gauge |Number of token ranges skipped on the node +|SkippedTablesCount |Gauge |Number of tables skipped +on the node + |TotalMVTablesConsideredForRepair |Gauge |Number of materialized views considered on the node From d214ad6a1c00e5b3ad70bf42b18a3631fe2a5ee1 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Mon, 14 Oct 2024 18:32:40 -0700 Subject: [PATCH 044/115] Fix the failed StorageServiceServerTest test cases --- .../autorepair/SSTableRepairedAtTest.java | 171 ++++++++++++++++++ .../service/StorageServiceServerTest.java | 125 ------------- 2 files changed, 171 insertions(+), 125 deletions(-) create mode 100644 test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java diff --git a/test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java b/test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java new file mode 100644 index 000000000000..bd14eea805b3 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.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.repair.autorepair; + +import java.net.UnknownHostException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.config.DatabaseDescriptor; +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.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.service.StorageService; + +import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + + +public class SSTableRepairedAtTest extends CQLTester +{ + public static final String TEST_KEYSPACE = "test_keyspace"; + public static ColumnFamilyStore table1; + public static ColumnFamilyStore table2; + + @BeforeClass + public static void setUp() throws ConfigurationException, UnknownHostException + { + requireNetwork(); + AutoRepairUtils.setup(); + StorageService.instance.doAutoRepairSetup(); + DatabaseDescriptor.setCDCEnabled(false); + } + + @Before + public void clearData() + { + SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", TEST_KEYSPACE)); + QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (key text, val text, primary key(key))", TEST_KEYSPACE, "table1")); + QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (key text, val text, primary key(key))", TEST_KEYSPACE, "table2")); + + Keyspace.open(TEST_KEYSPACE).getColumnFamilyStore("table1").truncateBlocking(); + Keyspace.open(TEST_KEYSPACE).getColumnFamilyStore("table2").truncateBlocking(); + + table1 = Keyspace.open(TEST_KEYSPACE).getColumnFamilyStore("table1"); + assert table1 != null; + table2 = Keyspace.open(TEST_KEYSPACE).getColumnFamilyStore("table2"); + assert table2 != null; + } + @Test + public void testGetTablesForKeyspace() + { + List result = StorageService.instance.getTablesForKeyspace(TEST_KEYSPACE); + + assertEquals(Arrays.asList(table1.name, table2.name), result.stream().sorted().collect(Collectors.toList())); + } + + @Test + public void testGetTablesForKeyspaceNotFound() + { + String missingKeyspace = "MISSING_KEYSPACE"; + try + { + StorageService.instance.getTablesForKeyspace(missingKeyspace); + fail("Expected an AssertionError to be thrown"); + } + catch (AssertionError e) + { + assertEquals("Unknown keyspace " + missingKeyspace, e.getMessage()); + } + } + + @Test + public void testMutateSSTableRepairedStateTableNotFound() + { + try + { + StorageService.instance.mutateSSTableRepairedState(true, false, TEST_KEYSPACE, Arrays.asList("MISSING_TABLE")); + fail("Expected an InvalidRequestException to be thrown"); + } + catch (InvalidRequestException e) + { + // Test passed + } + } + + @Test + public void testMutateSSTableRepairedStateTablePreview() + { + SchemaLoader.insertData(TEST_KEYSPACE, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + assertEquals(1, table1.getLiveSSTables().size()); + + List result = StorageService.instance.mutateSSTableRepairedState(true, true, TEST_KEYSPACE, Arrays.asList(table1.name)); + + assertEquals(1, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); + }); + } + + @Test + public void testMutateSSTableRepairedStateTableRepaired() + { + SchemaLoader.insertData(TEST_KEYSPACE, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + SchemaLoader.insertData(TEST_KEYSPACE, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + assertEquals(2, table1.getLiveSSTables().size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + }); + + List result = StorageService.instance.mutateSSTableRepairedState(true, false, TEST_KEYSPACE, Arrays.asList(table1.name)); + + assertEquals(2, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertTrue(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); + }); + } + + @Test + public void testMutateSSTableRepairedStateTableUnrepaired() throws Exception + { + SchemaLoader.insertData(TEST_KEYSPACE, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + SchemaLoader.insertData(TEST_KEYSPACE, table1.name, 0, 1); + table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + table1.getCompactionStrategyManager().mutateRepaired(table1.getLiveSSTables(), 1, null, false); + assertEquals(2, table1.getLiveSSTables().stream().filter(SSTableReader::isRepaired).count()); + + List result = StorageService.instance.mutateSSTableRepairedState(false, false, TEST_KEYSPACE, Arrays.asList(table1.name)); + + assertEquals(2, result.size()); + table1.getLiveSSTables().forEach(sstable -> { + assertFalse(sstable.isRepaired()); + assertTrue(result.contains(sstable.descriptor.baseFile().name())); + }); + } +} diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java index 181ef5de02d1..87534135801a 100644 --- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java +++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.net.UnknownHostException; - import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -30,13 +29,6 @@ import java.util.UUID; import com.google.common.collect.Sets; -import java.util.*; -import java.util.stream.Collectors; - -import org.apache.cassandra.SchemaLoader; -import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.Keyspace; - import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -54,8 +46,6 @@ import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.AbstractNetworkTopologySnitch; -import org.apache.cassandra.exceptions.InvalidRequestException; -import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.WithPartitioner; @@ -76,17 +66,10 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.GOSSIP_DISABLE_THREAD_VALIDATION; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - public class StorageServiceServerTest { - public static final String keyspace = "test_keyspace"; - public static ColumnFamilyStore table1; - public static ColumnFamilyStore table2; - static final String DC1 = "DC1"; static final String DC2 = "DC2"; static final String RACK = "rack1"; @@ -136,21 +119,6 @@ private Location location(InetAddressAndPort endpoint) id5 = InetAddressAndPort.getByName("127.0.0.5"); registerNodes(); ServerTestUtils.markCMS(); - - SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), - SchemaLoader.standardCFMD(keyspace, "table1").build(), - SchemaLoader.standardCFMD(keyspace, "table2").build()); - table1 = Keyspace.open(keyspace).getColumnFamilyStore("table1"); - assert table1 != null; - table2 = Keyspace.open(keyspace).getColumnFamilyStore("table2"); - assert table2 != null; - } - - @Before - public void clearData() - { - table1.truncateBlocking(); - table2.truncateBlocking(); } private static void registerNodes() @@ -646,97 +614,4 @@ public void testAuditLogEnableLoggerTransitions() throws Exception assertTrue(AuditLogManager.instance.isEnabled()); StorageService.instance.disableAuditLog(); } - - @Test - public void testGetTablesForKeyspace() - { - List result = StorageService.instance.getTablesForKeyspace(keyspace); - - assertEquals(Arrays.asList(table1.name, table2.name), result.stream().sorted().collect(Collectors.toList())); - } - - @Test - public void testGetTablesForKeyspaceNotFound() - { - String missingKeyspace = "MISSING_KEYSPACE"; - try - { - StorageService.instance.getTablesForKeyspace(missingKeyspace); - fail("Expected an AssertionError to be thrown"); - } - catch (AssertionError e) - { - assertEquals("Unknown keyspace " + missingKeyspace, e.getMessage()); - } - } - - @Test - public void testMutateSSTableRepairedStateTableNotFound() - { - try - { - StorageService.instance.mutateSSTableRepairedState(true, false, keyspace, Arrays.asList("MISSING_TABLE")); - fail("Expected an InvalidRequestException to be thrown"); - } - catch (InvalidRequestException e) - { - // Test passed - } - } - - @Test - public void testMutateSSTableRepairedStateTablePreview() - { - SchemaLoader.insertData(keyspace, table1.name, 0, 1); - table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); - assertEquals(1, table1.getLiveSSTables().size()); - - List result = StorageService.instance.mutateSSTableRepairedState(true, true, keyspace, Arrays.asList(table1.name)); - - assertEquals(1, result.size()); - table1.getLiveSSTables().forEach(sstable -> { - assertFalse(sstable.isRepaired()); - assertTrue(result.contains(sstable.descriptor.baseFile().name())); - }); - } - - @Test - public void testMutateSSTableRepairedStateTableRepaired() - { - SchemaLoader.insertData(keyspace, table1.name, 0, 1); - table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); - SchemaLoader.insertData(keyspace, table1.name, 0, 1); - table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); - assertEquals(2, table1.getLiveSSTables().size()); - table1.getLiveSSTables().forEach(sstable -> { - assertFalse(sstable.isRepaired()); - }); - - List result = StorageService.instance.mutateSSTableRepairedState(true, false, keyspace, Arrays.asList(table1.name)); - - assertEquals(2, result.size()); - table1.getLiveSSTables().forEach(sstable -> { - assertTrue(sstable.isRepaired()); - assertTrue(result.contains(sstable.descriptor.baseFile().name())); - }); - } - - @Test - public void testMutateSSTableRepairedStateTableUnrepaired() throws Exception - { - SchemaLoader.insertData(keyspace, table1.name, 0, 1); - table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); - SchemaLoader.insertData(keyspace, table1.name, 0, 1); - table1.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); - table1.getCompactionStrategyManager().mutateRepaired(table1.getLiveSSTables(), 1, null, false); - assertEquals(2, table1.getLiveSSTables().stream().filter(SSTableReader::isRepaired).count()); - - List result = StorageService.instance.mutateSSTableRepairedState(false, false, keyspace, Arrays.asList(table1.name)); - - assertEquals(2, result.size()); - table1.getLiveSSTables().forEach(sstable -> { - assertFalse(sstable.isRepaired()); - assertTrue(result.contains(sstable.descriptor.baseFile().name())); - }); - } } From c641fd4dd71507336f800f0052e1d422bb79b047 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 15 Oct 2024 14:50:14 -0700 Subject: [PATCH 045/115] Formatting: Add a new line towards the end SystemDistributedKeyspace.java --- .../org/apache/cassandra/schema/SystemDistributedKeyspace.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java index 42ac3aaa240b..1b775fb7589c 100644 --- a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java @@ -417,4 +417,4 @@ private enum BuildStatus { UNKNOWN, STARTED, SUCCESS } -} \ No newline at end of file +} From e340bacaf6805d9804c3362f09f1e3f3508a5bdf Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 15 Oct 2024 14:58:36 -0700 Subject: [PATCH 046/115] Formatting: Add a new line towards the end IAutoRepairTokenRangeSplitter.java --- .../repair/autorepair/IAutoRepairTokenRangeSplitter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java index dd02c4d4f7b6..169600eca405 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java @@ -96,4 +96,4 @@ public String toString() '}'; } } -} \ No newline at end of file +} From 2c007e0e254c9f7b1b282ff15b0a7423ff5df198 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 15 Oct 2024 16:27:41 -0700 Subject: [PATCH 047/115] Formatting: YamlConfigurationLoader.java & CassandraStreamReceiver.java --- .../apache/cassandra/config/YamlConfigurationLoader.java | 1 + .../cassandra/db/streaming/CassandraStreamReceiver.java | 7 ++----- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java b/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java index 7cb48b9b3b23..9bf4e415592c 100644 --- a/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java +++ b/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java @@ -455,3 +455,4 @@ public static LoaderOptions getDefaultLoaderOptions() return loaderOptions; } } + diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java index 218d830687a9..77c4a2418a9e 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java @@ -23,13 +23,9 @@ import java.util.List; import java.util.Set; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; -import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; -import org.apache.cassandra.io.sstable.SSTable; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +35,7 @@ import org.apache.cassandra.db.Mutation; import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.rows.ThrottledUnfilteredIterator; @@ -47,6 +44,7 @@ import org.apache.cassandra.dht.Bounds; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.ISSTableScanner; +import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.SSTableMultiWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.streaming.IncomingStream; @@ -193,7 +191,6 @@ private boolean cdcRequiresWriteCommitLog(ColumnFamilyStore cfs) * For CDC-enabled tables and write path for CDC is enabled, we want to ensure that the mutations are * run through the CommitLog, so they can be archived by the CDC process on discard. */ - @VisibleForTesting public boolean requiresWritePath(ColumnFamilyStore cfs) { return cdcRequiresWriteCommitLog(cfs) From a8fb1aabf1e33d825552b4f69da917a821375e5e Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Thu, 17 Oct 2024 16:47:38 -0700 Subject: [PATCH 048/115] Revert .circleci/config.yaml settings --- .circleci/config.yml | 38 +++++++++++++++++++------------------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 9da3ea0f9aea..864919b8f418 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -220,7 +220,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -338,7 +338,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -625,7 +625,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -3788,7 +3788,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -4100,7 +4100,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - run: name: Log Environment Information @@ -4413,7 +4413,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -4530,7 +4530,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -4781,7 +4781,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -4885,7 +4885,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -5046,7 +5046,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -5160,7 +5160,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -5278,7 +5278,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -6444,7 +6444,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -6997,7 +6997,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -7502,7 +7502,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -7843,7 +7843,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -7914,7 +7914,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - run: name: Log Environment Information @@ -8196,7 +8196,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra @@ -8833,7 +8833,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 1 steps: - attach_workspace: at: /home/cassandra From c984d46c33e6a03f70c91662f10ebaa417daa2f9 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Sun, 27 Oct 2024 15:58:02 -0700 Subject: [PATCH 049/115] Code formatting: Remove Group from naming --- .../apache/cassandra/service/AutoRepairService.java | 8 ++++---- .../cassandra/service/AutoRepairServiceMBean.java | 4 ++-- src/java/org/apache/cassandra/tools/NodeProbe.java | 8 ++++---- .../cassandra/tools/nodetool/SetAutoRepairConfig.java | 10 +++++----- .../cassandra/service/AutoRepairServiceSetterTest.java | 4 ++-- .../tools/nodetool/SetAutoRepairConfigTest.java | 4 ++-- 6 files changed, 19 insertions(+), 19 deletions(-) diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index d2395f1157b5..87e01d10bcfb 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -162,15 +162,15 @@ public void setPrimaryTokenRangeOnly(RepairType repairType, boolean primaryToken } @Override - public void setParallelRepairPercentageInGroup(RepairType repairType, int percentageInGroup) + public void setParallelRepairPercentage(RepairType repairType, int percentage) { - config.setParallelRepairPercentage(repairType, percentageInGroup); + config.setParallelRepairPercentage(repairType, percentage); } @Override - public void setParallelRepairCountInGroup(RepairType repairType, int countInGroup) + public void setParallelRepairCount(RepairType repairType, int count) { - config.setParallelRepairCount(repairType, countInGroup); + config.setParallelRepairCount(repairType, count); } public void setMVRepairEnabled(RepairType repairType, boolean enabled) diff --git a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java index 5e62e5630043..b6f08715d3e4 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java +++ b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java @@ -56,9 +56,9 @@ public interface AutoRepairServiceMBean public void setPrimaryTokenRangeOnly(RepairType repairType, boolean primaryTokenRangeOnly); - public void setParallelRepairPercentageInGroup(RepairType repairType, int percentageInGroup); + public void setParallelRepairPercentage(RepairType repairType, int percentage); - public void setParallelRepairCountInGroup(RepairType repairType, int countInGroup); + public void setParallelRepairCount(RepairType repairType, int count); public void setMVRepairEnabled(RepairType repairType, boolean enabled); diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index e5620ac056fb..c216d0efa655 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -2484,12 +2484,12 @@ public void setAutoRepairIgnoreDCs(AutoRepairConfig.RepairType repairType, Set validateLocalGroupHosts(String paramVal) + private Set retrieveHosts(String paramVal) { Set hosts = new HashSet<>(); for (String host : Splitter.on(',').split(paramVal)) diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java index 271d28b926de..58102ed7235b 100644 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java @@ -73,8 +73,8 @@ public static Collection testCases() { forEachRepairType(400, AutoRepairService.instance::setRepairSSTableCountHigherThreshold, config::getRepairSSTableCountHigherThreshold), forEachRepairType(ImmutableSet.of("dc1", "dc2"), AutoRepairService.instance::setIgnoreDCs, config::getIgnoreDCs), forEachRepairType(true, AutoRepairService.instance::setPrimaryTokenRangeOnly, config::getRepairPrimaryTokenRangeOnly), - forEachRepairType(600, AutoRepairService.instance::setParallelRepairPercentageInGroup, config::getParallelRepairPercentage), - forEachRepairType(700, AutoRepairService.instance::setParallelRepairCountInGroup, config::getParallelRepairCount), + forEachRepairType(600, AutoRepairService.instance::setParallelRepairPercentage, config::getParallelRepairPercentage), + forEachRepairType(700, AutoRepairService.instance::setParallelRepairCount, config::getParallelRepairCount), forEachRepairType(true, AutoRepairService.instance::setMVRepairEnabled, config::getMVRepairEnabled), forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setRepairPriorityForHosts, AutoRepairUtils::getPriorityHosts), forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setForceRepairForHosts, AutoRepairServiceSetterTest::isLocalHostForceRepair) diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java index 2a69cefcace9..5245062ad815 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java @@ -259,8 +259,8 @@ public static Collection testCases() forEachRepairType("sstable_upper_threshold", "4", (type) -> verify(probe, times(1)).setRepairSSTableCountHigherThreshold(type, 4)), forEachRepairType("table_max_repair_time", "5s", (type) -> verify(probe, times(1)).setAutoRepairTableMaxRepairTime(type, "5s")), forEachRepairType("repair_primary_token_range_only", "true", (type) -> verify(probe, times(1)).setPrimaryTokenRangeOnly(type, true)), - forEachRepairType("parallel_repair_count", "6", (type) -> verify(probe, times(1)).setParallelRepairCountInGroup(type, 6)), - forEachRepairType("parallel_repair_percentage", "7", (type) -> verify(probe, times(1)).setParallelRepairPercentageInGroup(type, 7)), + forEachRepairType("parallel_repair_count", "6", (type) -> verify(probe, times(1)).setParallelRepairCount(type, 6)), + forEachRepairType("parallel_repair_percentage", "7", (type) -> verify(probe, times(1)).setParallelRepairPercentage(type, 7)), forEachRepairType("mv_repair_enabled", "true", (type) -> verify(probe, times(1)).setMVRepairEnabled(type, true)), forEachRepairType("ignore_dcs", "dc1,dc2", (type) -> verify(probe, times(1)).setAutoRepairIgnoreDCs(type, ImmutableSet.of("dc1", "dc2"))) ).flatMap(Function.identity()).collect(Collectors.toList()); From ab452a6e99b1b8df5ad413f9ec6e8ee592c83ee6 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Sun, 27 Oct 2024 20:57:39 -0700 Subject: [PATCH 050/115] Enable repair scheduler through nodetool --- .../repair/autorepair/AutoRepairConfig.java | 8 +++++++- .../cassandra/service/AutoRepairService.java | 6 ++++++ .../service/AutoRepairServiceMBean.java | 2 ++ .../org/apache/cassandra/tools/NodeProbe.java | 5 +++++ .../tools/nodetool/SetAutoRepairConfig.java | 10 ++++++++-- .../tools/nodetool/SetAutoRepairConfigTest.java | 16 ++++++++++++++++ 6 files changed, 44 insertions(+), 3 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index 30433a1ad7ff..a80a875125a9 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -39,7 +39,7 @@ public class AutoRepairConfig implements Serializable // enable/disable auto repair globally, overrides all other settings. Cannot be modified dynamically. // if it is set to false, then no repair will be scheduled, including full and incremental repairs by this framework. // if it is set to true, then this repair scheduler will consult another config available for each RepairType, and based on that config, it will schedule repairs. - public final Boolean enabled; + public volatile Boolean enabled; // the interval between successive checks for repair scheduler to check if either the ongoing repair is completed or if // none is going, then check if it's time to schedule or wait public final DurationSpec.IntSecondsBound repair_check_interval = new DurationSpec.IntSecondsBound("5m"); @@ -108,6 +108,12 @@ public DurationSpec.IntSecondsBound getAutoRepairHistoryClearDeleteHostsBufferIn return history_clear_delete_hosts_buffer_interval; } + public void startScheduler() + { + enabled = true; + AutoRepair.instance.setup(); + } + public void setAutoRepairHistoryClearDeleteHostsBufferInterval(String duration) { history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound(duration); diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index 87e01d10bcfb..f80ce1b551bf 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -120,6 +120,12 @@ public void setRepairMinInterval(RepairType repairType, String minRepairInterval config.setRepairMinInterval(repairType, minRepairInterval); } + @Override + public void startScheduler() + { + config.startScheduler(); + } + public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) { config.setAutoRepairHistoryClearDeleteHostsBufferInterval(duration); diff --git a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java index b6f08715d3e4..121c9a480303 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java +++ b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java @@ -42,6 +42,8 @@ public interface AutoRepairServiceMBean public void setRepairMinInterval(RepairType repairType, String minRepairInterval); + void startScheduler(); + public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration); public void setAutoRepairMaxRetriesCount(int retries); diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index c216d0efa655..987616ddb338 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -2459,6 +2459,11 @@ public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) autoRepairProxy.setAutoRepairHistoryClearDeleteHostsBufferDuration(duration); } + public void startScheduler() + { + autoRepairProxy.startScheduler(); + } + public void setAutoRepairMaxRetriesCount(int retries) { autoRepairProxy.setAutoRepairMaxRetriesCount(retries); diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java index cdd3ca58a8fc..2929c944442a 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java @@ -43,7 +43,7 @@ public class SetAutoRepairConfig extends NodeToolCmd @VisibleForTesting @Arguments(title = " ", usage = " ", description = "autorepair param and value.\nPossible autorepair parameters are as following: " + - "[number_of_repair_threads|number_of_subranges|min_repair_interval|sstable_upper_threshold" + + "[start_scheduler|number_of_repair_threads|number_of_subranges|min_repair_interval|sstable_upper_threshold" + "|enabled|table_max_repair_time|priority_hosts|forcerepair_hosts|ignore_dcs" + "|history_clear_delete_hosts_buffer_interval|repair_primary_token_range_only" + "|parallel_repair_count|parallel_repair_percentage|mv_repair_enabled|repair_max_retries|repair_retry_backoff|repair_session_timeout]", @@ -64,7 +64,7 @@ public void execute(NodeProbe probe) String paramType = args.get(0); String paramVal = args.get(1); - if (!probe.getAutoRepairConfig().isAutoRepairSchedulingEnabled()) + if (!probe.getAutoRepairConfig().isAutoRepairSchedulingEnabled() && !paramType.equalsIgnoreCase("start_scheduler")) { out.println("Auto-repair is not enabled"); return; @@ -73,6 +73,12 @@ public void execute(NodeProbe probe) // options that do not require --repair-type option switch (paramType) { + case "start_scheduler": + if (Boolean.parseBoolean(paramVal)) + { + probe.startScheduler(); + } + return; case "history_clear_delete_hosts_buffer_interval": probe.setAutoRepairHistoryClearDeleteHostsBufferDuration(paramVal); return; diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java index 5245062ad815..6e2d79def026 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java @@ -115,6 +115,22 @@ public void testRetryBackoffInSec() verify(probe, times(1)).setAutoRepairRetryBackoff("3s"); } + + @Test + public void testStartScheduler() + { + cmd.args = ImmutableList.of("start_scheduler", "false"); + + cmd.execute(probe); + + verify(probe, times(0)).startScheduler(); + + cmd.args = ImmutableList.of("start_scheduler", "true"); + + cmd.execute(probe); + + verify(probe, times(1)).startScheduler(); + } } @RunWith(Parameterized.class) From e1357c16d4de8bc4fd52b5ed987f6fc25bce83cb Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 29 Oct 2024 11:44:27 -0700 Subject: [PATCH 051/115] Increment system_distributed GENERATION --- .../org/apache/cassandra/schema/SystemDistributedKeyspace.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java index 1b775fb7589c..9411f94c1924 100644 --- a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java @@ -86,7 +86,7 @@ private SystemDistributedKeyspace() * * // TODO: TCM - how do we evolve these tables? */ - public static final long GENERATION = 6; + public static final long GENERATION = 7; public static final String REPAIR_HISTORY = "repair_history"; From 793683db814edd79a7a9d0c9660e4f3e730858f7 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Mon, 4 Nov 2024 10:18:47 -0800 Subject: [PATCH 052/115] Option to disable write path during streaming for MV enabled tables; similar to CASSANDRA-17666 --- conf/cassandra.yaml | 7 +++++++ src/java/org/apache/cassandra/config/Config.java | 4 ++++ .../apache/cassandra/config/DatabaseDescriptor.java | 10 ++++++++++ .../db/streaming/CassandraStreamReceiver.java | 9 +++++---- 4 files changed, 26 insertions(+), 4 deletions(-) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index ef450de7fd09..6350a53d9893 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1969,6 +1969,13 @@ report_unconfirmed_repaired_data_mismatches: false # Materialized views are considered experimental and are not recommended for production use. materialized_views_enabled: false +# Specify whether Materialized View mutations are replayed through the write path on streaming, e.g. repair. +# When enabled, Materialized View data streamed to the destination node will be written into commit log first. When setting to false, +# the streamed Materialized View data is written into SSTables just the same as normal streaming. The default is true. +# If this is set to false, streaming will be considerably faster however it's possible that, in extreme situations +# (losing > quorum # nodes in a replica set), you may have data in your SSTables that never makes it to the Materialized View. +# materialized_views_on_repair_enabled: true + # Enables SASI index creation on this node. # SASI indexes are considered experimental and are not recommended for production use. sasi_indexes_enabled: false diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 7dde1bdcc594..cbe1b64bcac9 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -601,6 +601,10 @@ public static class SSTableConfig @Replaces(oldName = "enable_materialized_views", converter = Converters.IDENTITY, deprecated = true) public boolean materialized_views_enabled = false; + // When true, materialized views data in SSTable go through commit logs during internodes streaming, e.g. repair + // When false, it behaves the same as normal streaming. + public volatile boolean materialized_views_on_repair_enabled = true; + @Replaces(oldName = "enable_transient_replication", converter = Converters.IDENTITY, deprecated = true) public boolean transient_replication_enabled = false; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 2c6645ae8eed..431158c04b02 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -4206,6 +4206,16 @@ public static void setMaterializedViewsEnabled(boolean enableMaterializedViews) conf.materialized_views_enabled = enableMaterializedViews; } + public static boolean isMaterializedViewsOnRepairEnabled() + { + return conf.materialized_views_on_repair_enabled; + } + + public static void setMaterializedViewsOnRepairEnabled(boolean val) + { + conf.materialized_views_on_repair_enabled = val; + } + public static boolean getSASIIndexesEnabled() { return conf.sasi_indexes_enabled; diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java index 77c4a2418a9e..7d657b896171 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java @@ -26,16 +26,18 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.io.sstable.SSTable; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.Mutation; import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.filter.ColumnFilter; -import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.rows.ThrottledUnfilteredIterator; @@ -44,7 +46,6 @@ import org.apache.cassandra.dht.Bounds; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.ISSTableScanner; -import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.SSTableMultiWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.streaming.IncomingStream; @@ -195,7 +196,7 @@ public boolean requiresWritePath(ColumnFamilyStore cfs) { return cdcRequiresWriteCommitLog(cfs) || cfs.streamToMemtable() - || (session.streamOperation().requiresViewBuild() && hasViews(cfs)); + || (session.streamOperation().requiresViewBuild() && hasViews(cfs) && DatabaseDescriptor.isMaterializedViewsOnRepairEnabled()); } private void sendThroughWritePath(ColumnFamilyStore cfs, Collection readers) From c888f7345d06b43ce421d8e3c37d1f976ca557cd Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Mon, 4 Nov 2024 11:00:18 -0800 Subject: [PATCH 053/115] cr from tolbertam - replace MV/CDC repair replay JVM properties to yaml configs --- .../config/CassandraRelevantProperties.java | 7 ------- .../cassandra/service/AutoRepairService.java | 7 +++---- .../cassandra/streaming/StreamOperation.java | 5 ++--- .../test/repair/AutoRepairSchedulerTest.java | 12 ++---------- .../autorepair/AutoRepairParameterizedTest.java | 6 +++--- .../repair/autorepair/AutoRepairTest.java | 15 +++------------ .../service/AutoRepairServiceBasicTest.java | 9 +++------ .../service/AutoRepairServiceSetterTest.java | 3 +-- 8 files changed, 17 insertions(+), 47 deletions(-) diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index 7598f4e7cf9f..9862f227822d 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -507,14 +507,7 @@ public enum CassandraRelevantProperties STORAGE_HOOK("cassandra.storage_hook"), STORAGE_PORT("cassandra.storage_port"), STREAMING_HISTOGRAM_ROUND_SECONDS("cassandra.streaminghistogram.roundseconds", "60"), - - /** - * If set to true, mutations streamed during anti-entropy repair will be replayed via the regular write path for associated views. - */ - STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR("cassandra.streaming.requires_view_build_during_repair", "true"), - STREAMING_SESSION_PARALLELTRANSFERS("cassandra.streaming.session.parallelTransfers"), - STREAM_HOOK("cassandra.stream_hook"), /** Platform word size sun.arch.data.model. Examples: "32", "64", "unknown"*/ SUN_ARCH_DATA_MODEL("sun.arch.data.model"), diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index f80ce1b551bf..7a8342c2a7a4 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -17,7 +17,6 @@ */ package org.apache.cassandra.service; -import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.InetAddressAndPort; @@ -65,11 +64,11 @@ public void checkCanRun(RepairType repairType) if (repairType != RepairType.incremental) return; - if (CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.getBoolean()) - throw new ConfigurationException("Cannot run incremental repair while materialized view replay is enabled."); + if (DatabaseDescriptor.isMaterializedViewsOnRepairEnabled()) + throw new ConfigurationException("Cannot run incremental repair while materialized view replay is enabled. Set materialized_views_on_repair_enabled to false."); if (DatabaseDescriptor.isCDCOnRepairEnabled()) - throw new ConfigurationException("Cannot run incremental repair while CDC replay is enabled."); + throw new ConfigurationException("Cannot run incremental repair while CDC replay is enabled. Set cdc_on_repair_enabled to false."); } @Override diff --git a/src/java/org/apache/cassandra/streaming/StreamOperation.java b/src/java/org/apache/cassandra/streaming/StreamOperation.java index 8c308830d230..ea03c224cb52 100644 --- a/src/java/org/apache/cassandra/streaming/StreamOperation.java +++ b/src/java/org/apache/cassandra/streaming/StreamOperation.java @@ -17,8 +17,6 @@ */ package org.apache.cassandra.streaming; -import org.apache.cassandra.config.CassandraRelevantProperties; - public enum StreamOperation { OTHER("Other", true, false), // Fallback to avoid null types when deserializing from string @@ -28,7 +26,8 @@ public enum StreamOperation BOOTSTRAP("Bootstrap", false, true), REBUILD("Rebuild", false, true), BULK_LOAD("Bulk Load", true, false), - REPAIR("Repair", CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.getBoolean(), false); + REPAIR("Repair", true, false); + private final String description; private final boolean requiresViewBuild; diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java index 43c29f14e0de..9b583f006507 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java @@ -26,10 +26,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Uninterruptibles; -import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.schema.SystemDistributedKeyspace; -import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -53,8 +51,6 @@ public class AutoRepairSchedulerTest extends TestBaseImpl @BeforeClass public static void init() throws IOException { - CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); - // Define the expected date format pattern String pattern = "EEE MMM dd HH:mm:ss z yyyy"; // Create SimpleDateFormat object with the given pattern @@ -85,12 +81,6 @@ public static void init() throws IOException cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck text, v1 int, v2 int, PRIMARY KEY (pk, ck)) WITH read_repair='NONE'")); } - @AfterClass - public static void afterClass() - { - System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); - } - @Test public void testScheduler() throws ParseException { @@ -101,6 +91,8 @@ public void testScheduler() throws ParseException cluster.forEach(i -> i.runOnInstance(() -> { try { + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); AutoRepairService.instance.setup(); DatabaseDescriptor.setCDCOnRepairEnabled(false); AutoRepair.instance.setup(); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 490e7a102b94..3ec202f23e19 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -31,7 +31,6 @@ import com.google.common.collect.Sets; -import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.statements.schema.TableAttributes; import org.apache.cassandra.dht.Range; @@ -133,7 +132,8 @@ public void setup() QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); - CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); MockitoAnnotations.initMocks(this); Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE).truncateBlocking(); @@ -671,7 +671,7 @@ public void testRepairSuccessAfterRetry() public void testRepairThrowsForIRWithMVReplay() { AutoRepair.instance.setup(); - CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(true); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); if (repairType == AutoRepairConfig.RepairType.incremental) { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index be2b6cc47d75..df9f105b615f 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -21,8 +21,6 @@ import java.util.HashMap; import java.util.Map; -import org.apache.cassandra.config.CassandraRelevantProperties; -import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -57,19 +55,13 @@ public static void setupClass() throws Exception public void setup() { AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); - CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.full, true); DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); AutoRepairService.setup(); } - @After - public void after() - { - System.clearProperty("cassandra.streaming.requires_view_build_during_repair"); - } - @Test public void testSetup() { @@ -109,9 +101,8 @@ public void testSafeGuardSetupCall() @Test(expected = ConfigurationException.class) public void testSetupFailsWhenIREnabledWithCDCReplay() { - CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); DatabaseDescriptor.setCDCEnabled(true); DatabaseDescriptor.setCDCOnRepairEnabled(true); @@ -123,8 +114,8 @@ public void testSetupFailsWhenIREnabledWithCDCReplay() public void testSetupFailsWhenIREnabledWithMVReplay() { DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); - CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(true); DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); AutoRepair instance = new AutoRepair(); instance.setup(); } diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java index 704d09fc7dbc..054f136dad75 100644 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java @@ -18,7 +18,6 @@ package org.apache.cassandra.service; -import org.apache.cassandra.config.CassandraRelevantProperties; import org.junit.Before; import org.junit.Test; @@ -35,8 +34,8 @@ public class AutoRepairServiceBasicTest extends CQLTester { @Before public void setUp() { - CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); DatabaseDescriptor.setMaterializedViewsEnabled(false); DatabaseDescriptor.setCDCEnabled(false); config = new AutoRepairConfig(); @@ -91,8 +90,7 @@ public void testSetAutoRepairEnabledThrowsWithSchedulerDisabled() { @Test(expected = ConfigurationException.class) public void testSetAutoRepairEnabledThrowsForIRWithMVReplay() { autoRepairService.config = new AutoRepairConfig(true); - CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(true); - + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); } @@ -100,8 +98,7 @@ public void testSetAutoRepairEnabledThrowsForIRWithMVReplay() { public void testSetAutoRepairEnabledDoesNotThrowForIRWithMVReplayDisabled() { autoRepairService.config = new AutoRepairConfig(true); DatabaseDescriptor.setMaterializedViewsEnabled(true); - CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); - + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); } diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java index 58102ed7235b..f34e1f0a7071 100644 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java @@ -19,7 +19,6 @@ package org.apache.cassandra.service; import com.google.common.collect.ImmutableSet; -import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.QueryProcessor; @@ -125,7 +124,7 @@ public void prepare() { @Test public void testSettersTest() { - CassandraRelevantProperties.STREAMING_REQUIRES_VIEW_BUILD_DURING_REPAIR.setBoolean(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); DatabaseDescriptor.setCDCOnRepairEnabled(false); setter.accept(repairType, arg); assertEquals(arg, getter.apply(repairType)); From 206637c7d8b643d596ccb56173387868d1691251 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 5 Nov 2024 15:16:51 -0800 Subject: [PATCH 054/115] code formatting for StreamOperation.java --- src/java/org/apache/cassandra/streaming/StreamOperation.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/java/org/apache/cassandra/streaming/StreamOperation.java b/src/java/org/apache/cassandra/streaming/StreamOperation.java index ea03c224cb52..98a4070d2b0c 100644 --- a/src/java/org/apache/cassandra/streaming/StreamOperation.java +++ b/src/java/org/apache/cassandra/streaming/StreamOperation.java @@ -28,7 +28,6 @@ public enum StreamOperation BULK_LOAD("Bulk Load", true, false), REPAIR("Repair", true, false); - private final String description; private final boolean requiresViewBuild; private final boolean keepSSTableLevel; From 5fa4015ec7fc2e61149154865a5025bcb24715d8 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 5 Nov 2024 17:53:33 -0800 Subject: [PATCH 055/115] ant checkstyle: toLowerCase-->toLowerCaseLocalized --- src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java | 5 +++-- src/java/org/apache/cassandra/schema/AutoRepairParams.java | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java index bdb6deacf6ad..b097dd3414c4 100644 --- a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java +++ b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java @@ -25,6 +25,7 @@ import org.apache.cassandra.repair.autorepair.AutoRepair; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * Metrics related to AutoRepair. @@ -163,7 +164,7 @@ protected static class AutoRepairMetricsFactory implements MetricNameFactory protected AutoRepairMetricsFactory(RepairType repairType) { - this.repairType = repairType.toString().toLowerCase(); + this.repairType = toLowerCaseLocalized(repairType.toString()); } @Override @@ -178,7 +179,7 @@ public CassandraMetricsRegistry.MetricName createMetricName(String metricName) StringBuilder scope = new StringBuilder(); scope.append("repairType=").append(repairType); - return new CassandraMetricsRegistry.MetricName(DefaultNameFactory.GROUP_NAME, TYPE.toLowerCase(), + return new CassandraMetricsRegistry.MetricName(DefaultNameFactory.GROUP_NAME, toLowerCaseLocalized(TYPE), metricName, scope.toString(), mbeanName.toString()); } } diff --git a/src/java/org/apache/cassandra/schema/AutoRepairParams.java b/src/java/org/apache/cassandra/schema/AutoRepairParams.java index ae97e562635f..ea3802db93d2 100644 --- a/src/java/org/apache/cassandra/schema/AutoRepairParams.java +++ b/src/java/org/apache/cassandra/schema/AutoRepairParams.java @@ -29,6 +29,7 @@ import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import static java.lang.String.format; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public final class AutoRepairParams { @@ -39,7 +40,7 @@ public enum Option @Override public String toString() { - return name().toLowerCase(); + return toLowerCaseLocalized(name()); } } @@ -67,7 +68,7 @@ public static AutoRepairParams create(AutoRepairConfig.RepairType repairType, Ma { for (Map.Entry entry : options.entrySet()) { - if (!Option.ENABLED.toString().equals(entry.getKey().toLowerCase())) + if (!Option.ENABLED.toString().equals(toLowerCaseLocalized(entry.getKey()))) { throw new ConfigurationException(format("Unknown property '%s'", entry.getKey())); } From c6bd7df126977480bd8d861a949012ed44dc9963 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 5 Nov 2024 19:11:26 -0800 Subject: [PATCH 056/115] code formatting: DatabaseDescriptor.java --- src/java/org/apache/cassandra/config/DatabaseDescriptor.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 431158c04b02..af35ba246092 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -62,6 +62,7 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.google.common.util.concurrent.RateLimiter; + import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; From f9f6971336c26e1bcae4110f5e0926b3e92b4565 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Wed, 20 Nov 2024 21:01:04 -0800 Subject: [PATCH 057/115] Rename AutoRepair --> UnifiedRepair --- .../pages/managing/operating/metrics.adoc | 14 +- .../org/apache/cassandra/config/Config.java | 4 +- .../cassandra/config/DatabaseDescriptor.java | 6 +- .../statements/schema/TableAttributes.java | 8 +- .../metrics/CassandraMetricsRegistry.java | 2 +- ...Metrics.java => UnifiedRepairMetrics.java} | 44 +- ....java => UnifiedRepairMetricsManager.java} | 10 +- .../DefaultUnifiedRepairTokenSplitter.java} | 12 +- .../IUnifiedRepairTokenRangeSplitter.java} | 10 +- .../UnifiedRepair.java} | 120 +++--- .../UnifiedRepairConfig.java} | 46 +- .../UnifiedRepairState.java} | 38 +- .../UnifiedRepairUtils.java} | 186 ++++----- .../cassandra/schema/SchemaKeyspace.java | 10 +- .../schema/SystemDistributedKeyspace.java | 44 +- .../apache/cassandra/schema/TableParams.java | 76 ++-- ...irParams.java => UnifiedRepairParams.java} | 30 +- .../cassandra/service/CassandraDaemon.java | 2 +- .../cassandra/service/StorageService.java | 14 +- ...Service.java => UnifiedRepairService.java} | 50 +-- ...an.java => UnifiedRepairServiceMBean.java} | 20 +- .../tcm/sequences/BootstrapAndJoin.java | 4 +- .../tcm/sequences/BootstrapAndReplace.java | 4 +- .../tcm/sequences/ReplaceSameAddress.java | 4 +- .../org/apache/cassandra/tools/NodeProbe.java | 94 ++--- .../org/apache/cassandra/tools/NodeTool.java | 6 +- ...onfig.java => GetUnifiedRepairConfig.java} | 22 +- ...onfig.java => SetUnifiedRepairConfig.java} | 28 +- ...irStatus.java => UnifiedRepairStatus.java} | 14 +- .../apache/cassandra/utils/FBUtilities.java | 12 +- ...t.java => UnifiedRepairSchedulerTest.java} | 34 +- test/unit/org/apache/cassandra/Util.java | 8 +- .../config/DatabaseDescriptorRefTest.java | 16 +- .../config/YamlConfigurationLoaderTest.java | 14 +- .../SSTableRepairedAtTest.java | 6 +- .../UnifiedRepairConfigTest.java} | 78 ++-- ...efaultTokenSplitterParameterizedTest.java} | 28 +- .../UnifiedRepairKeyspaceTest.java} | 10 +- .../UnifiedRepairParameterizedTest.java} | 394 +++++++++--------- .../UnifiedRepairStateFactoryTest.java} | 12 +- .../UnifiedRepairStateTest.java} | 84 ++-- .../UnifiedRepairTest.java} | 36 +- .../UnifiedRepairUtilsTest.java} | 174 ++++---- .../service/AutoRepairServiceBasicTest.java | 118 ------ .../UnifiedRepairServiceBasicTest.java | 118 ++++++ ...> UnifiedRepairServiceRepairTypeTest.java} | 26 +- ...va => UnifiedRepairServiceSetterTest.java} | 60 +-- .../cassandra/tools/JMXStandardsTest.java | 6 +- ...t.java => SetUnifiedRepairConfigTest.java} | 64 +-- ...Test.java => UnifiedRepairStatusTest.java} | 24 +- 50 files changed, 1122 insertions(+), 1122 deletions(-) rename src/java/org/apache/cassandra/metrics/{AutoRepairMetrics.java => UnifiedRepairMetrics.java} (76%) rename src/java/org/apache/cassandra/metrics/{AutoRepairMetricsManager.java => UnifiedRepairMetricsManager.java} (70%) rename src/java/org/apache/cassandra/repair/{autorepair/DefaultAutoRepairTokenSplitter.java => unifiedrepair/DefaultUnifiedRepairTokenSplitter.java} (82%) rename src/java/org/apache/cassandra/repair/{autorepair/IAutoRepairTokenRangeSplitter.java => unifiedrepair/IUnifiedRepairTokenRangeSplitter.java} (87%) rename src/java/org/apache/cassandra/repair/{autorepair/AutoRepair.java => unifiedrepair/UnifiedRepair.java} (78%) rename src/java/org/apache/cassandra/repair/{autorepair/AutoRepairConfig.java => unifiedrepair/UnifiedRepairConfig.java} (91%) rename src/java/org/apache/cassandra/repair/{autorepair/AutoRepairState.java => unifiedrepair/UnifiedRepairState.java} (88%) rename src/java/org/apache/cassandra/repair/{autorepair/AutoRepairUtils.java => unifiedrepair/UnifiedRepairUtils.java} (83%) rename src/java/org/apache/cassandra/schema/{AutoRepairParams.java => UnifiedRepairParams.java} (74%) rename src/java/org/apache/cassandra/service/{AutoRepairService.java => UnifiedRepairService.java} (72%) rename src/java/org/apache/cassandra/service/{AutoRepairServiceMBean.java => UnifiedRepairServiceMBean.java} (74%) rename src/java/org/apache/cassandra/tools/nodetool/{GetAutoRepairConfig.java => GetUnifiedRepairConfig.java} (82%) rename src/java/org/apache/cassandra/tools/nodetool/{SetAutoRepairConfig.java => SetUnifiedRepairConfig.java} (83%) rename src/java/org/apache/cassandra/tools/nodetool/{AutoRepairStatus.java => UnifiedRepairStatus.java} (82%) rename test/distributed/org/apache/cassandra/distributed/test/repair/{AutoRepairSchedulerTest.java => UnifiedRepairSchedulerTest.java} (83%) rename test/unit/org/apache/cassandra/repair/{autorepair => unifiedrepair}/SSTableRepairedAtTest.java (97%) rename test/unit/org/apache/cassandra/repair/{autorepair/AutoRepairConfigTest.java => unifiedrepair/UnifiedRepairConfigTest.java} (77%) rename test/unit/org/apache/cassandra/repair/{autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java => unifiedrepair/UnifiedRepairDefaultTokenSplitterParameterizedTest.java} (84%) rename test/unit/org/apache/cassandra/repair/{autorepair/AutoRepairKeyspaceTest.java => unifiedrepair/UnifiedRepairKeyspaceTest.java} (87%) rename test/unit/org/apache/cassandra/repair/{autorepair/AutoRepairParameterizedTest.java => unifiedrepair/UnifiedRepairParameterizedTest.java} (52%) rename test/unit/org/apache/cassandra/repair/{autorepair/AutoRepairStateFactoryTest.java => unifiedrepair/UnifiedRepairStateFactoryTest.java} (76%) rename test/unit/org/apache/cassandra/repair/{autorepair/AutoRepairStateTest.java => unifiedrepair/UnifiedRepairStateTest.java} (72%) rename test/unit/org/apache/cassandra/repair/{autorepair/AutoRepairTest.java => unifiedrepair/UnifiedRepairTest.java} (79%) rename test/unit/org/apache/cassandra/repair/{autorepair/AutoRepairUtilsTest.java => unifiedrepair/UnifiedRepairUtilsTest.java} (70%) delete mode 100644 test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java create mode 100644 test/unit/org/apache/cassandra/service/UnifiedRepairServiceBasicTest.java rename test/unit/org/apache/cassandra/service/{AutoRepairServiceRepairTypeTest.java => UnifiedRepairServiceRepairTypeTest.java} (72%) rename test/unit/org/apache/cassandra/service/{AutoRepairServiceSetterTest.java => UnifiedRepairServiceSetterTest.java} (59%) rename test/unit/org/apache/cassandra/tools/nodetool/{SetAutoRepairConfigTest.java => SetUnifiedRepairConfigTest.java} (79%) rename test/unit/org/apache/cassandra/tools/nodetool/{AutoRepairStatusTest.java => UnifiedRepairStatusTest.java} (76%) diff --git a/doc/modules/cassandra/pages/managing/operating/metrics.adoc b/doc/modules/cassandra/pages/managing/operating/metrics.adoc index 2abeaefa6d39..8a5c286f37a6 100644 --- a/doc/modules/cassandra/pages/managing/operating/metrics.adoc +++ b/doc/modules/cassandra/pages/managing/operating/metrics.adoc @@ -1078,16 +1078,16 @@ partitions processed per logged batch partitions processed per unlogged batch |=== -== Automated Repair Metrics +== Unified Repair Metrics -Metrics specifc to automated repair. +Metrics specifc to unified repair. Reported name format: *Metric Name*:: -`org.apache.cassandra.metrics.AutoRepair.` +`org.apache.cassandra.metrics.UnifiedRepair.` *JMX MBean*:: -`org.apache.cassandra.metrics:type=AutoRepair name= repairType=` +`org.apache.cassandra.metrics:type=UnifiedRepair name= repairType=` [cols=",,",options="header",] |=== @@ -1118,15 +1118,15 @@ on the node views considered on the node |TotalDisabledRepairTables |Gauge |Number of tables on which -the automated repair has been disabled on the node +the unified repair has been disabled on the node |RepairTurnMyTurn |Counter |Represents the node's turn to repair |RepairTurnMyTurnDueToPriority |Counter |Represents the node's turn to repair -due to priority set in the automated repair +due to priority set in the unified repair |RepairTurnMyTurnForceRepair |Counter |Represents the node's turn to repair -due to force repair set in the automated repair +due to force repair set in the unified repair |=== diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index cbe1b64bcac9..25dc67a01c3f 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -33,7 +33,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -957,7 +957,7 @@ public static void setClientMode(boolean clientMode) public volatile boolean password_validator_reconfiguration_enabled = true; public volatile CustomGuardrailConfig password_validator = new CustomGuardrailConfig(); - public volatile AutoRepairConfig auto_repair = new AutoRepairConfig(); + public volatile UnifiedRepairConfig unified_repair = new UnifiedRepairConfig(); /** * The variants of paxos implementation and semantics supported by Cassandra. diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index af35ba246092..a35465c59751 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -107,7 +107,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.SeedProvider; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; import org.apache.cassandra.security.AbstractCryptoProvider; import org.apache.cassandra.security.EncryptionContext; import org.apache.cassandra.security.JREProvider; @@ -5361,9 +5361,9 @@ public static boolean isPasswordValidatorReconfigurationEnabled() return conf.password_validator_reconfiguration_enabled; } - public static AutoRepairConfig getAutoRepairConfig() + public static UnifiedRepairConfig getUnifiedRepairConfig() { - return conf.auto_repair; + return conf.unified_repair; } public static double getIncrementalRepairDiskHeadroomRejectRatio() diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index f47b7e4ba722..3b173a6514e4 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -25,8 +25,8 @@ import org.apache.cassandra.cql3.statements.PropertyDefinitions; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; -import org.apache.cassandra.schema.AutoRepairParams; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.schema.UnifiedRepairParams; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.schema.CompressionParams; @@ -154,10 +154,10 @@ private TableParams build(TableParams.Builder builder) builder.readRepair(ReadRepairStrategy.fromString(getString(READ_REPAIR))); if (hasOption(Option.REPAIR_FULL)) - builder.automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, getMap(Option.REPAIR_FULL))); + builder.unifiedRepairFull(UnifiedRepairParams.fromMap(UnifiedRepairConfig.RepairType.full, getMap(Option.REPAIR_FULL))); if (hasOption(Option.REPAIR_INCREMENTAL)) - builder.automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, getMap(Option.REPAIR_INCREMENTAL))); + builder.unifiedRepairIncremental(UnifiedRepairParams.fromMap(UnifiedRepairConfig.RepairType.incremental, getMap(Option.REPAIR_INCREMENTAL))); return builder.build(); } diff --git a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java index 919d3163e317..f9d96585715d 100644 --- a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java +++ b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java @@ -150,7 +150,7 @@ public class CassandraMetricsRegistry extends MetricRegistry .add(ThreadPoolMetrics.TYPE_NAME) .add(TrieMemtableMetricsView.TYPE_NAME) .add(UnweightedCacheMetrics.TYPE_NAME) - .add(AutoRepairMetrics.TYPE_NAME) + .add(UnifiedRepairMetrics.TYPE_NAME) .build(); } diff --git a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java b/src/java/org/apache/cassandra/metrics/UnifiedRepairMetrics.java similarity index 76% rename from src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java rename to src/java/org/apache/cassandra/metrics/UnifiedRepairMetrics.java index b097dd3414c4..d4929cbfefb0 100644 --- a/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java +++ b/src/java/org/apache/cassandra/metrics/UnifiedRepairMetrics.java @@ -20,19 +20,19 @@ import com.codahale.metrics.Counter; import com.codahale.metrics.Gauge; import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils; -import org.apache.cassandra.repair.autorepair.AutoRepair; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepair; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** - * Metrics related to AutoRepair. + * Metrics related to UnifiedRepair. */ -public class AutoRepairMetrics +public class UnifiedRepairMetrics { - public static final String TYPE_NAME = "autorepair"; + public static final String TYPE_NAME = "unifiedrepair"; public Gauge repairsInProgress; public Gauge nodeRepairTimeInSec; public Gauge clusterRepairTimeInSec; @@ -47,15 +47,15 @@ public class AutoRepairMetrics public Gauge totalMVTablesConsideredForRepair; public Gauge totalDisabledRepairTables; - public AutoRepairMetrics(RepairType repairType) + public UnifiedRepairMetrics(RepairType repairType) { - AutoRepairMetricsFactory factory = new AutoRepairMetricsFactory(repairType); + UnifiedRepairMetricsFactory factory = new UnifiedRepairMetricsFactory(repairType); repairsInProgress = Metrics.register(factory.createMetricName("RepairsInProgress"), new Gauge() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).isRepairInProgress() ? 1 : 0; + return UnifiedRepair.instance.getRepairState(repairType).isRepairInProgress() ? 1 : 0; } }); @@ -63,7 +63,7 @@ public Integer getValue() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).getNodeRepairTimeInSec(); + return UnifiedRepair.instance.getRepairState(repairType).getNodeRepairTimeInSec(); } }); @@ -71,7 +71,7 @@ public Integer getValue() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).getClusterRepairTimeInSec(); + return UnifiedRepair.instance.getRepairState(repairType).getClusterRepairTimeInSec(); } }); @@ -79,7 +79,7 @@ public Integer getValue() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).getSkippedTokenRangesCount(); + return UnifiedRepair.instance.getRepairState(repairType).getSkippedTokenRangesCount(); } }); @@ -87,7 +87,7 @@ public Integer getValue() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).getSkippedTablesCount(); + return UnifiedRepair.instance.getRepairState(repairType).getSkippedTablesCount(); } }); @@ -96,7 +96,7 @@ public Integer getValue() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).getLongestUnrepairedSec(); + return UnifiedRepair.instance.getRepairState(repairType).getLongestUnrepairedSec(); } }); @@ -104,7 +104,7 @@ public Integer getValue() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).getSucceededTokenRangesCount(); + return UnifiedRepair.instance.getRepairState(repairType).getSucceededTokenRangesCount(); } }); @@ -112,7 +112,7 @@ public Integer getValue() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).getFailedTokenRangesCount(); + return UnifiedRepair.instance.getRepairState(repairType).getFailedTokenRangesCount(); } }); @@ -124,7 +124,7 @@ public Integer getValue() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).getTotalMVTablesConsideredForRepair(); + return UnifiedRepair.instance.getRepairState(repairType).getTotalMVTablesConsideredForRepair(); } }); @@ -132,12 +132,12 @@ public Integer getValue() { public Integer getValue() { - return AutoRepair.instance.getRepairState(repairType).getTotalDisabledTablesRepairCount(); + return UnifiedRepair.instance.getRepairState(repairType).getTotalDisabledTablesRepairCount(); } }); } - public void recordTurn(AutoRepairUtils.RepairTurn turn) + public void recordTurn(UnifiedRepairUtils.RepairTurn turn) { switch (turn) { @@ -156,13 +156,13 @@ public void recordTurn(AutoRepairUtils.RepairTurn turn) } @VisibleForTesting - protected static class AutoRepairMetricsFactory implements MetricNameFactory + protected static class UnifiedRepairMetricsFactory implements MetricNameFactory { - private static final String TYPE = "AutoRepair"; + private static final String TYPE = "UnifiedRepair"; @VisibleForTesting protected final String repairType; - protected AutoRepairMetricsFactory(RepairType repairType) + protected UnifiedRepairMetricsFactory(RepairType repairType) { this.repairType = toLowerCaseLocalized(repairType.toString()); } diff --git a/src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java b/src/java/org/apache/cassandra/metrics/UnifiedRepairMetricsManager.java similarity index 70% rename from src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java rename to src/java/org/apache/cassandra/metrics/UnifiedRepairMetricsManager.java index e293945c9846..e8057595944d 100644 --- a/src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java +++ b/src/java/org/apache/cassandra/metrics/UnifiedRepairMetricsManager.java @@ -18,17 +18,17 @@ package org.apache.cassandra.metrics; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -public class AutoRepairMetricsManager +public class UnifiedRepairMetricsManager { - private static final Map metrics = new ConcurrentHashMap<>(); + private static final Map metrics = new ConcurrentHashMap<>(); - public static AutoRepairMetrics getMetrics(RepairType repairType) + public static UnifiedRepairMetrics getMetrics(RepairType repairType) { - return metrics.computeIfAbsent(repairType, k -> new AutoRepairMetrics(repairType)); + return metrics.computeIfAbsent(repairType, k -> new UnifiedRepairMetrics(repairType)); } } diff --git a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java b/src/java/org/apache/cassandra/repair/unifiedrepair/DefaultUnifiedRepairTokenSplitter.java similarity index 82% rename from src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java rename to src/java/org/apache/cassandra/repair/unifiedrepair/DefaultUnifiedRepairTokenSplitter.java index 9a884f61c581..98dc09f8ad7d 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java +++ b/src/java/org/apache/cassandra/repair/unifiedrepair/DefaultUnifiedRepairTokenSplitter.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.util.ArrayList; @@ -23,20 +23,20 @@ import java.util.Collections; import java.util.List; -import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.UnifiedRepairService; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.service.StorageService; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.splitEvenly; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.splitEvenly; -public class DefaultAutoRepairTokenSplitter implements IAutoRepairTokenRangeSplitter +public class DefaultUnifiedRepairTokenSplitter implements IUnifiedRepairTokenRangeSplitter { @Override - public List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) + public List getRepairAssignments(UnifiedRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) { - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); List repairAssignments = new ArrayList<>(); Collection> tokens = StorageService.instance.getPrimaryRanges(keyspaceName); diff --git a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/unifiedrepair/IUnifiedRepairTokenRangeSplitter.java similarity index 87% rename from src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java rename to src/java/org/apache/cassandra/repair/unifiedrepair/IUnifiedRepairTokenRangeSplitter.java index 169600eca405..251e6b96ec56 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/unifiedrepair/IUnifiedRepairTokenRangeSplitter.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.util.List; @@ -24,22 +24,22 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; -public interface IAutoRepairTokenRangeSplitter +public interface IUnifiedRepairTokenRangeSplitter { /** * Split the token range you wish to repair into multiple assignments. - * The autorepair framework will repair the list of returned subrange in a sequence. + * The unifiedrepair framework will repair the list of returned subrange in a sequence. * @param repairType The type of repair being executed * @param primaryRangeOnly Whether to repair only this node's primary ranges or all of its ranges. * @param keyspaceName The keyspace being repaired * @param tableNames The tables to repair * @return repair assignments broken up by range, keyspace and tables. */ - List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames); + List getRepairAssignments(UnifiedRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames); /** - * Defines a repair assignment to be issued by the autorepair framework. + * Defines a repair assignment to be issued by the unifiedrepair framework. */ class RepairAssignment { diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepair.java similarity index 78% rename from src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java rename to src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepair.java index 1abb71d69cac..02db8951fa64 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepair.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.util.ArrayList; import java.util.EnumMap; @@ -40,7 +40,7 @@ import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.utils.Clock; -import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; +import org.apache.cassandra.repair.unifiedrepair.IUnifiedRepairTokenRangeSplitter.RepairAssignment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,35 +53,35 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Tables; -import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.UnifiedRepairService; import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn; import org.apache.cassandra.utils.concurrent.Future; import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; -public class AutoRepair +public class UnifiedRepair { - private static final Logger logger = LoggerFactory.getLogger(AutoRepair.class); + private static final Logger logger = LoggerFactory.getLogger(UnifiedRepair.class); @VisibleForTesting protected static Supplier timeFunc = Clock.Global::currentTimeMillis; - public static AutoRepair instance = new AutoRepair(); + public static UnifiedRepair instance = new UnifiedRepair(); // Sleep for 5 seconds if repair finishes quickly to flush JMX metrics; it happens only for Cassandra nodes with tiny amount of data. public static DurationSpec.IntSecondsBound SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("5s"); @VisibleForTesting - protected final Map repairExecutors; + protected final Map repairExecutors; - protected final Map repairRunnableExecutors; + protected final Map repairRunnableExecutors; @VisibleForTesting - protected final Map repairStates; + protected final Map repairStates; @VisibleForTesting protected static Consumer> shuffleFunc = java.util.Collections::shuffle; @@ -89,23 +89,23 @@ public class AutoRepair @VisibleForTesting protected static BiConsumer sleepFunc = Uninterruptibles::sleepUninterruptibly; - protected final Map tokenRangeSplitters = new EnumMap<>(AutoRepairConfig.RepairType.class); + protected final Map tokenRangeSplitters = new EnumMap<>(UnifiedRepairConfig.RepairType.class); private boolean isSetupDone = false; @VisibleForTesting - protected AutoRepair() + protected UnifiedRepair() { - AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); - repairExecutors = new EnumMap<>(AutoRepairConfig.RepairType.class); - repairRunnableExecutors = new EnumMap<>(AutoRepairConfig.RepairType.class); - repairStates = new EnumMap<>(AutoRepairConfig.RepairType.class); - for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + UnifiedRepairConfig config = DatabaseDescriptor.getUnifiedRepairConfig(); + repairExecutors = new EnumMap<>(UnifiedRepairConfig.RepairType.class); + repairRunnableExecutors = new EnumMap<>(UnifiedRepairConfig.RepairType.class); + repairStates = new EnumMap<>(UnifiedRepairConfig.RepairType.class); + for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) { - repairExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-Repair-" + repairType, Thread.NORM_PRIORITY)); - repairRunnableExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-RepairRunnable-" + repairType, Thread.NORM_PRIORITY)); - repairStates.put(repairType, AutoRepairConfig.RepairType.getAutoRepairState(repairType)); - tokenRangeSplitters.put(repairType, FBUtilities.newAutoRepairTokenRangeSplitter(config.getTokenRangeSplitter(repairType))); + repairExecutors.put(repairType, executorFactory().scheduled(false, "UnifiedRepair-Repair-" + repairType, Thread.NORM_PRIORITY)); + repairRunnableExecutors.put(repairType, executorFactory().scheduled(false, "UnifiedRepair-RepairRunnable-" + repairType, Thread.NORM_PRIORITY)); + repairStates.put(repairType, UnifiedRepairConfig.RepairType.getUnifiedRepairState(repairType)); + tokenRangeSplitters.put(repairType, FBUtilities.newUnifiedRepairTokenRangeSplitter(config.getTokenRangeSplitter(repairType))); } } @@ -119,13 +119,13 @@ public void setup() { return; } - AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); - AutoRepairUtils.setup(); + UnifiedRepairConfig config = DatabaseDescriptor.getUnifiedRepairConfig(); + UnifiedRepairUtils.setup(); - for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) { - if (config.isAutoRepairEnabled(repairType)) - AutoRepairService.instance.checkCanRun(repairType); + if (config.isUnifiedRepairEnabled(repairType)) + UnifiedRepairService.instance.checkCanRun(repairType); repairExecutors.get(repairType).scheduleWithFixedDelay( () -> repair(repairType), @@ -138,26 +138,26 @@ public void setup() } // repairAsync runs a repair session of the given type asynchronously. - public void repairAsync(AutoRepairConfig.RepairType repairType) + public void repairAsync(UnifiedRepairConfig.RepairType repairType) { - if (!AutoRepairService.instance.getAutoRepairConfig().isAutoRepairEnabled(repairType)) + if (!UnifiedRepairService.instance.getUnifiedRepairConfig().isUnifiedRepairEnabled(repairType)) { - throw new ConfigurationException("Auto-repair is disabled for repair type " + repairType); + throw new ConfigurationException("Unified-repair is disabled for repair type " + repairType); } repairExecutors.get(repairType).submit(() -> repair(repairType)); } // repair runs a repair session of the given type synchronously. - public void repair(AutoRepairConfig.RepairType repairType) + public void repair(UnifiedRepairConfig.RepairType repairType) { - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); - if (!config.isAutoRepairEnabled(repairType)) + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + if (!config.isUnifiedRepairEnabled(repairType)) { - logger.debug("Auto-repair is disabled for repair type {}", repairType); + logger.debug("Unified-repair is disabled for repair type {}", repairType); return; } - AutoRepairService.instance.checkCanRun(repairType); - AutoRepairState repairState = repairStates.get(repairType); + UnifiedRepairService.instance.checkCanRun(repairType); + UnifiedRepairState repairState = repairStates.get(repairType); try { String localDC = DatabaseDescriptor.getLocalDataCenter(); @@ -168,16 +168,16 @@ public void repair(AutoRepairConfig.RepairType repairType) } // refresh the longest unrepaired node - repairState.setLongestUnrepairedNode(AutoRepairUtils.getHostWithLongestUnrepairTime(repairType)); + repairState.setLongestUnrepairedNode(UnifiedRepairUtils.getHostWithLongestUnrepairTime(repairType)); //consistency level to use for local query UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); - RepairTurn turn = AutoRepairUtils.myTurnToRunRepair(repairType, myId); + RepairTurn turn = UnifiedRepairUtils.myTurnToRunRepair(repairType, myId); if (turn == MY_TURN || turn == MY_TURN_DUE_TO_PRIORITY || turn == MY_TURN_FORCE_REPAIR) { repairState.recordTurn(turn); - // For normal auto repair, we will use primary range only repairs (Repair with -pr option). - // For some cases, we may set the auto_repair_primary_token_range_only flag to false then we will do repair + // For normal unified repair, we will use primary range only repairs (Repair with -pr option). + // For some cases, we may set the unified_repair_primary_token_range_only flag to false then we will do repair // without -pr. We may also do force repair for certain node that we want to repair all the data on one node // When doing force repair, we want to repair without -pr. boolean primaryRangeOnly = config.getRepairPrimaryTokenRangeOnly(repairType) @@ -190,7 +190,7 @@ public void repair(AutoRepairConfig.RepairType repairType) long startTime = timeFunc.get(); logger.info("My host id: {}, my turn to run repair...repair primary-ranges only? {}", myId, config.getRepairPrimaryTokenRangeOnly(repairType)); - AutoRepairUtils.updateStartAutoRepairHistory(repairType, myId, timeFunc.get(), turn); + UnifiedRepairUtils.updateStartUnifiedRepairHistory(repairType, myId, timeFunc.get(), turn); repairState.setRepairKeyspaceCount(0); repairState.setRepairInProgress(true); @@ -201,13 +201,13 @@ public void repair(AutoRepairConfig.RepairType repairType) List keyspaces = new ArrayList<>(); Keyspace.all().forEach(keyspaces::add); - // Auto-repair is likely to be run on multiple nodes independently, we want to avoid running multiple repair + // Unified-repair is likely to be run on multiple nodes independently, we want to avoid running multiple repair // sessions on overlapping datasets at the same time. Shuffling keyspaces reduces the likelihood of this happening. shuffleFunc.accept(keyspaces); for (Keyspace keyspace : keyspaces) { - if (!AutoRepairUtils.checkNodeContainsKeyspaceReplica(keyspace)) + if (!UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(keyspace)) { continue; } @@ -237,20 +237,20 @@ public void repair(AutoRepairConfig.RepairType repairType) tableStartTime = timeFunc.get(); } previousAssignment = curRepairAssignment; - if (!config.isAutoRepairEnabled(repairType)) + if (!config.isUnifiedRepairEnabled(repairType)) { - logger.error("Auto-repair for type {} is disabled hence not running repair", repairType); + logger.error("Unified-repair for type {} is disabled hence not running repair", repairType); repairState.setRepairInProgress(false); return; } - if (AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, keyspaceStartTime, tablesToBeRepairedList.size())) + if (UnifiedRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, keyspaceStartTime, tablesToBeRepairedList.size())) { collectectedRepairStats.skippedTokenRanges += totalRepairAssignments - totalProcessedAssignments; logger.info("Keyspace took too much time to repair hence skipping it {}", keyspaceName); break; } - if (repairOneTableAtATime && AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, tableStartTime)) + if (repairOneTableAtATime && UnifiedRepairUtils.tableMaxRepairTimeExceeded(repairType, tableStartTime)) { collectectedRepairStats.skippedTokenRanges += 1; logger.info("Table took too much time to repair hence skipping it table name {}.{}, token range {}", @@ -339,17 +339,17 @@ else if (retryCount < config.getRepairMaxRetries()) } catch (Exception e) { - logger.error("Exception in autorepair:", e); + logger.error("Exception in unifiedrepair:", e); } } - private boolean tooSoonToRunRepair(AutoRepairConfig.RepairType repairType, AutoRepairState repairState, AutoRepairConfig config, UUID myId) + private boolean tooSoonToRunRepair(UnifiedRepairConfig.RepairType repairType, UnifiedRepairState repairState, UnifiedRepairConfig config, UUID myId) { if (repairState.getLastRepairTime() == 0) { // the node has either just boooted or has not run repair before, // we should check for the node's repair history in the DB - repairState.setLastRepairTime(AutoRepairUtils.getLastRepairTimeForNode(repairType, myId)); + repairState.setLastRepairTime(UnifiedRepairUtils.getLastRepairTimeForNode(repairType, myId)); } /** check if it is too soon to run repair. one of the reason we * should not run frequent repair is that repair triggers @@ -365,7 +365,7 @@ private boolean tooSoonToRunRepair(AutoRepairConfig.RepairType repairType, AutoR return false; } - private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairConfig config, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, CollectectedRepairStats collectectedRepairStats) + private List retrieveTablesToBeRepaired(Keyspace keyspace, UnifiedRepairConfig config, UnifiedRepairConfig.RepairType repairType, UnifiedRepairState repairState, CollectectedRepairStats collectectedRepairStats) { Tables tables = keyspace.getMetadata().tables; List tablesToBeRepaired = new ArrayList<>(); @@ -377,7 +377,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon String tableName = tableMetadata.name; ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName); - if (!columnFamilyStore.metadata().params.automatedRepair.get(repairType).repairEnabled()) + if (!columnFamilyStore.metadata().params.unifiedRepair.get(repairType).repairEnabled()) { logger.info("Repair is disabled for keyspace {} for tables: {}", keyspace.getName(), tableName); repairState.setTotalDisabledTablesRepairCount(repairState.getTotalDisabledTablesRepairCount() + 1); @@ -385,7 +385,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon continue; } - // this is done to make autorepair safe as running repair on table with more sstables + // this is done to make unifiedrepair safe as running repair on table with more sstables // may have its own challenges int totalSSTables = columnFamilyStore.getLiveSSTables().size(); if (totalSSTables > config.getRepairSSTableCountHigherThreshold(repairType)) @@ -399,7 +399,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon tablesToBeRepaired.add(tableName); // See if we should repair MVs as well that are associated with this given table - List mvs = AutoRepairUtils.getAllMVs(repairType, keyspace, tableMetadata); + List mvs = UnifiedRepairUtils.getAllMVs(repairType, keyspace, tableMetadata); if (!mvs.isEmpty()) { tablesToBeRepaired.addAll(mvs); @@ -409,14 +409,14 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon return tablesToBeRepaired; } - private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, UUID myId, + private void cleanupAndUpdateStats(RepairTurn turn, UnifiedRepairConfig.RepairType repairType, UnifiedRepairState repairState, UUID myId, long startTime, CollectectedRepairStats collectectedRepairStats) throws InterruptedException { //if it was due to priority then remove it now if (turn == MY_TURN_DUE_TO_PRIORITY) { logger.info("Remove current host from priority list"); - AutoRepairUtils.removePriorityStatus(repairType, myId); + UnifiedRepairUtils.removePriorityStatus(repairType, myId); } repairState.setFailedTokenRangesCount(collectectedRepairStats.failedTokenRanges); @@ -446,10 +446,10 @@ private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType Thread.sleep(SLEEP_IF_REPAIR_FINISHES_QUICKLY.toMilliseconds()); } repairState.setRepairInProgress(false); - AutoRepairUtils.updateFinishAutoRepairHistory(repairType, myId, timeFunc.get()); + UnifiedRepairUtils.updateFinishUnifiedRepairHistory(repairType, myId, timeFunc.get()); } - public AutoRepairState getRepairState(AutoRepairConfig.RepairType repairType) + public UnifiedRepairState getRepairState(UnifiedRepairConfig.RepairType repairType) { return repairStates.get(repairType); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfig.java similarity index 91% rename from src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java rename to src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfig.java index a80a875125a9..33f37b8f57ee 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfig.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.io.Serializable; import java.util.ArrayList; @@ -34,9 +34,9 @@ import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.config.ParameterizedClass; -public class AutoRepairConfig implements Serializable +public class UnifiedRepairConfig implements Serializable { - // enable/disable auto repair globally, overrides all other settings. Cannot be modified dynamically. + // enable/disable unified repair globally, overrides all other settings. Cannot be modified dynamically. // if it is set to false, then no repair will be scheduled, including full and incremental repairs by this framework. // if it is set to true, then this repair scheduler will consult another config available for each RepairType, and based on that config, it will schedule repairs. public volatile Boolean enabled; @@ -61,7 +61,7 @@ public enum RepairType implements Serializable full, incremental; - public static AutoRepairState getAutoRepairState(RepairType repairType) + public static UnifiedRepairState getUnifiedRepairState(RepairType repairType) { switch (repairType) { @@ -78,12 +78,12 @@ public static AutoRepairState getAutoRepairState(RepairType repairType) // repair_type_overrides overrides the global_settings for a specific repair type public volatile Map repair_type_overrides = new EnumMap<>(RepairType.class); - public AutoRepairConfig() + public UnifiedRepairConfig() { this(false); } - public AutoRepairConfig(boolean enabled) + public UnifiedRepairConfig(boolean enabled) { this.enabled = enabled; global_settings = Options.getDefaultOptions(); @@ -98,12 +98,12 @@ public DurationSpec.IntSecondsBound getRepairCheckInterval() return repair_check_interval; } - public boolean isAutoRepairSchedulingEnabled() + public boolean isUnifiedRepairSchedulingEnabled() { return enabled; } - public DurationSpec.IntSecondsBound getAutoRepairHistoryClearDeleteHostsBufferInterval() + public DurationSpec.IntSecondsBound getUnifiedRepairHistoryClearDeleteHostsBufferInterval() { return history_clear_delete_hosts_buffer_interval; } @@ -111,10 +111,10 @@ public DurationSpec.IntSecondsBound getAutoRepairHistoryClearDeleteHostsBufferIn public void startScheduler() { enabled = true; - AutoRepair.instance.setup(); + UnifiedRepair.instance.setup(); } - public void setAutoRepairHistoryClearDeleteHostsBufferInterval(String duration) + public void setUnifiedRepairHistoryClearDeleteHostsBufferInterval(String duration) { history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound(duration); } @@ -139,12 +139,12 @@ public void setRepairRetryBackoff(String interval) repair_retry_backoff = new DurationSpec.LongSecondsBound(interval); } - public boolean isAutoRepairEnabled(RepairType repairType) + public boolean isUnifiedRepairEnabled(RepairType repairType) { return enabled && applyOverrides(repairType, opt -> opt.enabled); } - public void setAutoRepairEnabled(RepairType repairType, boolean enabled) + public void setUnifiedRepairEnabled(RepairType repairType, boolean enabled) { ensureOverrides(repairType); repair_type_overrides.get(repairType).enabled = enabled; @@ -205,15 +205,15 @@ public void setRepairSSTableCountHigherThreshold(RepairType repairType, int ssta repair_type_overrides.get(repairType).sstable_upper_threshold = sstableHigherThreshold; } - public DurationSpec.IntSecondsBound getAutoRepairTableMaxRepairTime(RepairType repairType) + public DurationSpec.IntSecondsBound getUnifiedRepairTableMaxRepairTime(RepairType repairType) { return applyOverrides(repairType, opt -> opt.table_max_repair_time); } - public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime) + public void setUnifiedRepairTableMaxRepairTime(RepairType repairType, String unifiedRepairTableMaxRepairTime) { ensureOverrides(repairType); - repair_type_overrides.get(repairType).table_max_repair_time = new DurationSpec.IntSecondsBound(autoRepairTableMaxRepairTime); + repair_type_overrides.get(repairType).table_max_repair_time = new DurationSpec.IntSecondsBound(unifiedRepairTableMaxRepairTime); } public Set getIgnoreDCs(RepairType repairType) @@ -309,11 +309,11 @@ public void setRepairSessionTimeout(RepairType repairType, String repairSessionT repair_type_overrides.get(repairType).repair_session_timeout = new DurationSpec.IntSecondsBound(repairSessionTimeout); } - // Options configures auto-repair behavior for a given repair type. + // Options configures unified-repair behavior for a given repair type. // All fields can be modified dynamically. public static class Options implements Serializable { - // defaultOptions defines the default auto-repair behavior when no overrides are defined + // defaultOptions defines the default unified-repair behavior when no overrides are defined @VisibleForTesting protected static final Options defaultOptions = getDefaultOptions(); @@ -339,16 +339,16 @@ protected static Options getDefaultOptions() opts.force_repair_new_node = false; opts.table_max_repair_time = new DurationSpec.IntSecondsBound("6h"); opts.mv_repair_enabled = false; - opts.token_range_splitter = new ParameterizedClass(DefaultAutoRepairTokenSplitter.class.getName(), Collections.emptyMap()); + opts.token_range_splitter = new ParameterizedClass(DefaultUnifiedRepairTokenSplitter.class.getName(), Collections.emptyMap()); opts.initial_scheduler_delay = new DurationSpec.IntSecondsBound("5m"); // 5 minutes opts.repair_session_timeout = new DurationSpec.IntSecondsBound("3h"); // 3 hours return opts; } - // enable/disable auto repair for the given repair type + // enable/disable unified repair for the given repair type public volatile Boolean enabled; - // auto repair is default repair table by table, if this is enabled, the framework will repair all the tables in a keyspace in one go. + // unified repair is default repair table by table, if this is enabled, the framework will repair all the tables in a keyspace in one go. public volatile Boolean repair_by_keyspace; // the number of subranges to split each to-be-repaired token range into, // the higher this number, the smaller the repair sessions will be @@ -387,7 +387,7 @@ protected static Options getDefaultOptions() // specifies a denylist of datacenters to repair // This is useful if you want to completely avoid running repairs in one or more data centers. By default, it is empty, i.e., the framework will repair nodes in all the datacenters. public volatile Set ignore_dcs; - // Set this 'true' if AutoRepair should repair only the primary ranges owned by this node; else, 'false' + // Set this 'true' if UnifiedRepair should repair only the primary ranges owned by this node; else, 'false' // It is the same as -pr in nodetool repair options. public volatile Boolean repair_primary_token_range_only; // configures whether to force immediate repair on new nodes @@ -401,9 +401,9 @@ protected static Options getDefaultOptions() // So, the idea here is to penalize the outliers instead of good candidates. This can easily be configured with a higher value if we want to disable the functionality. public volatile DurationSpec.IntSecondsBound table_max_repair_time; // the default is 'true'. - // This flag determines whether the auto-repair framework needs to run anti-entropy, a.k.a, repair on the MV table or not. + // This flag determines whether the unified-repair framework needs to run anti-entropy, a.k.a, repair on the MV table or not. public volatile Boolean mv_repair_enabled; - // the default is DefaultAutoRepairTokenSplitter. The class should implement IAutoRepairTokenRangeSplitter. + // the default is DefaultUnifiedRepairTokenSplitter. The class should implement IUnifiedRepairTokenRangeSplitter. // The default implementation splits the tokens based on the token ranges owned by this node divided by the number of 'number_of_subranges' public volatile ParameterizedClass token_range_splitter; // the minimum delay after a node starts before the scheduler starts running repair diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairState.java similarity index 88% rename from src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java rename to src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairState.java index 84701ca78fbc..035906289592 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairState.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import com.google.common.annotations.VisibleForTesting; @@ -26,14 +26,14 @@ import org.apache.cassandra.db.view.TableViews; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.metrics.AutoRepairMetricsManager; -import org.apache.cassandra.metrics.AutoRepairMetrics; +import org.apache.cassandra.metrics.UnifiedRepairMetricsManager; +import org.apache.cassandra.metrics.UnifiedRepairMetrics; import org.apache.cassandra.repair.RepairCoordinator; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.UnifiedRepairHistory; import org.apache.cassandra.repair.RepairParallelism; import org.apache.cassandra.repair.messages.RepairOption; -import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.UnifiedRepairService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.utils.Clock; @@ -55,10 +55,10 @@ import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition; -// AutoRepairState represents the state of automated repair for a given repair type. -public abstract class AutoRepairState implements ProgressListener +// UnifiedRepairState represents the state of unified repair for a given repair type. +public abstract class UnifiedRepairState implements ProgressListener { - protected static final Logger logger = LoggerFactory.getLogger(AutoRepairState.class); + protected static final Logger logger = LoggerFactory.getLogger(UnifiedRepairState.class); private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS"); @VisibleForTesting protected static Supplier timeFunc = Clock.Global::currentTimeMillis; @@ -93,16 +93,16 @@ public abstract class AutoRepairState implements ProgressListener protected int skippedTablesCount = 0; @VisibleForTesting - protected AutoRepairHistory longestUnrepairedNode; + protected UnifiedRepairHistory longestUnrepairedNode; @VisibleForTesting protected Condition condition = newOneTimeCondition(); @VisibleForTesting protected boolean success = true; - protected final AutoRepairMetrics metrics; + protected final UnifiedRepairMetrics metrics; - protected AutoRepairState(RepairType repairType) + protected UnifiedRepairState(RepairType repairType) { - metrics = AutoRepairMetricsManager.getMetrics(repairType); + metrics = UnifiedRepairMetricsManager.getMetrics(repairType); this.repairType = repairType; } @@ -229,7 +229,7 @@ public int getRepairKeyspaceCount() return repairKeyspaceCount; } - public void setLongestUnrepairedNode(AutoRepairHistory longestUnrepairedNode) + public void setLongestUnrepairedNode(UnifiedRepairHistory longestUnrepairedNode) { this.longestUnrepairedNode = longestUnrepairedNode; } @@ -279,7 +279,7 @@ public boolean isSuccess() return success; } - public void recordTurn(AutoRepairUtils.RepairTurn turn) + public void recordTurn(UnifiedRepairUtils.RepairTurn turn) { metrics.recordTurn(turn); } @@ -300,7 +300,7 @@ public void resetWaitCondition() } } -class IncrementalRepairState extends AutoRepairState +class IncrementalRepairState extends UnifiedRepairState { public IncrementalRepairState() { @@ -311,7 +311,7 @@ public IncrementalRepairState() public RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) { RepairOption option = new RepairOption(RepairParallelism.PARALLEL, primaryRangeOnly, true, false, - AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), ranges, + UnifiedRepairService.instance.getUnifiedRepairConfig().getRepairThreads(repairType), ranges, !ranges.isEmpty(), false, false, PreviewKind.NONE, true, true, false, false); option.getColumnFamilies().addAll(filterOutUnsafeTables(keyspace, tables)); @@ -345,7 +345,7 @@ protected List filterOutUnsafeTables(String keyspaceName, List t } } -class FullRepairState extends AutoRepairState +class FullRepairState extends UnifiedRepairState { public FullRepairState() { @@ -356,7 +356,7 @@ public FullRepairState() public RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) { RepairOption option = new RepairOption(RepairParallelism.PARALLEL, primaryRangeOnly, false, false, - AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), ranges, + UnifiedRepairService.instance.getUnifiedRepairConfig().getRepairThreads(repairType), ranges, !ranges.isEmpty(), false, false, PreviewKind.NONE, true, true, false, false); option.getColumnFamilies().addAll(tables); diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtils.java similarity index 83% rename from src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java rename to src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtils.java index 8849b7847aef..2738aa6dc8de 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtils.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.util.ArrayList; import java.util.Arrays; @@ -60,7 +60,7 @@ import org.apache.cassandra.schema.ViewMetadata; import org.apache.cassandra.serializers.SetSerializer; import org.apache.cassandra.serializers.UUIDSerializer; -import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.UnifiedRepairService; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.StorageService; @@ -72,21 +72,21 @@ import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.NOT_MY_TURN; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; /** - * This class serves as a utility class for AutoRepair. It contains various helper APIs + * This class serves as a utility class for UnifiedRepair. It contains various helper APIs * to store/retrieve repair status, decide whose turn is next, etc. */ -public class AutoRepairUtils +public class UnifiedRepairUtils { - private static final Logger logger = LoggerFactory.getLogger(AutoRepairUtils.class); + private static final Logger logger = LoggerFactory.getLogger(UnifiedRepairUtils.class); static final String COL_REPAIR_TYPE = "repair_type"; static final String COL_HOST_ID = "host_id"; static final String COL_REPAIR_START_TS = "repair_start_ts"; @@ -99,56 +99,56 @@ public class AutoRepairUtils final static String SELECT_REPAIR_HISTORY = String.format( "SELECT * FROM %s.%s WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_TYPE); final static String SELECT_REPAIR_PRIORITY = String.format( "SELECT * FROM %s.%s WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, COL_REPAIR_TYPE); final static String DEL_REPAIR_PRIORITY = String.format( "DELETE %s[?] FROM %s.%s WHERE %s = ?", COL_REPAIR_PRIORITY, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, COL_REPAIR_TYPE); final static String ADD_PRIORITY_HOST = String.format( "UPDATE %s.%s SET %s = %s + ? WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_TYPE); final static String INSERT_NEW_REPAIR_HISTORY = String.format( "INSERT INTO %s.%s (%s, %s, %s, %s, %s, %s) values (?, ? ,?, ?, {}, ?) IF NOT EXISTS", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID, COL_REPAIR_START_TS, COL_REPAIR_FINISH_TS, COL_DELETE_HOSTS, COL_DELETE_HOSTS_UPDATE_TIME); final static String ADD_HOST_ID_TO_DELETE_HOSTS = String.format( "UPDATE %s.%s SET %s = %s + ?, %s = ? WHERE %s = ? AND %s = ? IF EXISTS" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_DELETE_HOSTS, COL_DELETE_HOSTS, COL_DELETE_HOSTS_UPDATE_TIME, COL_REPAIR_TYPE, COL_HOST_ID); - final static String DEL_AUTO_REPAIR_HISTORY = String.format( + final static String DEL_UNIFIED_REPAIR_HISTORY = String.format( "DELETE FROM %s.%s WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID); final static String RECORD_START_REPAIR_HISTORY = String.format( "UPDATE %s.%s SET %s= ?, repair_turn = ? WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_START_TS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_START_TS, COL_REPAIR_TYPE, COL_HOST_ID); final static String RECORD_FINISH_REPAIR_HISTORY = String.format( "UPDATE %s.%s SET %s= ?, %s=false WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_FINISH_TS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_FINISH_TS, COL_FORCE_REPAIR, COL_REPAIR_TYPE, COL_HOST_ID); final static String CLEAR_DELETE_HOSTS = String.format( "UPDATE %s.%s SET %s= {} WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_DELETE_HOSTS, COL_REPAIR_TYPE, COL_HOST_ID); final static String SET_FORCE_REPAIR = String.format( "UPDATE %s.%s SET %s=true WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_FORCE_REPAIR, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_FORCE_REPAIR, COL_REPAIR_TYPE, COL_HOST_ID); final static String SELECT_LAST_REPAIR_TIME_FOR_NODE = String.format( "SELECT %s FROM %s.%s WHERE %s = ? AND %s = ?", COL_REPAIR_FINISH_TS, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID); + SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID); static ModificationStatement delStatementRepairHistory; static SelectStatement selectStatementRepairHistory; @@ -198,14 +198,14 @@ public static void setup() .forInternalCalls()); clearDeleteHostsStatement = (ModificationStatement) QueryProcessor.getStatement(CLEAR_DELETE_HOSTS, ClientState .forInternalCalls()); - delStatementRepairHistory = (ModificationStatement) QueryProcessor.getStatement(DEL_AUTO_REPAIR_HISTORY, ClientState + delStatementRepairHistory = (ModificationStatement) QueryProcessor.getStatement(DEL_UNIFIED_REPAIR_HISTORY, ClientState .forInternalCalls()); - Keyspace autoRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); - internalQueryCL = autoRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? + Keyspace unifiedRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); + internalQueryCL = unifiedRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.ONE; } - public static class AutoRepairHistory + public static class UnifiedRepairHistory { UUID hostId; String repairTurn; @@ -215,8 +215,8 @@ public static class AutoRepairHistory long deleteHostsUpdateTime; boolean forceRepair; - public AutoRepairHistory(UUID hostId, String repairTurn, long lastRepairStartTime, long lastRepairFinishTime, - Set deleteHosts, long deleteHostsUpateTime, boolean forceRepair) + public UnifiedRepairHistory(UUID hostId, String repairTurn, long lastRepairStartTime, long lastRepairFinishTime, + Set deleteHosts, long deleteHostsUpateTime, boolean forceRepair) { this.hostId = hostId; this.repairTurn = repairTurn; @@ -259,15 +259,15 @@ public static class CurrentRepairStatus public Set hostIdsWithOnGoingRepair; // hosts that is running repair public Set hostIdsWithOnGoingForceRepair; // hosts that is running repair because of force repair Set priority; - List historiesWithoutOnGoingRepair; // hosts that is NOT running repair + List historiesWithoutOnGoingRepair; // hosts that is NOT running repair - public CurrentRepairStatus(List repairHistories, Set priority) + public CurrentRepairStatus(List repairHistories, Set priority) { hostIdsWithOnGoingRepair = new HashSet<>(); hostIdsWithOnGoingForceRepair = new HashSet<>(); historiesWithoutOnGoingRepair = new ArrayList<>(); - for (AutoRepairHistory history : repairHistories) + for (UnifiedRepairHistory history : repairHistories) { if (history.isRepairRunning()) { @@ -300,7 +300,7 @@ public String toString() } @VisibleForTesting - public static List getAutoRepairHistory(RepairType repairType) + public static List getUnifiedRepairHistory(RepairType repairType) { UntypedResultSet repairHistoryResult; @@ -308,7 +308,7 @@ public static List getAutoRepairHistory(RepairType repairType QueryOptions.forInternalCalls(internalQueryCL, Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()))), Dispatcher.RequestTime.forImmediateExecution()); repairHistoryResult = UntypedResultSet.create(repairStatusRows.result); - List repairHistories = new ArrayList<>(); + List repairHistories = new ArrayList<>(); if (repairHistoryResult.size() > 0) { for (UntypedResultSet.Row row : repairHistoryResult) @@ -322,8 +322,8 @@ public static List getAutoRepairHistory(RepairType repairType Set deleteHosts = row.getSet(COL_DELETE_HOSTS, UUIDType.instance); long deleteHostsUpdateTime = row.getLong(COL_DELETE_HOSTS_UPDATE_TIME, 0); Boolean forceRepair = row.has(COL_FORCE_REPAIR) ? row.getBoolean(COL_FORCE_REPAIR) : false; - repairHistories.add(new AutoRepairHistory(hostId, repairTurn, lastRepairStartTime, lastRepairFinishTime, - deleteHosts, deleteHostsUpdateTime, forceRepair)); + repairHistories.add(new UnifiedRepairHistory(hostId, repairTurn, lastRepairStartTime, lastRepairFinishTime, + deleteHosts, deleteHostsUpdateTime, forceRepair)); } return repairHistories; } @@ -373,8 +373,8 @@ public static void setForceRepair(RepairType repairType, UUID hostId) public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType) { - List autoRepairHistories = getAutoRepairHistory(repairType); - return getCurrentRepairStatus(repairType, autoRepairHistories); + List unifiedRepairHistories = getUnifiedRepairHistory(repairType); + return getCurrentRepairStatus(repairType, unifiedRepairHistories); } public static long getLastRepairTimeForNode(RepairType repairType, UUID hostId) @@ -396,11 +396,11 @@ public static long getLastRepairTimeForNode(RepairType repairType, UUID hostId) return repairTime.one().getLong(COL_REPAIR_FINISH_TS); } - public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType, List autoRepairHistories) + public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType, List unifiedRepairHistories) { - if (autoRepairHistories != null) + if (unifiedRepairHistories != null) { - CurrentRepairStatus status = new CurrentRepairStatus(autoRepairHistories, getPriorityHostIds(repairType)); + CurrentRepairStatus status = new CurrentRepairStatus(unifiedRepairHistories, getPriorityHostIds(repairType)); return status; } @@ -414,7 +414,7 @@ protected static TreeSet getHostIdsInCurrentRing(RepairType repairType, Se for (NodeAddresses node : allNodesInRing) { String nodeDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(node.broadcastAddress); - if (AutoRepairService.instance.getAutoRepairConfig().getIgnoreDCs(repairType).contains(nodeDC)) + if (UnifiedRepairService.instance.getUnifiedRepairConfig().getIgnoreDCs(repairType).contains(nodeDC)) { logger.info("Ignore node {} because its datacenter is {}", node, nodeDC); continue; @@ -442,39 +442,39 @@ public static TreeSet getHostIdsInCurrentRing(RepairType repairType) } // This function will return the host ID for the node which has not been repaired for longest time - public static AutoRepairHistory getHostWithLongestUnrepairTime(RepairType repairType) + public static UnifiedRepairHistory getHostWithLongestUnrepairTime(RepairType repairType) { - List autoRepairHistories = getAutoRepairHistory(repairType); - return getHostWithLongestUnrepairTime(autoRepairHistories); + List unifiedRepairHistories = getUnifiedRepairHistory(repairType); + return getHostWithLongestUnrepairTime(unifiedRepairHistories); } - private static AutoRepairHistory getHostWithLongestUnrepairTime(List autoRepairHistories) + private static UnifiedRepairHistory getHostWithLongestUnrepairTime(List unifiedRepairHistories) { - if (autoRepairHistories == null) + if (unifiedRepairHistories == null) { return null; } - AutoRepairHistory rst = null; + UnifiedRepairHistory rst = null; long oldestTimestamp = Long.MAX_VALUE; - for (AutoRepairHistory autoRepairHistory : autoRepairHistories) + for (UnifiedRepairHistory unifiedRepairHistory : unifiedRepairHistories) { - if (autoRepairHistory.lastRepairFinishTime < oldestTimestamp) + if (unifiedRepairHistory.lastRepairFinishTime < oldestTimestamp) { - rst = autoRepairHistory; - oldestTimestamp = autoRepairHistory.lastRepairFinishTime; + rst = unifiedRepairHistory; + oldestTimestamp = unifiedRepairHistory.lastRepairFinishTime; } } return rst; } - public static int getMaxNumberOfNodeRunAutoRepair(RepairType repairType, int groupSize) + public static int getMaxNumberOfNodeRunUnifiedRepair(RepairType repairType, int groupSize) { - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); if (groupSize == 0) { return Math.max(config.getParallelRepairCount(repairType), 1); } - // we will use the max number from config between auto_repair_parallel_repair_count_in_group and auto_repair_parallel_repair_percentage_in_group + // we will use the max number from config between unified_repair_parallel_repair_count_in_group and unified_repair_parallel_repair_percentage_in_group int value = Math.max(groupSize * config.getParallelRepairPercentage(repairType) / 100, config.getParallelRepairCount(repairType)); // make sure at least one node getting repaired @@ -491,33 +491,33 @@ public static RepairTurn myTurnToRunRepair(RepairType repairType, UUID myId) TreeSet hostIdsInCurrentRing = getHostIdsInCurrentRing(repairType, allNodesInRing); logger.info("Total nodes qualified for repair {}", hostIdsInCurrentRing.size()); - List autoRepairHistories = getAutoRepairHistory(repairType); - Set autoRepairHistoryIds = new HashSet<>(); + List unifiedRepairHistories = getUnifiedRepairHistory(repairType); + Set unifiedRepairHistoryIds = new HashSet<>(); // 1. Remove any node that is not part of group based on goissip info - if (autoRepairHistories != null) + if (unifiedRepairHistories != null) { - for (AutoRepairHistory nodeHistory : autoRepairHistories) + for (UnifiedRepairHistory nodeHistory : unifiedRepairHistories) { - autoRepairHistoryIds.add(nodeHistory.hostId); + unifiedRepairHistoryIds.add(nodeHistory.hostId); // clear delete_hosts if the node's delete hosts is not growing for more than two hours - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); if (nodeHistory.deleteHosts.size() > 0 - && config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds() < TimeUnit.MILLISECONDS.toSeconds( + && config.getUnifiedRepairHistoryClearDeleteHostsBufferInterval().toSeconds() < TimeUnit.MILLISECONDS.toSeconds( currentTimeMillis() - nodeHistory.deleteHostsUpdateTime )) { clearDeleteHosts(repairType, nodeHistory.hostId); logger.info("Delete hosts for {} for repair type {} has not been updated for more than {} seconds. Delete hosts has been cleared. Delete hosts before clear {}" - , nodeHistory.hostId, repairType, config.getAutoRepairHistoryClearDeleteHostsBufferInterval(), nodeHistory.deleteHosts); + , nodeHistory.hostId, repairType, config.getUnifiedRepairHistoryClearDeleteHostsBufferInterval(), nodeHistory.deleteHosts); } else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) { if (nodeHistory.deleteHosts.size() > Math.max(2, hostIdsInCurrentRing.size() * 0.5)) { // More than half of the groups thinks the record should be deleted - logger.info("{} think {} is orphan node, will delete auto repair history for repair type {}.", nodeHistory.deleteHosts, nodeHistory.hostId, repairType); - deleteAutoRepairHistory(repairType, nodeHistory.hostId); + logger.info("{} think {} is orphan node, will delete unified repair history for repair type {}.", nodeHistory.deleteHosts, nodeHistory.hostId, repairType); + deleteUnifiedRepairHistory(repairType, nodeHistory.hostId); } else { @@ -529,23 +529,23 @@ else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) } } - // 2. Add node to auto repair history table if a node is in gossip info + // 2. Add node to unified repair history table if a node is in gossip info for (UUID hostId : hostIdsInCurrentRing) { - if (!autoRepairHistoryIds.contains(hostId)) + if (!unifiedRepairHistoryIds.contains(hostId)) { - logger.info("{} for repair type {} doesn't exist in the auto repair history table, insert a new record.", repairType, hostId); + logger.info("{} for repair type {} doesn't exist in the unified repair history table, insert a new record.", repairType, hostId); insertNewRepairHistory(repairType, hostId, currentTimeMillis(), currentTimeMillis()); } } //get current repair status - CurrentRepairStatus currentRepairStatus = getCurrentRepairStatus(repairType, autoRepairHistories); + CurrentRepairStatus currentRepairStatus = getCurrentRepairStatus(repairType, unifiedRepairHistories); if (currentRepairStatus != null) { logger.info("Latest repair status {}", currentRepairStatus); //check if I am forced to run repair - for (AutoRepairHistory history : currentRepairStatus.historiesWithoutOnGoingRepair) + for (UnifiedRepairHistory history : currentRepairStatus.historiesWithoutOnGoingRepair) { if (history.forceRepair && history.hostId.equals(myId)) { @@ -554,24 +554,24 @@ else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) } } - int parallelRepairNumber = getMaxNumberOfNodeRunAutoRepair(repairType, - autoRepairHistories == null ? 0 : autoRepairHistories.size()); + int parallelRepairNumber = getMaxNumberOfNodeRunUnifiedRepair(repairType, + unifiedRepairHistories == null ? 0 : unifiedRepairHistories.size()); logger.info("Will run repairs concurrently on {} node(s)", parallelRepairNumber); if (currentRepairStatus == null || parallelRepairNumber > currentRepairStatus.hostIdsWithOnGoingRepair.size()) { // more repairs can be run, I might be the new one - if (autoRepairHistories != null) + if (unifiedRepairHistories != null) { - logger.info("Auto repair history table has {} records", autoRepairHistories.size()); + logger.info("Unified repair history table has {} records", unifiedRepairHistories.size()); } else { // try to fetch again - autoRepairHistories = getAutoRepairHistory(repairType); - currentRepairStatus = getCurrentRepairStatus(repairType, autoRepairHistories); - if (autoRepairHistories == null || currentRepairStatus == null) + unifiedRepairHistories = getUnifiedRepairHistory(repairType); + currentRepairStatus = getCurrentRepairStatus(repairType, unifiedRepairHistories); + if (unifiedRepairHistories == null || currentRepairStatus == null) { logger.error("No record found"); return NOT_MY_TURN; @@ -611,7 +611,7 @@ else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) } // get the longest unrepaired node from the nodes which are not running repair - AutoRepairHistory defaultNodeToBeRepaired = getHostWithLongestUnrepairTime(currentRepairStatus.historiesWithoutOnGoingRepair); + UnifiedRepairHistory defaultNodeToBeRepaired = getHostWithLongestUnrepairTime(currentRepairStatus.historiesWithoutOnGoingRepair); //check who is next, which is helpful for debugging logger.info("Next node to be repaired for repair type {} by default: {}", repairType, defaultNodeToBeRepaired); if (defaultNodeToBeRepaired != null && defaultNodeToBeRepaired.hostId.equals(myId)) @@ -633,7 +633,7 @@ else if (currentRepairStatus.hostIdsWithOnGoingForceRepair.contains(myId)) return NOT_MY_TURN; } - static void deleteAutoRepairHistory(RepairType repairType, UUID hostId) + static void deleteUnifiedRepairHistory(RepairType repairType, UUID hostId) { //delete the given hostId delStatementRepairHistory.execute(QueryState.forInternalCalls(), @@ -642,7 +642,7 @@ static void deleteAutoRepairHistory(RepairType repairType, UUID hostId) ByteBufferUtil.bytes(hostId))), Dispatcher.RequestTime.forImmediateExecution()); } - static void updateStartAutoRepairHistory(RepairType repairType, UUID myId, long timestamp, RepairTurn turn) + static void updateStartUnifiedRepairHistory(RepairType repairType, UUID myId, long timestamp, RepairTurn turn) { recordStartRepairHistoryStatement.execute(QueryState.forInternalCalls(), QueryOptions.forInternalCalls(internalQueryCL, @@ -653,7 +653,7 @@ static void updateStartAutoRepairHistory(RepairType repairType, UUID myId, long )), Dispatcher.RequestTime.forImmediateExecution()); } - static void updateFinishAutoRepairHistory(RepairType repairType, UUID myId, long timestamp) + static void updateFinishUnifiedRepairHistory(RepairType repairType, UUID myId, long timestamp) { recordFinishRepairHistoryStatement.execute(QueryState.forInternalCalls(), QueryOptions.forInternalCalls(internalQueryCL, @@ -662,15 +662,15 @@ static void updateFinishAutoRepairHistory(RepairType repairType, UUID myId, long ByteBufferUtil.bytes(myId) )), Dispatcher.RequestTime.forImmediateExecution()); // Do not remove beblow log, the log is used by dtest - logger.info("Auto repair finished for {}", myId); + logger.info("Unified repair finished for {}", myId); } public static void insertNewRepairHistory(RepairType repairType, UUID hostId, long startTime, long finishTime) { try { - Keyspace autoRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); - ConsistencyLevel cl = autoRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? + Keyspace unifiedRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); + ConsistencyLevel cl = unifiedRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? ConsistencyLevel.LOCAL_SERIAL : null; UntypedResultSet resultSet; @@ -687,7 +687,7 @@ public static void insertNewRepairHistory(RepairType repairType, UUID hostId, lo boolean applied = resultSet.one().getBoolean(ModificationStatement.CAS_RESULT_COLUMN.toString()); if (applied) { - logger.info("Successfully inserted a new auto repair history record for host id: {}", hostId); + logger.info("Successfully inserted a new unified repair history record for host id: {}", hostId); } else { @@ -810,21 +810,21 @@ public static boolean tableMaxRepairTimeExceeded(RepairType repairType, long sta { long tableRepairTimeSoFar = TimeUnit.MILLISECONDS.toSeconds (currentTimeMillis() - startTime); - return AutoRepairService.instance.getAutoRepairConfig().getAutoRepairTableMaxRepairTime(repairType).toSeconds() < + return UnifiedRepairService.instance.getUnifiedRepairConfig().getUnifiedRepairTableMaxRepairTime(repairType).toSeconds() < tableRepairTimeSoFar; } public static boolean keyspaceMaxRepairTimeExceeded(RepairType repairType, long startTime, int numOfTablesToBeRepaired) { long keyspaceRepairTimeSoFar = TimeUnit.MILLISECONDS.toSeconds((currentTimeMillis() - startTime)); - return (long) AutoRepairService.instance.getAutoRepairConfig().getAutoRepairTableMaxRepairTime(repairType).toSeconds() * + return (long) UnifiedRepairService.instance.getUnifiedRepairConfig().getUnifiedRepairTableMaxRepairTime(repairType).toSeconds() * numOfTablesToBeRepaired < keyspaceRepairTimeSoFar; } public static List getAllMVs(RepairType repairType, Keyspace keyspace, TableMetadata tableMetadata) { List allMvs = new ArrayList<>(); - if (AutoRepairService.instance.getAutoRepairConfig().getMVRepairEnabled(repairType) && keyspace.getMetadata().views != null) + if (UnifiedRepairService.instance.getUnifiedRepairConfig().getMVRepairEnabled(repairType) && keyspace.getMetadata().views != null) { Iterator views = keyspace.getMetadata().views.forTable(tableMetadata.id).iterator(); while (views.hasNext()) @@ -839,12 +839,12 @@ public static List getAllMVs(RepairType repairType, Keyspace keyspace, T public static void runRepairOnNewlyBootstrappedNodeIfEnabled() { - AutoRepairConfig repairConfig = DatabaseDescriptor.getAutoRepairConfig(); - if (repairConfig.isAutoRepairSchedulingEnabled()) + UnifiedRepairConfig repairConfig = DatabaseDescriptor.getUnifiedRepairConfig(); + if (repairConfig.isUnifiedRepairSchedulingEnabled()) { - for (AutoRepairConfig.RepairType rType : AutoRepairConfig.RepairType.values()) - if (repairConfig.isAutoRepairEnabled(rType) && repairConfig.getForceRepairNewNode(rType)) - AutoRepairUtils.setForceRepairNewNode(rType); + for (UnifiedRepairConfig.RepairType rType : UnifiedRepairConfig.RepairType.values()) + if (repairConfig.isUnifiedRepairEnabled(rType) && repairConfig.getForceRepairNewNode(rType)) + UnifiedRepairUtils.setForceRepairNewNode(rType); } } diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index fe9a35f74519..796e5d819e3f 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -44,7 +44,7 @@ import org.apache.cassandra.db.partitions.*; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.exceptions.InvalidRequestException; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; import org.apache.cassandra.schema.ColumnMetadata.ClusteringOrder; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; @@ -565,8 +565,8 @@ private static void addTableParamsToRowBuilder(TableParams params, Row.SimpleBui .add("compression", params.compression.asMap()) .add("read_repair", params.readRepair.toString()) .add("extensions", params.extensions) - .add("repair_full", params.automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) - .add("repair_incremental", params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); + .add("repair_full", params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full).asMap()) + .add("repair_incremental", params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).asMap()); // Only add CDC-enabled flag to schema if it's enabled on the node. This is to work around RTE's post-8099 if a 3.8+ @@ -1051,8 +1051,8 @@ static TableParams createTableParamsFromRow(UntypedResultSet.Row row) SpeculativeRetryPolicy.fromString("99PERCENTILE")) .cdc(row.has("cdc") && row.getBoolean("cdc")) .readRepair(getReadRepairStrategy(row)) - .automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, row.getFrozenTextMap("repair_full"))) - .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, row.getFrozenTextMap("repair_incremental"))); + .unifiedRepairFull(UnifiedRepairParams.fromMap(UnifiedRepairConfig.RepairType.full, row.getFrozenTextMap("repair_full"))) + .unifiedRepairIncremental(UnifiedRepairParams.fromMap(UnifiedRepairConfig.RepairType.incremental, row.getFrozenTextMap("repair_incremental"))); // allow_auto_snapshot column was introduced in 4.2 if (row.has("allow_auto_snapshot")) diff --git a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java index 9411f94c1924..0189cdac952a 100644 --- a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java @@ -96,11 +96,11 @@ private SystemDistributedKeyspace() public static final String PARTITION_DENYLIST_TABLE = "partition_denylist"; - public static final String AUTO_REPAIR_HISTORY = "auto_repair_history"; + public static final String UNIFIED_REPAIR_HISTORY = "unified_repair_history"; - public static final String AUTO_REPAIR_PRIORITY = "auto_repair_priority"; + public static final String UNIFIED_REPAIR_PRIORITY = "unified_repair_priority"; - public static final Set TABLE_NAMES = ImmutableSet.of(REPAIR_HISTORY, PARENT_REPAIR_HISTORY, VIEW_BUILD_STATUS, PARTITION_DENYLIST_TABLE, AUTO_REPAIR_HISTORY, AUTO_REPAIR_PRIORITY); + public static final Set TABLE_NAMES = ImmutableSet.of(REPAIR_HISTORY, PARENT_REPAIR_HISTORY, VIEW_BUILD_STATUS, PARTITION_DENYLIST_TABLE, UNIFIED_REPAIR_HISTORY, UNIFIED_REPAIR_PRIORITY); public static final String REPAIR_HISTORY_CQL = "CREATE TABLE IF NOT EXISTS %s (" + "keyspace_name text," @@ -163,27 +163,27 @@ private SystemDistributedKeyspace() private static final TableMetadata PartitionDenylistTable = parse(PARTITION_DENYLIST_TABLE, "Partition keys which have been denied access", PARTITION_DENYLIST_CQL).build(); - public static final String AUTO_REPAIR_HISTORY_CQL = "CREATE TABLE IF NOT EXISTS %s (" - + "host_id uuid," - + "repair_type text," - + "repair_turn text," - + "repair_start_ts timestamp," - + "repair_finish_ts timestamp," - + "delete_hosts set," - + "delete_hosts_update_time timestamp," - + "force_repair boolean," - + "PRIMARY KEY (repair_type, host_id))"; + public static final String UNIFIED_REPAIR_HISTORY_CQL = "CREATE TABLE IF NOT EXISTS %s (" + + "host_id uuid," + + "repair_type text," + + "repair_turn text," + + "repair_start_ts timestamp," + + "repair_finish_ts timestamp," + + "delete_hosts set," + + "delete_hosts_update_time timestamp," + + "force_repair boolean," + + "PRIMARY KEY (repair_type, host_id))"; - private static final TableMetadata AutoRepairHistoryTable = - parse(AUTO_REPAIR_HISTORY, "Auto repair history for each node", AUTO_REPAIR_HISTORY_CQL).build(); + private static final TableMetadata UnifiedRepairHistoryTable = + parse(UNIFIED_REPAIR_HISTORY, "Unified repair history for each node", UNIFIED_REPAIR_HISTORY_CQL).build(); - public static final String AUTO_REPAIR_PRIORITY_CQL = "CREATE TABLE IF NOT EXISTS %s (" - + "repair_type text," - + "repair_priority set," - + "PRIMARY KEY (repair_type))"; + public static final String UNIFIED_REPAIR_PRIORITY_CQL = "CREATE TABLE IF NOT EXISTS %s (" + + "repair_type text," + + "repair_priority set," + + "PRIMARY KEY (repair_type))"; - private static final TableMetadata AutoRepairPriorityTable = - parse(AUTO_REPAIR_PRIORITY, "Auto repair priority for each group", AUTO_REPAIR_PRIORITY_CQL).build(); + private static final TableMetadata UnifiedRepairPriorityTable = + parse(UNIFIED_REPAIR_PRIORITY, "Unified repair priority for each group", UNIFIED_REPAIR_PRIORITY_CQL).build(); private static TableMetadata.Builder parse(String table, String description, String cql) @@ -198,7 +198,7 @@ public static KeyspaceMetadata metadata() { return KeyspaceMetadata.create(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, KeyspaceParams.simple(Math.max(DEFAULT_RF, DatabaseDescriptor.getDefaultKeyspaceRF())), - Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus, PartitionDenylistTable, AutoRepairHistoryTable, AutoRepairPriorityTable)); + Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus, PartitionDenylistTable, UnifiedRepairHistoryTable, UnifiedRepairPriorityTable)); } public static void startParentRepair(TimeUUID parent_id, String keyspaceName, String[] cfnames, RepairOption options) diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index 9f698ecb9114..8794fca25389 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -35,7 +35,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; import org.apache.cassandra.service.reads.PercentileSpeculativeRetryPolicy; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; @@ -102,7 +102,7 @@ public String toString() public final boolean cdc; public final ReadRepairStrategy readRepair; - public final Map automatedRepair; + public final Map unifiedRepair; private TableParams(Builder builder) { @@ -127,11 +127,11 @@ private TableParams(Builder builder) extensions = builder.extensions; cdc = builder.cdc; readRepair = builder.readRepair; - automatedRepair = new EnumMap(AutoRepairConfig.RepairType.class) + unifiedRepair = new EnumMap(UnifiedRepairConfig.RepairType.class) { { - put(AutoRepairConfig.RepairType.full, builder.automatedRepairFull); - put(AutoRepairConfig.RepairType.incremental, builder.automatedRepairIncremental); + put(UnifiedRepairConfig.RepairType.full, builder.unifiedRepairFull); + put(UnifiedRepairConfig.RepairType.incremental, builder.unifiedRepairIncremental); } }; } @@ -162,8 +162,8 @@ public static Builder builder(TableParams params) .extensions(params.extensions) .cdc(params.cdc) .readRepair(params.readRepair) - .automatedRepairFull(params.automatedRepair.get(AutoRepairConfig.RepairType.full)) - .automatedRepairIncremental(params.automatedRepair.get(AutoRepairConfig.RepairType.incremental)) + .unifiedRepairFull(params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full)) + .unifiedRepairIncremental(params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental)) ; } @@ -220,7 +220,7 @@ public void validate() if (cdc && memtable.factory().writesShouldSkipCommitLog()) fail("CDC cannot work if writes skip the commit log. Check your memtable configuration."); - for (Map.Entry entry : automatedRepair.entrySet()) + for (Map.Entry entry : unifiedRepair.entrySet()) { entry.getValue().validate(); } @@ -243,25 +243,25 @@ public boolean equals(Object o) TableParams p = (TableParams) o; return comment.equals(p.comment) - && additionalWritePolicy.equals(p.additionalWritePolicy) - && allowAutoSnapshot == p.allowAutoSnapshot - && bloomFilterFpChance == p.bloomFilterFpChance - && crcCheckChance == p.crcCheckChance - && gcGraceSeconds == p.gcGraceSeconds - && incrementalBackups == p.incrementalBackups - && defaultTimeToLive == p.defaultTimeToLive - && memtableFlushPeriodInMs == p.memtableFlushPeriodInMs - && minIndexInterval == p.minIndexInterval - && maxIndexInterval == p.maxIndexInterval - && speculativeRetry.equals(p.speculativeRetry) - && caching.equals(p.caching) - && compaction.equals(p.compaction) - && compression.equals(p.compression) - && memtable.equals(p.memtable) - && extensions.equals(p.extensions) - && cdc == p.cdc - && readRepair == p.readRepair - && automatedRepair.equals(p.automatedRepair); + && additionalWritePolicy.equals(p.additionalWritePolicy) + && allowAutoSnapshot == p.allowAutoSnapshot + && bloomFilterFpChance == p.bloomFilterFpChance + && crcCheckChance == p.crcCheckChance + && gcGraceSeconds == p.gcGraceSeconds + && incrementalBackups == p.incrementalBackups + && defaultTimeToLive == p.defaultTimeToLive + && memtableFlushPeriodInMs == p.memtableFlushPeriodInMs + && minIndexInterval == p.minIndexInterval + && maxIndexInterval == p.maxIndexInterval + && speculativeRetry.equals(p.speculativeRetry) + && caching.equals(p.caching) + && compaction.equals(p.compaction) + && compression.equals(p.compression) + && memtable.equals(p.memtable) + && extensions.equals(p.extensions) + && cdc == p.cdc + && readRepair == p.readRepair + && unifiedRepair.equals(p.unifiedRepair); } @Override @@ -286,7 +286,7 @@ public int hashCode() extensions, cdc, readRepair, - automatedRepair); + unifiedRepair); } @Override @@ -312,8 +312,8 @@ public String toString() .add(EXTENSIONS.toString(), extensions) .add(CDC.toString(), cdc) .add(READ_REPAIR.toString(), readRepair) - .add(Option.REPAIR_FULL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.full)) - .add(Option.REPAIR_INCREMENTAL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.incremental)) + .add(Option.REPAIR_FULL.toString(), unifiedRepair.get(UnifiedRepairConfig.RepairType.full)) + .add(Option.REPAIR_INCREMENTAL.toString(), unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental)) .toString(); } @@ -367,9 +367,9 @@ public void appendCqlTo(CqlBuilder builder, boolean isView) .newLine() .append("AND speculative_retry = ").appendWithSingleQuotes(speculativeRetry.toString()) .newLine() - .append("AND repair_full = ").append(automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) + .append("AND repair_full = ").append(unifiedRepair.get(UnifiedRepairConfig.RepairType.full).asMap()) .newLine() - .append("AND repair_incremental = ").append(automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); + .append("AND repair_incremental = ").append(unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).asMap()); } public static final class Builder @@ -394,8 +394,8 @@ public static final class Builder private boolean cdc; private ReadRepairStrategy readRepair = ReadRepairStrategy.BLOCKING; - private AutoRepairParams automatedRepairFull = new AutoRepairParams(AutoRepairConfig.RepairType.full); - private AutoRepairParams automatedRepairIncremental = new AutoRepairParams(AutoRepairConfig.RepairType.incremental); + private UnifiedRepairParams unifiedRepairFull = new UnifiedRepairParams(UnifiedRepairConfig.RepairType.full); + private UnifiedRepairParams unifiedRepairIncremental = new UnifiedRepairParams(UnifiedRepairConfig.RepairType.incremental); public Builder() { @@ -520,15 +520,15 @@ public Builder extensions(Map val) return this; } - public Builder automatedRepairFull(AutoRepairParams val) + public Builder unifiedRepairFull(UnifiedRepairParams val) { - automatedRepairFull = val; + unifiedRepairFull = val; return this; } - public Builder automatedRepairIncremental(AutoRepairParams val) + public Builder unifiedRepairIncremental(UnifiedRepairParams val) { - automatedRepairIncremental = val; + unifiedRepairIncremental = val; return this; } } diff --git a/src/java/org/apache/cassandra/schema/AutoRepairParams.java b/src/java/org/apache/cassandra/schema/UnifiedRepairParams.java similarity index 74% rename from src/java/org/apache/cassandra/schema/AutoRepairParams.java rename to src/java/org/apache/cassandra/schema/UnifiedRepairParams.java index ea3802db93d2..27459ad30be0 100644 --- a/src/java/org/apache/cassandra/schema/AutoRepairParams.java +++ b/src/java/org/apache/cassandra/schema/UnifiedRepairParams.java @@ -26,12 +26,12 @@ import org.apache.commons.lang3.StringUtils; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; import static java.lang.String.format; import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; -public final class AutoRepairParams +public final class UnifiedRepairParams { public enum Option { @@ -44,23 +44,23 @@ public String toString() } } - public static final Map> DEFAULT_OPTIONS = - ImmutableMap.of(AutoRepairConfig.RepairType.full, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), - AutoRepairConfig.RepairType.incremental, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true))); + public static final Map> DEFAULT_OPTIONS = + ImmutableMap.of(UnifiedRepairConfig.RepairType.full, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), + UnifiedRepairConfig.RepairType.incremental, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true))); - public final AutoRepairConfig.RepairType type; + public final UnifiedRepairConfig.RepairType type; - private Map> options = DEFAULT_OPTIONS; + private Map> options = DEFAULT_OPTIONS; - AutoRepairParams(AutoRepairConfig.RepairType type) + UnifiedRepairParams(UnifiedRepairConfig.RepairType type) { this.type = type; } - public static AutoRepairParams create(AutoRepairConfig.RepairType repairType, Map options) + public static UnifiedRepairParams create(UnifiedRepairConfig.RepairType repairType, Map options) { - Map> optionsMap = new HashMap<>(); - for (Map.Entry> entry : DEFAULT_OPTIONS.entrySet()) + Map> optionsMap = new HashMap<>(); + for (Map.Entry> entry : DEFAULT_OPTIONS.entrySet()) { optionsMap.put(entry.getKey(), new HashMap<>(entry.getValue())); } @@ -75,7 +75,7 @@ public static AutoRepairParams create(AutoRepairConfig.RepairType repairType, Ma optionsMap.get(repairType).put(entry.getKey(), entry.getValue()); } } - AutoRepairParams repairParams = new AutoRepairParams(repairType); + UnifiedRepairParams repairParams = new UnifiedRepairParams(repairType); repairParams.options = optionsMap; return repairParams; } @@ -109,7 +109,7 @@ public Map options() return options.get(type); } - public static AutoRepairParams fromMap(AutoRepairConfig.RepairType repairType, Map map) + public static UnifiedRepairParams fromMap(UnifiedRepairConfig.RepairType repairType, Map map) { return create(repairType, map); } @@ -133,10 +133,10 @@ public boolean equals(Object o) if (this == o) return true; - if (!(o instanceof AutoRepairParams)) + if (!(o instanceof UnifiedRepairParams)) return false; - AutoRepairParams cp = (AutoRepairParams) o; + UnifiedRepairParams cp = (UnifiedRepairParams) o; return options.equals(cp.options); } diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index 35197c3e1866..3df8b57a9671 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -405,7 +405,7 @@ protected void setup() AuditLogManager.instance.initialize(); - StorageService.instance.doAutoRepairSetup(); + StorageService.instance.doUnifiedRepairSetup(); // schedule periodic background compaction task submission. this is simply a backstop against compactions stalling // due to scheduling errors or race conditions diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 7eb10e0542b0..3af0889a46aa 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -76,7 +76,7 @@ import com.google.common.util.concurrent.Uninterruptibles; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.repair.autorepair.AutoRepair; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepair; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1130,15 +1130,15 @@ public void doAuthSetup(boolean async) } } - public void doAutoRepairSetup() + public void doUnifiedRepairSetup() { - AutoRepairService.setup(); - if (DatabaseDescriptor.getAutoRepairConfig().isAutoRepairSchedulingEnabled()) + UnifiedRepairService.setup(); + if (DatabaseDescriptor.getUnifiedRepairConfig().isUnifiedRepairSchedulingEnabled()) { - logger.info("Enable auto-repair scheduling"); - AutoRepair.instance.setup(); + logger.info("Enable unified-repair scheduling"); + UnifiedRepair.instance.setup(); } - logger.info("AutoRepair setup complete!"); + logger.info("UnifiedRepair setup complete!"); } diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/UnifiedRepairService.java similarity index 72% rename from src/java/org/apache/cassandra/service/AutoRepairService.java rename to src/java/org/apache/cassandra/service/UnifiedRepairService.java index 7a8342c2a7a4..489dd6873011 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/UnifiedRepairService.java @@ -20,9 +20,9 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; import org.apache.cassandra.utils.MBeanWrapper; import java.util.HashSet; @@ -32,23 +32,23 @@ import com.google.common.annotations.VisibleForTesting; -public class AutoRepairService implements AutoRepairServiceMBean +public class UnifiedRepairService implements UnifiedRepairServiceMBean { - public static final String MBEAN_NAME = "org.apache.cassandra.db:type=AutoRepairService"; + public static final String MBEAN_NAME = "org.apache.cassandra.db:type=UnifiedRepairService"; @VisibleForTesting - protected AutoRepairConfig config; + protected UnifiedRepairConfig config; - public static final AutoRepairService instance = new AutoRepairService(); + public static final UnifiedRepairService instance = new UnifiedRepairService(); @VisibleForTesting - protected AutoRepairService() + protected UnifiedRepairService() { } public static void setup() { - instance.config = DatabaseDescriptor.getAutoRepairConfig(); + instance.config = DatabaseDescriptor.getUnifiedRepairConfig(); } static @@ -58,8 +58,8 @@ public static void setup() public void checkCanRun(RepairType repairType) { - if (!config.isAutoRepairSchedulingEnabled()) - throw new ConfigurationException("Auto-repair scheduller is disabled."); + if (!config.isUnifiedRepairSchedulingEnabled()) + throw new ConfigurationException("Unified-repair scheduller is disabled."); if (repairType != RepairType.incremental) return; @@ -72,16 +72,16 @@ public void checkCanRun(RepairType repairType) } @Override - public AutoRepairConfig getAutoRepairConfig() + public UnifiedRepairConfig getUnifiedRepairConfig() { return config; } @Override - public void setAutoRepairEnabled(RepairType repairType, boolean enabled) + public void setUnifiedRepairEnabled(RepairType repairType, boolean enabled) { checkCanRun(repairType); - config.setAutoRepairEnabled(repairType, enabled); + config.setUnifiedRepairEnabled(repairType, enabled); } @Override @@ -93,18 +93,18 @@ public void setRepairThreads(RepairType repairType, int repairThreads) @Override public void setRepairPriorityForHosts(RepairType repairType, Set hosts) { - AutoRepairUtils.addPriorityHosts(repairType, hosts); + UnifiedRepairUtils.addPriorityHosts(repairType, hosts); } @Override public Set getRepairHostPriority(RepairType repairType) { - return AutoRepairUtils.getPriorityHosts(repairType); + return UnifiedRepairUtils.getPriorityHosts(repairType); } @Override public void setForceRepairForHosts(RepairType repairType, Set hosts) { - AutoRepairUtils.setForceRepair(repairType, hosts); + UnifiedRepairUtils.setForceRepair(repairType, hosts); } @Override @@ -125,19 +125,19 @@ public void startScheduler() config.startScheduler(); } - public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) + public void setUnifiedRepairHistoryClearDeleteHostsBufferDuration(String duration) { - config.setAutoRepairHistoryClearDeleteHostsBufferInterval(duration); + config.setUnifiedRepairHistoryClearDeleteHostsBufferInterval(duration); } @Override - public void setAutoRepairMaxRetriesCount(int retries) + public void setUnifiedRepairMaxRetriesCount(int retries) { config.setRepairMaxRetries(retries); } @Override - public void setAutoRepairRetryBackoff(String interval) + public void setUnifiedRepairRetryBackoff(String interval) { config.setRepairRetryBackoff(interval); } @@ -149,9 +149,9 @@ public void setRepairSSTableCountHigherThreshold(RepairType repairType, int ssta } @Override - public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime) + public void setUnifiedRepairTableMaxRepairTime(RepairType repairType, String unifiedRepairTableMaxRepairTime) { - config.setAutoRepairTableMaxRepairTime(repairType, autoRepairTableMaxRepairTime); + config.setUnifiedRepairTableMaxRepairTime(repairType, unifiedRepairTableMaxRepairTime); } @Override @@ -192,12 +192,12 @@ public void setRepairSessionTimeout(RepairType repairType, String timeout) public Set getOnGoingRepairHostIds(RepairType rType) { Set hostIds = new HashSet<>(); - List histories = AutoRepairUtils.getAutoRepairHistory(rType); + List histories = UnifiedRepairUtils.getUnifiedRepairHistory(rType); if (histories == null) { return hostIds; } - AutoRepairUtils.CurrentRepairStatus currentRepairStatus = new AutoRepairUtils.CurrentRepairStatus(histories, AutoRepairUtils.getPriorityHostIds(rType)); + UnifiedRepairUtils.CurrentRepairStatus currentRepairStatus = new UnifiedRepairUtils.CurrentRepairStatus(histories, UnifiedRepairUtils.getPriorityHostIds(rType)); for (UUID id : currentRepairStatus.hostIdsWithOnGoingRepair) { hostIds.add(id.toString()); diff --git a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java b/src/java/org/apache/cassandra/service/UnifiedRepairServiceMBean.java similarity index 74% rename from src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java rename to src/java/org/apache/cassandra/service/UnifiedRepairServiceMBean.java index 121c9a480303..ec5a71bdcddd 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java +++ b/src/java/org/apache/cassandra/service/UnifiedRepairServiceMBean.java @@ -18,17 +18,17 @@ package org.apache.cassandra.service; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; import java.util.Set; -public interface AutoRepairServiceMBean +public interface UnifiedRepairServiceMBean { /** - * Enable or disable auto-repair for a given repair type + * Enable or disable unified-repair for a given repair type */ - public void setAutoRepairEnabled(RepairType repairType, boolean enabled); + public void setUnifiedRepairEnabled(RepairType repairType, boolean enabled); public void setRepairThreads(RepairType repairType, int repairThreads); @@ -44,15 +44,15 @@ public interface AutoRepairServiceMBean void startScheduler(); - public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration); + public void setUnifiedRepairHistoryClearDeleteHostsBufferDuration(String duration); - public void setAutoRepairMaxRetriesCount(int retries); + public void setUnifiedRepairMaxRetriesCount(int retries); - public void setAutoRepairRetryBackoff(String interval); + public void setUnifiedRepairRetryBackoff(String interval); public void setRepairSSTableCountHigherThreshold(RepairType repairType, int ssTableHigherThreshold); - public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime); + public void setUnifiedRepairTableMaxRepairTime(RepairType repairType, String unifiedRepairTableMaxRepairTime); public void setIgnoreDCs(RepairType repairType, Set ignorDCs); @@ -64,7 +64,7 @@ public interface AutoRepairServiceMBean public void setMVRepairEnabled(RepairType repairType, boolean enabled); - public AutoRepairConfig getAutoRepairConfig(); + public UnifiedRepairConfig getUnifiedRepairConfig(); public void setRepairSessionTimeout(RepairType repairType, String timeout); diff --git a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java index d3ffb9f92cf0..9119892af483 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java +++ b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java @@ -39,7 +39,7 @@ import org.apache.cassandra.locator.EndpointsByReplica; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.StreamState; @@ -219,7 +219,7 @@ public SequenceState executeNext() return halted(); } // this node might have just bootstrapped; check if we should run repair immediately - AutoRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); + UnifiedRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED); } else diff --git a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java index 93251b354f2f..a827bfcf7c12 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java +++ b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java @@ -43,7 +43,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.EndpointsByReplica; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; @@ -218,7 +218,7 @@ public SequenceState executeNext() return halted(); } // this node might have just bootstrapped; check if we should run repair immediately - AutoRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); + UnifiedRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED); } else diff --git a/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java b/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java index aad049ae7fa2..19d57af35d24 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java +++ b/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java @@ -29,7 +29,7 @@ import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.locator.EndpointsByReplica; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; @@ -87,7 +87,7 @@ public static void streamData(NodeId nodeId, ClusterMetadata metadata, boolean s throw new IllegalStateException("Could not finish join for during replacement"); } // this node might have just bootstrapped; check if we should run repair immediately - AutoRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); + UnifiedRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); } if (finishJoiningRing) diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 987616ddb338..30aeaf63aa6e 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -101,10 +101,10 @@ import org.apache.cassandra.metrics.ThreadPoolMetrics; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.MessagingServiceMBean; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; import org.apache.cassandra.service.ActiveRepairServiceMBean; -import org.apache.cassandra.service.AutoRepairService; -import org.apache.cassandra.service.AutoRepairServiceMBean; +import org.apache.cassandra.service.UnifiedRepairService; +import org.apache.cassandra.service.UnifiedRepairServiceMBean; import org.apache.cassandra.service.CacheService; import org.apache.cassandra.service.CacheServiceMBean; import org.apache.cassandra.tcm.CMSOperationsMBean; @@ -176,7 +176,7 @@ public class NodeProbe implements AutoCloseable protected CIDRGroupsMappingManagerMBean cmbProxy; protected PermissionsCacheMBean pcProxy; protected RolesCacheMBean rcProxy; - protected AutoRepairServiceMBean autoRepairProxy; + protected UnifiedRepairServiceMBean unifiedRepairProxy; protected Output output; private boolean failed; @@ -319,8 +319,8 @@ protected void connect() throws IOException name = new ObjectName(CIDRFilteringMetricsTable.MBEAN_NAME); cfmProxy = JMX.newMBeanProxy(mbeanServerConn, name, CIDRFilteringMetricsTableMBean.class); - name = new ObjectName(AutoRepairService.MBEAN_NAME); - autoRepairProxy = JMX.newMBeanProxy(mbeanServerConn, name, AutoRepairServiceMBean.class); + name = new ObjectName(UnifiedRepairService.MBEAN_NAME); + unifiedRepairProxy = JMX.newMBeanProxy(mbeanServerConn, name, UnifiedRepairServiceMBean.class); } catch (MalformedObjectNameException e) { @@ -2416,95 +2416,95 @@ public void abortBootstrap(String nodeId, String endpoint) ssProxy.abortBootstrap(nodeId, endpoint); } - public AutoRepairConfig getAutoRepairConfig() { - return autoRepairProxy.getAutoRepairConfig(); + public UnifiedRepairConfig getUnifiedRepairConfig() { + return unifiedRepairProxy.getUnifiedRepairConfig(); } - public void setAutoRepairEnabled(AutoRepairConfig.RepairType repairType, boolean enabled) + public void setUnifiedRepairEnabled(UnifiedRepairConfig.RepairType repairType, boolean enabled) { - autoRepairProxy.setAutoRepairEnabled(repairType, enabled); + unifiedRepairProxy.setUnifiedRepairEnabled(repairType, enabled); } - public void setRepairThreads(AutoRepairConfig.RepairType repairType, int repairThreads) + public void setRepairThreads(UnifiedRepairConfig.RepairType repairType, int repairThreads) { - autoRepairProxy.setRepairThreads(repairType, repairThreads); + unifiedRepairProxy.setRepairThreads(repairType, repairThreads); } - public void setRepairPriorityForHosts(AutoRepairConfig.RepairType repairType, Set hosts) + public void setRepairPriorityForHosts(UnifiedRepairConfig.RepairType repairType, Set hosts) { - autoRepairProxy.setRepairPriorityForHosts(repairType, hosts); + unifiedRepairProxy.setRepairPriorityForHosts(repairType, hosts); } - public Set getRepairPriorityForHosts(AutoRepairConfig.RepairType repairType) + public Set getRepairPriorityForHosts(UnifiedRepairConfig.RepairType repairType) { - return autoRepairProxy.getRepairHostPriority(repairType); + return unifiedRepairProxy.getRepairHostPriority(repairType); } - public void setForceRepairForHosts(AutoRepairConfig.RepairType repairType, Set hosts){ - autoRepairProxy.setForceRepairForHosts(repairType, hosts); + public void setForceRepairForHosts(UnifiedRepairConfig.RepairType repairType, Set hosts){ + unifiedRepairProxy.setForceRepairForHosts(repairType, hosts); } - public void setRepairSubRangeNum(AutoRepairConfig.RepairType repairType, int repairSubRanges) + public void setRepairSubRangeNum(UnifiedRepairConfig.RepairType repairType, int repairSubRanges) { - autoRepairProxy.setRepairSubRangeNum(repairType, repairSubRanges); + unifiedRepairProxy.setRepairSubRangeNum(repairType, repairSubRanges); } - public void setRepairMinInterval(AutoRepairConfig.RepairType repairType, String minRepairInterval) + public void setRepairMinInterval(UnifiedRepairConfig.RepairType repairType, String minRepairInterval) { - autoRepairProxy.setRepairMinInterval(repairType, minRepairInterval); + unifiedRepairProxy.setRepairMinInterval(repairType, minRepairInterval); } - public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) + public void setUnifiedRepairHistoryClearDeleteHostsBufferDuration(String duration) { - autoRepairProxy.setAutoRepairHistoryClearDeleteHostsBufferDuration(duration); + unifiedRepairProxy.setUnifiedRepairHistoryClearDeleteHostsBufferDuration(duration); } public void startScheduler() { - autoRepairProxy.startScheduler(); + unifiedRepairProxy.startScheduler(); } - public void setAutoRepairMaxRetriesCount(int retries) + public void setUnifiedRepairMaxRetriesCount(int retries) { - autoRepairProxy.setAutoRepairMaxRetriesCount(retries); + unifiedRepairProxy.setUnifiedRepairMaxRetriesCount(retries); } - public void setAutoRepairRetryBackoff(String interval) + public void setUnifiedRepairRetryBackoff(String interval) { - autoRepairProxy.setAutoRepairRetryBackoff(interval); + unifiedRepairProxy.setUnifiedRepairRetryBackoff(interval); } - public void setRepairSSTableCountHigherThreshold(AutoRepairConfig.RepairType repairType, int ssTableHigherThreshold) + public void setRepairSSTableCountHigherThreshold(UnifiedRepairConfig.RepairType repairType, int ssTableHigherThreshold) { - autoRepairProxy.setRepairSSTableCountHigherThreshold(repairType, ssTableHigherThreshold); + unifiedRepairProxy.setRepairSSTableCountHigherThreshold(repairType, ssTableHigherThreshold); } - public void setAutoRepairTableMaxRepairTime(AutoRepairConfig.RepairType repairType, String autoRepairTableMaxRepairTime) + public void setUnifiedRepairTableMaxRepairTime(UnifiedRepairConfig.RepairType repairType, String unifiedRepairTableMaxRepairTime) { - autoRepairProxy.setAutoRepairTableMaxRepairTime(repairType, autoRepairTableMaxRepairTime); + unifiedRepairProxy.setUnifiedRepairTableMaxRepairTime(repairType, unifiedRepairTableMaxRepairTime); } - public void setAutoRepairIgnoreDCs(AutoRepairConfig.RepairType repairType, Set ignoreDCs) + public void setUnifiedRepairIgnoreDCs(UnifiedRepairConfig.RepairType repairType, Set ignoreDCs) { - autoRepairProxy.setIgnoreDCs(repairType, ignoreDCs); + unifiedRepairProxy.setIgnoreDCs(repairType, ignoreDCs); } - public void setParallelRepairPercentage(AutoRepairConfig.RepairType repairType, int percentage) { - autoRepairProxy.setParallelRepairPercentage(repairType, percentage); + public void setParallelRepairPercentage(UnifiedRepairConfig.RepairType repairType, int percentage) { + unifiedRepairProxy.setParallelRepairPercentage(repairType, percentage); } - public void setParallelRepairCount(AutoRepairConfig.RepairType repairType, int count) { - autoRepairProxy.setParallelRepairCount(repairType, count); + public void setParallelRepairCount(UnifiedRepairConfig.RepairType repairType, int count) { + unifiedRepairProxy.setParallelRepairCount(repairType, count); } - public void setPrimaryTokenRangeOnly(AutoRepairConfig.RepairType repairType, boolean primaryTokenRangeOnly) + public void setPrimaryTokenRangeOnly(UnifiedRepairConfig.RepairType repairType, boolean primaryTokenRangeOnly) { - autoRepairProxy.setPrimaryTokenRangeOnly(repairType, primaryTokenRangeOnly); + unifiedRepairProxy.setPrimaryTokenRangeOnly(repairType, primaryTokenRangeOnly); } - public void setMVRepairEnabled(AutoRepairConfig.RepairType repairType, boolean enabled) + public void setMVRepairEnabled(UnifiedRepairConfig.RepairType repairType, boolean enabled) { - autoRepairProxy.setMVRepairEnabled(repairType, enabled); + unifiedRepairProxy.setMVRepairEnabled(repairType, enabled); } public List mutateSSTableRepairedState(boolean repair, boolean preview, String keyspace, List tables) throws InvalidRequestException @@ -2516,14 +2516,14 @@ public List getTablesForKeyspace(String keyspace) { return ssProxy.getTablesForKeyspace(keyspace); } - public void setRepairSessionTimeout(AutoRepairConfig.RepairType repairType, String timeout) + public void setRepairSessionTimeout(UnifiedRepairConfig.RepairType repairType, String timeout) { - autoRepairProxy.setRepairSessionTimeout(repairType, timeout); + unifiedRepairProxy.setRepairSessionTimeout(repairType, timeout); } - public Set getOnGoingRepairHostIds(AutoRepairConfig.RepairType type) + public Set getOnGoingRepairHostIds(UnifiedRepairConfig.RepairType type) { - return autoRepairProxy.getOnGoingRepairHostIds(type); + return unifiedRepairProxy.getOnGoingRepairHostIds(type); } } diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index 80f8ed8f09e1..9c002f877e06 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -95,7 +95,6 @@ public int execute(String... args) { List> commands = newArrayList( AbortBootstrap.class, - AutoRepairStatus.class, Assassinate.class, CassHelp.class, CIDRFilteringStats.class, @@ -135,7 +134,6 @@ public int execute(String... args) GcStats.class, GetAuditLog.class, GetAuthCacheConfig.class, - GetAutoRepairConfig.class, GetBatchlogReplayTrottle.class, GetCIDRGroupsOfIP.class, GetColumnIndexSize.class, @@ -156,6 +154,7 @@ public int execute(String... args) GetStreamThroughput.class, GetTimeout.class, GetTraceProbability.class, + GetUnifiedRepairConfig.class, GossipInfo.class, Import.class, Info.class, @@ -198,7 +197,6 @@ public int execute(String... args) Ring.class, Scrub.class, SetAuthCacheConfig.class, - SetAutoRepairConfig.class, SetBatchlogReplayThrottle.class, SetCacheCapacity.class, SetCacheKeysToSave.class, @@ -217,6 +215,7 @@ public int execute(String... args) SetStreamThroughput.class, SetTimeout.class, SetTraceProbability.class, + SetUnifiedRepairConfig.class, Sjk.class, Snapshot.class, SSTableRepairedSet.class, @@ -233,6 +232,7 @@ public int execute(String... args) TopPartitions.class, TpStats.class, TruncateHints.class, + UnifiedRepairStatus.class, UpdateCIDRGroup.class, UpgradeSSTable.class, Verify.class, diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/GetUnifiedRepairConfig.java similarity index 82% rename from src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java rename to src/java/org/apache/cassandra/tools/nodetool/GetUnifiedRepairConfig.java index 48f8d54de3f6..336063eb2405 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/GetUnifiedRepairConfig.java @@ -20,15 +20,15 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import io.airlift.airline.Command; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; import java.io.PrintStream; -@Command(name = "getautorepairconfig", description = "Print autorepair configurations") -public class GetAutoRepairConfig extends NodeToolCmd +@Command(name = "getunifiedrepairconfig", description = "Print unifiedrepair configurations") +public class GetUnifiedRepairConfig extends NodeToolCmd { @VisibleForTesting protected static PrintStream out = System.out; @@ -36,17 +36,17 @@ public class GetAutoRepairConfig extends NodeToolCmd @Override public void execute(NodeProbe probe) { - AutoRepairConfig config = probe.getAutoRepairConfig(); - if (config == null || !config.isAutoRepairSchedulingEnabled()) + UnifiedRepairConfig config = probe.getUnifiedRepairConfig(); + if (config == null || !config.isUnifiedRepairSchedulingEnabled()) { - out.println("Auto-repair is not enabled"); + out.println("Unified-repair is not enabled"); return; } StringBuilder sb = new StringBuilder(); sb.append("repair scheduler configuration:"); sb.append("\n\trepair eligibility check interval: " + config.getRepairCheckInterval()); - sb.append("\n\tTTL for repair history for dead nodes: " + config.getAutoRepairHistoryClearDeleteHostsBufferInterval()); + sb.append("\n\tTTL for repair history for dead nodes: " + config.getUnifiedRepairHistoryClearDeleteHostsBufferInterval()); sb.append("\n\tmax retries for repair: " + config.getRepairMaxRetries()); sb.append("\n\tretry backoff: " + config.getRepairRetryBackoff()); for (RepairType repairType : RepairType.values()) @@ -57,17 +57,17 @@ public void execute(NodeProbe probe) out.println(sb); } - private String formatRepairTypeConfig(NodeProbe probe, RepairType repairType, AutoRepairConfig config) + private String formatRepairTypeConfig(NodeProbe probe, RepairType repairType, UnifiedRepairConfig config) { StringBuilder sb = new StringBuilder(); sb.append("\nconfiguration for repair type: " + repairType); - sb.append("\n\tenabled: " + config.isAutoRepairEnabled(repairType)); + sb.append("\n\tenabled: " + config.isUnifiedRepairEnabled(repairType)); sb.append("\n\tminimum repair interval: " + config.getRepairMinInterval(repairType)); sb.append("\n\trepair threads: " + config.getRepairThreads(repairType)); sb.append("\n\tnumber of repair subranges: " + config.getRepairSubRangeNum(repairType)); sb.append("\n\tpriority hosts: " + Joiner.on(',').skipNulls().join(probe.getRepairPriorityForHosts(repairType))); sb.append("\n\tsstable count higher threshold: " + config.getRepairSSTableCountHigherThreshold(repairType)); - sb.append("\n\ttable max repair time in sec: " + config.getAutoRepairTableMaxRepairTime(repairType)); + sb.append("\n\ttable max repair time in sec: " + config.getUnifiedRepairTableMaxRepairTime(repairType)); sb.append("\n\tignore datacenters: " + Joiner.on(',').skipNulls().join(config.getIgnoreDCs(repairType))); sb.append("\n\trepair primary token-range: " + config.getRepairPrimaryTokenRangeOnly(repairType)); sb.append("\n\tnumber of parallel repairs within group: " + config.getParallelRepairCount(repairType)); diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/SetUnifiedRepairConfig.java similarity index 83% rename from src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java rename to src/java/org/apache/cassandra/tools/nodetool/SetUnifiedRepairConfig.java index 2929c944442a..fe123c3de1ac 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/SetUnifiedRepairConfig.java @@ -24,7 +24,7 @@ import io.airlift.airline.Command; import io.airlift.airline.Option; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; @@ -37,12 +37,12 @@ import static com.google.common.base.Preconditions.checkArgument; -@Command(name = "setautorepairconfig", description = "sets the autorepair configuration") -public class SetAutoRepairConfig extends NodeToolCmd +@Command(name = "setunifiedrepairconfig", description = "sets the unifiedrepair configuration") +public class SetUnifiedRepairConfig extends NodeToolCmd { @VisibleForTesting - @Arguments(title = " ", usage = " ", - description = "autorepair param and value.\nPossible autorepair parameters are as following: " + + @Arguments(title = " ", usage = " ", + description = "unifiedrepair param and value.\nPossible unifiedrepair parameters are as following: " + "[start_scheduler|number_of_repair_threads|number_of_subranges|min_repair_interval|sstable_upper_threshold" + "|enabled|table_max_repair_time|priority_hosts|forcerepair_hosts|ignore_dcs" + "|history_clear_delete_hosts_buffer_interval|repair_primary_token_range_only" + @@ -60,13 +60,13 @@ public class SetAutoRepairConfig extends NodeToolCmd @Override public void execute(NodeProbe probe) { - checkArgument(args.size() == 2, "setautorepairconfig requires param-type, and value args."); + checkArgument(args.size() == 2, "setunifiedrepairconfig requires param-type, and value args."); String paramType = args.get(0); String paramVal = args.get(1); - if (!probe.getAutoRepairConfig().isAutoRepairSchedulingEnabled() && !paramType.equalsIgnoreCase("start_scheduler")) + if (!probe.getUnifiedRepairConfig().isUnifiedRepairSchedulingEnabled() && !paramType.equalsIgnoreCase("start_scheduler")) { - out.println("Auto-repair is not enabled"); + out.println("Unified-repair is not enabled"); return; } @@ -80,13 +80,13 @@ public void execute(NodeProbe probe) } return; case "history_clear_delete_hosts_buffer_interval": - probe.setAutoRepairHistoryClearDeleteHostsBufferDuration(paramVal); + probe.setUnifiedRepairHistoryClearDeleteHostsBufferDuration(paramVal); return; case "repair_max_retries": - probe.setAutoRepairMaxRetriesCount(Integer.parseInt(paramVal)); + probe.setUnifiedRepairMaxRetriesCount(Integer.parseInt(paramVal)); return; case "repair_retry_backoff": - probe.setAutoRepairRetryBackoff(paramVal); + probe.setUnifiedRepairRetryBackoff(paramVal); return; default: // proceed to options that require --repair-type option @@ -99,7 +99,7 @@ public void execute(NodeProbe probe) switch (paramType) { case "enabled": - probe.setAutoRepairEnabled(repairType, Boolean.parseBoolean(paramVal)); + probe.setUnifiedRepairEnabled(repairType, Boolean.parseBoolean(paramVal)); break; case "number_of_repair_threads": probe.setRepairThreads(repairType, Integer.parseInt(paramVal)); @@ -114,7 +114,7 @@ public void execute(NodeProbe probe) probe.setRepairSSTableCountHigherThreshold(repairType, Integer.parseInt(paramVal)); break; case "table_max_repair_time": - probe.setAutoRepairTableMaxRepairTime(repairType, paramVal); + probe.setUnifiedRepairTableMaxRepairTime(repairType, paramVal); break; case "priority_hosts": hosts = retrieveHosts(paramVal); @@ -136,7 +136,7 @@ public void execute(NodeProbe probe) { ignoreDCs.add(dc); } - probe.setAutoRepairIgnoreDCs(repairType, ignoreDCs); + probe.setUnifiedRepairIgnoreDCs(repairType, ignoreDCs); break; case "repair_primary_token_range_only": probe.setPrimaryTokenRangeOnly(repairType, Boolean.parseBoolean(paramVal)); diff --git a/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java b/src/java/org/apache/cassandra/tools/nodetool/UnifiedRepairStatus.java similarity index 82% rename from src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java rename to src/java/org/apache/cassandra/tools/nodetool/UnifiedRepairStatus.java index 7b96102c6698..b40fe887e5d8 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java +++ b/src/java/org/apache/cassandra/tools/nodetool/UnifiedRepairStatus.java @@ -25,19 +25,19 @@ import io.airlift.airline.Command; import io.airlift.airline.Option; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool; import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; import static com.google.common.base.Preconditions.checkArgument; -@Command(name = "autorepairstatus", description = "Print autorepair status") -public class AutoRepairStatus extends NodeTool.NodeToolCmd +@Command(name = "unifiedrepairstatus", description = "Print unifiedrepair status") +public class UnifiedRepairStatus extends NodeTool.NodeToolCmd { @VisibleForTesting @Option(title = "repair type", name = { "-t", "--repair-type" }, description = "Repair type") - protected AutoRepairConfig.RepairType repairType; + protected UnifiedRepairConfig.RepairType repairType; @Override public void execute(NodeProbe probe) @@ -45,10 +45,10 @@ public void execute(NodeProbe probe) checkArgument(repairType != null, "--repair-type is required."); PrintStream out = probe.output().out; - AutoRepairConfig config = probe.getAutoRepairConfig(); - if (config == null || !config.isAutoRepairSchedulingEnabled()) + UnifiedRepairConfig config = probe.getUnifiedRepairConfig(); + if (config == null || !config.isUnifiedRepairSchedulingEnabled()) { - out.println("Auto-repair is not enabled"); + out.println("Unified-repair is not enabled"); return; } diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index b2dcedde48d4..d5332ec76953 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -69,7 +69,7 @@ import org.apache.cassandra.config.ParameterizedClass; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter; +import org.apache.cassandra.repair.unifiedrepair.IUnifiedRepairTokenRangeSplitter; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -721,11 +721,11 @@ public static AbstractCryptoProvider newCryptoProvider(String className, Map parameters = parameterizedClass.parameters != null ? parameterizedClass.parameters : Collections.emptyMap(); // first attempt to initialize with Map arguments. - return (IAutoRepairTokenRangeSplitter) tokenRangeSplitterClass.getConstructor(Map.class).newInstance(parameters); + return (IUnifiedRepairTokenRangeSplitter) tokenRangeSplitterClass.getConstructor(Map.class).newInstance(parameters); } catch (NoSuchMethodException nsme) { // fall back on no argument constructor. - return (IAutoRepairTokenRangeSplitter) tokenRangeSplitterClass.getConstructor().newInstance(); + return (IUnifiedRepairTokenRangeSplitter) tokenRangeSplitterClass.getConstructor().newInstance(); } } catch (Exception ex) { - throw new ConfigurationException("Unable to create instance of IAutoRepairTokenRangeSplitter for " + className, ex); + throw new ConfigurationException("Unable to create instance of IUnifiedRepairTokenRangeSplitter for " + className, ex); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/UnifiedRepairSchedulerTest.java similarity index 83% rename from test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java rename to test/distributed/org/apache/cassandra/distributed/test/repair/UnifiedRepairSchedulerTest.java index 9b583f006507..8460ec119cee 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/UnifiedRepairSchedulerTest.java @@ -35,14 +35,14 @@ import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.test.TestBaseImpl; -import org.apache.cassandra.repair.autorepair.AutoRepair; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; -import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepair; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.service.UnifiedRepairService; import static org.apache.cassandra.schema.SchemaConstants.DISTRIBUTED_KEYSPACE_NAME; import static org.junit.Assert.assertEquals; -public class AutoRepairSchedulerTest extends TestBaseImpl +public class UnifiedRepairSchedulerTest extends TestBaseImpl { private static Cluster cluster; @@ -57,25 +57,25 @@ public static void init() throws IOException sdf = new SimpleDateFormat(pattern); sdf.setLenient(false); cluster = Cluster.build(3).withConfig(config -> config - .set("auto_repair", + .set("unified_repair", ImmutableMap.of( "repair_type_overrides", - ImmutableMap.of(AutoRepairConfig.RepairType.full.toString(), - ImmutableMap.of( + ImmutableMap.of(UnifiedRepairConfig.RepairType.full.toString(), + ImmutableMap.of( "initial_scheduler_delay", "5s", "enabled", "true", "parallel_repair_count", "1", "parallel_repair_percentage", "0", "min_repair_interval", "1s"), - AutoRepairConfig.RepairType.incremental.toString(), - ImmutableMap.of( + UnifiedRepairConfig.RepairType.incremental.toString(), + ImmutableMap.of( "initial_scheduler_delay", "5s", "enabled", "true", "parallel_repair_count", "1", "parallel_repair_percentage", "0", "min_repair_interval", "1s")))) - .set("auto_repair.enabled", "true") - .set("auto_repair.repair_check_interval", "10s")).start(); + .set("unified_repair.enabled", "true") + .set("unified_repair.repair_check_interval", "10s")).start(); cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};"); cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck text, v1 int, v2 int, PRIMARY KEY (pk, ck)) WITH read_repair='NONE'")); @@ -85,7 +85,7 @@ public static void init() throws IOException public void testScheduler() throws ParseException { // ensure there was no history of previous repair runs through the scheduler - Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY), ConsistencyLevel.QUORUM); + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY), ConsistencyLevel.QUORUM); assertEquals(0, rows.length); cluster.forEach(i -> i.runOnInstance(() -> { @@ -93,9 +93,9 @@ public void testScheduler() throws ParseException { DatabaseDescriptor.setCDCOnRepairEnabled(false); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); - AutoRepairService.instance.setup(); + UnifiedRepairService.instance.setup(); DatabaseDescriptor.setCDCOnRepairEnabled(false); - AutoRepair.instance.setup(); + UnifiedRepair.instance.setup(); } catch (Exception e) { @@ -105,13 +105,13 @@ public void testScheduler() throws ParseException // wait for a couple of minutes for repair to go through on all three nodes Uninterruptibles.sleepUninterruptibly(2, TimeUnit.MINUTES); - validate(AutoRepairConfig.RepairType.full.toString()); - validate(AutoRepairConfig.RepairType.incremental.toString()); + validate(UnifiedRepairConfig.RepairType.full.toString()); + validate(UnifiedRepairConfig.RepairType.incremental.toString()); } private void validate(String repairType) throws ParseException { - Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s where repair_type='%s'", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType), ConsistencyLevel.QUORUM); + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s where repair_type='%s'", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType), ConsistencyLevel.QUORUM); assertEquals(3, rows.length); for (int node = 0; node < rows.length; node++) { diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java index 74fe87dbaf16..71fa6c46175b 100644 --- a/test/unit/org/apache/cassandra/Util.java +++ b/test/unit/org/apache/cassandra/Util.java @@ -62,7 +62,7 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.config.Config; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; import org.apache.commons.lang3.StringUtils; import org.junit.Assume; import org.slf4j.Logger; @@ -1288,11 +1288,11 @@ public static RuntimeException testMustBeImplementedForSSTableFormat() return new UnsupportedOperationException("Test must be implemented for sstable format " + DatabaseDescriptor.getSelectedSSTableFormat().getClass().getName()); } - // Replaces the global auto-repair config with a new config where auto-repair schedulling is enabled/disabled - public static void setAutoRepairEnabled(boolean enabled) throws Exception + // Replaces the global unified-repair config with a new config where unified-repair schedulling is enabled/disabled + public static void setUnifiedRepairEnabled(boolean enabled) throws Exception { Config config = DatabaseDescriptor.getRawConfig(); - config.auto_repair = new AutoRepairConfig(enabled); + config.unified_repair = new UnifiedRepairConfig(enabled); Field configField = DatabaseDescriptor.class.getDeclaredField("conf"); configField.setAccessible(true); configField.set(null, config); diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java index 490f2534baf2..42aa930968f6 100644 --- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java +++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java @@ -103,14 +103,14 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.config.ConfigurationLoader", "org.apache.cassandra.config.Config$CorruptedTombstoneStrategy", "org.apache.cassandra.config.Config$BatchlogEndpointStrategy", - "org.apache.cassandra.repair.autorepair.AutoRepairConfig", - "org.apache.cassandra.repair.autorepair.AutoRepairConfig$Options", - "org.apache.cassandra.repair.autorepair.DefaultAutoRepairTokenSplitter", - "org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter", - "org.apache.cassandra.repair.autorepair.FullRepairState", - "org.apache.cassandra.repair.autorepair.IncrementalRepairState", - "org.apache.cassandra.repair.autorepair.AutoRepairConfig$RepairType", - "org.apache.cassandra.repair.autorepair.AutoRepairState", + "org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig", + "org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig$Options", + "org.apache.cassandra.repair.unifiedrepair.DefaultUnifiedRepairTokenSplitter", + "org.apache.cassandra.repair.unifiedrepair.IUnifiedRepairTokenRangeSplitter", + "org.apache.cassandra.repair.unifiedrepair.FullRepairState", + "org.apache.cassandra.repair.unifiedrepair.IncrementalRepairState", + "org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig$RepairType", + "org.apache.cassandra.repair.unifiedrepair.UnifiedRepairState", "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/config/YamlConfigurationLoaderTest.java b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java index 141c762ba461..946496cfe0bf 100644 --- a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java +++ b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java @@ -42,7 +42,7 @@ import static org.apache.cassandra.config.YamlConfigurationLoader.SYSTEM_PROPERTY_PREFIX; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -273,7 +273,7 @@ public void fromMapTest() Map encryptionOptions = ImmutableMap.of("cipher_suites", Collections.singletonList("FakeCipher"), "optional", false, "enabled", true); - Map autoRepairConfig = ImmutableMap.of("enabled", true, + Map unifiedRepairConfig = ImmutableMap.of("enabled", true, "global_settings", ImmutableMap.of("number_of_repair_threads", 1), "repair_type_overrides", ImmutableMap.of( @@ -287,7 +287,7 @@ public void fromMapTest() .put("internode_socket_send_buffer_size", "5B") .put("internode_socket_receive_buffer_size", "5B") .put("commitlog_sync_group_window_in_ms", "42") - .put("auto_repair", autoRepairConfig) + .put("unified_repair", unifiedRepairConfig) .build(); Config config = YamlConfigurationLoader.fromMap(map, Config.class); @@ -298,9 +298,9 @@ public void fromMapTest() assertEquals(true, config.client_encryption_options.enabled); // Check a nested object assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_send_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_receive_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) - assertEquals(true, config.auto_repair.enabled); - assertEquals(new DurationSpec.IntSecondsBound("6h"), config.auto_repair.getAutoRepairTableMaxRepairTime(AutoRepairConfig.RepairType.incremental)); - config.auto_repair.setMVRepairEnabled(AutoRepairConfig.RepairType.incremental, false); + assertEquals(true, config.unified_repair.enabled); + assertEquals(new DurationSpec.IntSecondsBound("6h"), config.unified_repair.getUnifiedRepairTableMaxRepairTime(UnifiedRepairConfig.RepairType.incremental)); + config.unified_repair.setMVRepairEnabled(UnifiedRepairConfig.RepairType.incremental, false); } @Test @@ -497,4 +497,4 @@ public static Config load(String path) } return new YamlConfigurationLoader().loadConfig(url); } -} \ No newline at end of file +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java b/test/unit/org/apache/cassandra/repair/unifiedrepair/SSTableRepairedAtTest.java similarity index 97% rename from test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java rename to test/unit/org/apache/cassandra/repair/unifiedrepair/SSTableRepairedAtTest.java index bd14eea805b3..14011f85896d 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java +++ b/test/unit/org/apache/cassandra/repair/unifiedrepair/SSTableRepairedAtTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.net.UnknownHostException; import java.util.Arrays; @@ -55,8 +55,8 @@ public class SSTableRepairedAtTest extends CQLTester public static void setUp() throws ConfigurationException, UnknownHostException { requireNetwork(); - AutoRepairUtils.setup(); - StorageService.instance.doAutoRepairSetup(); + UnifiedRepairUtils.setup(); + StorageService.instance.doUnifiedRepairSetup(); DatabaseDescriptor.setCDCEnabled(false); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfigTest.java similarity index 77% rename from test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java rename to test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfigTest.java index 4f1dd029ef5e..e7ce54a1cea5 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfigTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.util.EnumMap; import java.util.Objects; @@ -35,7 +35,7 @@ import org.apache.cassandra.config.ParameterizedClass; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.Options; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.Options; import org.apache.cassandra.utils.FBUtilities; import static org.junit.Assert.assertEquals; @@ -44,79 +44,79 @@ import static org.junit.Assert.assertTrue; @RunWith(Parameterized.class) -public class AutoRepairConfigTest extends CQLTester +public class UnifiedRepairConfigTest extends CQLTester { - private AutoRepairConfig config; + private UnifiedRepairConfig config; private Set testSet = ImmutableSet.of("dc1"); @Parameterized.Parameter - public AutoRepairConfig.RepairType repairType; + public UnifiedRepairConfig.RepairType repairType; @Parameterized.Parameters public static Object[] repairTypes() { - return AutoRepairConfig.RepairType.values(); + return UnifiedRepairConfig.RepairType.values(); } @Before public void setUp() { - config = new AutoRepairConfig(true); + config = new UnifiedRepairConfig(true); config.repair_type_overrides = null; - AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + UnifiedRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); } @Test - public void autoRepairConfigDefaultsAreNotNull() + public void unifiedRepairConfigDefaultsAreNotNull() { - AutoRepairConfig config = new AutoRepairConfig(); + UnifiedRepairConfig config = new UnifiedRepairConfig(); assertNotNull(config.global_settings); } @Test - public void autoRepairConfigRepairTypesAreNotNull() + public void unifiedRepairConfigRepairTypesAreNotNull() { - AutoRepairConfig config = new AutoRepairConfig(); - for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + UnifiedRepairConfig config = new UnifiedRepairConfig(); + for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) { assertNotNull(config.repair_type_overrides.get(repairType)); } } @Test - public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsEnabled() + public void testIsUnifiedRepairEnabledReturnsTrueWhenRepairIsEnabled() { config.global_settings.enabled = true; - assertTrue(config.isAutoRepairEnabled(repairType)); + assertTrue(config.isUnifiedRepairEnabled(repairType)); } @Test - public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledGlobally() + public void testIsUnifiedRepairEnabledReturnsTrueWhenRepairIsDisabledGlobally() { - config = new AutoRepairConfig(false); + config = new UnifiedRepairConfig(false); config.global_settings.enabled = true; - assertFalse(config.isAutoRepairEnabled(repairType)); + assertFalse(config.isUnifiedRepairEnabled(repairType)); } @Test - public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledForRepairType() + public void testIsUnifiedRepairEnabledReturnsTrueWhenRepairIsDisabledForRepairType() { config.global_settings.enabled = true; - config.repair_type_overrides = new EnumMap<>(AutoRepairConfig.RepairType.class); + config.repair_type_overrides = new EnumMap<>(UnifiedRepairConfig.RepairType.class); config.repair_type_overrides.put(repairType, new Options()); config.repair_type_overrides.get(repairType).enabled = false; - assertFalse(config.isAutoRepairEnabled(repairType)); + assertFalse(config.isUnifiedRepairEnabled(repairType)); } @Test - public void testSetAutoRepairEnabledNoMVOrCDC() + public void testSetUnifiedRepairEnabledNoMVOrCDC() { DatabaseDescriptor.setCDCEnabled(false); DatabaseDescriptor.setMaterializedViewsEnabled(false); - config.setAutoRepairEnabled(repairType, true); + config.setUnifiedRepairEnabled(repairType, true); assertTrue(config.repair_type_overrides.get(repairType).enabled); } @@ -194,19 +194,19 @@ public void testSetRepairMinFrequencyInHours() } @Test - public void testGetAutoRepairHistoryClearDeleteHostsBufferInSec() + public void testGetUnifiedRepairHistoryClearDeleteHostsBufferInSec() { config.history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound("5s"); - int result = config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds(); + int result = config.getUnifiedRepairHistoryClearDeleteHostsBufferInterval().toSeconds(); assertEquals(5, result); } @Test - public void testSetAutoRepairHistoryClearDeleteHostsBufferInSec() + public void testSetUnifiedRepairHistoryClearDeleteHostsBufferInSec() { - config.setAutoRepairHistoryClearDeleteHostsBufferInterval("5s"); + config.setUnifiedRepairHistoryClearDeleteHostsBufferInterval("5s"); assert Objects.equals(config.history_clear_delete_hosts_buffer_interval, new DurationSpec.IntSecondsBound("5s")); } @@ -230,19 +230,19 @@ public void testSetRepairSSTableCountHigherThreshold() } @Test - public void testGetAutoRepairTableMaxRepairTimeInSec() + public void testGetUnifiedRepairTableMaxRepairTimeInSec() { config.global_settings.table_max_repair_time = new DurationSpec.IntSecondsBound("5s"); - DurationSpec.IntSecondsBound result = config.getAutoRepairTableMaxRepairTime(repairType); + DurationSpec.IntSecondsBound result = config.getUnifiedRepairTableMaxRepairTime(repairType); assertEquals(5, result.toSeconds()); } @Test - public void testSetAutoRepairTableMaxRepairTimeInSec() + public void testSetUnifiedRepairTableMaxRepairTimeInSec() { - config.setAutoRepairTableMaxRepairTime(repairType, "5s"); + config.setUnifiedRepairTableMaxRepairTime(repairType, "5s"); assert config.repair_type_overrides.get(repairType).table_max_repair_time.toSeconds() == 5; } @@ -356,19 +356,19 @@ public void testGetForceRepairNewNode() } @Test - public void testIsAutoRepairSchedulingEnabledDefault() + public void testIsUnifiedRepairSchedulingEnabledDefault() { - config = new AutoRepairConfig(); + config = new UnifiedRepairConfig(); - boolean result = config.isAutoRepairSchedulingEnabled(); + boolean result = config.isUnifiedRepairSchedulingEnabled(); assertFalse(result); } @Test - public void testIsAutoRepairSchedulingEnabledTrue() + public void testIsUnifiedRepairSchedulingEnabledTrue() { - boolean result = config.isAutoRepairSchedulingEnabled(); + boolean result = config.isUnifiedRepairSchedulingEnabled(); assertTrue(result); } @@ -386,16 +386,16 @@ public void testGetDefaultOptionsTokenRangeSplitter() { Options defaultOptions = Options.getDefaultOptions(); - ParameterizedClass expectedDefault = new ParameterizedClass(DefaultAutoRepairTokenSplitter.class.getName(), Collections.emptyMap()); + ParameterizedClass expectedDefault = new ParameterizedClass(DefaultUnifiedRepairTokenSplitter.class.getName(), Collections.emptyMap()); assertEquals(expectedDefault, defaultOptions.token_range_splitter); - assertEquals(DefaultAutoRepairTokenSplitter.class.getName(), FBUtilities.newAutoRepairTokenRangeSplitter(defaultOptions.token_range_splitter).getClass().getName()); + assertEquals(DefaultUnifiedRepairTokenSplitter.class.getName(), FBUtilities.newUnifiedRepairTokenRangeSplitter(defaultOptions.token_range_splitter).getClass().getName()); } @Test(expected = ConfigurationException.class) public void testInvalidTokenRangeSplitter() { - FBUtilities.newAutoRepairTokenRangeSplitter(new ParameterizedClass("invalid-class", Collections.emptyMap())); + FBUtilities.newUnifiedRepairTokenRangeSplitter(new ParameterizedClass("invalid-class", Collections.emptyMap())); } @Test diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairDefaultTokenSplitterParameterizedTest.java similarity index 84% rename from test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java rename to test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairDefaultTokenSplitterParameterizedTest.java index 804d0a712b48..f14f6acefeac 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairDefaultTokenSplitterParameterizedTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.util.ArrayList; import java.util.Arrays; @@ -37,8 +37,8 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.index.sai.disk.format.Version; -import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; -import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.repair.unifiedrepair.IUnifiedRepairTokenRangeSplitter.RepairAssignment; +import org.apache.cassandra.service.UnifiedRepairService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; @@ -48,7 +48,7 @@ import static org.junit.Assert.assertEquals; @RunWith(Parameterized.class) -public class AutoRepairDefaultTokenSplitterParameterizedTest +public class UnifiedRepairDefaultTokenSplitterParameterizedTest { private static final String KEYSPACE = "ks"; private static final String TABLE1 = "tbl1"; @@ -56,12 +56,12 @@ public class AutoRepairDefaultTokenSplitterParameterizedTest private static final String TABLE3 = "tbl3"; @Parameterized.Parameter() - public AutoRepairConfig.RepairType repairType; + public UnifiedRepairConfig.RepairType repairType; @Parameterized.Parameters(name = "repairType={0}") - public static Collection repairTypes() + public static Collection repairTypes() { - return Arrays.asList(AutoRepairConfig.RepairType.values()); + return Arrays.asList(UnifiedRepairConfig.RepairType.values()); } @BeforeClass @@ -83,7 +83,7 @@ public static void setupClass() throws Exception ServerTestUtils.registerLocal(tokens); // Ensure that the on-disk format statics are loaded before the test run Version.LATEST.onDiskFormat(); - StorageService.instance.doAutoRepairSetup(); + StorageService.instance.doUnifiedRepairSetup(); SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE)); @@ -111,7 +111,7 @@ private static void appendExpectedTokens(long left, long right, int numberOfSpli @Test public void testTokenRangesSplitByTable() { - AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(repairType, false); + UnifiedRepairService.instance.getUnifiedRepairConfig().setRepairByKeyspace(repairType, false); int totalTokenRanges = 3; Collection> tokens = StorageService.instance.getPrimaryRanges(KEYSPACE); assertEquals(totalTokenRanges, tokens.size()); @@ -125,9 +125,9 @@ public void testTokenRangesSplitByTable() appendExpectedTokens(256, 1024, numberOfSplits, expectedToken); } - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); config.setRepairSubRangeNum(repairType, numberOfSplits); - List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); + List assignments = new DefaultUnifiedRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); assertEquals(totalTokenRanges * numberOfSplits * tables.size(), assignments.size()); assertEquals(expectedToken.size(), assignments.size()); @@ -146,7 +146,7 @@ public void testTokenRangesSplitByTable() @Test public void testTokenRangesSplitByKeyspace() { - AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(repairType, true); + UnifiedRepairService.instance.getUnifiedRepairConfig().setRepairByKeyspace(repairType, true); int totalTokenRanges = 3; Collection> tokens = StorageService.instance.getPrimaryRanges(KEYSPACE); assertEquals(totalTokenRanges, tokens.size()); @@ -157,9 +157,9 @@ public void testTokenRangesSplitByKeyspace() appendExpectedTokens(0, 256, numberOfSplits, expectedToken); appendExpectedTokens(256, 1024, numberOfSplits, expectedToken); - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); config.setRepairSubRangeNum(repairType, numberOfSplits); - List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); + List assignments = new DefaultUnifiedRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); assertEquals(totalTokenRanges * numberOfSplits, assignments.size()); assertEquals(expectedToken.size(), assignments.size()); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairKeyspaceTest.java similarity index 87% rename from test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java rename to test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairKeyspaceTest.java index 1337cf3dd2d3..748cc88b2a2e 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairKeyspaceTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.util.HashSet; import java.util.Iterator; @@ -29,7 +29,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; -public class AutoRepairKeyspaceTest +public class UnifiedRepairKeyspaceTest { @BeforeClass public static void setupDatabaseDescriptor() @@ -38,7 +38,7 @@ public static void setupDatabaseDescriptor() } @Test - public void testEnsureAutoRepairTablesArePresent() + public void testEnsureUnifiedRepairTablesArePresent() { KeyspaceMetadata keyspaceMetadata = SystemDistributedKeyspace.metadata(); Iterator iter = keyspaceMetadata.tables.iterator(); @@ -48,7 +48,7 @@ public void testEnsureAutoRepairTablesArePresent() actualDistributedTablesIter.add(iter.next().name); } - Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.AUTO_REPAIR_HISTORY)); - Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY)); + Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY)); + Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY)); } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairParameterizedTest.java similarity index 52% rename from test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java rename to test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairParameterizedTest.java index 3ec202f23e19..4a8d375f5988 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairParameterizedTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.util.ArrayList; import java.util.Arrays; @@ -37,7 +37,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.repair.RepairCoordinator; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; +import org.apache.cassandra.repair.unifiedrepair.IUnifiedRepairTokenRangeSplitter.RepairAssignment; import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.apache.cassandra.service.StorageService; @@ -56,20 +56,20 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.metrics.AutoRepairMetricsManager; -import org.apache.cassandra.metrics.AutoRepairMetrics; +import org.apache.cassandra.metrics.UnifiedRepairMetricsManager; +import org.apache.cassandra.metrics.UnifiedRepairMetrics; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.UnifiedRepairService; import org.apache.cassandra.utils.FBUtilities; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; -import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.Util.setUnifiedRepairEnabled; import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.NOT_MY_TURN; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -81,43 +81,43 @@ import static org.mockito.Mockito.when; @RunWith(Parameterized.class) -public class AutoRepairParameterizedTest extends CQLTester +public class UnifiedRepairParameterizedTest extends CQLTester { private static final String KEYSPACE = "ks"; private static final String TABLE = "tbl"; - private static final String TABLE_DISABLED_AUTO_REPAIR = "tbl_disabled_auto_repair"; + private static final String TABLE_DISABLED_UNIFIED_REPAIR = "tbl_disabled_unified_repair"; private static final String MV = "mv"; private static TableMetadata cfm; - private static TableMetadata cfmDisabledAutoRepair; + private static TableMetadata cfmDisabledUnifiedRepair; private static Keyspace keyspace; private static int timeFuncCalls; @Mock ScheduledExecutorPlus mockExecutor; @Mock - AutoRepairState autoRepairState; + UnifiedRepairState unifiedRepairState; @Mock RepairCoordinator repairRunnable; - private static AutoRepairConfig defaultConfig; + private static UnifiedRepairConfig defaultConfig; @Parameterized.Parameter() - public AutoRepairConfig.RepairType repairType; + public UnifiedRepairConfig.RepairType repairType; @Parameterized.Parameters(name = "repairType={0}") - public static Collection repairTypes() + public static Collection repairTypes() { - return Arrays.asList(AutoRepairConfig.RepairType.values()); + return Arrays.asList(UnifiedRepairConfig.RepairType.values()); } @BeforeClass public static void setupClass() throws Exception { SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); - AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); - setAutoRepairEnabled(true); + UnifiedRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + setUnifiedRepairEnabled(true); requireNetwork(); - AutoRepairUtils.setup(); - StorageService.instance.doAutoRepairSetup(); + UnifiedRepairUtils.setup(); + StorageService.instance.doUnifiedRepairSetup(); DatabaseDescriptor.setCDCEnabled(false); } @@ -127,7 +127,7 @@ public void setup() SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE)); QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i))", KEYSPACE, TABLE)); - QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i)) WITH repair_full = {'enabled': 'false'} AND repair_incremental = {'enabled': 'false'}", KEYSPACE, TABLE_DISABLED_AUTO_REPAIR)); + QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i)) WITH repair_full = {'enabled': 'false'} AND repair_incremental = {'enabled': 'false'}", KEYSPACE, TABLE_DISABLED_UNIFIED_REPAIR)); QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); @@ -142,23 +142,23 @@ public void setup() Keyspace.open(KEYSPACE).getColumnFamilyStore(MV).truncateBlocking(); Keyspace.open(KEYSPACE).getColumnFamilyStore(MV).disableAutoCompaction(); - Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY).truncateBlocking(); - Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_HISTORY).truncateBlocking(); + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY).truncateBlocking(); + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY).truncateBlocking(); - AutoRepair.instance = new AutoRepair(); + UnifiedRepair.instance = new UnifiedRepair(); executeCQL(); timeFuncCalls = 0; - AutoRepair.timeFunc = System::currentTimeMillis; + UnifiedRepair.timeFunc = System::currentTimeMillis; resetCounters(); resetConfig(); - AutoRepair.shuffleFunc = java.util.Collections::shuffle; + UnifiedRepair.shuffleFunc = java.util.Collections::shuffle; keyspace = Keyspace.open(KEYSPACE); cfm = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE).metadata(); - cfmDisabledAutoRepair = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE_DISABLED_AUTO_REPAIR).metadata(); + cfmDisabledUnifiedRepair = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE_DISABLED_UNIFIED_REPAIR).metadata(); DatabaseDescriptor.setCDCOnRepairEnabled(false); } @@ -170,7 +170,7 @@ public void tearDown() private void resetCounters() { - AutoRepairMetrics metrics = AutoRepairMetricsManager.getMetrics(repairType); + UnifiedRepairMetrics metrics = UnifiedRepairMetricsManager.getMetrics(repairType); Metrics.removeMatching((name, metric) -> name.startsWith("repairTurn")); metrics.repairTurnMyTurn = Metrics.counter(String.format("repairTurnMyTurn-%s", repairType)); metrics.repairTurnMyTurnForceRepair = Metrics.counter(String.format("repairTurnMyTurnForceRepair-%s", repairType)); @@ -180,15 +180,15 @@ private void resetCounters() private void resetConfig() { // prepare a fresh default config - defaultConfig = new AutoRepairConfig(true); - for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + defaultConfig = new UnifiedRepairConfig(true); + for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) { - defaultConfig.setAutoRepairEnabled(repairType, true); + defaultConfig.setUnifiedRepairEnabled(repairType, true); defaultConfig.setMVRepairEnabled(repairType, false); } - // reset the AutoRepairService config to default - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + // reset the UnifiedRepairService config to default + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); config.repair_type_overrides = defaultConfig.repair_type_overrides; config.global_settings = defaultConfig.global_settings; config.history_clear_delete_hosts_buffer_interval = defaultConfig.history_clear_delete_hosts_buffer_interval; @@ -200,24 +200,24 @@ private void executeCQL() QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, s) VALUES ('k', 's')"); QueryProcessor.executeInternal("SELECT s FROM ks.tbl WHERE k='k'"); Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME) - .getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY) + .getColumnFamilyStore(SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY) .forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); } @Test(expected = ConfigurationException.class) public void testRepairAsyncWithRepairTypeDisabled() { - AutoRepairService.instance.getAutoRepairConfig().setAutoRepairEnabled(repairType, false); + UnifiedRepairService.instance.getUnifiedRepairConfig().setUnifiedRepairEnabled(repairType, false); - AutoRepair.instance.repairAsync(repairType); + UnifiedRepair.instance.repairAsync(repairType); } @Test public void testRepairAsync() { - AutoRepair.instance.repairExecutors.put(repairType, mockExecutor); + UnifiedRepair.instance.repairExecutors.put(repairType, mockExecutor); - AutoRepair.instance.repairAsync(repairType); + UnifiedRepair.instance.repairAsync(repairType); verify(mockExecutor, Mockito.times(1)).submit(any(Runnable.class)); } @@ -226,17 +226,17 @@ public void testRepairAsync() public void testRepairTurn() { UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); - Assert.assertTrue("Expected my turn for the repair", AutoRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); + Assert.assertTrue("Expected my turn for the repair", UnifiedRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); } @Test public void testRepair() { - AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); - AutoRepair.instance.repair(repairType); - assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); - long lastRepairTime = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + UnifiedRepairService.instance.getUnifiedRepairConfig().setRepairMinInterval(repairType, "0s"); + UnifiedRepair.instance.repair(repairType); + assertEquals(0, UnifiedRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + long lastRepairTime = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); //if repair was done then lastRepairTime should be non-zero Assert.assertTrue(String.format("Expected lastRepairTime > 0, actual value lastRepairTime %d", lastRepairTime), lastRepairTime > 0); @@ -245,113 +245,113 @@ public void testRepair() @Test public void testTooFrequentRepairs() { - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); //in the first round let repair run config.setRepairMinInterval(repairType, "0s"); - AutoRepair.instance.repair(repairType); - long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); - int consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); + UnifiedRepair.instance.repair(repairType); + long lastRepairTime1 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); + int consideredTables = UnifiedRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); Assert.assertNotSame(String.format("Expected total repaired tables > 0, actual value %s ", consideredTables), consideredTables, 0); //if repair was done in last 24 hours then it should not trigger another repair config.setRepairMinInterval(repairType, "24h"); - AutoRepair.instance.repair(repairType); - long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + UnifiedRepair.instance.repair(repairType); + long lastRepairTime2 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertEquals(String.format("Expected repair time to be same, actual value lastRepairTime1 %d, " + "lastRepairTime2 %d", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); - assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertEquals(0, UnifiedRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); } @Test public void testNonFrequentRepairs() { - Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); - AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + Integer prevMetricsCount = UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + UnifiedRepairState state = UnifiedRepair.instance.repairStates.get(repairType); long prevCount = state.getTotalMVTablesConsideredForRepair(); - AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); - AutoRepair.instance.repair(repairType); - long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + UnifiedRepairService.instance.getUnifiedRepairConfig().setRepairMinInterval(repairType, "0s"); + UnifiedRepair.instance.repair(repairType); + long lastRepairTime1 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertTrue(String.format("Expected lastRepairTime1 > 0, actual value lastRepairTime1 %d", lastRepairTime1), lastRepairTime1 > 0); UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); Assert.assertTrue("Expected my turn for the repair", - AutoRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); - AutoRepair.instance.repair(repairType); - long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + UnifiedRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); + UnifiedRepair.instance.repair(repairType); + long lastRepairTime2 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertNotSame(String.format("Expected repair time to be same, actual value lastRepairTime1 %d, " + "lastRepairTime2 ", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); - assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + assertEquals(prevMetricsCount, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); } @Test public void testGetPriorityHosts() { - Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); - AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + Integer prevMetricsCount = UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + UnifiedRepairState state = UnifiedRepair.instance.repairStates.get(repairType); long prevCount = state.getTotalMVTablesConsideredForRepair(); - AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); + UnifiedRepairService.instance.getUnifiedRepairConfig().setRepairMinInterval(repairType, "0s"); Assert.assertSame(String.format("Priority host count is not same, actual value %d, expected value %d", - AutoRepairUtils.getPriorityHosts(repairType).size(), 0), AutoRepairUtils.getPriorityHosts(repairType).size(), 0); + UnifiedRepairUtils.getPriorityHosts(repairType).size(), 0), UnifiedRepairUtils.getPriorityHosts(repairType).size(), 0); UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); - Assert.assertTrue("Expected my turn for the repair", AutoRepairUtils.myTurnToRunRepair(repairType, myId) != + Assert.assertTrue("Expected my turn for the repair", UnifiedRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); - AutoRepair.instance.repair(repairType); - AutoRepairUtils.addPriorityHosts(repairType, Sets.newHashSet(FBUtilities.getBroadcastAddressAndPort())); - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); + UnifiedRepairUtils.addPriorityHosts(repairType, Sets.newHashSet(FBUtilities.getBroadcastAddressAndPort())); + UnifiedRepair.instance.repair(repairType); Assert.assertSame(String.format("Priority host count is not same actual value %d, expected value %d", - AutoRepairUtils.getPriorityHosts(repairType).size(), 0), AutoRepairUtils.getPriorityHosts(repairType).size(), 0); + UnifiedRepairUtils.getPriorityHosts(repairType).size(), 0), UnifiedRepairUtils.getPriorityHosts(repairType).size(), 0); assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); - assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + assertEquals(prevMetricsCount, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); } @Test - public void testCheckAutoRepairStartStop() throws Throwable + public void testCheckUnifiedRepairStartStop() throws Throwable { - Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); - AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + Integer prevMetricsCount = UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + UnifiedRepairState state = UnifiedRepair.instance.repairStates.get(repairType); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); long prevCount = state.getTotalMVTablesConsideredForRepair(); config.setRepairMinInterval(repairType, "0s"); - config.setAutoRepairEnabled(repairType, false); - long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); - AutoRepair.instance.repair(repairType); - long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + config.setUnifiedRepairEnabled(repairType, false); + long lastRepairTime1 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); + UnifiedRepair.instance.repair(repairType); + long lastRepairTime2 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); //Since repair has not happened, both the last repair times should be same Assert.assertEquals(String.format("Expected lastRepairTime1 %d, and lastRepairTime2 %d to be same", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); - config.setAutoRepairEnabled(repairType, true); - AutoRepair.instance.repair(repairType); + config.setUnifiedRepairEnabled(repairType, true); + UnifiedRepair.instance.repair(repairType); //since repair is done now, so lastRepairTime1/lastRepairTime2 and lastRepairTime3 should not be same - long lastRepairTime3 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + long lastRepairTime3 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertNotSame(String.format("Expected lastRepairTime1 %d, and lastRepairTime3 %d to be not same", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime3); assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); - assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + assertEquals(prevMetricsCount, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); } @Test public void testRepairPrimaryRangesByDefault() { Assert.assertTrue("Expected primary range repair only", - AutoRepairService.instance.getAutoRepairConfig().getRepairPrimaryTokenRangeOnly(repairType)); + UnifiedRepairService.instance.getUnifiedRepairConfig().getRepairPrimaryTokenRangeOnly(repairType)); } @Test public void testGetAllMVs() { - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); config.setMVRepairEnabled(repairType, false); assertFalse(config.getMVRepairEnabled(repairType)); - assertEquals(0, AutoRepairUtils.getAllMVs(repairType, keyspace, cfm).size()); + assertEquals(0, UnifiedRepairUtils.getAllMVs(repairType, keyspace, cfm).size()); config.setMVRepairEnabled(repairType, true); assertTrue(config.getMVRepairEnabled(repairType)); - assertEquals(Arrays.asList(MV), AutoRepairUtils.getAllMVs(repairType, keyspace, cfm)); + assertEquals(Arrays.asList(MV), UnifiedRepairUtils.getAllMVs(repairType, keyspace, cfm)); config.setMVRepairEnabled(repairType, false); } @@ -359,32 +359,32 @@ public void testGetAllMVs() @Test public void testMVRepair() { - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); config.setMVRepairEnabled(repairType, true); config.setRepairMinInterval(repairType, "0s"); - AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); - AutoRepair.instance.repair(repairType); - assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); - assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + UnifiedRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); + UnifiedRepair.instance.repair(repairType); + assertEquals(1, UnifiedRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); config.setMVRepairEnabled(repairType, false); - AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); - AutoRepair.instance.repair(repairType); - assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + UnifiedRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); + UnifiedRepair.instance.repair(repairType); + assertEquals(0, UnifiedRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); config.setMVRepairEnabled(repairType, true); - AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); - AutoRepair.instance.repair(repairType); - assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); - assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + UnifiedRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); + UnifiedRepair.instance.repair(repairType); + assertEquals(1, UnifiedRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); } @Test public void testSkipRepairSSTableCountHigherThreshold() { - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); - AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + UnifiedRepairState state = UnifiedRepair.instance.repairStates.get(repairType); ColumnFamilyStore cfsBaseTable = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE); ColumnFamilyStore cfsMVTable = Keyspace.open(KEYSPACE).getColumnFamilyStore(MV); Set preBaseTable = cfsBaseTable.getLiveSSTables(); @@ -412,89 +412,89 @@ public void testSkipRepairSSTableCountHigherThreshold() config.setMVRepairEnabled(repairType, true); config.setRepairSSTableCountHigherThreshold(repairType, 9); assertEquals(0, state.getSkippedTokenRangesCount()); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); state.setLastRepairTime(0); - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); assertEquals(0, state.getTotalMVTablesConsideredForRepair()); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); // skipping both the tables - one table is due to its repair has been disabled, and another one due to high sstable count assertEquals(0, state.getSkippedTokenRangesCount()); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); assertEquals(2, state.getSkippedTablesCount()); - assertEquals(2, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + assertEquals(2, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); // set it to higher value, and this time, the tables should not be skipped config.setRepairSSTableCountHigherThreshold(repairType, beforeCount); state.setLastRepairTime(0); state.setSkippedTablesCount(0); state.setTotalMVTablesConsideredForRepair(0); - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); assertEquals(1, state.getTotalMVTablesConsideredForRepair()); - assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); assertEquals(0, state.getSkippedTokenRangesCount()); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); assertEquals(1, state.getSkippedTablesCount()); - assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); } @Test public void testGetRepairState() { - assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getRepairKeyspaceCount()); + assertEquals(0, UnifiedRepair.instance.repairStates.get(repairType).getRepairKeyspaceCount()); - AutoRepairState state = AutoRepair.instance.getRepairState(repairType); + UnifiedRepairState state = UnifiedRepair.instance.getRepairState(repairType); state.setRepairKeyspaceCount(100); - assertEquals(100L, AutoRepair.instance.getRepairState(repairType).getRepairKeyspaceCount()); + assertEquals(100L, UnifiedRepair.instance.getRepairState(repairType).getRepairKeyspaceCount()); } @Test public void testMetrics() { - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); config.setMVRepairEnabled(repairType, true); config.setRepairMinInterval(repairType, "0s"); - config.setAutoRepairTableMaxRepairTime(repairType, "0s"); - AutoRepair.timeFunc = () -> { + config.setUnifiedRepairTableMaxRepairTime(repairType, "0s"); + UnifiedRepair.timeFunc = () -> { timeFuncCalls++; return timeFuncCalls * 1000L; }; - AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(1000L); + UnifiedRepair.instance.repairStates.get(repairType).setLastRepairTime(1000L); - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); - assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); - assertTrue(AutoRepairMetricsManager.getMetrics(repairType).nodeRepairTimeInSec.getValue() > 0); - assertTrue(AutoRepairMetricsManager.getMetrics(repairType).clusterRepairTimeInSec.getValue() > 0); - assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).repairTurnMyTurn.getCount()); - assertTrue(AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue() > 0); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue().intValue()); + assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).nodeRepairTimeInSec.getValue() > 0); + assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).clusterRepairTimeInSec.getValue() > 0); + assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).repairTurnMyTurn.getCount()); + assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue() > 0); + assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue().intValue()); - config.setAutoRepairTableMaxRepairTime(repairType, String.valueOf(Integer.MAX_VALUE-1) + 's'); - AutoRepair.instance.repairStates.put(repairType, autoRepairState); - when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) + config.setUnifiedRepairTableMaxRepairTime(repairType, String.valueOf(Integer.MAX_VALUE - 1) + 's'); + UnifiedRepair.instance.repairStates.put(repairType, unifiedRepairState); + when(unifiedRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) .thenReturn(repairRunnable); - when(autoRepairState.getFailedTokenRangesCount()).thenReturn(10); - when(autoRepairState.getSucceededTokenRangesCount()).thenReturn(11); - when(autoRepairState.getLongestUnrepairedSec()).thenReturn(10); + when(unifiedRepairState.getFailedTokenRangesCount()).thenReturn(10); + when(unifiedRepairState.getSucceededTokenRangesCount()).thenReturn(11); + when(unifiedRepairState.getLongestUnrepairedSec()).thenReturn(10); - AutoRepair.instance.repair(repairType); - assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); - assertTrue(AutoRepairMetricsManager.getMetrics(repairType).failedTokenRangesCount.getValue() > 0); - assertTrue(AutoRepairMetricsManager.getMetrics(repairType).succeededTokenRangesCount.getValue() > 0); - assertTrue(AutoRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue() > 0); + UnifiedRepair.instance.repair(repairType); + assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).failedTokenRangesCount.getValue() > 0); + assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).succeededTokenRangesCount.getValue() > 0); + assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue() > 0); } @Test public void testRepairWaitsForRepairToFinishBeforeSchedullingNewSession() throws Exception { - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); config.setMVRepairEnabled(repairType, false); config.setRepairRetryBackoff("0s"); - when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) + when(unifiedRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) .thenReturn(repairRunnable); - AutoRepair.instance.repairStates.put(repairType, autoRepairState); - when(autoRepairState.getLastRepairTime()).thenReturn((long) 0); + UnifiedRepair.instance.repairStates.put(repairType, unifiedRepairState); + when(unifiedRepairState.getLastRepairTime()).thenReturn((long) 0); AtomicInteger resetWaitConditionCalls = new AtomicInteger(); AtomicInteger waitForRepairCompletedCalls = new AtomicInteger(); doAnswer(invocation -> { @@ -502,36 +502,36 @@ public void testRepairWaitsForRepairToFinishBeforeSchedullingNewSession() throws assertEquals("waitForRepairToComplete was called before resetWaitCondition", resetWaitConditionCalls.get(), waitForRepairCompletedCalls.get() + 1); return null; - }).when(autoRepairState).resetWaitCondition(); + }).when(unifiedRepairState).resetWaitCondition(); doAnswer(invocation -> { waitForRepairCompletedCalls.getAndIncrement(); assertEquals("resetWaitCondition was not called before waitForRepairToComplete", resetWaitConditionCalls.get(), waitForRepairCompletedCalls.get()); return null; - }).when(autoRepairState).waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); + }).when(unifiedRepairState).waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); - AutoRepair.instance.repair(repairType); - AutoRepair.instance.repair(repairType); - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); } @Test - public void testDisabledAutoRepairForATableThroughTableLevelConfiguration() + public void testDisabledUnifiedRepairForATableThroughTableLevelConfiguration() { - Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); - Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); - Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); - Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); + Assert.assertTrue(cfm.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full).repairEnabled()); + Assert.assertTrue(cfm.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).repairEnabled()); + Assert.assertFalse(cfmDisabledUnifiedRepair.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full).repairEnabled()); + Assert.assertFalse(cfmDisabledUnifiedRepair.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).repairEnabled()); - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); config.setRepairMinInterval(repairType, "0s"); - int disabledTablesRepairCountBefore = AutoRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); - AutoRepair.instance.repair(repairType); - int consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); + int disabledTablesRepairCountBefore = UnifiedRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); + UnifiedRepair.instance.repair(repairType); + int consideredTables = UnifiedRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); Assert.assertNotSame(String.format("Expected total repaired tables > 0, actual value %s ", consideredTables), consideredTables, 0); - int disabledTablesRepairCountAfter = AutoRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); - Assert.assertTrue(String.format("A table %s should be skipped from auto repair, expected value: %d, actual value %d ", TABLE_DISABLED_AUTO_REPAIR, disabledTablesRepairCountBefore + 1, disabledTablesRepairCountAfter), + int disabledTablesRepairCountAfter = UnifiedRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); + Assert.assertTrue(String.format("A table %s should be skipped from unified repair, expected value: %d, actual value %d ", TABLE_DISABLED_UNIFIED_REPAIR, disabledTablesRepairCountBefore + 1, disabledTablesRepairCountAfter), disabledTablesRepairCountBefore < disabledTablesRepairCountAfter); } @@ -542,7 +542,7 @@ public void testTokenRangesNoSplit() assertEquals(1, tokens.size()); List> expectedToken = new ArrayList<>(tokens); - List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, Collections.singletonList(TABLE)); + List assignments = new DefaultUnifiedRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, Collections.singletonList(TABLE)); assertEquals(1, assignments.size()); assertEquals(expectedToken.get(0).left, assignments.get(0).getTokenRange().left); assertEquals(expectedToken.get(0).right, assignments.get(0).getTokenRange().right); @@ -556,12 +556,12 @@ public void testTableAttribute() } @Test - public void testDefaultAutomatedRepair() + public void testDefaultUnifiedRepair() { - Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); - Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); - Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); - Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); + Assert.assertTrue(cfm.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full).repairEnabled()); + Assert.assertTrue(cfm.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).repairEnabled()); + Assert.assertFalse(cfmDisabledUnifiedRepair.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full).repairEnabled()); + Assert.assertFalse(cfmDisabledUnifiedRepair.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).repairEnabled()); } @Test @@ -569,7 +569,7 @@ public void testRepairShufflesKeyspacesAndTables() { AtomicInteger shuffleKeyspacesCall = new AtomicInteger(); AtomicInteger shuffleTablesCall = new AtomicInteger(); - AutoRepair.shuffleFunc = (List list) -> { + UnifiedRepair.shuffleFunc = (List list) -> { if (!list.isEmpty()) { assertTrue(list.get(0) instanceof Keyspace || list.get(0) instanceof String); @@ -585,9 +585,9 @@ else if (list.get(0) instanceof String) } }; - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); config.setRepairMinInterval(repairType, "0s"); - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); assertEquals(1, shuffleKeyspacesCall.get()); assertEquals(5, shuffleTablesCall.get()); @@ -596,88 +596,88 @@ else if (list.get(0) instanceof String) @Test public void testRepairTakesLastRepairTimeFromDB() { - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); config.setMVRepairEnabled(repairType, true); long lastRepairTime = System.currentTimeMillis() - 1000; - AutoRepairUtils.insertNewRepairHistory(repairType, 0, lastRepairTime); - AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(0); + UnifiedRepairUtils.insertNewRepairHistory(repairType, 0, lastRepairTime); + UnifiedRepair.instance.repairStates.get(repairType).setLastRepairTime(0); config.setRepairMinInterval(repairType, "1h"); - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); // repair scheduler should not attempt to run repair as last repair time in DB is current time - 1s - assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair()); + assertEquals(0, UnifiedRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair()); // repair scheduler should load the repair time from the DB - assertEquals(lastRepairTime, AutoRepair.instance.repairStates.get(repairType).getLastRepairTime()); + assertEquals(lastRepairTime, UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime()); } @Test public void testRepairMaxRetries() { - when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); - when(autoRepairState.isSuccess()).thenReturn(false); - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + when(unifiedRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + when(unifiedRepairState.isSuccess()).thenReturn(false); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); AtomicInteger sleepCalls = new AtomicInteger(); - AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + UnifiedRepair.sleepFunc = (Long duration, TimeUnit unit) -> { sleepCalls.getAndIncrement(); assertEquals(TimeUnit.SECONDS, unit); assertEquals(config.getRepairRetryBackoff().toSeconds(), (long) duration); }; config.setRepairMinInterval(repairType, "0s"); - AutoRepair.instance.repairStates.put(repairType, autoRepairState); + UnifiedRepair.instance.repairStates.put(repairType, unifiedRepairState); - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); //system_auth.role_permissions,system_auth.network_permissions,system_auth.role_members,system_auth.roles, // system_auth.resource_role_permissons_index,system_traces.sessions,system_traces.events,ks.tbl, - // system_distributed.auto_repair_priority,system_distributed.repair_history,system_distributed.auto_repair_history, + // system_distributed.unified_repair_priority,system_distributed.repair_history,system_distributed.unified_repair_history, // system_distributed.view_build_status,system_distributed.parent_repair_history,system_distributed.partition_denylist int exptedTablesGoingThroughRepair = 18; assertEquals(config.getRepairMaxRetries()*exptedTablesGoingThroughRepair, sleepCalls.get()); - verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(0); - verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); - verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(exptedTablesGoingThroughRepair); + verify(unifiedRepairState, Mockito.times(1)).setSucceededTokenRangesCount(0); + verify(unifiedRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); + verify(unifiedRepairState, Mockito.times(1)).setFailedTokenRangesCount(exptedTablesGoingThroughRepair); } @Test public void testRepairSuccessAfterRetry() { - when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + when(unifiedRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); - AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); AtomicInteger sleepCalls = new AtomicInteger(); - AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + UnifiedRepair.sleepFunc = (Long duration, TimeUnit unit) -> { sleepCalls.getAndIncrement(); assertEquals(TimeUnit.SECONDS, unit); assertEquals(config.getRepairRetryBackoff().toSeconds(), (long) duration); }; - when(autoRepairState.isSuccess()).then((invocationOnMock) -> { + when(unifiedRepairState.isSuccess()).then((invocationOnMock) -> { if (sleepCalls.get() == 0) { return false; } return true; }); config.setRepairMinInterval(repairType, "0s"); - AutoRepair.instance.repairStates.put(repairType, autoRepairState); - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repairStates.put(repairType, unifiedRepairState); + UnifiedRepair.instance.repair(repairType); assertEquals(1, sleepCalls.get()); - verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(18); - verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); - verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(0); + verify(unifiedRepairState, Mockito.times(1)).setSucceededTokenRangesCount(18); + verify(unifiedRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); + verify(unifiedRepairState, Mockito.times(1)).setFailedTokenRangesCount(0); } @Test public void testRepairThrowsForIRWithMVReplay() { - AutoRepair.instance.setup(); + UnifiedRepair.instance.setup(); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); - if (repairType == AutoRepairConfig.RepairType.incremental) + if (repairType == UnifiedRepairConfig.RepairType.incremental) { try { - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); fail("Expected ConfigurationException"); } catch (ConfigurationException ignored) @@ -686,7 +686,7 @@ public void testRepairThrowsForIRWithMVReplay() } else { - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); } } @@ -694,14 +694,14 @@ public void testRepairThrowsForIRWithMVReplay() @Test public void testRepairThrowsForIRWithCDCReplay() { - AutoRepair.instance.setup(); + UnifiedRepair.instance.setup(); DatabaseDescriptor.setCDCOnRepairEnabled(true); - if (repairType == AutoRepairConfig.RepairType.incremental) + if (repairType == UnifiedRepairConfig.RepairType.incremental) { try { - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); fail("Expected ConfigurationException"); } catch (ConfigurationException ignored) @@ -710,7 +710,7 @@ public void testRepairThrowsForIRWithCDCReplay() } else { - AutoRepair.instance.repair(repairType); + UnifiedRepair.instance.repair(repairType); } } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateFactoryTest.java similarity index 76% rename from test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java rename to test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateFactoryTest.java index a0e5bdc45294..c1ee56e1065d 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java +++ b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateFactoryTest.java @@ -16,24 +16,24 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import org.junit.Test; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; -public class AutoRepairStateFactoryTest +public class UnifiedRepairStateFactoryTest { @Test public void testGetRepairState() { - AutoRepairState state = RepairType.getAutoRepairState(RepairType.full); + UnifiedRepairState state = RepairType.getUnifiedRepairState(RepairType.full); assert state instanceof FullRepairState; - state = RepairType.getAutoRepairState(RepairType.incremental); + state = RepairType.getUnifiedRepairState(RepairType.incremental); assert state instanceof IncrementalRepairState; } @@ -42,7 +42,7 @@ public void testGetRepairState() { public void testGetRepairStateSupportsAllRepairTypes() { for (RepairType repairType : RepairType.values()) { try { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); assertNotNull(state); } catch (IllegalArgumentException e) { assertNull(e); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateTest.java similarity index 72% rename from test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java rename to test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateTest.java index f0974dd83c1a..df60ffc4e484 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java +++ b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.util.Arrays; import java.util.Collection; @@ -33,9 +33,9 @@ import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; -import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.UnifiedRepairHistory; +import org.apache.cassandra.service.UnifiedRepairService; import org.apache.cassandra.utils.concurrent.Condition; import org.apache.cassandra.utils.progress.ProgressEvent; import org.apache.cassandra.utils.progress.ProgressEventType; @@ -50,7 +50,7 @@ import static org.mockito.MockitoAnnotations.initMocks; @RunWith(Parameterized.class) -public class AutoRepairStateTest extends CQLTester +public class UnifiedRepairStateTest extends CQLTester { private static final String testTable = "test"; @@ -68,15 +68,15 @@ public static Collection repairTypes() @Before public void setUp() { - AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + UnifiedRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); initMocks(this); createTable(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int)", KEYSPACE, testTable)); } @Test public void testGetRepairRunnable() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); - AutoRepairService.setup(); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + UnifiedRepairService.setup(); Runnable runnable = state.getRepairRunnable(KEYSPACE, ImmutableList.of(testTable), ImmutableSet.of(), false); @@ -86,7 +86,7 @@ public void testGetRepairRunnable() { @Test public void testProgressError() throws InterruptedException { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); when(progressEvent.getType()).thenReturn(ProgressEventType.ERROR); state.progress("test", progressEvent); @@ -98,7 +98,7 @@ public void testProgressError() throws InterruptedException @Test public void testProgress_progress() throws InterruptedException { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); when(progressEvent.getType()).thenReturn(ProgressEventType.PROGRESS); state.progress("test", progressEvent); @@ -111,7 +111,7 @@ public void testProgress_progress() throws InterruptedException @Test public void testProgress_complete() throws InterruptedException { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); when(progressEvent.getType()).thenReturn(ProgressEventType.COMPLETE); state.progress("test", progressEvent); @@ -123,7 +123,7 @@ public void testProgress_complete() throws InterruptedException @Test public void testWaitForRepairToComplete() throws Exception { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.condition.signalAll(); Condition finishedCondition = Condition.newOneTimeCondition(); Callable waitForRepairToComplete = () -> { @@ -139,7 +139,7 @@ public void testWaitForRepairToComplete() throws Exception @Test public void testGetLastRepairTime() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.lastRepairTimeInMs = 1; assertEquals(1, state.getLastRepairTime()); @@ -147,7 +147,7 @@ public void testGetLastRepairTime() { @Test public void testSetTotalTablesConsideredForRepair() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.setTotalTablesConsideredForRepair(1); @@ -156,7 +156,7 @@ public void testSetTotalTablesConsideredForRepair() { @Test public void testGetTotalTablesConsideredForRepair() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.totalTablesConsideredForRepair = 1; assertEquals(1, state.getTotalTablesConsideredForRepair()); @@ -164,7 +164,7 @@ public void testGetTotalTablesConsideredForRepair() { @Test public void testSetLastRepairTimeInMs() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.setLastRepairTime(1); @@ -173,7 +173,7 @@ public void testSetLastRepairTimeInMs() { @Test public void testGetClusterRepairTimeInSec() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.clusterRepairTimeInSec = 1; assertEquals(1, state.getClusterRepairTimeInSec()); @@ -181,7 +181,7 @@ public void testGetClusterRepairTimeInSec() { @Test public void testGetNodeRepairTimeInSec() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.nodeRepairTimeInSec = 1; assertEquals(1, state.getNodeRepairTimeInSec()); @@ -189,7 +189,7 @@ public void testGetNodeRepairTimeInSec() { @Test public void testSetRepairInProgress() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.setRepairInProgress(true); @@ -198,7 +198,7 @@ public void testSetRepairInProgress() { @Test public void testIsRepairInProgress() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.repairInProgress = true; assertTrue(state.isRepairInProgress()); @@ -206,7 +206,7 @@ public void testIsRepairInProgress() { @Test public void testSetSkippedTokenRangesCount() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.setSkippedTokenRangesCount(1); @@ -215,7 +215,7 @@ public void testSetSkippedTokenRangesCount() { @Test public void testGetSkippedTokenRangesCount() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.skippedTokenRangesCount = 1; assertEquals(1, state.getSkippedTokenRangesCount()); @@ -223,7 +223,7 @@ public void testGetSkippedTokenRangesCount() { @Test public void testGetLongestUnrepairedSecNull() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.longestUnrepairedNode = null; try @@ -236,10 +236,10 @@ public void testGetLongestUnrepairedSecNull() { @Test public void testGetLongestUnrepairedSec() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); - state.longestUnrepairedNode = new AutoRepairHistory(UUID.randomUUID(), "", 0, 1000, - null, 0, false); - AutoRepairState.timeFunc = () -> 2000L; + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + state.longestUnrepairedNode = new UnifiedRepairHistory(UUID.randomUUID(), "", 0, 1000, + null, 0, false); + UnifiedRepairState.timeFunc = () -> 2000L; try { @@ -251,7 +251,7 @@ public void testGetLongestUnrepairedSec() { @Test public void testSetTotalMVTablesConsideredForRepair() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.setTotalMVTablesConsideredForRepair(1); @@ -260,7 +260,7 @@ public void testSetTotalMVTablesConsideredForRepair() { @Test public void testGetTotalMVTablesConsideredForRepair() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.totalMVTablesConsideredForRepair = 1; assertEquals(1, state.getTotalMVTablesConsideredForRepair()); @@ -268,7 +268,7 @@ public void testGetTotalMVTablesConsideredForRepair() { @Test public void testSetNodeRepairTimeInSec() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.setNodeRepairTimeInSec(1); @@ -277,7 +277,7 @@ public void testSetNodeRepairTimeInSec() { @Test public void testSetClusterRepairTimeInSec() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.setClusterRepairTimeInSec(1); @@ -286,7 +286,7 @@ public void testSetClusterRepairTimeInSec() { @Test public void testSetRepairKeyspaceCount() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.setRepairKeyspaceCount(1); @@ -294,7 +294,7 @@ public void testSetRepairKeyspaceCount() { } @Test public void testGetRepairKeyspaceCount() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.repairKeyspaceCount = 1; assertEquals(1, state.getRepairKeyspaceCount()); @@ -302,8 +302,8 @@ public void testGetRepairKeyspaceCount() { @Test public void testSetLongestUnrepairedNode() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); - AutoRepairHistory history = new AutoRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + UnifiedRepairHistory history = new UnifiedRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); state.setLongestUnrepairedNode(history); @@ -312,7 +312,7 @@ public void testSetLongestUnrepairedNode() { @Test public void testSetSucceededTokenRangesCount() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.setSucceededTokenRangesCount(1); @@ -321,7 +321,7 @@ public void testSetSucceededTokenRangesCount() { @Test public void testGetSucceededTokenRangesCount() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.succeededTokenRangesCount = 1; assertEquals(1, state.getSucceededTokenRangesCount()); @@ -329,7 +329,7 @@ public void testGetSucceededTokenRangesCount() { @Test public void testSetFailedTokenRangesCount() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.setFailedTokenRangesCount(1); @@ -338,7 +338,7 @@ public void testSetFailedTokenRangesCount() { @Test public void testGetFailedTokenRangesCount() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.failedTokenRangesCount = 1; assertEquals(1, state.getFailedTokenRangesCount()); @@ -346,7 +346,7 @@ public void testGetFailedTokenRangesCount() { @Test public void isSuccess() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.success = true; assertTrue(state.isSuccess()); @@ -359,7 +359,7 @@ public void isSuccess() { @Test public void testWaitForRepairToCompleteDoesNotSetSuccessWhenProgressReceivesError() throws InterruptedException { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); when(progressEvent.getType()).thenReturn(ProgressEventType.ERROR); state.progress("test", progressEvent); @@ -372,7 +372,7 @@ public void testWaitForRepairToCompleteDoesNotSetSuccessWhenProgressReceivesErro @Test public void testResetWaitCondition() { - AutoRepairState state = RepairType.getAutoRepairState(repairType); + UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); state.condition.signalAll(); assertTrue(state.condition.isSignalled()); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairTest.java similarity index 79% rename from test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java rename to test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairTest.java index df9f105b615f..4e6e755a09f4 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.util.HashMap; import java.util.Map; @@ -34,38 +34,38 @@ import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.ReplicationParams; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; import org.apache.cassandra.schema.SchemaTestUtil; -import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.UnifiedRepairService; -import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.Util.setUnifiedRepairEnabled; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -public class AutoRepairTest extends CQLTester +public class UnifiedRepairTest extends CQLTester { @BeforeClass public static void setupClass() throws Exception { - setAutoRepairEnabled(true); + setUnifiedRepairEnabled(true); requireNetwork(); } @Before public void setup() { - AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + UnifiedRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); DatabaseDescriptor.setCDCOnRepairEnabled(false); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.full, true); - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); - AutoRepairService.setup(); + DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(RepairType.full, true); + DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(RepairType.incremental, true); + UnifiedRepairService.setup(); } @Test public void testSetup() { - AutoRepair instance = new AutoRepair(); + UnifiedRepair instance = new UnifiedRepair(); instance.setup(); assertEquals(RepairType.values().length, instance.repairExecutors.size()); @@ -81,7 +81,7 @@ public void testSetup() @Test public void testSafeGuardSetupCall() { - AutoRepair instance = new AutoRepair(); + UnifiedRepair instance = new UnifiedRepair(); // only one should be setup, and rest should be ignored instance.setup(); @@ -101,22 +101,22 @@ public void testSafeGuardSetupCall() @Test(expected = ConfigurationException.class) public void testSetupFailsWhenIREnabledWithCDCReplay() { - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(RepairType.incremental, true); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); DatabaseDescriptor.setCDCEnabled(true); DatabaseDescriptor.setCDCOnRepairEnabled(true); - AutoRepair instance = new AutoRepair(); + UnifiedRepair instance = new UnifiedRepair(); instance.setup(); } @Test(expected = ConfigurationException.class) public void testSetupFailsWhenIREnabledWithMVReplay() { - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(RepairType.incremental, true); DatabaseDescriptor.setCDCOnRepairEnabled(false); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); - AutoRepair instance = new AutoRepair(); + UnifiedRepair instance = new UnifiedRepair(); instance.setup(); } @@ -141,14 +141,14 @@ public void testCheckNTSreplicationNodeInsideOutsideDC() // case 1 : // node reside in "datacenter1" // keyspace has replica in "datacenter1" - Assert.assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + Assert.assertTrue(UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(ks)); } else if (ks.getName().equals(ksname2)) { // case 2 : // node reside in "datacenter1" // keyspace has replica in "datacenter2" - Assert.assertFalse(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + Assert.assertFalse(UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(ks)); } } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtilsTest.java similarity index 70% rename from test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java rename to test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtilsTest.java index 9a3c551c9bf6..2b6bd0e14114 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java +++ b/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtilsTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.autorepair; +package org.apache.cassandra.repair.unifiedrepair; import java.util.List; import java.util.Set; @@ -36,9 +36,9 @@ import org.apache.cassandra.db.marshal.UUIDType; import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils.CurrentRepairStatus; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.UnifiedRepairHistory; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.CurrentRepairStatus; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.QueryProcessor; @@ -49,14 +49,14 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.Util.setUnifiedRepairEnabled; import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_DELETE_HOSTS; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_FORCE_REPAIR; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_FINISH_TS; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_PRIORITY; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_START_TS; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_TURN; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_DELETE_HOSTS; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_FORCE_REPAIR; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_REPAIR_FINISH_TS; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_REPAIR_PRIORITY; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_REPAIR_START_TS; +import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_REPAIR_TURN; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -64,7 +64,7 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.when; -public class AutoRepairUtilsTest extends CQLTester +public class UnifiedRepairUtilsTest extends CQLTester { static RepairType repairType = RepairType.incremental; static UUID hostId; @@ -80,12 +80,12 @@ public class AutoRepairUtilsTest extends CQLTester public static void setupClass() throws Exception { SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); - setAutoRepairEnabled(true); + setUnifiedRepairEnabled(true); requireNetwork(); defaultSnitch = DatabaseDescriptor.getEndpointSnitch(); localEndpoint = FBUtilities.getBroadcastAddressAndPort(); hostId = StorageService.instance.getHostIdForEndpoint(localEndpoint); - StorageService.instance.doAutoRepairSetup(); + StorageService.instance.doUnifiedRepairSetup(); } @Before @@ -95,15 +95,15 @@ public void setup() QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", "ks")); QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i))", "ks", "tbl")); - AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + UnifiedRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); MockitoAnnotations.initMocks(this); DatabaseDescriptor.setEndpointSnitch(defaultSnitch); QueryProcessor.executeInternal(String.format( "TRUNCATE %s.%s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY)); + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY)); QueryProcessor.executeInternal(String.format( "TRUNCATE %s.%s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY)); + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY)); } @Test @@ -111,14 +111,14 @@ public void testSetForceRepair() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, force_repair) VALUES ('%s', %s, false)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); - AutoRepairUtils.setForceRepair(repairType, ImmutableSet.of(localEndpoint)); + UnifiedRepairUtils.setForceRepair(repairType, ImmutableSet.of(localEndpoint)); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT force_repair FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -128,11 +128,11 @@ public void testSetForceRepair() @Test public void testSetForceRepairNewNode() { - AutoRepairUtils.setForceRepairNewNode(repairType); + UnifiedRepairUtils.setForceRepairNewNode(repairType); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT force_repair FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -145,14 +145,14 @@ public void testClearDeleteHosts() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, delete_hosts, delete_hosts_update_time) VALUES ('%s', %s, { %s }, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId, hostId)); - AutoRepairUtils.clearDeleteHosts(repairType, hostId); + UnifiedRepairUtils.clearDeleteHosts(repairType, hostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT delete_hosts FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -161,23 +161,23 @@ public void testClearDeleteHosts() } @Test - public void testGetAutoRepairHistoryForLocalGroup() + public void testGetUnifiedRepairHistoryForLocalGroup() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, force_repair) VALUES ('%s', %s, false)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); - List history = AutoRepairUtils.getAutoRepairHistory(repairType); + List history = UnifiedRepairUtils.getUnifiedRepairHistory(repairType); assertNotNull(history); assertEquals(1, history.size()); assertEquals(hostId, history.get(0).hostId); } @Test - public void testGetAutoRepairHistoryForLocalGroup_empty_history() + public void testGetUnifiedRepairHistoryForLocalGroup_empty_history() { - List history = AutoRepairUtils.getAutoRepairHistory(repairType); + List history = UnifiedRepairUtils.getUnifiedRepairHistory(repairType); assertNull(history); } @@ -189,22 +189,22 @@ public void testGetCurrentRepairStatus() UUID regularRepair = UUID.randomUUID(); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, force_repair, repair_start_ts) VALUES ('%s', %s, true, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), forceRepair)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, repair_start_ts) VALUES ('%s', %s, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), regularRepair)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, repairType.toString(), regularRepair)); - CurrentRepairStatus status = AutoRepairUtils.getCurrentRepairStatus(repairType); + CurrentRepairStatus status = UnifiedRepairUtils.getCurrentRepairStatus(repairType); assertNotNull(status); assertEquals(1, status.historiesWithoutOnGoingRepair.size()); @@ -220,7 +220,7 @@ public void testGetCurrentRepairStatus() @Test public void testGetHostIdsInCurrentRing() { - TreeSet hosts = AutoRepairUtils.getHostIdsInCurrentRing(repairType); + TreeSet hosts = UnifiedRepairUtils.getHostIdsInCurrentRing(repairType); assertNotNull(hosts); assertEquals(1, hosts.size()); @@ -232,13 +232,13 @@ public void testGetHostIdsInCurrentRing_multiple_nodes() { InetAddressAndPort ignoredEndpoint = localEndpoint.withPort(localEndpoint.getPort() + 1); InetAddressAndPort deadEndpoint = localEndpoint.withPort(localEndpoint.getPort() + 2); - DatabaseDescriptor.getAutoRepairConfig().setIgnoreDCs(repairType, ImmutableSet.of("dc2")); + DatabaseDescriptor.getUnifiedRepairConfig().setIgnoreDCs(repairType, ImmutableSet.of("dc2")); DatabaseDescriptor.setEndpointSnitch(snitchMock); when(snitchMock.getDatacenter(localEndpoint)).thenReturn("dc1"); when(snitchMock.getDatacenter(ignoredEndpoint)).thenReturn("dc2"); when(snitchMock.getDatacenter(deadEndpoint)).thenReturn("dc1"); - TreeSet hosts = AutoRepairUtils.getHostIdsInCurrentRing(repairType, ImmutableSet.of(new NodeAddresses(localEndpoint), new NodeAddresses(ignoredEndpoint), new NodeAddresses(deadEndpoint))); + TreeSet hosts = UnifiedRepairUtils.getHostIdsInCurrentRing(repairType, ImmutableSet.of(new NodeAddresses(localEndpoint), new NodeAddresses(ignoredEndpoint), new NodeAddresses(deadEndpoint))); assertNotNull(hosts); assertEquals(1, hosts.size()); @@ -251,93 +251,93 @@ public void testGetHostWithLongestUnrepairTime() UUID otherHostId = UUID.randomUUID(); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, repair_finish_ts) VALUES ('%s', %s, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), otherHostId)); - AutoRepairHistory history = AutoRepairUtils.getHostWithLongestUnrepairTime(repairType); + UnifiedRepairHistory history = UnifiedRepairUtils.getHostWithLongestUnrepairTime(repairType); assertEquals(hostId, history.hostId); } @Test - public void testGetMaxNumberOfNodeRunAutoRepairInGroup_0_group_size() + public void testGetMaxNumberOfNodeRunUnifiedRepairInGroup_0_group_size() { - DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 2); + DatabaseDescriptor.getUnifiedRepairConfig().setParallelRepairCount(repairType, 2); - int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepair(repairType, 0); + int count = UnifiedRepairUtils.getMaxNumberOfNodeRunUnifiedRepair(repairType, 0); assertEquals(2, count); } @Test - public void testGetMaxNumberOfNodeRunAutoRepairInGroup_percentage() + public void testGetMaxNumberOfNodeRunUnifiedRepairInGroup_percentage() { - DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 2); - DatabaseDescriptor.getAutoRepairConfig().setParallelRepairPercentage(repairType, 50); + DatabaseDescriptor.getUnifiedRepairConfig().setParallelRepairCount(repairType, 2); + DatabaseDescriptor.getUnifiedRepairConfig().setParallelRepairPercentage(repairType, 50); - int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepair(repairType, 10); + int count = UnifiedRepairUtils.getMaxNumberOfNodeRunUnifiedRepair(repairType, 10); assertEquals(5, count); } @Test - public void testDeleteAutoRepairHistory() + public void testDeleteUnifiedRepairHistory() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); - AutoRepairUtils.deleteAutoRepairHistory(repairType, hostId); + UnifiedRepairUtils.deleteUnifiedRepairHistory(repairType, hostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(0, result.size()); } @Test - public void testUpdateStartAutoRepairHistory() + public void testUpdateStartUnifiedRepairHistory() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); - AutoRepairUtils.updateStartAutoRepairHistory(repairType, hostId, 123, AutoRepairUtils.RepairTurn.MY_TURN); + UnifiedRepairUtils.updateStartUnifiedRepairHistory(repairType, hostId, 123, UnifiedRepairUtils.RepairTurn.MY_TURN); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT repair_start_ts, repair_turn FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); UntypedResultSet.Row row = result.one(); assertEquals(123, row.getLong(COL_REPAIR_START_TS, 0)); - assertEquals(AutoRepairUtils.RepairTurn.MY_TURN.toString(), row.getString(COL_REPAIR_TURN)); + assertEquals(UnifiedRepairUtils.RepairTurn.MY_TURN.toString(), row.getString(COL_REPAIR_TURN)); } @Test - public void testUpdateFinishAutoRepairHistory() + public void testUpdateFinishUnifiedRepairHistory() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); - AutoRepairUtils.updateFinishAutoRepairHistory(repairType, hostId, 123); + UnifiedRepairUtils.updateFinishUnifiedRepairHistory(repairType, hostId, 123); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT repair_finish_ts FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -350,14 +350,14 @@ public void testAddHostIdToDeleteHosts() UUID otherHostId = UUID.randomUUID(); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), otherHostId)); - AutoRepairUtils.addHostIdToDeleteHosts(repairType, hostId, otherHostId); + UnifiedRepairUtils.addHostIdToDeleteHosts(repairType, hostId, otherHostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType.toString(), otherHostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -370,11 +370,11 @@ public void testAddHostIdToDeleteHosts() @Test public void testAddPriorityHost() { - AutoRepairUtils.addPriorityHosts(repairType, ImmutableSet.of(localEndpoint)); + UnifiedRepairUtils.addPriorityHosts(repairType, ImmutableSet.of(localEndpoint)); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s'", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, repairType.toString())); assertNotNull(result); assertEquals(1, result.size()); @@ -389,14 +389,14 @@ public void testRemovePriorityStatus() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, repairType.toString(), hostId)); - AutoRepairUtils.removePriorityStatus(repairType, hostId); + UnifiedRepairUtils.removePriorityStatus(repairType, hostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s'", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, repairType.toString())); assertNotNull(result); assertEquals(1, result.size()); @@ -409,10 +409,10 @@ public void testGetPriorityHosts() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, repairType.toString(), hostId)); - Set hosts = AutoRepairUtils.getPriorityHosts(repairType); + Set hosts = UnifiedRepairUtils.getPriorityHosts(repairType); assertNotNull(hosts); assertEquals(1, hosts.size()); @@ -424,29 +424,29 @@ public void testCheckNodeContainsKeyspaceReplica() { Keyspace ks = Keyspace.open("ks"); - assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + assertTrue(UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(ks)); } @Test public void testTableMaxRepairTimeExceeded() { - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairTableMaxRepairTime(repairType, "0s"); + DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairTableMaxRepairTime(repairType, "0s"); - assertTrue(AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, 0)); + assertTrue(UnifiedRepairUtils.tableMaxRepairTimeExceeded(repairType, 0)); } @Test public void testKeyspaceMaxRepairTimeExceeded() { - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairTableMaxRepairTime(repairType, "0s"); + DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairTableMaxRepairTime(repairType, "0s"); - assertTrue(AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, 0, 1)); + assertTrue(UnifiedRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, 0, 1)); } @Test public void testGetLastRepairFinishTime() { - AutoRepairHistory history = new AutoRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); + UnifiedRepairHistory history = new UnifiedRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); assertEquals(0, history.getLastRepairFinishTime()); @@ -460,21 +460,21 @@ public void testMyTurnToRunRepairShouldReturnMyTurnWhenRepairOngoing() { UUID myID = UUID.randomUUID(); UUID otherID = UUID.randomUUID(); - DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 5); + DatabaseDescriptor.getUnifiedRepairConfig().setParallelRepairCount(repairType, 5); long currentMillis = System.currentTimeMillis(); // finish time less than start time means that repair is ongoing - AutoRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); + UnifiedRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); // finish time is larger than start time means that repair for other node is finished - AutoRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); + UnifiedRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); - assertEquals(AutoRepairUtils.RepairTurn.MY_TURN, AutoRepairUtils.myTurnToRunRepair(repairType, myID)); + assertEquals(UnifiedRepairUtils.RepairTurn.MY_TURN, UnifiedRepairUtils.myTurnToRunRepair(repairType, myID)); } @Test public void testLocalStrategyAndNetworkKeyspace() { - assertFalse(AutoRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open("system"))); - assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open(KEYSPACE))); + assertFalse(UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open("system"))); + assertTrue(UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open(KEYSPACE))); } @Test @@ -483,10 +483,10 @@ public void testGetLastRepairTimeForNode() UUID myID = UUID.randomUUID(); UUID otherID = UUID.randomUUID(); long currentMillis = System.currentTimeMillis(); - AutoRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); - AutoRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); + UnifiedRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); + UnifiedRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); - assertEquals(currentMillis - 100, AutoRepairUtils.getLastRepairTimeForNode(repairType, myID)); + assertEquals(currentMillis - 100, UnifiedRepairUtils.getLastRepairTimeForNode(repairType, myID)); } @Test @@ -494,6 +494,6 @@ public void testGetLastRepairTimeForNodeWhenHistoryIsEmpty() { UUID myID = UUID.randomUUID(); - assertEquals(0, AutoRepairUtils.getLastRepairTimeForNode(repairType, myID)); + assertEquals(0, UnifiedRepairUtils.getLastRepairTimeForNode(repairType, myID)); } } diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java deleted file mode 100644 index 054f136dad75..000000000000 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java +++ /dev/null @@ -1,118 +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; - -import org.junit.Before; -import org.junit.Test; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; - -import static org.junit.Assert.assertEquals; - -public class AutoRepairServiceBasicTest extends CQLTester { - private static AutoRepairService autoRepairService; - private static AutoRepairConfig config; - - @Before - public void setUp() { - DatabaseDescriptor.setCDCOnRepairEnabled(false); - DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); - DatabaseDescriptor.setMaterializedViewsEnabled(false); - DatabaseDescriptor.setCDCEnabled(false); - config = new AutoRepairConfig(); - autoRepairService = new AutoRepairService(); - autoRepairService.config = config; - } - - @Test - public void testSetup() { - AutoRepairService.instance.config = null; - - AutoRepairService.setup(); - - assertEquals(DatabaseDescriptor.getAutoRepairConfig(), AutoRepairService.instance.config); - } - - @Test - public void testGetAutoRepairConfigReturnsConfig() { - assertEquals(config, autoRepairService.getAutoRepairConfig()); - } - - @Test - public void testsetAutoRepairHistoryClearDeleteHostsBufferInSecV2() { - autoRepairService.setAutoRepairHistoryClearDeleteHostsBufferDuration("100s"); - - assertEquals(100, config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds()); - } - - - @Test - public void testsetAutoRepairMaxRetriesCount() { - autoRepairService.setAutoRepairMaxRetriesCount(101); - - assertEquals(101, config.getRepairMaxRetries()); - } - - - @Test - public void testsetAutoRepairRetryBackoffInSec() { - autoRepairService.setAutoRepairRetryBackoff("102s"); - - assertEquals(102, config.getRepairRetryBackoff().toSeconds()); - } - - @Test(expected = ConfigurationException.class) - public void testSetAutoRepairEnabledThrowsWithSchedulerDisabled() { - autoRepairService.config = new AutoRepairConfig(false); - - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); - } - - @Test(expected = ConfigurationException.class) - public void testSetAutoRepairEnabledThrowsForIRWithMVReplay() { - autoRepairService.config = new AutoRepairConfig(true); - DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); - } - - @Test - public void testSetAutoRepairEnabledDoesNotThrowForIRWithMVReplayDisabled() { - autoRepairService.config = new AutoRepairConfig(true); - DatabaseDescriptor.setMaterializedViewsEnabled(true); - DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); - } - - @Test(expected = ConfigurationException.class) - public void testSetAutoRepairEnabledThrowsForIRWithCDCReplay() { - autoRepairService.config = new AutoRepairConfig(true); - DatabaseDescriptor.setCDCOnRepairEnabled(true); - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); - } - - @Test - public void testSetAutoRepairEnabledDoesNotThrowForIRWithCDCReplayDisabled() { - autoRepairService.config = new AutoRepairConfig(true); - DatabaseDescriptor.setCDCEnabled(true); - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); - } -} diff --git a/test/unit/org/apache/cassandra/service/UnifiedRepairServiceBasicTest.java b/test/unit/org/apache/cassandra/service/UnifiedRepairServiceBasicTest.java new file mode 100644 index 000000000000..dc7c91d0edef --- /dev/null +++ b/test/unit/org/apache/cassandra/service/UnifiedRepairServiceBasicTest.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 org.apache.cassandra.service; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; + +import static org.junit.Assert.assertEquals; + +public class UnifiedRepairServiceBasicTest extends CQLTester { + private static UnifiedRepairService unifiedRepairService; + private static UnifiedRepairConfig config; + + @Before + public void setUp() { + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + DatabaseDescriptor.setCDCEnabled(false); + config = new UnifiedRepairConfig(); + unifiedRepairService = new UnifiedRepairService(); + unifiedRepairService.config = config; + } + + @Test + public void testSetup() { + UnifiedRepairService.instance.config = null; + + UnifiedRepairService.setup(); + + assertEquals(DatabaseDescriptor.getUnifiedRepairConfig(), UnifiedRepairService.instance.config); + } + + @Test + public void testGetUnifiedRepairConfigReturnsConfig() { + assertEquals(config, unifiedRepairService.getUnifiedRepairConfig()); + } + + @Test + public void testsetUnifiedRepairHistoryClearDeleteHostsBufferInSecV2() { + unifiedRepairService.setUnifiedRepairHistoryClearDeleteHostsBufferDuration("100s"); + + assertEquals(100, config.getUnifiedRepairHistoryClearDeleteHostsBufferInterval().toSeconds()); + } + + + @Test + public void testsetUnifiedRepairMaxRetriesCount() { + unifiedRepairService.setUnifiedRepairMaxRetriesCount(101); + + assertEquals(101, config.getRepairMaxRetries()); + } + + + @Test + public void testsetUnifiedRepairRetryBackoffInSec() { + unifiedRepairService.setUnifiedRepairRetryBackoff("102s"); + + assertEquals(102, config.getRepairRetryBackoff().toSeconds()); + } + + @Test(expected = ConfigurationException.class) + public void testSetUnifiedRepairEnabledThrowsWithSchedulerDisabled() { + unifiedRepairService.config = new UnifiedRepairConfig(false); + + unifiedRepairService.setUnifiedRepairEnabled(UnifiedRepairConfig.RepairType.incremental, true); + } + + @Test(expected = ConfigurationException.class) + public void testSetUnifiedRepairEnabledThrowsForIRWithMVReplay() { + unifiedRepairService.config = new UnifiedRepairConfig(true); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); + unifiedRepairService.setUnifiedRepairEnabled(UnifiedRepairConfig.RepairType.incremental, true); + } + + @Test + public void testSetUnifiedRepairEnabledDoesNotThrowForIRWithMVReplayDisabled() { + unifiedRepairService.config = new UnifiedRepairConfig(true); + DatabaseDescriptor.setMaterializedViewsEnabled(true); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); + unifiedRepairService.setUnifiedRepairEnabled(UnifiedRepairConfig.RepairType.incremental, true); + } + + @Test(expected = ConfigurationException.class) + public void testSetUnifiedRepairEnabledThrowsForIRWithCDCReplay() { + unifiedRepairService.config = new UnifiedRepairConfig(true); + DatabaseDescriptor.setCDCOnRepairEnabled(true); + unifiedRepairService.setUnifiedRepairEnabled(UnifiedRepairConfig.RepairType.incremental, true); + } + + @Test + public void testSetUnifiedRepairEnabledDoesNotThrowForIRWithCDCReplayDisabled() { + unifiedRepairService.config = new UnifiedRepairConfig(true); + DatabaseDescriptor.setCDCEnabled(true); + unifiedRepairService.setUnifiedRepairEnabled(UnifiedRepairConfig.RepairType.incremental, true); + } +} diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java b/test/unit/org/apache/cassandra/service/UnifiedRepairServiceRepairTypeTest.java similarity index 72% rename from test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java rename to test/unit/org/apache/cassandra/service/UnifiedRepairServiceRepairTypeTest.java index 7c8645149adc..d91c89ee8619 100644 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java +++ b/test/unit/org/apache/cassandra/service/UnifiedRepairServiceRepairTypeTest.java @@ -20,8 +20,8 @@ import com.google.common.collect.ImmutableSet; import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -33,44 +33,44 @@ import java.util.Set; import java.util.UUID; -import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.Util.setUnifiedRepairEnabled; import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; import static org.junit.Assert.assertEquals; @RunWith(Parameterized.class) -public class AutoRepairServiceRepairTypeTest extends CQLTester { +public class UnifiedRepairServiceRepairTypeTest extends CQLTester { @Parameterized.Parameter() - public AutoRepairConfig.RepairType repairType; + public UnifiedRepairConfig.RepairType repairType; private final UUID host1 = UUID.fromString("00000000-0000-0000-0000-000000000001"); private final UUID host2 = UUID.fromString("00000000-0000-0000-0000-000000000002"); - private AutoRepairService instance; + private UnifiedRepairService instance; @Parameterized.Parameters(name = "repairType={0}") - public static Collection repairTypes() { - return Arrays.asList(AutoRepairConfig.RepairType.values()); + public static Collection repairTypes() { + return Arrays.asList(UnifiedRepairConfig.RepairType.values()); } @BeforeClass public static void setupClass() throws Exception { SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); - setAutoRepairEnabled(true); + setUnifiedRepairEnabled(true); requireNetwork(); } @Before public void setUpTest() { - AutoRepairUtils.setup(); - instance = new AutoRepairService(); + UnifiedRepairUtils.setup(); + instance = new UnifiedRepairService(); } @Test public void testGetOnGoingRepairHostIdsTest() { long now = System.currentTimeMillis(); - AutoRepairUtils.insertNewRepairHistory(repairType, host1, now, now - 1000000); - AutoRepairUtils.insertNewRepairHistory(repairType, host2, now, now - 1000000); + UnifiedRepairUtils.insertNewRepairHistory(repairType, host1, now, now - 1000000); + UnifiedRepairUtils.insertNewRepairHistory(repairType, host2, now, now - 1000000); Set hosts = instance.getOnGoingRepairHostIds(repairType); diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java b/test/unit/org/apache/cassandra/service/UnifiedRepairServiceSetterTest.java similarity index 59% rename from test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java rename to test/unit/org/apache/cassandra/service/UnifiedRepairServiceSetterTest.java index f34e1f0a7071..57b5500b665e 100644 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java +++ b/test/unit/org/apache/cassandra/service/UnifiedRepairServiceSetterTest.java @@ -24,8 +24,8 @@ import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; -import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.junit.Before; @@ -43,48 +43,48 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.Util.setUnifiedRepairEnabled; import static org.junit.Assert.assertEquals; @RunWith(Parameterized.class) -public class AutoRepairServiceSetterTest extends CQLTester { - private static final AutoRepairConfig config = new AutoRepairConfig(true); +public class UnifiedRepairServiceSetterTest extends CQLTester { + private static final UnifiedRepairConfig config = new UnifiedRepairConfig(true); @Parameterized.Parameter - public AutoRepairConfig.RepairType repairType; + public UnifiedRepairConfig.RepairType repairType; @Parameterized.Parameter(1) public T arg; @Parameterized.Parameter(2) - public BiConsumer setter; + public BiConsumer setter; @Parameterized.Parameter(3) - public Function getter; + public Function getter; @Parameterized.Parameters(name = "{index}: repairType={0}, arg={1}") public static Collection testCases() { DatabaseDescriptor.setConfig(DatabaseDescriptor.loadConfig()); return Stream.of( - forEachRepairType(true, AutoRepairService.instance::setAutoRepairEnabled, config::isAutoRepairEnabled), - forEachRepairType(100, AutoRepairService.instance::setRepairThreads, config::getRepairThreads), - forEachRepairType(200, AutoRepairService.instance::setRepairSubRangeNum, config::getRepairSubRangeNum), - forEachRepairType(400, AutoRepairService.instance::setRepairSSTableCountHigherThreshold, config::getRepairSSTableCountHigherThreshold), - forEachRepairType(ImmutableSet.of("dc1", "dc2"), AutoRepairService.instance::setIgnoreDCs, config::getIgnoreDCs), - forEachRepairType(true, AutoRepairService.instance::setPrimaryTokenRangeOnly, config::getRepairPrimaryTokenRangeOnly), - forEachRepairType(600, AutoRepairService.instance::setParallelRepairPercentage, config::getParallelRepairPercentage), - forEachRepairType(700, AutoRepairService.instance::setParallelRepairCount, config::getParallelRepairCount), - forEachRepairType(true, AutoRepairService.instance::setMVRepairEnabled, config::getMVRepairEnabled), - forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setRepairPriorityForHosts, AutoRepairUtils::getPriorityHosts), - forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setForceRepairForHosts, AutoRepairServiceSetterTest::isLocalHostForceRepair) + forEachRepairType(true, UnifiedRepairService.instance::setUnifiedRepairEnabled, config::isUnifiedRepairEnabled), + forEachRepairType(100, UnifiedRepairService.instance::setRepairThreads, config::getRepairThreads), + forEachRepairType(200, UnifiedRepairService.instance::setRepairSubRangeNum, config::getRepairSubRangeNum), + forEachRepairType(400, UnifiedRepairService.instance::setRepairSSTableCountHigherThreshold, config::getRepairSSTableCountHigherThreshold), + forEachRepairType(ImmutableSet.of("dc1", "dc2"), UnifiedRepairService.instance::setIgnoreDCs, config::getIgnoreDCs), + forEachRepairType(true, UnifiedRepairService.instance::setPrimaryTokenRangeOnly, config::getRepairPrimaryTokenRangeOnly), + forEachRepairType(600, UnifiedRepairService.instance::setParallelRepairPercentage, config::getParallelRepairPercentage), + forEachRepairType(700, UnifiedRepairService.instance::setParallelRepairCount, config::getParallelRepairCount), + forEachRepairType(true, UnifiedRepairService.instance::setMVRepairEnabled, config::getMVRepairEnabled), + forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), UnifiedRepairService.instance::setRepairPriorityForHosts, UnifiedRepairUtils::getPriorityHosts), + forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), UnifiedRepairService.instance::setForceRepairForHosts, UnifiedRepairServiceSetterTest::isLocalHostForceRepair) ).flatMap(Function.identity()).collect(Collectors.toList()); } - private static Set isLocalHostForceRepair(AutoRepairConfig.RepairType type) { + private static Set isLocalHostForceRepair(UnifiedRepairConfig.RepairType type) { UUID hostId = StorageService.instance.getHostIdForEndpoint(InetAddressAndPort.getLocalHost()); UntypedResultSet resultSet = QueryProcessor.executeInternal(String.format( - "SELECT force_repair FROM %s.%s WHERE host_id = %s and repair_type = '%s'", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, hostId, type)); + "SELECT force_repair FROM %s.%s WHERE host_id = %s and repair_type = '%s'", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, hostId, type)); if (!resultSet.isEmpty() && resultSet.one().getBoolean("force_repair")) { return ImmutableSet.of(InetAddressAndPort.getLocalHost()); @@ -92,9 +92,9 @@ private static Set isLocalHostForceRepair(AutoRepairConfig.R return ImmutableSet.of(); } - private static Stream forEachRepairType(T arg, BiConsumer setter, Function getter) { - Object[][] testCases = new Object[AutoRepairConfig.RepairType.values().length][4]; - for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { + private static Stream forEachRepairType(T arg, BiConsumer setter, Function getter) { + Object[][] testCases = new Object[UnifiedRepairConfig.RepairType.values().length][4]; + for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) { testCases[repairType.ordinal()] = new Object[]{repairType, arg, setter, getter}; } @@ -104,22 +104,22 @@ private static Stream forEachRepairType(T arg, BiConsumer verifyFunc; + public Consumer verifyFunc; @Parameterized.Parameters(name = "repairType={0},paramType={1}") public static Collection testCases() { return Stream.of( - forEachRepairType("enabled", "true", (type) -> verify(probe, times(1)).setAutoRepairEnabled(type, true)), + forEachRepairType("enabled", "true", (type) -> verify(probe, times(1)).setUnifiedRepairEnabled(type, true)), forEachRepairType("number_of_repair_threads", "1", (type) -> verify(probe, times(1)).setRepairThreads(type, 1)), forEachRepairType("number_of_subranges", "2", (type) -> verify(probe, times(1)).setRepairSubRangeNum(type, 2)), forEachRepairType("min_repair_interval", "3h", (type) -> verify(probe, times(1)).setRepairMinInterval(type, "3h")), forEachRepairType("sstable_upper_threshold", "4", (type) -> verify(probe, times(1)).setRepairSSTableCountHigherThreshold(type, 4)), - forEachRepairType("table_max_repair_time", "5s", (type) -> verify(probe, times(1)).setAutoRepairTableMaxRepairTime(type, "5s")), + forEachRepairType("table_max_repair_time", "5s", (type) -> verify(probe, times(1)).setUnifiedRepairTableMaxRepairTime(type, "5s")), forEachRepairType("repair_primary_token_range_only", "true", (type) -> verify(probe, times(1)).setPrimaryTokenRangeOnly(type, true)), forEachRepairType("parallel_repair_count", "6", (type) -> verify(probe, times(1)).setParallelRepairCount(type, 6)), forEachRepairType("parallel_repair_percentage", "7", (type) -> verify(probe, times(1)).setParallelRepairPercentage(type, 7)), forEachRepairType("mv_repair_enabled", "true", (type) -> verify(probe, times(1)).setMVRepairEnabled(type, true)), - forEachRepairType("ignore_dcs", "dc1,dc2", (type) -> verify(probe, times(1)).setAutoRepairIgnoreDCs(type, ImmutableSet.of("dc1", "dc2"))) + forEachRepairType("ignore_dcs", "dc1,dc2", (type) -> verify(probe, times(1)).setUnifiedRepairIgnoreDCs(type, ImmutableSet.of("dc1", "dc2"))) ).flatMap(Function.identity()).collect(Collectors.toList()); } - private static Stream forEachRepairType(String paramType, String paramVal, Consumer verifyFunc) + private static Stream forEachRepairType(String paramType, String paramVal, Consumer verifyFunc) { - Object[][] testCases = new Object[AutoRepairConfig.RepairType.values().length][4]; - for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) + Object[][] testCases = new Object[UnifiedRepairConfig.RepairType.values().length][4]; + for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) { testCases[repairType.ordinal()] = new Object[]{ repairType, paramType, paramVal, verifyFunc }; } @@ -316,12 +316,12 @@ public void test() verifyFunc.accept(repairType); - // test scenario when auto repair is disabled - when(probe.getAutoRepairConfig()).thenReturn(new AutoRepairConfig(false)); + // test scenario when unified repair is disabled + when(probe.getUnifiedRepairConfig()).thenReturn(new UnifiedRepairConfig(false)); cmd.execute(probe); - // test new calls are not made when auto repair is disabled + // test new calls are not made when unified repair is disabled verifyFunc.accept(repairType); } } diff --git a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java b/test/unit/org/apache/cassandra/tools/nodetool/UnifiedRepairStatusTest.java similarity index 76% rename from test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java rename to test/unit/org/apache/cassandra/tools/nodetool/UnifiedRepairStatusTest.java index 16b12ee2350e..a84e24a3a9ae 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/UnifiedRepairStatusTest.java @@ -30,33 +30,33 @@ import org.junit.runners.Parameterized; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.Output; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import static org.apache.cassandra.Util.setAutoRepairEnabled; +import static org.apache.cassandra.Util.setUnifiedRepairEnabled; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.when; @RunWith(Parameterized.class) -public class AutoRepairStatusTest +public class UnifiedRepairStatusTest { @Mock private static NodeProbe probe; private ByteArrayOutputStream cmdOutput; - private static AutoRepairStatus cmd; + private static UnifiedRepairStatus cmd; @Parameterized.Parameter() - public AutoRepairConfig.RepairType repairType; + public UnifiedRepairConfig.RepairType repairType; @Parameterized.Parameters(name = "repairType={0}") - public static Collection repairTypes() + public static Collection repairTypes() { - return Arrays.asList(AutoRepairConfig.RepairType.values()); + return Arrays.asList(UnifiedRepairConfig.RepairType.values()); } @Before @@ -66,13 +66,13 @@ public void setUp() throws Exception cmdOutput = new ByteArrayOutputStream(); PrintStream out = new PrintStream(cmdOutput); when(probe.output()).thenReturn(new Output(out, out)); - cmd = new AutoRepairStatus(); + cmd = new UnifiedRepairStatus(); DatabaseDescriptor.daemonInitialization(); DatabaseDescriptor.loadConfig(); - setAutoRepairEnabled(true); - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.full, true); - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); - when(probe.getAutoRepairConfig()).thenReturn(DatabaseDescriptor.getAutoRepairConfig()); + setUnifiedRepairEnabled(true); + DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(UnifiedRepairConfig.RepairType.full, true); + DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(UnifiedRepairConfig.RepairType.incremental, true); + when(probe.getUnifiedRepairConfig()).thenReturn(DatabaseDescriptor.getUnifiedRepairConfig()); } @Test(expected = IllegalArgumentException.class) From 5c7f3944f5d06413d7b7e360026d4d015c33d9e1 Mon Sep 17 00:00:00 2001 From: Jaydeepkumar Chovatia Date: Thu, 21 Nov 2024 09:17:30 -0800 Subject: [PATCH 058/115] Revert "Rename AutoRepair --> UnifiedRepair" This reverts commit f9f6971336c26e1bcae4110f5e0926b3e92b4565. --- .../pages/managing/operating/metrics.adoc | 14 +- .../org/apache/cassandra/config/Config.java | 4 +- .../cassandra/config/DatabaseDescriptor.java | 6 +- .../statements/schema/TableAttributes.java | 8 +- ...airMetrics.java => AutoRepairMetrics.java} | 44 +- ...ger.java => AutoRepairMetricsManager.java} | 10 +- .../metrics/CassandraMetricsRegistry.java | 2 +- .../AutoRepair.java} | 120 +++--- .../AutoRepairConfig.java} | 46 +- .../AutoRepairState.java} | 38 +- .../AutoRepairUtils.java} | 186 ++++----- .../DefaultAutoRepairTokenSplitter.java} | 12 +- .../IAutoRepairTokenRangeSplitter.java} | 10 +- ...epairParams.java => AutoRepairParams.java} | 30 +- .../cassandra/schema/SchemaKeyspace.java | 10 +- .../schema/SystemDistributedKeyspace.java | 44 +- .../apache/cassandra/schema/TableParams.java | 76 ++-- ...airService.java => AutoRepairService.java} | 50 +-- ...MBean.java => AutoRepairServiceMBean.java} | 20 +- .../cassandra/service/CassandraDaemon.java | 2 +- .../cassandra/service/StorageService.java | 14 +- .../tcm/sequences/BootstrapAndJoin.java | 4 +- .../tcm/sequences/BootstrapAndReplace.java | 4 +- .../tcm/sequences/ReplaceSameAddress.java | 4 +- .../org/apache/cassandra/tools/NodeProbe.java | 94 ++--- .../org/apache/cassandra/tools/NodeTool.java | 6 +- ...epairStatus.java => AutoRepairStatus.java} | 14 +- ...irConfig.java => GetAutoRepairConfig.java} | 22 +- ...irConfig.java => SetAutoRepairConfig.java} | 28 +- .../apache/cassandra/utils/FBUtilities.java | 12 +- ...Test.java => AutoRepairSchedulerTest.java} | 34 +- test/unit/org/apache/cassandra/Util.java | 8 +- .../config/DatabaseDescriptorRefTest.java | 16 +- .../config/YamlConfigurationLoaderTest.java | 14 +- .../AutoRepairConfigTest.java} | 78 ++-- ...efaultTokenSplitterParameterizedTest.java} | 28 +- .../AutoRepairKeyspaceTest.java} | 10 +- .../AutoRepairParameterizedTest.java} | 394 +++++++++--------- .../AutoRepairStateFactoryTest.java} | 12 +- .../AutoRepairStateTest.java} | 84 ++-- .../AutoRepairTest.java} | 36 +- .../AutoRepairUtilsTest.java} | 174 ++++---- .../SSTableRepairedAtTest.java | 6 +- .../service/AutoRepairServiceBasicTest.java | 118 ++++++ ...a => AutoRepairServiceRepairTypeTest.java} | 26 +- ....java => AutoRepairServiceSetterTest.java} | 60 +-- .../UnifiedRepairServiceBasicTest.java | 118 ------ .../cassandra/tools/JMXStandardsTest.java | 6 +- ...tusTest.java => AutoRepairStatusTest.java} | 24 +- ...Test.java => SetAutoRepairConfigTest.java} | 64 +-- 50 files changed, 1122 insertions(+), 1122 deletions(-) rename src/java/org/apache/cassandra/metrics/{UnifiedRepairMetrics.java => AutoRepairMetrics.java} (76%) rename src/java/org/apache/cassandra/metrics/{UnifiedRepairMetricsManager.java => AutoRepairMetricsManager.java} (70%) rename src/java/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepair.java => autorepair/AutoRepair.java} (78%) rename src/java/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepairConfig.java => autorepair/AutoRepairConfig.java} (91%) rename src/java/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepairState.java => autorepair/AutoRepairState.java} (88%) rename src/java/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepairUtils.java => autorepair/AutoRepairUtils.java} (83%) rename src/java/org/apache/cassandra/repair/{unifiedrepair/DefaultUnifiedRepairTokenSplitter.java => autorepair/DefaultAutoRepairTokenSplitter.java} (82%) rename src/java/org/apache/cassandra/repair/{unifiedrepair/IUnifiedRepairTokenRangeSplitter.java => autorepair/IAutoRepairTokenRangeSplitter.java} (87%) rename src/java/org/apache/cassandra/schema/{UnifiedRepairParams.java => AutoRepairParams.java} (74%) rename src/java/org/apache/cassandra/service/{UnifiedRepairService.java => AutoRepairService.java} (72%) rename src/java/org/apache/cassandra/service/{UnifiedRepairServiceMBean.java => AutoRepairServiceMBean.java} (74%) rename src/java/org/apache/cassandra/tools/nodetool/{UnifiedRepairStatus.java => AutoRepairStatus.java} (82%) rename src/java/org/apache/cassandra/tools/nodetool/{GetUnifiedRepairConfig.java => GetAutoRepairConfig.java} (82%) rename src/java/org/apache/cassandra/tools/nodetool/{SetUnifiedRepairConfig.java => SetAutoRepairConfig.java} (83%) rename test/distributed/org/apache/cassandra/distributed/test/repair/{UnifiedRepairSchedulerTest.java => AutoRepairSchedulerTest.java} (83%) rename test/unit/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepairConfigTest.java => autorepair/AutoRepairConfigTest.java} (77%) rename test/unit/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepairDefaultTokenSplitterParameterizedTest.java => autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java} (84%) rename test/unit/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepairKeyspaceTest.java => autorepair/AutoRepairKeyspaceTest.java} (87%) rename test/unit/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepairParameterizedTest.java => autorepair/AutoRepairParameterizedTest.java} (52%) rename test/unit/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepairStateFactoryTest.java => autorepair/AutoRepairStateFactoryTest.java} (76%) rename test/unit/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepairStateTest.java => autorepair/AutoRepairStateTest.java} (72%) rename test/unit/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepairTest.java => autorepair/AutoRepairTest.java} (79%) rename test/unit/org/apache/cassandra/repair/{unifiedrepair/UnifiedRepairUtilsTest.java => autorepair/AutoRepairUtilsTest.java} (70%) rename test/unit/org/apache/cassandra/repair/{unifiedrepair => autorepair}/SSTableRepairedAtTest.java (97%) create mode 100644 test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java rename test/unit/org/apache/cassandra/service/{UnifiedRepairServiceRepairTypeTest.java => AutoRepairServiceRepairTypeTest.java} (72%) rename test/unit/org/apache/cassandra/service/{UnifiedRepairServiceSetterTest.java => AutoRepairServiceSetterTest.java} (59%) delete mode 100644 test/unit/org/apache/cassandra/service/UnifiedRepairServiceBasicTest.java rename test/unit/org/apache/cassandra/tools/nodetool/{UnifiedRepairStatusTest.java => AutoRepairStatusTest.java} (76%) rename test/unit/org/apache/cassandra/tools/nodetool/{SetUnifiedRepairConfigTest.java => SetAutoRepairConfigTest.java} (79%) diff --git a/doc/modules/cassandra/pages/managing/operating/metrics.adoc b/doc/modules/cassandra/pages/managing/operating/metrics.adoc index 8a5c286f37a6..2abeaefa6d39 100644 --- a/doc/modules/cassandra/pages/managing/operating/metrics.adoc +++ b/doc/modules/cassandra/pages/managing/operating/metrics.adoc @@ -1078,16 +1078,16 @@ partitions processed per logged batch partitions processed per unlogged batch |=== -== Unified Repair Metrics +== Automated Repair Metrics -Metrics specifc to unified repair. +Metrics specifc to automated repair. Reported name format: *Metric Name*:: -`org.apache.cassandra.metrics.UnifiedRepair.` +`org.apache.cassandra.metrics.AutoRepair.` *JMX MBean*:: -`org.apache.cassandra.metrics:type=UnifiedRepair name= repairType=` +`org.apache.cassandra.metrics:type=AutoRepair name= repairType=` [cols=",,",options="header",] |=== @@ -1118,15 +1118,15 @@ on the node views considered on the node |TotalDisabledRepairTables |Gauge |Number of tables on which -the unified repair has been disabled on the node +the automated repair has been disabled on the node |RepairTurnMyTurn |Counter |Represents the node's turn to repair |RepairTurnMyTurnDueToPriority |Counter |Represents the node's turn to repair -due to priority set in the unified repair +due to priority set in the automated repair |RepairTurnMyTurnForceRepair |Counter |Represents the node's turn to repair -due to force repair set in the unified repair +due to force repair set in the automated repair |=== diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 25dc67a01c3f..cbe1b64bcac9 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -33,7 +33,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -957,7 +957,7 @@ public static void setClientMode(boolean clientMode) public volatile boolean password_validator_reconfiguration_enabled = true; public volatile CustomGuardrailConfig password_validator = new CustomGuardrailConfig(); - public volatile UnifiedRepairConfig unified_repair = new UnifiedRepairConfig(); + public volatile AutoRepairConfig auto_repair = new AutoRepairConfig(); /** * The variants of paxos implementation and semantics supported by Cassandra. diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index a35465c59751..af35ba246092 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -107,7 +107,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.SeedProvider; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.security.AbstractCryptoProvider; import org.apache.cassandra.security.EncryptionContext; import org.apache.cassandra.security.JREProvider; @@ -5361,9 +5361,9 @@ public static boolean isPasswordValidatorReconfigurationEnabled() return conf.password_validator_reconfiguration_enabled; } - public static UnifiedRepairConfig getUnifiedRepairConfig() + public static AutoRepairConfig getAutoRepairConfig() { - return conf.unified_repair; + return conf.auto_repair; } public static double getIncrementalRepairDiskHeadroomRejectRatio() diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index 3b173a6514e4..f47b7e4ba722 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -25,8 +25,8 @@ import org.apache.cassandra.cql3.statements.PropertyDefinitions; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; -import org.apache.cassandra.schema.UnifiedRepairParams; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.schema.AutoRepairParams; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.schema.CompressionParams; @@ -154,10 +154,10 @@ private TableParams build(TableParams.Builder builder) builder.readRepair(ReadRepairStrategy.fromString(getString(READ_REPAIR))); if (hasOption(Option.REPAIR_FULL)) - builder.unifiedRepairFull(UnifiedRepairParams.fromMap(UnifiedRepairConfig.RepairType.full, getMap(Option.REPAIR_FULL))); + builder.automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, getMap(Option.REPAIR_FULL))); if (hasOption(Option.REPAIR_INCREMENTAL)) - builder.unifiedRepairIncremental(UnifiedRepairParams.fromMap(UnifiedRepairConfig.RepairType.incremental, getMap(Option.REPAIR_INCREMENTAL))); + builder.automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, getMap(Option.REPAIR_INCREMENTAL))); return builder.build(); } diff --git a/src/java/org/apache/cassandra/metrics/UnifiedRepairMetrics.java b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java similarity index 76% rename from src/java/org/apache/cassandra/metrics/UnifiedRepairMetrics.java rename to src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java index d4929cbfefb0..b097dd3414c4 100644 --- a/src/java/org/apache/cassandra/metrics/UnifiedRepairMetrics.java +++ b/src/java/org/apache/cassandra/metrics/AutoRepairMetrics.java @@ -20,19 +20,19 @@ import com.codahale.metrics.Counter; import com.codahale.metrics.Gauge; import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepair; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; +import org.apache.cassandra.repair.autorepair.AutoRepair; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** - * Metrics related to UnifiedRepair. + * Metrics related to AutoRepair. */ -public class UnifiedRepairMetrics +public class AutoRepairMetrics { - public static final String TYPE_NAME = "unifiedrepair"; + public static final String TYPE_NAME = "autorepair"; public Gauge repairsInProgress; public Gauge nodeRepairTimeInSec; public Gauge clusterRepairTimeInSec; @@ -47,15 +47,15 @@ public class UnifiedRepairMetrics public Gauge totalMVTablesConsideredForRepair; public Gauge totalDisabledRepairTables; - public UnifiedRepairMetrics(RepairType repairType) + public AutoRepairMetrics(RepairType repairType) { - UnifiedRepairMetricsFactory factory = new UnifiedRepairMetricsFactory(repairType); + AutoRepairMetricsFactory factory = new AutoRepairMetricsFactory(repairType); repairsInProgress = Metrics.register(factory.createMetricName("RepairsInProgress"), new Gauge() { public Integer getValue() { - return UnifiedRepair.instance.getRepairState(repairType).isRepairInProgress() ? 1 : 0; + return AutoRepair.instance.getRepairState(repairType).isRepairInProgress() ? 1 : 0; } }); @@ -63,7 +63,7 @@ public Integer getValue() { public Integer getValue() { - return UnifiedRepair.instance.getRepairState(repairType).getNodeRepairTimeInSec(); + return AutoRepair.instance.getRepairState(repairType).getNodeRepairTimeInSec(); } }); @@ -71,7 +71,7 @@ public Integer getValue() { public Integer getValue() { - return UnifiedRepair.instance.getRepairState(repairType).getClusterRepairTimeInSec(); + return AutoRepair.instance.getRepairState(repairType).getClusterRepairTimeInSec(); } }); @@ -79,7 +79,7 @@ public Integer getValue() { public Integer getValue() { - return UnifiedRepair.instance.getRepairState(repairType).getSkippedTokenRangesCount(); + return AutoRepair.instance.getRepairState(repairType).getSkippedTokenRangesCount(); } }); @@ -87,7 +87,7 @@ public Integer getValue() { public Integer getValue() { - return UnifiedRepair.instance.getRepairState(repairType).getSkippedTablesCount(); + return AutoRepair.instance.getRepairState(repairType).getSkippedTablesCount(); } }); @@ -96,7 +96,7 @@ public Integer getValue() { public Integer getValue() { - return UnifiedRepair.instance.getRepairState(repairType).getLongestUnrepairedSec(); + return AutoRepair.instance.getRepairState(repairType).getLongestUnrepairedSec(); } }); @@ -104,7 +104,7 @@ public Integer getValue() { public Integer getValue() { - return UnifiedRepair.instance.getRepairState(repairType).getSucceededTokenRangesCount(); + return AutoRepair.instance.getRepairState(repairType).getSucceededTokenRangesCount(); } }); @@ -112,7 +112,7 @@ public Integer getValue() { public Integer getValue() { - return UnifiedRepair.instance.getRepairState(repairType).getFailedTokenRangesCount(); + return AutoRepair.instance.getRepairState(repairType).getFailedTokenRangesCount(); } }); @@ -124,7 +124,7 @@ public Integer getValue() { public Integer getValue() { - return UnifiedRepair.instance.getRepairState(repairType).getTotalMVTablesConsideredForRepair(); + return AutoRepair.instance.getRepairState(repairType).getTotalMVTablesConsideredForRepair(); } }); @@ -132,12 +132,12 @@ public Integer getValue() { public Integer getValue() { - return UnifiedRepair.instance.getRepairState(repairType).getTotalDisabledTablesRepairCount(); + return AutoRepair.instance.getRepairState(repairType).getTotalDisabledTablesRepairCount(); } }); } - public void recordTurn(UnifiedRepairUtils.RepairTurn turn) + public void recordTurn(AutoRepairUtils.RepairTurn turn) { switch (turn) { @@ -156,13 +156,13 @@ public void recordTurn(UnifiedRepairUtils.RepairTurn turn) } @VisibleForTesting - protected static class UnifiedRepairMetricsFactory implements MetricNameFactory + protected static class AutoRepairMetricsFactory implements MetricNameFactory { - private static final String TYPE = "UnifiedRepair"; + private static final String TYPE = "AutoRepair"; @VisibleForTesting protected final String repairType; - protected UnifiedRepairMetricsFactory(RepairType repairType) + protected AutoRepairMetricsFactory(RepairType repairType) { this.repairType = toLowerCaseLocalized(repairType.toString()); } diff --git a/src/java/org/apache/cassandra/metrics/UnifiedRepairMetricsManager.java b/src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java similarity index 70% rename from src/java/org/apache/cassandra/metrics/UnifiedRepairMetricsManager.java rename to src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java index e8057595944d..e293945c9846 100644 --- a/src/java/org/apache/cassandra/metrics/UnifiedRepairMetricsManager.java +++ b/src/java/org/apache/cassandra/metrics/AutoRepairMetricsManager.java @@ -18,17 +18,17 @@ package org.apache.cassandra.metrics; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -public class UnifiedRepairMetricsManager +public class AutoRepairMetricsManager { - private static final Map metrics = new ConcurrentHashMap<>(); + private static final Map metrics = new ConcurrentHashMap<>(); - public static UnifiedRepairMetrics getMetrics(RepairType repairType) + public static AutoRepairMetrics getMetrics(RepairType repairType) { - return metrics.computeIfAbsent(repairType, k -> new UnifiedRepairMetrics(repairType)); + return metrics.computeIfAbsent(repairType, k -> new AutoRepairMetrics(repairType)); } } diff --git a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java index f9d96585715d..919d3163e317 100644 --- a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java +++ b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java @@ -150,7 +150,7 @@ public class CassandraMetricsRegistry extends MetricRegistry .add(ThreadPoolMetrics.TYPE_NAME) .add(TrieMemtableMetricsView.TYPE_NAME) .add(UnweightedCacheMetrics.TYPE_NAME) - .add(UnifiedRepairMetrics.TYPE_NAME) + .add(AutoRepairMetrics.TYPE_NAME) .build(); } diff --git a/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java similarity index 78% rename from src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepair.java rename to src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 02db8951fa64..1abb71d69cac 100644 --- a/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.util.ArrayList; import java.util.EnumMap; @@ -40,7 +40,7 @@ import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.utils.Clock; -import org.apache.cassandra.repair.unifiedrepair.IUnifiedRepairTokenRangeSplitter.RepairAssignment; +import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,35 +53,35 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Tables; -import org.apache.cassandra.service.UnifiedRepairService; +import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn; import org.apache.cassandra.utils.concurrent.Future; import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; -public class UnifiedRepair +public class AutoRepair { - private static final Logger logger = LoggerFactory.getLogger(UnifiedRepair.class); + private static final Logger logger = LoggerFactory.getLogger(AutoRepair.class); @VisibleForTesting protected static Supplier timeFunc = Clock.Global::currentTimeMillis; - public static UnifiedRepair instance = new UnifiedRepair(); + public static AutoRepair instance = new AutoRepair(); // Sleep for 5 seconds if repair finishes quickly to flush JMX metrics; it happens only for Cassandra nodes with tiny amount of data. public static DurationSpec.IntSecondsBound SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("5s"); @VisibleForTesting - protected final Map repairExecutors; + protected final Map repairExecutors; - protected final Map repairRunnableExecutors; + protected final Map repairRunnableExecutors; @VisibleForTesting - protected final Map repairStates; + protected final Map repairStates; @VisibleForTesting protected static Consumer> shuffleFunc = java.util.Collections::shuffle; @@ -89,23 +89,23 @@ public class UnifiedRepair @VisibleForTesting protected static BiConsumer sleepFunc = Uninterruptibles::sleepUninterruptibly; - protected final Map tokenRangeSplitters = new EnumMap<>(UnifiedRepairConfig.RepairType.class); + protected final Map tokenRangeSplitters = new EnumMap<>(AutoRepairConfig.RepairType.class); private boolean isSetupDone = false; @VisibleForTesting - protected UnifiedRepair() + protected AutoRepair() { - UnifiedRepairConfig config = DatabaseDescriptor.getUnifiedRepairConfig(); - repairExecutors = new EnumMap<>(UnifiedRepairConfig.RepairType.class); - repairRunnableExecutors = new EnumMap<>(UnifiedRepairConfig.RepairType.class); - repairStates = new EnumMap<>(UnifiedRepairConfig.RepairType.class); - for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) + AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); + repairExecutors = new EnumMap<>(AutoRepairConfig.RepairType.class); + repairRunnableExecutors = new EnumMap<>(AutoRepairConfig.RepairType.class); + repairStates = new EnumMap<>(AutoRepairConfig.RepairType.class); + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { - repairExecutors.put(repairType, executorFactory().scheduled(false, "UnifiedRepair-Repair-" + repairType, Thread.NORM_PRIORITY)); - repairRunnableExecutors.put(repairType, executorFactory().scheduled(false, "UnifiedRepair-RepairRunnable-" + repairType, Thread.NORM_PRIORITY)); - repairStates.put(repairType, UnifiedRepairConfig.RepairType.getUnifiedRepairState(repairType)); - tokenRangeSplitters.put(repairType, FBUtilities.newUnifiedRepairTokenRangeSplitter(config.getTokenRangeSplitter(repairType))); + repairExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-Repair-" + repairType, Thread.NORM_PRIORITY)); + repairRunnableExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-RepairRunnable-" + repairType, Thread.NORM_PRIORITY)); + repairStates.put(repairType, AutoRepairConfig.RepairType.getAutoRepairState(repairType)); + tokenRangeSplitters.put(repairType, FBUtilities.newAutoRepairTokenRangeSplitter(config.getTokenRangeSplitter(repairType))); } } @@ -119,13 +119,13 @@ public void setup() { return; } - UnifiedRepairConfig config = DatabaseDescriptor.getUnifiedRepairConfig(); - UnifiedRepairUtils.setup(); + AutoRepairConfig config = DatabaseDescriptor.getAutoRepairConfig(); + AutoRepairUtils.setup(); - for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { - if (config.isUnifiedRepairEnabled(repairType)) - UnifiedRepairService.instance.checkCanRun(repairType); + if (config.isAutoRepairEnabled(repairType)) + AutoRepairService.instance.checkCanRun(repairType); repairExecutors.get(repairType).scheduleWithFixedDelay( () -> repair(repairType), @@ -138,26 +138,26 @@ public void setup() } // repairAsync runs a repair session of the given type asynchronously. - public void repairAsync(UnifiedRepairConfig.RepairType repairType) + public void repairAsync(AutoRepairConfig.RepairType repairType) { - if (!UnifiedRepairService.instance.getUnifiedRepairConfig().isUnifiedRepairEnabled(repairType)) + if (!AutoRepairService.instance.getAutoRepairConfig().isAutoRepairEnabled(repairType)) { - throw new ConfigurationException("Unified-repair is disabled for repair type " + repairType); + throw new ConfigurationException("Auto-repair is disabled for repair type " + repairType); } repairExecutors.get(repairType).submit(() -> repair(repairType)); } // repair runs a repair session of the given type synchronously. - public void repair(UnifiedRepairConfig.RepairType repairType) + public void repair(AutoRepairConfig.RepairType repairType) { - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); - if (!config.isUnifiedRepairEnabled(repairType)) + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + if (!config.isAutoRepairEnabled(repairType)) { - logger.debug("Unified-repair is disabled for repair type {}", repairType); + logger.debug("Auto-repair is disabled for repair type {}", repairType); return; } - UnifiedRepairService.instance.checkCanRun(repairType); - UnifiedRepairState repairState = repairStates.get(repairType); + AutoRepairService.instance.checkCanRun(repairType); + AutoRepairState repairState = repairStates.get(repairType); try { String localDC = DatabaseDescriptor.getLocalDataCenter(); @@ -168,16 +168,16 @@ public void repair(UnifiedRepairConfig.RepairType repairType) } // refresh the longest unrepaired node - repairState.setLongestUnrepairedNode(UnifiedRepairUtils.getHostWithLongestUnrepairTime(repairType)); + repairState.setLongestUnrepairedNode(AutoRepairUtils.getHostWithLongestUnrepairTime(repairType)); //consistency level to use for local query UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); - RepairTurn turn = UnifiedRepairUtils.myTurnToRunRepair(repairType, myId); + RepairTurn turn = AutoRepairUtils.myTurnToRunRepair(repairType, myId); if (turn == MY_TURN || turn == MY_TURN_DUE_TO_PRIORITY || turn == MY_TURN_FORCE_REPAIR) { repairState.recordTurn(turn); - // For normal unified repair, we will use primary range only repairs (Repair with -pr option). - // For some cases, we may set the unified_repair_primary_token_range_only flag to false then we will do repair + // For normal auto repair, we will use primary range only repairs (Repair with -pr option). + // For some cases, we may set the auto_repair_primary_token_range_only flag to false then we will do repair // without -pr. We may also do force repair for certain node that we want to repair all the data on one node // When doing force repair, we want to repair without -pr. boolean primaryRangeOnly = config.getRepairPrimaryTokenRangeOnly(repairType) @@ -190,7 +190,7 @@ public void repair(UnifiedRepairConfig.RepairType repairType) long startTime = timeFunc.get(); logger.info("My host id: {}, my turn to run repair...repair primary-ranges only? {}", myId, config.getRepairPrimaryTokenRangeOnly(repairType)); - UnifiedRepairUtils.updateStartUnifiedRepairHistory(repairType, myId, timeFunc.get(), turn); + AutoRepairUtils.updateStartAutoRepairHistory(repairType, myId, timeFunc.get(), turn); repairState.setRepairKeyspaceCount(0); repairState.setRepairInProgress(true); @@ -201,13 +201,13 @@ public void repair(UnifiedRepairConfig.RepairType repairType) List keyspaces = new ArrayList<>(); Keyspace.all().forEach(keyspaces::add); - // Unified-repair is likely to be run on multiple nodes independently, we want to avoid running multiple repair + // Auto-repair is likely to be run on multiple nodes independently, we want to avoid running multiple repair // sessions on overlapping datasets at the same time. Shuffling keyspaces reduces the likelihood of this happening. shuffleFunc.accept(keyspaces); for (Keyspace keyspace : keyspaces) { - if (!UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(keyspace)) + if (!AutoRepairUtils.checkNodeContainsKeyspaceReplica(keyspace)) { continue; } @@ -237,20 +237,20 @@ public void repair(UnifiedRepairConfig.RepairType repairType) tableStartTime = timeFunc.get(); } previousAssignment = curRepairAssignment; - if (!config.isUnifiedRepairEnabled(repairType)) + if (!config.isAutoRepairEnabled(repairType)) { - logger.error("Unified-repair for type {} is disabled hence not running repair", repairType); + logger.error("Auto-repair for type {} is disabled hence not running repair", repairType); repairState.setRepairInProgress(false); return; } - if (UnifiedRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, keyspaceStartTime, tablesToBeRepairedList.size())) + if (AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, keyspaceStartTime, tablesToBeRepairedList.size())) { collectectedRepairStats.skippedTokenRanges += totalRepairAssignments - totalProcessedAssignments; logger.info("Keyspace took too much time to repair hence skipping it {}", keyspaceName); break; } - if (repairOneTableAtATime && UnifiedRepairUtils.tableMaxRepairTimeExceeded(repairType, tableStartTime)) + if (repairOneTableAtATime && AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, tableStartTime)) { collectectedRepairStats.skippedTokenRanges += 1; logger.info("Table took too much time to repair hence skipping it table name {}.{}, token range {}", @@ -339,17 +339,17 @@ else if (retryCount < config.getRepairMaxRetries()) } catch (Exception e) { - logger.error("Exception in unifiedrepair:", e); + logger.error("Exception in autorepair:", e); } } - private boolean tooSoonToRunRepair(UnifiedRepairConfig.RepairType repairType, UnifiedRepairState repairState, UnifiedRepairConfig config, UUID myId) + private boolean tooSoonToRunRepair(AutoRepairConfig.RepairType repairType, AutoRepairState repairState, AutoRepairConfig config, UUID myId) { if (repairState.getLastRepairTime() == 0) { // the node has either just boooted or has not run repair before, // we should check for the node's repair history in the DB - repairState.setLastRepairTime(UnifiedRepairUtils.getLastRepairTimeForNode(repairType, myId)); + repairState.setLastRepairTime(AutoRepairUtils.getLastRepairTimeForNode(repairType, myId)); } /** check if it is too soon to run repair. one of the reason we * should not run frequent repair is that repair triggers @@ -365,7 +365,7 @@ private boolean tooSoonToRunRepair(UnifiedRepairConfig.RepairType repairType, Un return false; } - private List retrieveTablesToBeRepaired(Keyspace keyspace, UnifiedRepairConfig config, UnifiedRepairConfig.RepairType repairType, UnifiedRepairState repairState, CollectectedRepairStats collectectedRepairStats) + private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairConfig config, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, CollectectedRepairStats collectectedRepairStats) { Tables tables = keyspace.getMetadata().tables; List tablesToBeRepaired = new ArrayList<>(); @@ -377,7 +377,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, UnifiedRepair String tableName = tableMetadata.name; ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName); - if (!columnFamilyStore.metadata().params.unifiedRepair.get(repairType).repairEnabled()) + if (!columnFamilyStore.metadata().params.automatedRepair.get(repairType).repairEnabled()) { logger.info("Repair is disabled for keyspace {} for tables: {}", keyspace.getName(), tableName); repairState.setTotalDisabledTablesRepairCount(repairState.getTotalDisabledTablesRepairCount() + 1); @@ -385,7 +385,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, UnifiedRepair continue; } - // this is done to make unifiedrepair safe as running repair on table with more sstables + // this is done to make autorepair safe as running repair on table with more sstables // may have its own challenges int totalSSTables = columnFamilyStore.getLiveSSTables().size(); if (totalSSTables > config.getRepairSSTableCountHigherThreshold(repairType)) @@ -399,7 +399,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, UnifiedRepair tablesToBeRepaired.add(tableName); // See if we should repair MVs as well that are associated with this given table - List mvs = UnifiedRepairUtils.getAllMVs(repairType, keyspace, tableMetadata); + List mvs = AutoRepairUtils.getAllMVs(repairType, keyspace, tableMetadata); if (!mvs.isEmpty()) { tablesToBeRepaired.addAll(mvs); @@ -409,14 +409,14 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, UnifiedRepair return tablesToBeRepaired; } - private void cleanupAndUpdateStats(RepairTurn turn, UnifiedRepairConfig.RepairType repairType, UnifiedRepairState repairState, UUID myId, + private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, UUID myId, long startTime, CollectectedRepairStats collectectedRepairStats) throws InterruptedException { //if it was due to priority then remove it now if (turn == MY_TURN_DUE_TO_PRIORITY) { logger.info("Remove current host from priority list"); - UnifiedRepairUtils.removePriorityStatus(repairType, myId); + AutoRepairUtils.removePriorityStatus(repairType, myId); } repairState.setFailedTokenRangesCount(collectectedRepairStats.failedTokenRanges); @@ -446,10 +446,10 @@ private void cleanupAndUpdateStats(RepairTurn turn, UnifiedRepairConfig.RepairTy Thread.sleep(SLEEP_IF_REPAIR_FINISHES_QUICKLY.toMilliseconds()); } repairState.setRepairInProgress(false); - UnifiedRepairUtils.updateFinishUnifiedRepairHistory(repairType, myId, timeFunc.get()); + AutoRepairUtils.updateFinishAutoRepairHistory(repairType, myId, timeFunc.get()); } - public UnifiedRepairState getRepairState(UnifiedRepairConfig.RepairType repairType) + public AutoRepairState getRepairState(AutoRepairConfig.RepairType repairType) { return repairStates.get(repairType); } diff --git a/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java similarity index 91% rename from src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfig.java rename to src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index 33f37b8f57ee..a80a875125a9 100644 --- a/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.io.Serializable; import java.util.ArrayList; @@ -34,9 +34,9 @@ import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.config.ParameterizedClass; -public class UnifiedRepairConfig implements Serializable +public class AutoRepairConfig implements Serializable { - // enable/disable unified repair globally, overrides all other settings. Cannot be modified dynamically. + // enable/disable auto repair globally, overrides all other settings. Cannot be modified dynamically. // if it is set to false, then no repair will be scheduled, including full and incremental repairs by this framework. // if it is set to true, then this repair scheduler will consult another config available for each RepairType, and based on that config, it will schedule repairs. public volatile Boolean enabled; @@ -61,7 +61,7 @@ public enum RepairType implements Serializable full, incremental; - public static UnifiedRepairState getUnifiedRepairState(RepairType repairType) + public static AutoRepairState getAutoRepairState(RepairType repairType) { switch (repairType) { @@ -78,12 +78,12 @@ public static UnifiedRepairState getUnifiedRepairState(RepairType repairType) // repair_type_overrides overrides the global_settings for a specific repair type public volatile Map repair_type_overrides = new EnumMap<>(RepairType.class); - public UnifiedRepairConfig() + public AutoRepairConfig() { this(false); } - public UnifiedRepairConfig(boolean enabled) + public AutoRepairConfig(boolean enabled) { this.enabled = enabled; global_settings = Options.getDefaultOptions(); @@ -98,12 +98,12 @@ public DurationSpec.IntSecondsBound getRepairCheckInterval() return repair_check_interval; } - public boolean isUnifiedRepairSchedulingEnabled() + public boolean isAutoRepairSchedulingEnabled() { return enabled; } - public DurationSpec.IntSecondsBound getUnifiedRepairHistoryClearDeleteHostsBufferInterval() + public DurationSpec.IntSecondsBound getAutoRepairHistoryClearDeleteHostsBufferInterval() { return history_clear_delete_hosts_buffer_interval; } @@ -111,10 +111,10 @@ public DurationSpec.IntSecondsBound getUnifiedRepairHistoryClearDeleteHostsBuffe public void startScheduler() { enabled = true; - UnifiedRepair.instance.setup(); + AutoRepair.instance.setup(); } - public void setUnifiedRepairHistoryClearDeleteHostsBufferInterval(String duration) + public void setAutoRepairHistoryClearDeleteHostsBufferInterval(String duration) { history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound(duration); } @@ -139,12 +139,12 @@ public void setRepairRetryBackoff(String interval) repair_retry_backoff = new DurationSpec.LongSecondsBound(interval); } - public boolean isUnifiedRepairEnabled(RepairType repairType) + public boolean isAutoRepairEnabled(RepairType repairType) { return enabled && applyOverrides(repairType, opt -> opt.enabled); } - public void setUnifiedRepairEnabled(RepairType repairType, boolean enabled) + public void setAutoRepairEnabled(RepairType repairType, boolean enabled) { ensureOverrides(repairType); repair_type_overrides.get(repairType).enabled = enabled; @@ -205,15 +205,15 @@ public void setRepairSSTableCountHigherThreshold(RepairType repairType, int ssta repair_type_overrides.get(repairType).sstable_upper_threshold = sstableHigherThreshold; } - public DurationSpec.IntSecondsBound getUnifiedRepairTableMaxRepairTime(RepairType repairType) + public DurationSpec.IntSecondsBound getAutoRepairTableMaxRepairTime(RepairType repairType) { return applyOverrides(repairType, opt -> opt.table_max_repair_time); } - public void setUnifiedRepairTableMaxRepairTime(RepairType repairType, String unifiedRepairTableMaxRepairTime) + public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime) { ensureOverrides(repairType); - repair_type_overrides.get(repairType).table_max_repair_time = new DurationSpec.IntSecondsBound(unifiedRepairTableMaxRepairTime); + repair_type_overrides.get(repairType).table_max_repair_time = new DurationSpec.IntSecondsBound(autoRepairTableMaxRepairTime); } public Set getIgnoreDCs(RepairType repairType) @@ -309,11 +309,11 @@ public void setRepairSessionTimeout(RepairType repairType, String repairSessionT repair_type_overrides.get(repairType).repair_session_timeout = new DurationSpec.IntSecondsBound(repairSessionTimeout); } - // Options configures unified-repair behavior for a given repair type. + // Options configures auto-repair behavior for a given repair type. // All fields can be modified dynamically. public static class Options implements Serializable { - // defaultOptions defines the default unified-repair behavior when no overrides are defined + // defaultOptions defines the default auto-repair behavior when no overrides are defined @VisibleForTesting protected static final Options defaultOptions = getDefaultOptions(); @@ -339,16 +339,16 @@ protected static Options getDefaultOptions() opts.force_repair_new_node = false; opts.table_max_repair_time = new DurationSpec.IntSecondsBound("6h"); opts.mv_repair_enabled = false; - opts.token_range_splitter = new ParameterizedClass(DefaultUnifiedRepairTokenSplitter.class.getName(), Collections.emptyMap()); + opts.token_range_splitter = new ParameterizedClass(DefaultAutoRepairTokenSplitter.class.getName(), Collections.emptyMap()); opts.initial_scheduler_delay = new DurationSpec.IntSecondsBound("5m"); // 5 minutes opts.repair_session_timeout = new DurationSpec.IntSecondsBound("3h"); // 3 hours return opts; } - // enable/disable unified repair for the given repair type + // enable/disable auto repair for the given repair type public volatile Boolean enabled; - // unified repair is default repair table by table, if this is enabled, the framework will repair all the tables in a keyspace in one go. + // auto repair is default repair table by table, if this is enabled, the framework will repair all the tables in a keyspace in one go. public volatile Boolean repair_by_keyspace; // the number of subranges to split each to-be-repaired token range into, // the higher this number, the smaller the repair sessions will be @@ -387,7 +387,7 @@ protected static Options getDefaultOptions() // specifies a denylist of datacenters to repair // This is useful if you want to completely avoid running repairs in one or more data centers. By default, it is empty, i.e., the framework will repair nodes in all the datacenters. public volatile Set ignore_dcs; - // Set this 'true' if UnifiedRepair should repair only the primary ranges owned by this node; else, 'false' + // Set this 'true' if AutoRepair should repair only the primary ranges owned by this node; else, 'false' // It is the same as -pr in nodetool repair options. public volatile Boolean repair_primary_token_range_only; // configures whether to force immediate repair on new nodes @@ -401,9 +401,9 @@ protected static Options getDefaultOptions() // So, the idea here is to penalize the outliers instead of good candidates. This can easily be configured with a higher value if we want to disable the functionality. public volatile DurationSpec.IntSecondsBound table_max_repair_time; // the default is 'true'. - // This flag determines whether the unified-repair framework needs to run anti-entropy, a.k.a, repair on the MV table or not. + // This flag determines whether the auto-repair framework needs to run anti-entropy, a.k.a, repair on the MV table or not. public volatile Boolean mv_repair_enabled; - // the default is DefaultUnifiedRepairTokenSplitter. The class should implement IUnifiedRepairTokenRangeSplitter. + // the default is DefaultAutoRepairTokenSplitter. The class should implement IAutoRepairTokenRangeSplitter. // The default implementation splits the tokens based on the token ranges owned by this node divided by the number of 'number_of_subranges' public volatile ParameterizedClass token_range_splitter; // the minimum delay after a node starts before the scheduler starts running repair diff --git a/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java similarity index 88% rename from src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairState.java rename to src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index 035906289592..84701ca78fbc 100644 --- a/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import com.google.common.annotations.VisibleForTesting; @@ -26,14 +26,14 @@ import org.apache.cassandra.db.view.TableViews; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.metrics.UnifiedRepairMetricsManager; -import org.apache.cassandra.metrics.UnifiedRepairMetrics; +import org.apache.cassandra.metrics.AutoRepairMetricsManager; +import org.apache.cassandra.metrics.AutoRepairMetrics; import org.apache.cassandra.repair.RepairCoordinator; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.UnifiedRepairHistory; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; import org.apache.cassandra.repair.RepairParallelism; import org.apache.cassandra.repair.messages.RepairOption; -import org.apache.cassandra.service.UnifiedRepairService; +import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.utils.Clock; @@ -55,10 +55,10 @@ import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition; -// UnifiedRepairState represents the state of unified repair for a given repair type. -public abstract class UnifiedRepairState implements ProgressListener +// AutoRepairState represents the state of automated repair for a given repair type. +public abstract class AutoRepairState implements ProgressListener { - protected static final Logger logger = LoggerFactory.getLogger(UnifiedRepairState.class); + protected static final Logger logger = LoggerFactory.getLogger(AutoRepairState.class); private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS"); @VisibleForTesting protected static Supplier timeFunc = Clock.Global::currentTimeMillis; @@ -93,16 +93,16 @@ public abstract class UnifiedRepairState implements ProgressListener protected int skippedTablesCount = 0; @VisibleForTesting - protected UnifiedRepairHistory longestUnrepairedNode; + protected AutoRepairHistory longestUnrepairedNode; @VisibleForTesting protected Condition condition = newOneTimeCondition(); @VisibleForTesting protected boolean success = true; - protected final UnifiedRepairMetrics metrics; + protected final AutoRepairMetrics metrics; - protected UnifiedRepairState(RepairType repairType) + protected AutoRepairState(RepairType repairType) { - metrics = UnifiedRepairMetricsManager.getMetrics(repairType); + metrics = AutoRepairMetricsManager.getMetrics(repairType); this.repairType = repairType; } @@ -229,7 +229,7 @@ public int getRepairKeyspaceCount() return repairKeyspaceCount; } - public void setLongestUnrepairedNode(UnifiedRepairHistory longestUnrepairedNode) + public void setLongestUnrepairedNode(AutoRepairHistory longestUnrepairedNode) { this.longestUnrepairedNode = longestUnrepairedNode; } @@ -279,7 +279,7 @@ public boolean isSuccess() return success; } - public void recordTurn(UnifiedRepairUtils.RepairTurn turn) + public void recordTurn(AutoRepairUtils.RepairTurn turn) { metrics.recordTurn(turn); } @@ -300,7 +300,7 @@ public void resetWaitCondition() } } -class IncrementalRepairState extends UnifiedRepairState +class IncrementalRepairState extends AutoRepairState { public IncrementalRepairState() { @@ -311,7 +311,7 @@ public IncrementalRepairState() public RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) { RepairOption option = new RepairOption(RepairParallelism.PARALLEL, primaryRangeOnly, true, false, - UnifiedRepairService.instance.getUnifiedRepairConfig().getRepairThreads(repairType), ranges, + AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), ranges, !ranges.isEmpty(), false, false, PreviewKind.NONE, true, true, false, false); option.getColumnFamilies().addAll(filterOutUnsafeTables(keyspace, tables)); @@ -345,7 +345,7 @@ protected List filterOutUnsafeTables(String keyspaceName, List t } } -class FullRepairState extends UnifiedRepairState +class FullRepairState extends AutoRepairState { public FullRepairState() { @@ -356,7 +356,7 @@ public FullRepairState() public RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) { RepairOption option = new RepairOption(RepairParallelism.PARALLEL, primaryRangeOnly, false, false, - UnifiedRepairService.instance.getUnifiedRepairConfig().getRepairThreads(repairType), ranges, + AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), ranges, !ranges.isEmpty(), false, false, PreviewKind.NONE, true, true, false, false); option.getColumnFamilies().addAll(tables); diff --git a/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java similarity index 83% rename from src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtils.java rename to src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index 2738aa6dc8de..8849b7847aef 100644 --- a/src/java/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.util.ArrayList; import java.util.Arrays; @@ -60,7 +60,7 @@ import org.apache.cassandra.schema.ViewMetadata; import org.apache.cassandra.serializers.SetSerializer; import org.apache.cassandra.serializers.UUIDSerializer; -import org.apache.cassandra.service.UnifiedRepairService; +import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.StorageService; @@ -72,21 +72,21 @@ import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.NOT_MY_TURN; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_FORCE_REPAIR; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; /** - * This class serves as a utility class for UnifiedRepair. It contains various helper APIs + * This class serves as a utility class for AutoRepair. It contains various helper APIs * to store/retrieve repair status, decide whose turn is next, etc. */ -public class UnifiedRepairUtils +public class AutoRepairUtils { - private static final Logger logger = LoggerFactory.getLogger(UnifiedRepairUtils.class); + private static final Logger logger = LoggerFactory.getLogger(AutoRepairUtils.class); static final String COL_REPAIR_TYPE = "repair_type"; static final String COL_HOST_ID = "host_id"; static final String COL_REPAIR_START_TS = "repair_start_ts"; @@ -99,56 +99,56 @@ public class UnifiedRepairUtils final static String SELECT_REPAIR_HISTORY = String.format( "SELECT * FROM %s.%s WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE); final static String SELECT_REPAIR_PRIORITY = String.format( "SELECT * FROM %s.%s WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); final static String DEL_REPAIR_PRIORITY = String.format( "DELETE %s[?] FROM %s.%s WHERE %s = ?", COL_REPAIR_PRIORITY, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_TYPE); final static String ADD_PRIORITY_HOST = String.format( "UPDATE %s.%s SET %s = %s + ? WHERE %s = ?", SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_TYPE); + SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_PRIORITY, COL_REPAIR_TYPE); final static String INSERT_NEW_REPAIR_HISTORY = String.format( "INSERT INTO %s.%s (%s, %s, %s, %s, %s, %s) values (?, ? ,?, ?, {}, ?) IF NOT EXISTS", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_TYPE, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID, COL_REPAIR_START_TS, COL_REPAIR_FINISH_TS, COL_DELETE_HOSTS, COL_DELETE_HOSTS_UPDATE_TIME); final static String ADD_HOST_ID_TO_DELETE_HOSTS = String.format( "UPDATE %s.%s SET %s = %s + ?, %s = ? WHERE %s = ? AND %s = ? IF EXISTS" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_DELETE_HOSTS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, COL_DELETE_HOSTS, COL_DELETE_HOSTS_UPDATE_TIME, COL_REPAIR_TYPE, COL_HOST_ID); - final static String DEL_UNIFIED_REPAIR_HISTORY = String.format( + final static String DEL_AUTO_REPAIR_HISTORY = String.format( "DELETE FROM %s.%s WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_TYPE, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID); final static String RECORD_START_REPAIR_HISTORY = String.format( "UPDATE %s.%s SET %s= ?, repair_turn = ? WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_START_TS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_START_TS, COL_REPAIR_TYPE, COL_HOST_ID); final static String RECORD_FINISH_REPAIR_HISTORY = String.format( "UPDATE %s.%s SET %s= ?, %s=false WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_FINISH_TS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_FINISH_TS, COL_FORCE_REPAIR, COL_REPAIR_TYPE, COL_HOST_ID); final static String CLEAR_DELETE_HOSTS = String.format( "UPDATE %s.%s SET %s= {} WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_DELETE_HOSTS, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_DELETE_HOSTS, COL_REPAIR_TYPE, COL_HOST_ID); final static String SET_FORCE_REPAIR = String.format( "UPDATE %s.%s SET %s=true WHERE %s = ? AND %s = ?" - , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_FORCE_REPAIR, + , SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_FORCE_REPAIR, COL_REPAIR_TYPE, COL_HOST_ID); final static String SELECT_LAST_REPAIR_TIME_FOR_NODE = String.format( "SELECT %s FROM %s.%s WHERE %s = ? AND %s = ?", COL_REPAIR_FINISH_TS, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, - SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID); + SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, COL_REPAIR_TYPE, COL_HOST_ID); static ModificationStatement delStatementRepairHistory; static SelectStatement selectStatementRepairHistory; @@ -198,14 +198,14 @@ public static void setup() .forInternalCalls()); clearDeleteHostsStatement = (ModificationStatement) QueryProcessor.getStatement(CLEAR_DELETE_HOSTS, ClientState .forInternalCalls()); - delStatementRepairHistory = (ModificationStatement) QueryProcessor.getStatement(DEL_UNIFIED_REPAIR_HISTORY, ClientState + delStatementRepairHistory = (ModificationStatement) QueryProcessor.getStatement(DEL_AUTO_REPAIR_HISTORY, ClientState .forInternalCalls()); - Keyspace unifiedRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); - internalQueryCL = unifiedRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? + Keyspace autoRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); + internalQueryCL = autoRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.ONE; } - public static class UnifiedRepairHistory + public static class AutoRepairHistory { UUID hostId; String repairTurn; @@ -215,8 +215,8 @@ public static class UnifiedRepairHistory long deleteHostsUpdateTime; boolean forceRepair; - public UnifiedRepairHistory(UUID hostId, String repairTurn, long lastRepairStartTime, long lastRepairFinishTime, - Set deleteHosts, long deleteHostsUpateTime, boolean forceRepair) + public AutoRepairHistory(UUID hostId, String repairTurn, long lastRepairStartTime, long lastRepairFinishTime, + Set deleteHosts, long deleteHostsUpateTime, boolean forceRepair) { this.hostId = hostId; this.repairTurn = repairTurn; @@ -259,15 +259,15 @@ public static class CurrentRepairStatus public Set hostIdsWithOnGoingRepair; // hosts that is running repair public Set hostIdsWithOnGoingForceRepair; // hosts that is running repair because of force repair Set priority; - List historiesWithoutOnGoingRepair; // hosts that is NOT running repair + List historiesWithoutOnGoingRepair; // hosts that is NOT running repair - public CurrentRepairStatus(List repairHistories, Set priority) + public CurrentRepairStatus(List repairHistories, Set priority) { hostIdsWithOnGoingRepair = new HashSet<>(); hostIdsWithOnGoingForceRepair = new HashSet<>(); historiesWithoutOnGoingRepair = new ArrayList<>(); - for (UnifiedRepairHistory history : repairHistories) + for (AutoRepairHistory history : repairHistories) { if (history.isRepairRunning()) { @@ -300,7 +300,7 @@ public String toString() } @VisibleForTesting - public static List getUnifiedRepairHistory(RepairType repairType) + public static List getAutoRepairHistory(RepairType repairType) { UntypedResultSet repairHistoryResult; @@ -308,7 +308,7 @@ public static List getUnifiedRepairHistory(RepairType repa QueryOptions.forInternalCalls(internalQueryCL, Lists.newArrayList(ByteBufferUtil.bytes(repairType.toString()))), Dispatcher.RequestTime.forImmediateExecution()); repairHistoryResult = UntypedResultSet.create(repairStatusRows.result); - List repairHistories = new ArrayList<>(); + List repairHistories = new ArrayList<>(); if (repairHistoryResult.size() > 0) { for (UntypedResultSet.Row row : repairHistoryResult) @@ -322,8 +322,8 @@ public static List getUnifiedRepairHistory(RepairType repa Set deleteHosts = row.getSet(COL_DELETE_HOSTS, UUIDType.instance); long deleteHostsUpdateTime = row.getLong(COL_DELETE_HOSTS_UPDATE_TIME, 0); Boolean forceRepair = row.has(COL_FORCE_REPAIR) ? row.getBoolean(COL_FORCE_REPAIR) : false; - repairHistories.add(new UnifiedRepairHistory(hostId, repairTurn, lastRepairStartTime, lastRepairFinishTime, - deleteHosts, deleteHostsUpdateTime, forceRepair)); + repairHistories.add(new AutoRepairHistory(hostId, repairTurn, lastRepairStartTime, lastRepairFinishTime, + deleteHosts, deleteHostsUpdateTime, forceRepair)); } return repairHistories; } @@ -373,8 +373,8 @@ public static void setForceRepair(RepairType repairType, UUID hostId) public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType) { - List unifiedRepairHistories = getUnifiedRepairHistory(repairType); - return getCurrentRepairStatus(repairType, unifiedRepairHistories); + List autoRepairHistories = getAutoRepairHistory(repairType); + return getCurrentRepairStatus(repairType, autoRepairHistories); } public static long getLastRepairTimeForNode(RepairType repairType, UUID hostId) @@ -396,11 +396,11 @@ public static long getLastRepairTimeForNode(RepairType repairType, UUID hostId) return repairTime.one().getLong(COL_REPAIR_FINISH_TS); } - public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType, List unifiedRepairHistories) + public static CurrentRepairStatus getCurrentRepairStatus(RepairType repairType, List autoRepairHistories) { - if (unifiedRepairHistories != null) + if (autoRepairHistories != null) { - CurrentRepairStatus status = new CurrentRepairStatus(unifiedRepairHistories, getPriorityHostIds(repairType)); + CurrentRepairStatus status = new CurrentRepairStatus(autoRepairHistories, getPriorityHostIds(repairType)); return status; } @@ -414,7 +414,7 @@ protected static TreeSet getHostIdsInCurrentRing(RepairType repairType, Se for (NodeAddresses node : allNodesInRing) { String nodeDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(node.broadcastAddress); - if (UnifiedRepairService.instance.getUnifiedRepairConfig().getIgnoreDCs(repairType).contains(nodeDC)) + if (AutoRepairService.instance.getAutoRepairConfig().getIgnoreDCs(repairType).contains(nodeDC)) { logger.info("Ignore node {} because its datacenter is {}", node, nodeDC); continue; @@ -442,39 +442,39 @@ public static TreeSet getHostIdsInCurrentRing(RepairType repairType) } // This function will return the host ID for the node which has not been repaired for longest time - public static UnifiedRepairHistory getHostWithLongestUnrepairTime(RepairType repairType) + public static AutoRepairHistory getHostWithLongestUnrepairTime(RepairType repairType) { - List unifiedRepairHistories = getUnifiedRepairHistory(repairType); - return getHostWithLongestUnrepairTime(unifiedRepairHistories); + List autoRepairHistories = getAutoRepairHistory(repairType); + return getHostWithLongestUnrepairTime(autoRepairHistories); } - private static UnifiedRepairHistory getHostWithLongestUnrepairTime(List unifiedRepairHistories) + private static AutoRepairHistory getHostWithLongestUnrepairTime(List autoRepairHistories) { - if (unifiedRepairHistories == null) + if (autoRepairHistories == null) { return null; } - UnifiedRepairHistory rst = null; + AutoRepairHistory rst = null; long oldestTimestamp = Long.MAX_VALUE; - for (UnifiedRepairHistory unifiedRepairHistory : unifiedRepairHistories) + for (AutoRepairHistory autoRepairHistory : autoRepairHistories) { - if (unifiedRepairHistory.lastRepairFinishTime < oldestTimestamp) + if (autoRepairHistory.lastRepairFinishTime < oldestTimestamp) { - rst = unifiedRepairHistory; - oldestTimestamp = unifiedRepairHistory.lastRepairFinishTime; + rst = autoRepairHistory; + oldestTimestamp = autoRepairHistory.lastRepairFinishTime; } } return rst; } - public static int getMaxNumberOfNodeRunUnifiedRepair(RepairType repairType, int groupSize) + public static int getMaxNumberOfNodeRunAutoRepair(RepairType repairType, int groupSize) { - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); if (groupSize == 0) { return Math.max(config.getParallelRepairCount(repairType), 1); } - // we will use the max number from config between unified_repair_parallel_repair_count_in_group and unified_repair_parallel_repair_percentage_in_group + // we will use the max number from config between auto_repair_parallel_repair_count_in_group and auto_repair_parallel_repair_percentage_in_group int value = Math.max(groupSize * config.getParallelRepairPercentage(repairType) / 100, config.getParallelRepairCount(repairType)); // make sure at least one node getting repaired @@ -491,33 +491,33 @@ public static RepairTurn myTurnToRunRepair(RepairType repairType, UUID myId) TreeSet hostIdsInCurrentRing = getHostIdsInCurrentRing(repairType, allNodesInRing); logger.info("Total nodes qualified for repair {}", hostIdsInCurrentRing.size()); - List unifiedRepairHistories = getUnifiedRepairHistory(repairType); - Set unifiedRepairHistoryIds = new HashSet<>(); + List autoRepairHistories = getAutoRepairHistory(repairType); + Set autoRepairHistoryIds = new HashSet<>(); // 1. Remove any node that is not part of group based on goissip info - if (unifiedRepairHistories != null) + if (autoRepairHistories != null) { - for (UnifiedRepairHistory nodeHistory : unifiedRepairHistories) + for (AutoRepairHistory nodeHistory : autoRepairHistories) { - unifiedRepairHistoryIds.add(nodeHistory.hostId); + autoRepairHistoryIds.add(nodeHistory.hostId); // clear delete_hosts if the node's delete hosts is not growing for more than two hours - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); if (nodeHistory.deleteHosts.size() > 0 - && config.getUnifiedRepairHistoryClearDeleteHostsBufferInterval().toSeconds() < TimeUnit.MILLISECONDS.toSeconds( + && config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds() < TimeUnit.MILLISECONDS.toSeconds( currentTimeMillis() - nodeHistory.deleteHostsUpdateTime )) { clearDeleteHosts(repairType, nodeHistory.hostId); logger.info("Delete hosts for {} for repair type {} has not been updated for more than {} seconds. Delete hosts has been cleared. Delete hosts before clear {}" - , nodeHistory.hostId, repairType, config.getUnifiedRepairHistoryClearDeleteHostsBufferInterval(), nodeHistory.deleteHosts); + , nodeHistory.hostId, repairType, config.getAutoRepairHistoryClearDeleteHostsBufferInterval(), nodeHistory.deleteHosts); } else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) { if (nodeHistory.deleteHosts.size() > Math.max(2, hostIdsInCurrentRing.size() * 0.5)) { // More than half of the groups thinks the record should be deleted - logger.info("{} think {} is orphan node, will delete unified repair history for repair type {}.", nodeHistory.deleteHosts, nodeHistory.hostId, repairType); - deleteUnifiedRepairHistory(repairType, nodeHistory.hostId); + logger.info("{} think {} is orphan node, will delete auto repair history for repair type {}.", nodeHistory.deleteHosts, nodeHistory.hostId, repairType); + deleteAutoRepairHistory(repairType, nodeHistory.hostId); } else { @@ -529,23 +529,23 @@ else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) } } - // 2. Add node to unified repair history table if a node is in gossip info + // 2. Add node to auto repair history table if a node is in gossip info for (UUID hostId : hostIdsInCurrentRing) { - if (!unifiedRepairHistoryIds.contains(hostId)) + if (!autoRepairHistoryIds.contains(hostId)) { - logger.info("{} for repair type {} doesn't exist in the unified repair history table, insert a new record.", repairType, hostId); + logger.info("{} for repair type {} doesn't exist in the auto repair history table, insert a new record.", repairType, hostId); insertNewRepairHistory(repairType, hostId, currentTimeMillis(), currentTimeMillis()); } } //get current repair status - CurrentRepairStatus currentRepairStatus = getCurrentRepairStatus(repairType, unifiedRepairHistories); + CurrentRepairStatus currentRepairStatus = getCurrentRepairStatus(repairType, autoRepairHistories); if (currentRepairStatus != null) { logger.info("Latest repair status {}", currentRepairStatus); //check if I am forced to run repair - for (UnifiedRepairHistory history : currentRepairStatus.historiesWithoutOnGoingRepair) + for (AutoRepairHistory history : currentRepairStatus.historiesWithoutOnGoingRepair) { if (history.forceRepair && history.hostId.equals(myId)) { @@ -554,24 +554,24 @@ else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) } } - int parallelRepairNumber = getMaxNumberOfNodeRunUnifiedRepair(repairType, - unifiedRepairHistories == null ? 0 : unifiedRepairHistories.size()); + int parallelRepairNumber = getMaxNumberOfNodeRunAutoRepair(repairType, + autoRepairHistories == null ? 0 : autoRepairHistories.size()); logger.info("Will run repairs concurrently on {} node(s)", parallelRepairNumber); if (currentRepairStatus == null || parallelRepairNumber > currentRepairStatus.hostIdsWithOnGoingRepair.size()) { // more repairs can be run, I might be the new one - if (unifiedRepairHistories != null) + if (autoRepairHistories != null) { - logger.info("Unified repair history table has {} records", unifiedRepairHistories.size()); + logger.info("Auto repair history table has {} records", autoRepairHistories.size()); } else { // try to fetch again - unifiedRepairHistories = getUnifiedRepairHistory(repairType); - currentRepairStatus = getCurrentRepairStatus(repairType, unifiedRepairHistories); - if (unifiedRepairHistories == null || currentRepairStatus == null) + autoRepairHistories = getAutoRepairHistory(repairType); + currentRepairStatus = getCurrentRepairStatus(repairType, autoRepairHistories); + if (autoRepairHistories == null || currentRepairStatus == null) { logger.error("No record found"); return NOT_MY_TURN; @@ -611,7 +611,7 @@ else if (!hostIdsInCurrentRing.contains(nodeHistory.hostId)) } // get the longest unrepaired node from the nodes which are not running repair - UnifiedRepairHistory defaultNodeToBeRepaired = getHostWithLongestUnrepairTime(currentRepairStatus.historiesWithoutOnGoingRepair); + AutoRepairHistory defaultNodeToBeRepaired = getHostWithLongestUnrepairTime(currentRepairStatus.historiesWithoutOnGoingRepair); //check who is next, which is helpful for debugging logger.info("Next node to be repaired for repair type {} by default: {}", repairType, defaultNodeToBeRepaired); if (defaultNodeToBeRepaired != null && defaultNodeToBeRepaired.hostId.equals(myId)) @@ -633,7 +633,7 @@ else if (currentRepairStatus.hostIdsWithOnGoingForceRepair.contains(myId)) return NOT_MY_TURN; } - static void deleteUnifiedRepairHistory(RepairType repairType, UUID hostId) + static void deleteAutoRepairHistory(RepairType repairType, UUID hostId) { //delete the given hostId delStatementRepairHistory.execute(QueryState.forInternalCalls(), @@ -642,7 +642,7 @@ static void deleteUnifiedRepairHistory(RepairType repairType, UUID hostId) ByteBufferUtil.bytes(hostId))), Dispatcher.RequestTime.forImmediateExecution()); } - static void updateStartUnifiedRepairHistory(RepairType repairType, UUID myId, long timestamp, RepairTurn turn) + static void updateStartAutoRepairHistory(RepairType repairType, UUID myId, long timestamp, RepairTurn turn) { recordStartRepairHistoryStatement.execute(QueryState.forInternalCalls(), QueryOptions.forInternalCalls(internalQueryCL, @@ -653,7 +653,7 @@ static void updateStartUnifiedRepairHistory(RepairType repairType, UUID myId, lo )), Dispatcher.RequestTime.forImmediateExecution()); } - static void updateFinishUnifiedRepairHistory(RepairType repairType, UUID myId, long timestamp) + static void updateFinishAutoRepairHistory(RepairType repairType, UUID myId, long timestamp) { recordFinishRepairHistoryStatement.execute(QueryState.forInternalCalls(), QueryOptions.forInternalCalls(internalQueryCL, @@ -662,15 +662,15 @@ static void updateFinishUnifiedRepairHistory(RepairType repairType, UUID myId, l ByteBufferUtil.bytes(myId) )), Dispatcher.RequestTime.forImmediateExecution()); // Do not remove beblow log, the log is used by dtest - logger.info("Unified repair finished for {}", myId); + logger.info("Auto repair finished for {}", myId); } public static void insertNewRepairHistory(RepairType repairType, UUID hostId, long startTime, long finishTime) { try { - Keyspace unifiedRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); - ConsistencyLevel cl = unifiedRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? + Keyspace autoRepairKS = Schema.instance.getKeyspaceInstance(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME); + ConsistencyLevel cl = autoRepairKS.getReplicationStrategy().getClass() == NetworkTopologyStrategy.class ? ConsistencyLevel.LOCAL_SERIAL : null; UntypedResultSet resultSet; @@ -687,7 +687,7 @@ public static void insertNewRepairHistory(RepairType repairType, UUID hostId, lo boolean applied = resultSet.one().getBoolean(ModificationStatement.CAS_RESULT_COLUMN.toString()); if (applied) { - logger.info("Successfully inserted a new unified repair history record for host id: {}", hostId); + logger.info("Successfully inserted a new auto repair history record for host id: {}", hostId); } else { @@ -810,21 +810,21 @@ public static boolean tableMaxRepairTimeExceeded(RepairType repairType, long sta { long tableRepairTimeSoFar = TimeUnit.MILLISECONDS.toSeconds (currentTimeMillis() - startTime); - return UnifiedRepairService.instance.getUnifiedRepairConfig().getUnifiedRepairTableMaxRepairTime(repairType).toSeconds() < + return AutoRepairService.instance.getAutoRepairConfig().getAutoRepairTableMaxRepairTime(repairType).toSeconds() < tableRepairTimeSoFar; } public static boolean keyspaceMaxRepairTimeExceeded(RepairType repairType, long startTime, int numOfTablesToBeRepaired) { long keyspaceRepairTimeSoFar = TimeUnit.MILLISECONDS.toSeconds((currentTimeMillis() - startTime)); - return (long) UnifiedRepairService.instance.getUnifiedRepairConfig().getUnifiedRepairTableMaxRepairTime(repairType).toSeconds() * + return (long) AutoRepairService.instance.getAutoRepairConfig().getAutoRepairTableMaxRepairTime(repairType).toSeconds() * numOfTablesToBeRepaired < keyspaceRepairTimeSoFar; } public static List getAllMVs(RepairType repairType, Keyspace keyspace, TableMetadata tableMetadata) { List allMvs = new ArrayList<>(); - if (UnifiedRepairService.instance.getUnifiedRepairConfig().getMVRepairEnabled(repairType) && keyspace.getMetadata().views != null) + if (AutoRepairService.instance.getAutoRepairConfig().getMVRepairEnabled(repairType) && keyspace.getMetadata().views != null) { Iterator views = keyspace.getMetadata().views.forTable(tableMetadata.id).iterator(); while (views.hasNext()) @@ -839,12 +839,12 @@ public static List getAllMVs(RepairType repairType, Keyspace keyspace, T public static void runRepairOnNewlyBootstrappedNodeIfEnabled() { - UnifiedRepairConfig repairConfig = DatabaseDescriptor.getUnifiedRepairConfig(); - if (repairConfig.isUnifiedRepairSchedulingEnabled()) + AutoRepairConfig repairConfig = DatabaseDescriptor.getAutoRepairConfig(); + if (repairConfig.isAutoRepairSchedulingEnabled()) { - for (UnifiedRepairConfig.RepairType rType : UnifiedRepairConfig.RepairType.values()) - if (repairConfig.isUnifiedRepairEnabled(rType) && repairConfig.getForceRepairNewNode(rType)) - UnifiedRepairUtils.setForceRepairNewNode(rType); + for (AutoRepairConfig.RepairType rType : AutoRepairConfig.RepairType.values()) + if (repairConfig.isAutoRepairEnabled(rType) && repairConfig.getForceRepairNewNode(rType)) + AutoRepairUtils.setForceRepairNewNode(rType); } } diff --git a/src/java/org/apache/cassandra/repair/unifiedrepair/DefaultUnifiedRepairTokenSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java similarity index 82% rename from src/java/org/apache/cassandra/repair/unifiedrepair/DefaultUnifiedRepairTokenSplitter.java rename to src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java index 98dc09f8ad7d..9a884f61c581 100644 --- a/src/java/org/apache/cassandra/repair/unifiedrepair/DefaultUnifiedRepairTokenSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.util.ArrayList; @@ -23,20 +23,20 @@ import java.util.Collections; import java.util.List; -import org.apache.cassandra.service.UnifiedRepairService; +import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.service.StorageService; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.splitEvenly; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.splitEvenly; -public class DefaultUnifiedRepairTokenSplitter implements IUnifiedRepairTokenRangeSplitter +public class DefaultAutoRepairTokenSplitter implements IAutoRepairTokenRangeSplitter { @Override - public List getRepairAssignments(UnifiedRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) + public List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) { - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); List repairAssignments = new ArrayList<>(); Collection> tokens = StorageService.instance.getPrimaryRanges(keyspaceName); diff --git a/src/java/org/apache/cassandra/repair/unifiedrepair/IUnifiedRepairTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java similarity index 87% rename from src/java/org/apache/cassandra/repair/unifiedrepair/IUnifiedRepairTokenRangeSplitter.java rename to src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java index 251e6b96ec56..169600eca405 100644 --- a/src/java/org/apache/cassandra/repair/unifiedrepair/IUnifiedRepairTokenRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.util.List; @@ -24,22 +24,22 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; -public interface IUnifiedRepairTokenRangeSplitter +public interface IAutoRepairTokenRangeSplitter { /** * Split the token range you wish to repair into multiple assignments. - * The unifiedrepair framework will repair the list of returned subrange in a sequence. + * The autorepair framework will repair the list of returned subrange in a sequence. * @param repairType The type of repair being executed * @param primaryRangeOnly Whether to repair only this node's primary ranges or all of its ranges. * @param keyspaceName The keyspace being repaired * @param tableNames The tables to repair * @return repair assignments broken up by range, keyspace and tables. */ - List getRepairAssignments(UnifiedRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames); + List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames); /** - * Defines a repair assignment to be issued by the unifiedrepair framework. + * Defines a repair assignment to be issued by the autorepair framework. */ class RepairAssignment { diff --git a/src/java/org/apache/cassandra/schema/UnifiedRepairParams.java b/src/java/org/apache/cassandra/schema/AutoRepairParams.java similarity index 74% rename from src/java/org/apache/cassandra/schema/UnifiedRepairParams.java rename to src/java/org/apache/cassandra/schema/AutoRepairParams.java index 27459ad30be0..ea3802db93d2 100644 --- a/src/java/org/apache/cassandra/schema/UnifiedRepairParams.java +++ b/src/java/org/apache/cassandra/schema/AutoRepairParams.java @@ -26,12 +26,12 @@ import org.apache.commons.lang3.StringUtils; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import static java.lang.String.format; import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; -public final class UnifiedRepairParams +public final class AutoRepairParams { public enum Option { @@ -44,23 +44,23 @@ public String toString() } } - public static final Map> DEFAULT_OPTIONS = - ImmutableMap.of(UnifiedRepairConfig.RepairType.full, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), - UnifiedRepairConfig.RepairType.incremental, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true))); + public static final Map> DEFAULT_OPTIONS = + ImmutableMap.of(AutoRepairConfig.RepairType.full, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), + AutoRepairConfig.RepairType.incremental, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true))); - public final UnifiedRepairConfig.RepairType type; + public final AutoRepairConfig.RepairType type; - private Map> options = DEFAULT_OPTIONS; + private Map> options = DEFAULT_OPTIONS; - UnifiedRepairParams(UnifiedRepairConfig.RepairType type) + AutoRepairParams(AutoRepairConfig.RepairType type) { this.type = type; } - public static UnifiedRepairParams create(UnifiedRepairConfig.RepairType repairType, Map options) + public static AutoRepairParams create(AutoRepairConfig.RepairType repairType, Map options) { - Map> optionsMap = new HashMap<>(); - for (Map.Entry> entry : DEFAULT_OPTIONS.entrySet()) + Map> optionsMap = new HashMap<>(); + for (Map.Entry> entry : DEFAULT_OPTIONS.entrySet()) { optionsMap.put(entry.getKey(), new HashMap<>(entry.getValue())); } @@ -75,7 +75,7 @@ public static UnifiedRepairParams create(UnifiedRepairConfig.RepairType repairTy optionsMap.get(repairType).put(entry.getKey(), entry.getValue()); } } - UnifiedRepairParams repairParams = new UnifiedRepairParams(repairType); + AutoRepairParams repairParams = new AutoRepairParams(repairType); repairParams.options = optionsMap; return repairParams; } @@ -109,7 +109,7 @@ public Map options() return options.get(type); } - public static UnifiedRepairParams fromMap(UnifiedRepairConfig.RepairType repairType, Map map) + public static AutoRepairParams fromMap(AutoRepairConfig.RepairType repairType, Map map) { return create(repairType, map); } @@ -133,10 +133,10 @@ public boolean equals(Object o) if (this == o) return true; - if (!(o instanceof UnifiedRepairParams)) + if (!(o instanceof AutoRepairParams)) return false; - UnifiedRepairParams cp = (UnifiedRepairParams) o; + AutoRepairParams cp = (AutoRepairParams) o; return options.equals(cp.options); } diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index 796e5d819e3f..fe9a35f74519 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -44,7 +44,7 @@ import org.apache.cassandra.db.partitions.*; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.exceptions.InvalidRequestException; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; import org.apache.cassandra.schema.ColumnMetadata.ClusteringOrder; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; @@ -565,8 +565,8 @@ private static void addTableParamsToRowBuilder(TableParams params, Row.SimpleBui .add("compression", params.compression.asMap()) .add("read_repair", params.readRepair.toString()) .add("extensions", params.extensions) - .add("repair_full", params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full).asMap()) - .add("repair_incremental", params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).asMap()); + .add("repair_full", params.automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) + .add("repair_incremental", params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); // Only add CDC-enabled flag to schema if it's enabled on the node. This is to work around RTE's post-8099 if a 3.8+ @@ -1051,8 +1051,8 @@ static TableParams createTableParamsFromRow(UntypedResultSet.Row row) SpeculativeRetryPolicy.fromString("99PERCENTILE")) .cdc(row.has("cdc") && row.getBoolean("cdc")) .readRepair(getReadRepairStrategy(row)) - .unifiedRepairFull(UnifiedRepairParams.fromMap(UnifiedRepairConfig.RepairType.full, row.getFrozenTextMap("repair_full"))) - .unifiedRepairIncremental(UnifiedRepairParams.fromMap(UnifiedRepairConfig.RepairType.incremental, row.getFrozenTextMap("repair_incremental"))); + .automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, row.getFrozenTextMap("repair_full"))) + .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, row.getFrozenTextMap("repair_incremental"))); // allow_auto_snapshot column was introduced in 4.2 if (row.has("allow_auto_snapshot")) diff --git a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java index 0189cdac952a..9411f94c1924 100644 --- a/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SystemDistributedKeyspace.java @@ -96,11 +96,11 @@ private SystemDistributedKeyspace() public static final String PARTITION_DENYLIST_TABLE = "partition_denylist"; - public static final String UNIFIED_REPAIR_HISTORY = "unified_repair_history"; + public static final String AUTO_REPAIR_HISTORY = "auto_repair_history"; - public static final String UNIFIED_REPAIR_PRIORITY = "unified_repair_priority"; + public static final String AUTO_REPAIR_PRIORITY = "auto_repair_priority"; - public static final Set TABLE_NAMES = ImmutableSet.of(REPAIR_HISTORY, PARENT_REPAIR_HISTORY, VIEW_BUILD_STATUS, PARTITION_DENYLIST_TABLE, UNIFIED_REPAIR_HISTORY, UNIFIED_REPAIR_PRIORITY); + public static final Set TABLE_NAMES = ImmutableSet.of(REPAIR_HISTORY, PARENT_REPAIR_HISTORY, VIEW_BUILD_STATUS, PARTITION_DENYLIST_TABLE, AUTO_REPAIR_HISTORY, AUTO_REPAIR_PRIORITY); public static final String REPAIR_HISTORY_CQL = "CREATE TABLE IF NOT EXISTS %s (" + "keyspace_name text," @@ -163,27 +163,27 @@ private SystemDistributedKeyspace() private static final TableMetadata PartitionDenylistTable = parse(PARTITION_DENYLIST_TABLE, "Partition keys which have been denied access", PARTITION_DENYLIST_CQL).build(); - public static final String UNIFIED_REPAIR_HISTORY_CQL = "CREATE TABLE IF NOT EXISTS %s (" - + "host_id uuid," - + "repair_type text," - + "repair_turn text," - + "repair_start_ts timestamp," - + "repair_finish_ts timestamp," - + "delete_hosts set," - + "delete_hosts_update_time timestamp," - + "force_repair boolean," - + "PRIMARY KEY (repair_type, host_id))"; + public static final String AUTO_REPAIR_HISTORY_CQL = "CREATE TABLE IF NOT EXISTS %s (" + + "host_id uuid," + + "repair_type text," + + "repair_turn text," + + "repair_start_ts timestamp," + + "repair_finish_ts timestamp," + + "delete_hosts set," + + "delete_hosts_update_time timestamp," + + "force_repair boolean," + + "PRIMARY KEY (repair_type, host_id))"; - private static final TableMetadata UnifiedRepairHistoryTable = - parse(UNIFIED_REPAIR_HISTORY, "Unified repair history for each node", UNIFIED_REPAIR_HISTORY_CQL).build(); + private static final TableMetadata AutoRepairHistoryTable = + parse(AUTO_REPAIR_HISTORY, "Auto repair history for each node", AUTO_REPAIR_HISTORY_CQL).build(); - public static final String UNIFIED_REPAIR_PRIORITY_CQL = "CREATE TABLE IF NOT EXISTS %s (" - + "repair_type text," - + "repair_priority set," - + "PRIMARY KEY (repair_type))"; + public static final String AUTO_REPAIR_PRIORITY_CQL = "CREATE TABLE IF NOT EXISTS %s (" + + "repair_type text," + + "repair_priority set," + + "PRIMARY KEY (repair_type))"; - private static final TableMetadata UnifiedRepairPriorityTable = - parse(UNIFIED_REPAIR_PRIORITY, "Unified repair priority for each group", UNIFIED_REPAIR_PRIORITY_CQL).build(); + private static final TableMetadata AutoRepairPriorityTable = + parse(AUTO_REPAIR_PRIORITY, "Auto repair priority for each group", AUTO_REPAIR_PRIORITY_CQL).build(); private static TableMetadata.Builder parse(String table, String description, String cql) @@ -198,7 +198,7 @@ public static KeyspaceMetadata metadata() { return KeyspaceMetadata.create(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, KeyspaceParams.simple(Math.max(DEFAULT_RF, DatabaseDescriptor.getDefaultKeyspaceRF())), - Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus, PartitionDenylistTable, UnifiedRepairHistoryTable, UnifiedRepairPriorityTable)); + Tables.of(RepairHistory, ParentRepairHistory, ViewBuildStatus, PartitionDenylistTable, AutoRepairHistoryTable, AutoRepairPriorityTable)); } public static void startParentRepair(TimeUUID parent_id, String keyspaceName, String[] cfnames, RepairOption options) diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index 8794fca25389..9f698ecb9114 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -35,7 +35,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.service.reads.PercentileSpeculativeRetryPolicy; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; @@ -102,7 +102,7 @@ public String toString() public final boolean cdc; public final ReadRepairStrategy readRepair; - public final Map unifiedRepair; + public final Map automatedRepair; private TableParams(Builder builder) { @@ -127,11 +127,11 @@ private TableParams(Builder builder) extensions = builder.extensions; cdc = builder.cdc; readRepair = builder.readRepair; - unifiedRepair = new EnumMap(UnifiedRepairConfig.RepairType.class) + automatedRepair = new EnumMap(AutoRepairConfig.RepairType.class) { { - put(UnifiedRepairConfig.RepairType.full, builder.unifiedRepairFull); - put(UnifiedRepairConfig.RepairType.incremental, builder.unifiedRepairIncremental); + put(AutoRepairConfig.RepairType.full, builder.automatedRepairFull); + put(AutoRepairConfig.RepairType.incremental, builder.automatedRepairIncremental); } }; } @@ -162,8 +162,8 @@ public static Builder builder(TableParams params) .extensions(params.extensions) .cdc(params.cdc) .readRepair(params.readRepair) - .unifiedRepairFull(params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full)) - .unifiedRepairIncremental(params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental)) + .automatedRepairFull(params.automatedRepair.get(AutoRepairConfig.RepairType.full)) + .automatedRepairIncremental(params.automatedRepair.get(AutoRepairConfig.RepairType.incremental)) ; } @@ -220,7 +220,7 @@ public void validate() if (cdc && memtable.factory().writesShouldSkipCommitLog()) fail("CDC cannot work if writes skip the commit log. Check your memtable configuration."); - for (Map.Entry entry : unifiedRepair.entrySet()) + for (Map.Entry entry : automatedRepair.entrySet()) { entry.getValue().validate(); } @@ -243,25 +243,25 @@ public boolean equals(Object o) TableParams p = (TableParams) o; return comment.equals(p.comment) - && additionalWritePolicy.equals(p.additionalWritePolicy) - && allowAutoSnapshot == p.allowAutoSnapshot - && bloomFilterFpChance == p.bloomFilterFpChance - && crcCheckChance == p.crcCheckChance - && gcGraceSeconds == p.gcGraceSeconds - && incrementalBackups == p.incrementalBackups - && defaultTimeToLive == p.defaultTimeToLive - && memtableFlushPeriodInMs == p.memtableFlushPeriodInMs - && minIndexInterval == p.minIndexInterval - && maxIndexInterval == p.maxIndexInterval - && speculativeRetry.equals(p.speculativeRetry) - && caching.equals(p.caching) - && compaction.equals(p.compaction) - && compression.equals(p.compression) - && memtable.equals(p.memtable) - && extensions.equals(p.extensions) - && cdc == p.cdc - && readRepair == p.readRepair - && unifiedRepair.equals(p.unifiedRepair); + && additionalWritePolicy.equals(p.additionalWritePolicy) + && allowAutoSnapshot == p.allowAutoSnapshot + && bloomFilterFpChance == p.bloomFilterFpChance + && crcCheckChance == p.crcCheckChance + && gcGraceSeconds == p.gcGraceSeconds + && incrementalBackups == p.incrementalBackups + && defaultTimeToLive == p.defaultTimeToLive + && memtableFlushPeriodInMs == p.memtableFlushPeriodInMs + && minIndexInterval == p.minIndexInterval + && maxIndexInterval == p.maxIndexInterval + && speculativeRetry.equals(p.speculativeRetry) + && caching.equals(p.caching) + && compaction.equals(p.compaction) + && compression.equals(p.compression) + && memtable.equals(p.memtable) + && extensions.equals(p.extensions) + && cdc == p.cdc + && readRepair == p.readRepair + && automatedRepair.equals(p.automatedRepair); } @Override @@ -286,7 +286,7 @@ public int hashCode() extensions, cdc, readRepair, - unifiedRepair); + automatedRepair); } @Override @@ -312,8 +312,8 @@ public String toString() .add(EXTENSIONS.toString(), extensions) .add(CDC.toString(), cdc) .add(READ_REPAIR.toString(), readRepair) - .add(Option.REPAIR_FULL.toString(), unifiedRepair.get(UnifiedRepairConfig.RepairType.full)) - .add(Option.REPAIR_INCREMENTAL.toString(), unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental)) + .add(Option.REPAIR_FULL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.full)) + .add(Option.REPAIR_INCREMENTAL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.incremental)) .toString(); } @@ -367,9 +367,9 @@ public void appendCqlTo(CqlBuilder builder, boolean isView) .newLine() .append("AND speculative_retry = ").appendWithSingleQuotes(speculativeRetry.toString()) .newLine() - .append("AND repair_full = ").append(unifiedRepair.get(UnifiedRepairConfig.RepairType.full).asMap()) + .append("AND repair_full = ").append(automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) .newLine() - .append("AND repair_incremental = ").append(unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).asMap()); + .append("AND repair_incremental = ").append(automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); } public static final class Builder @@ -394,8 +394,8 @@ public static final class Builder private boolean cdc; private ReadRepairStrategy readRepair = ReadRepairStrategy.BLOCKING; - private UnifiedRepairParams unifiedRepairFull = new UnifiedRepairParams(UnifiedRepairConfig.RepairType.full); - private UnifiedRepairParams unifiedRepairIncremental = new UnifiedRepairParams(UnifiedRepairConfig.RepairType.incremental); + private AutoRepairParams automatedRepairFull = new AutoRepairParams(AutoRepairConfig.RepairType.full); + private AutoRepairParams automatedRepairIncremental = new AutoRepairParams(AutoRepairConfig.RepairType.incremental); public Builder() { @@ -520,15 +520,15 @@ public Builder extensions(Map val) return this; } - public Builder unifiedRepairFull(UnifiedRepairParams val) + public Builder automatedRepairFull(AutoRepairParams val) { - unifiedRepairFull = val; + automatedRepairFull = val; return this; } - public Builder unifiedRepairIncremental(UnifiedRepairParams val) + public Builder automatedRepairIncremental(AutoRepairParams val) { - unifiedRepairIncremental = val; + automatedRepairIncremental = val; return this; } } diff --git a/src/java/org/apache/cassandra/service/UnifiedRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java similarity index 72% rename from src/java/org/apache/cassandra/service/UnifiedRepairService.java rename to src/java/org/apache/cassandra/service/AutoRepairService.java index 489dd6873011..7a8342c2a7a4 100644 --- a/src/java/org/apache/cassandra/service/UnifiedRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -20,9 +20,9 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.utils.MBeanWrapper; import java.util.HashSet; @@ -32,23 +32,23 @@ import com.google.common.annotations.VisibleForTesting; -public class UnifiedRepairService implements UnifiedRepairServiceMBean +public class AutoRepairService implements AutoRepairServiceMBean { - public static final String MBEAN_NAME = "org.apache.cassandra.db:type=UnifiedRepairService"; + public static final String MBEAN_NAME = "org.apache.cassandra.db:type=AutoRepairService"; @VisibleForTesting - protected UnifiedRepairConfig config; + protected AutoRepairConfig config; - public static final UnifiedRepairService instance = new UnifiedRepairService(); + public static final AutoRepairService instance = new AutoRepairService(); @VisibleForTesting - protected UnifiedRepairService() + protected AutoRepairService() { } public static void setup() { - instance.config = DatabaseDescriptor.getUnifiedRepairConfig(); + instance.config = DatabaseDescriptor.getAutoRepairConfig(); } static @@ -58,8 +58,8 @@ public static void setup() public void checkCanRun(RepairType repairType) { - if (!config.isUnifiedRepairSchedulingEnabled()) - throw new ConfigurationException("Unified-repair scheduller is disabled."); + if (!config.isAutoRepairSchedulingEnabled()) + throw new ConfigurationException("Auto-repair scheduller is disabled."); if (repairType != RepairType.incremental) return; @@ -72,16 +72,16 @@ public void checkCanRun(RepairType repairType) } @Override - public UnifiedRepairConfig getUnifiedRepairConfig() + public AutoRepairConfig getAutoRepairConfig() { return config; } @Override - public void setUnifiedRepairEnabled(RepairType repairType, boolean enabled) + public void setAutoRepairEnabled(RepairType repairType, boolean enabled) { checkCanRun(repairType); - config.setUnifiedRepairEnabled(repairType, enabled); + config.setAutoRepairEnabled(repairType, enabled); } @Override @@ -93,18 +93,18 @@ public void setRepairThreads(RepairType repairType, int repairThreads) @Override public void setRepairPriorityForHosts(RepairType repairType, Set hosts) { - UnifiedRepairUtils.addPriorityHosts(repairType, hosts); + AutoRepairUtils.addPriorityHosts(repairType, hosts); } @Override public Set getRepairHostPriority(RepairType repairType) { - return UnifiedRepairUtils.getPriorityHosts(repairType); + return AutoRepairUtils.getPriorityHosts(repairType); } @Override public void setForceRepairForHosts(RepairType repairType, Set hosts) { - UnifiedRepairUtils.setForceRepair(repairType, hosts); + AutoRepairUtils.setForceRepair(repairType, hosts); } @Override @@ -125,19 +125,19 @@ public void startScheduler() config.startScheduler(); } - public void setUnifiedRepairHistoryClearDeleteHostsBufferDuration(String duration) + public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) { - config.setUnifiedRepairHistoryClearDeleteHostsBufferInterval(duration); + config.setAutoRepairHistoryClearDeleteHostsBufferInterval(duration); } @Override - public void setUnifiedRepairMaxRetriesCount(int retries) + public void setAutoRepairMaxRetriesCount(int retries) { config.setRepairMaxRetries(retries); } @Override - public void setUnifiedRepairRetryBackoff(String interval) + public void setAutoRepairRetryBackoff(String interval) { config.setRepairRetryBackoff(interval); } @@ -149,9 +149,9 @@ public void setRepairSSTableCountHigherThreshold(RepairType repairType, int ssta } @Override - public void setUnifiedRepairTableMaxRepairTime(RepairType repairType, String unifiedRepairTableMaxRepairTime) + public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime) { - config.setUnifiedRepairTableMaxRepairTime(repairType, unifiedRepairTableMaxRepairTime); + config.setAutoRepairTableMaxRepairTime(repairType, autoRepairTableMaxRepairTime); } @Override @@ -192,12 +192,12 @@ public void setRepairSessionTimeout(RepairType repairType, String timeout) public Set getOnGoingRepairHostIds(RepairType rType) { Set hostIds = new HashSet<>(); - List histories = UnifiedRepairUtils.getUnifiedRepairHistory(rType); + List histories = AutoRepairUtils.getAutoRepairHistory(rType); if (histories == null) { return hostIds; } - UnifiedRepairUtils.CurrentRepairStatus currentRepairStatus = new UnifiedRepairUtils.CurrentRepairStatus(histories, UnifiedRepairUtils.getPriorityHostIds(rType)); + AutoRepairUtils.CurrentRepairStatus currentRepairStatus = new AutoRepairUtils.CurrentRepairStatus(histories, AutoRepairUtils.getPriorityHostIds(rType)); for (UUID id : currentRepairStatus.hostIdsWithOnGoingRepair) { hostIds.add(id.toString()); diff --git a/src/java/org/apache/cassandra/service/UnifiedRepairServiceMBean.java b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java similarity index 74% rename from src/java/org/apache/cassandra/service/UnifiedRepairServiceMBean.java rename to src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java index ec5a71bdcddd..121c9a480303 100644 --- a/src/java/org/apache/cassandra/service/UnifiedRepairServiceMBean.java +++ b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java @@ -18,17 +18,17 @@ package org.apache.cassandra.service; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; import java.util.Set; -public interface UnifiedRepairServiceMBean +public interface AutoRepairServiceMBean { /** - * Enable or disable unified-repair for a given repair type + * Enable or disable auto-repair for a given repair type */ - public void setUnifiedRepairEnabled(RepairType repairType, boolean enabled); + public void setAutoRepairEnabled(RepairType repairType, boolean enabled); public void setRepairThreads(RepairType repairType, int repairThreads); @@ -44,15 +44,15 @@ public interface UnifiedRepairServiceMBean void startScheduler(); - public void setUnifiedRepairHistoryClearDeleteHostsBufferDuration(String duration); + public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration); - public void setUnifiedRepairMaxRetriesCount(int retries); + public void setAutoRepairMaxRetriesCount(int retries); - public void setUnifiedRepairRetryBackoff(String interval); + public void setAutoRepairRetryBackoff(String interval); public void setRepairSSTableCountHigherThreshold(RepairType repairType, int ssTableHigherThreshold); - public void setUnifiedRepairTableMaxRepairTime(RepairType repairType, String unifiedRepairTableMaxRepairTime); + public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime); public void setIgnoreDCs(RepairType repairType, Set ignorDCs); @@ -64,7 +64,7 @@ public interface UnifiedRepairServiceMBean public void setMVRepairEnabled(RepairType repairType, boolean enabled); - public UnifiedRepairConfig getUnifiedRepairConfig(); + public AutoRepairConfig getAutoRepairConfig(); public void setRepairSessionTimeout(RepairType repairType, String timeout); diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index 3df8b57a9671..35197c3e1866 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -405,7 +405,7 @@ protected void setup() AuditLogManager.instance.initialize(); - StorageService.instance.doUnifiedRepairSetup(); + StorageService.instance.doAutoRepairSetup(); // schedule periodic background compaction task submission. this is simply a backstop against compactions stalling // due to scheduling errors or race conditions diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 3af0889a46aa..7eb10e0542b0 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -76,7 +76,7 @@ import com.google.common.util.concurrent.Uninterruptibles; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepair; +import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1130,15 +1130,15 @@ public void doAuthSetup(boolean async) } } - public void doUnifiedRepairSetup() + public void doAutoRepairSetup() { - UnifiedRepairService.setup(); - if (DatabaseDescriptor.getUnifiedRepairConfig().isUnifiedRepairSchedulingEnabled()) + AutoRepairService.setup(); + if (DatabaseDescriptor.getAutoRepairConfig().isAutoRepairSchedulingEnabled()) { - logger.info("Enable unified-repair scheduling"); - UnifiedRepair.instance.setup(); + logger.info("Enable auto-repair scheduling"); + AutoRepair.instance.setup(); } - logger.info("UnifiedRepair setup complete!"); + logger.info("AutoRepair setup complete!"); } diff --git a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java index 9119892af483..d3ffb9f92cf0 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java +++ b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java @@ -39,7 +39,7 @@ import org.apache.cassandra.locator.EndpointsByReplica; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.StreamState; @@ -219,7 +219,7 @@ public SequenceState executeNext() return halted(); } // this node might have just bootstrapped; check if we should run repair immediately - UnifiedRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); + AutoRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED); } else diff --git a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java index a827bfcf7c12..93251b354f2f 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java +++ b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java @@ -43,7 +43,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.EndpointsByReplica; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; @@ -218,7 +218,7 @@ public SequenceState executeNext() return halted(); } // this node might have just bootstrapped; check if we should run repair immediately - UnifiedRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); + AutoRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED); } else diff --git a/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java b/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java index 19d57af35d24..aad049ae7fa2 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java +++ b/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java @@ -29,7 +29,7 @@ import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.locator.EndpointsByReplica; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; @@ -87,7 +87,7 @@ public static void streamData(NodeId nodeId, ClusterMetadata metadata, boolean s throw new IllegalStateException("Could not finish join for during replacement"); } // this node might have just bootstrapped; check if we should run repair immediately - UnifiedRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); + AutoRepairUtils.runRepairOnNewlyBootstrappedNodeIfEnabled(); } if (finishJoiningRing) diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 30aeaf63aa6e..987616ddb338 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -101,10 +101,10 @@ import org.apache.cassandra.metrics.ThreadPoolMetrics; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.MessagingServiceMBean; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.service.ActiveRepairServiceMBean; -import org.apache.cassandra.service.UnifiedRepairService; -import org.apache.cassandra.service.UnifiedRepairServiceMBean; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.AutoRepairServiceMBean; import org.apache.cassandra.service.CacheService; import org.apache.cassandra.service.CacheServiceMBean; import org.apache.cassandra.tcm.CMSOperationsMBean; @@ -176,7 +176,7 @@ public class NodeProbe implements AutoCloseable protected CIDRGroupsMappingManagerMBean cmbProxy; protected PermissionsCacheMBean pcProxy; protected RolesCacheMBean rcProxy; - protected UnifiedRepairServiceMBean unifiedRepairProxy; + protected AutoRepairServiceMBean autoRepairProxy; protected Output output; private boolean failed; @@ -319,8 +319,8 @@ protected void connect() throws IOException name = new ObjectName(CIDRFilteringMetricsTable.MBEAN_NAME); cfmProxy = JMX.newMBeanProxy(mbeanServerConn, name, CIDRFilteringMetricsTableMBean.class); - name = new ObjectName(UnifiedRepairService.MBEAN_NAME); - unifiedRepairProxy = JMX.newMBeanProxy(mbeanServerConn, name, UnifiedRepairServiceMBean.class); + name = new ObjectName(AutoRepairService.MBEAN_NAME); + autoRepairProxy = JMX.newMBeanProxy(mbeanServerConn, name, AutoRepairServiceMBean.class); } catch (MalformedObjectNameException e) { @@ -2416,95 +2416,95 @@ public void abortBootstrap(String nodeId, String endpoint) ssProxy.abortBootstrap(nodeId, endpoint); } - public UnifiedRepairConfig getUnifiedRepairConfig() { - return unifiedRepairProxy.getUnifiedRepairConfig(); + public AutoRepairConfig getAutoRepairConfig() { + return autoRepairProxy.getAutoRepairConfig(); } - public void setUnifiedRepairEnabled(UnifiedRepairConfig.RepairType repairType, boolean enabled) + public void setAutoRepairEnabled(AutoRepairConfig.RepairType repairType, boolean enabled) { - unifiedRepairProxy.setUnifiedRepairEnabled(repairType, enabled); + autoRepairProxy.setAutoRepairEnabled(repairType, enabled); } - public void setRepairThreads(UnifiedRepairConfig.RepairType repairType, int repairThreads) + public void setRepairThreads(AutoRepairConfig.RepairType repairType, int repairThreads) { - unifiedRepairProxy.setRepairThreads(repairType, repairThreads); + autoRepairProxy.setRepairThreads(repairType, repairThreads); } - public void setRepairPriorityForHosts(UnifiedRepairConfig.RepairType repairType, Set hosts) + public void setRepairPriorityForHosts(AutoRepairConfig.RepairType repairType, Set hosts) { - unifiedRepairProxy.setRepairPriorityForHosts(repairType, hosts); + autoRepairProxy.setRepairPriorityForHosts(repairType, hosts); } - public Set getRepairPriorityForHosts(UnifiedRepairConfig.RepairType repairType) + public Set getRepairPriorityForHosts(AutoRepairConfig.RepairType repairType) { - return unifiedRepairProxy.getRepairHostPriority(repairType); + return autoRepairProxy.getRepairHostPriority(repairType); } - public void setForceRepairForHosts(UnifiedRepairConfig.RepairType repairType, Set hosts){ - unifiedRepairProxy.setForceRepairForHosts(repairType, hosts); + public void setForceRepairForHosts(AutoRepairConfig.RepairType repairType, Set hosts){ + autoRepairProxy.setForceRepairForHosts(repairType, hosts); } - public void setRepairSubRangeNum(UnifiedRepairConfig.RepairType repairType, int repairSubRanges) + public void setRepairSubRangeNum(AutoRepairConfig.RepairType repairType, int repairSubRanges) { - unifiedRepairProxy.setRepairSubRangeNum(repairType, repairSubRanges); + autoRepairProxy.setRepairSubRangeNum(repairType, repairSubRanges); } - public void setRepairMinInterval(UnifiedRepairConfig.RepairType repairType, String minRepairInterval) + public void setRepairMinInterval(AutoRepairConfig.RepairType repairType, String minRepairInterval) { - unifiedRepairProxy.setRepairMinInterval(repairType, minRepairInterval); + autoRepairProxy.setRepairMinInterval(repairType, minRepairInterval); } - public void setUnifiedRepairHistoryClearDeleteHostsBufferDuration(String duration) + public void setAutoRepairHistoryClearDeleteHostsBufferDuration(String duration) { - unifiedRepairProxy.setUnifiedRepairHistoryClearDeleteHostsBufferDuration(duration); + autoRepairProxy.setAutoRepairHistoryClearDeleteHostsBufferDuration(duration); } public void startScheduler() { - unifiedRepairProxy.startScheduler(); + autoRepairProxy.startScheduler(); } - public void setUnifiedRepairMaxRetriesCount(int retries) + public void setAutoRepairMaxRetriesCount(int retries) { - unifiedRepairProxy.setUnifiedRepairMaxRetriesCount(retries); + autoRepairProxy.setAutoRepairMaxRetriesCount(retries); } - public void setUnifiedRepairRetryBackoff(String interval) + public void setAutoRepairRetryBackoff(String interval) { - unifiedRepairProxy.setUnifiedRepairRetryBackoff(interval); + autoRepairProxy.setAutoRepairRetryBackoff(interval); } - public void setRepairSSTableCountHigherThreshold(UnifiedRepairConfig.RepairType repairType, int ssTableHigherThreshold) + public void setRepairSSTableCountHigherThreshold(AutoRepairConfig.RepairType repairType, int ssTableHigherThreshold) { - unifiedRepairProxy.setRepairSSTableCountHigherThreshold(repairType, ssTableHigherThreshold); + autoRepairProxy.setRepairSSTableCountHigherThreshold(repairType, ssTableHigherThreshold); } - public void setUnifiedRepairTableMaxRepairTime(UnifiedRepairConfig.RepairType repairType, String unifiedRepairTableMaxRepairTime) + public void setAutoRepairTableMaxRepairTime(AutoRepairConfig.RepairType repairType, String autoRepairTableMaxRepairTime) { - unifiedRepairProxy.setUnifiedRepairTableMaxRepairTime(repairType, unifiedRepairTableMaxRepairTime); + autoRepairProxy.setAutoRepairTableMaxRepairTime(repairType, autoRepairTableMaxRepairTime); } - public void setUnifiedRepairIgnoreDCs(UnifiedRepairConfig.RepairType repairType, Set ignoreDCs) + public void setAutoRepairIgnoreDCs(AutoRepairConfig.RepairType repairType, Set ignoreDCs) { - unifiedRepairProxy.setIgnoreDCs(repairType, ignoreDCs); + autoRepairProxy.setIgnoreDCs(repairType, ignoreDCs); } - public void setParallelRepairPercentage(UnifiedRepairConfig.RepairType repairType, int percentage) { - unifiedRepairProxy.setParallelRepairPercentage(repairType, percentage); + public void setParallelRepairPercentage(AutoRepairConfig.RepairType repairType, int percentage) { + autoRepairProxy.setParallelRepairPercentage(repairType, percentage); } - public void setParallelRepairCount(UnifiedRepairConfig.RepairType repairType, int count) { - unifiedRepairProxy.setParallelRepairCount(repairType, count); + public void setParallelRepairCount(AutoRepairConfig.RepairType repairType, int count) { + autoRepairProxy.setParallelRepairCount(repairType, count); } - public void setPrimaryTokenRangeOnly(UnifiedRepairConfig.RepairType repairType, boolean primaryTokenRangeOnly) + public void setPrimaryTokenRangeOnly(AutoRepairConfig.RepairType repairType, boolean primaryTokenRangeOnly) { - unifiedRepairProxy.setPrimaryTokenRangeOnly(repairType, primaryTokenRangeOnly); + autoRepairProxy.setPrimaryTokenRangeOnly(repairType, primaryTokenRangeOnly); } - public void setMVRepairEnabled(UnifiedRepairConfig.RepairType repairType, boolean enabled) + public void setMVRepairEnabled(AutoRepairConfig.RepairType repairType, boolean enabled) { - unifiedRepairProxy.setMVRepairEnabled(repairType, enabled); + autoRepairProxy.setMVRepairEnabled(repairType, enabled); } public List mutateSSTableRepairedState(boolean repair, boolean preview, String keyspace, List tables) throws InvalidRequestException @@ -2516,14 +2516,14 @@ public List getTablesForKeyspace(String keyspace) { return ssProxy.getTablesForKeyspace(keyspace); } - public void setRepairSessionTimeout(UnifiedRepairConfig.RepairType repairType, String timeout) + public void setRepairSessionTimeout(AutoRepairConfig.RepairType repairType, String timeout) { - unifiedRepairProxy.setRepairSessionTimeout(repairType, timeout); + autoRepairProxy.setRepairSessionTimeout(repairType, timeout); } - public Set getOnGoingRepairHostIds(UnifiedRepairConfig.RepairType type) + public Set getOnGoingRepairHostIds(AutoRepairConfig.RepairType type) { - return unifiedRepairProxy.getOnGoingRepairHostIds(type); + return autoRepairProxy.getOnGoingRepairHostIds(type); } } diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index 9c002f877e06..80f8ed8f09e1 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -95,6 +95,7 @@ public int execute(String... args) { List> commands = newArrayList( AbortBootstrap.class, + AutoRepairStatus.class, Assassinate.class, CassHelp.class, CIDRFilteringStats.class, @@ -134,6 +135,7 @@ public int execute(String... args) GcStats.class, GetAuditLog.class, GetAuthCacheConfig.class, + GetAutoRepairConfig.class, GetBatchlogReplayTrottle.class, GetCIDRGroupsOfIP.class, GetColumnIndexSize.class, @@ -154,7 +156,6 @@ public int execute(String... args) GetStreamThroughput.class, GetTimeout.class, GetTraceProbability.class, - GetUnifiedRepairConfig.class, GossipInfo.class, Import.class, Info.class, @@ -197,6 +198,7 @@ public int execute(String... args) Ring.class, Scrub.class, SetAuthCacheConfig.class, + SetAutoRepairConfig.class, SetBatchlogReplayThrottle.class, SetCacheCapacity.class, SetCacheKeysToSave.class, @@ -215,7 +217,6 @@ public int execute(String... args) SetStreamThroughput.class, SetTimeout.class, SetTraceProbability.class, - SetUnifiedRepairConfig.class, Sjk.class, Snapshot.class, SSTableRepairedSet.class, @@ -232,7 +233,6 @@ public int execute(String... args) TopPartitions.class, TpStats.class, TruncateHints.class, - UnifiedRepairStatus.class, UpdateCIDRGroup.class, UpgradeSSTable.class, Verify.class, diff --git a/src/java/org/apache/cassandra/tools/nodetool/UnifiedRepairStatus.java b/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java similarity index 82% rename from src/java/org/apache/cassandra/tools/nodetool/UnifiedRepairStatus.java rename to src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java index b40fe887e5d8..7b96102c6698 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/UnifiedRepairStatus.java +++ b/src/java/org/apache/cassandra/tools/nodetool/AutoRepairStatus.java @@ -25,19 +25,19 @@ import io.airlift.airline.Command; import io.airlift.airline.Option; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool; import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; import static com.google.common.base.Preconditions.checkArgument; -@Command(name = "unifiedrepairstatus", description = "Print unifiedrepair status") -public class UnifiedRepairStatus extends NodeTool.NodeToolCmd +@Command(name = "autorepairstatus", description = "Print autorepair status") +public class AutoRepairStatus extends NodeTool.NodeToolCmd { @VisibleForTesting @Option(title = "repair type", name = { "-t", "--repair-type" }, description = "Repair type") - protected UnifiedRepairConfig.RepairType repairType; + protected AutoRepairConfig.RepairType repairType; @Override public void execute(NodeProbe probe) @@ -45,10 +45,10 @@ public void execute(NodeProbe probe) checkArgument(repairType != null, "--repair-type is required."); PrintStream out = probe.output().out; - UnifiedRepairConfig config = probe.getUnifiedRepairConfig(); - if (config == null || !config.isUnifiedRepairSchedulingEnabled()) + AutoRepairConfig config = probe.getAutoRepairConfig(); + if (config == null || !config.isAutoRepairSchedulingEnabled()) { - out.println("Unified-repair is not enabled"); + out.println("Auto-repair is not enabled"); return; } diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetUnifiedRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java similarity index 82% rename from src/java/org/apache/cassandra/tools/nodetool/GetUnifiedRepairConfig.java rename to src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java index 336063eb2405..48f8d54de3f6 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/GetUnifiedRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java @@ -20,15 +20,15 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import io.airlift.airline.Command; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; import java.io.PrintStream; -@Command(name = "getunifiedrepairconfig", description = "Print unifiedrepair configurations") -public class GetUnifiedRepairConfig extends NodeToolCmd +@Command(name = "getautorepairconfig", description = "Print autorepair configurations") +public class GetAutoRepairConfig extends NodeToolCmd { @VisibleForTesting protected static PrintStream out = System.out; @@ -36,17 +36,17 @@ public class GetUnifiedRepairConfig extends NodeToolCmd @Override public void execute(NodeProbe probe) { - UnifiedRepairConfig config = probe.getUnifiedRepairConfig(); - if (config == null || !config.isUnifiedRepairSchedulingEnabled()) + AutoRepairConfig config = probe.getAutoRepairConfig(); + if (config == null || !config.isAutoRepairSchedulingEnabled()) { - out.println("Unified-repair is not enabled"); + out.println("Auto-repair is not enabled"); return; } StringBuilder sb = new StringBuilder(); sb.append("repair scheduler configuration:"); sb.append("\n\trepair eligibility check interval: " + config.getRepairCheckInterval()); - sb.append("\n\tTTL for repair history for dead nodes: " + config.getUnifiedRepairHistoryClearDeleteHostsBufferInterval()); + sb.append("\n\tTTL for repair history for dead nodes: " + config.getAutoRepairHistoryClearDeleteHostsBufferInterval()); sb.append("\n\tmax retries for repair: " + config.getRepairMaxRetries()); sb.append("\n\tretry backoff: " + config.getRepairRetryBackoff()); for (RepairType repairType : RepairType.values()) @@ -57,17 +57,17 @@ public void execute(NodeProbe probe) out.println(sb); } - private String formatRepairTypeConfig(NodeProbe probe, RepairType repairType, UnifiedRepairConfig config) + private String formatRepairTypeConfig(NodeProbe probe, RepairType repairType, AutoRepairConfig config) { StringBuilder sb = new StringBuilder(); sb.append("\nconfiguration for repair type: " + repairType); - sb.append("\n\tenabled: " + config.isUnifiedRepairEnabled(repairType)); + sb.append("\n\tenabled: " + config.isAutoRepairEnabled(repairType)); sb.append("\n\tminimum repair interval: " + config.getRepairMinInterval(repairType)); sb.append("\n\trepair threads: " + config.getRepairThreads(repairType)); sb.append("\n\tnumber of repair subranges: " + config.getRepairSubRangeNum(repairType)); sb.append("\n\tpriority hosts: " + Joiner.on(',').skipNulls().join(probe.getRepairPriorityForHosts(repairType))); sb.append("\n\tsstable count higher threshold: " + config.getRepairSSTableCountHigherThreshold(repairType)); - sb.append("\n\ttable max repair time in sec: " + config.getUnifiedRepairTableMaxRepairTime(repairType)); + sb.append("\n\ttable max repair time in sec: " + config.getAutoRepairTableMaxRepairTime(repairType)); sb.append("\n\tignore datacenters: " + Joiner.on(',').skipNulls().join(config.getIgnoreDCs(repairType))); sb.append("\n\trepair primary token-range: " + config.getRepairPrimaryTokenRangeOnly(repairType)); sb.append("\n\tnumber of parallel repairs within group: " + config.getParallelRepairCount(repairType)); diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetUnifiedRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java similarity index 83% rename from src/java/org/apache/cassandra/tools/nodetool/SetUnifiedRepairConfig.java rename to src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java index fe123c3de1ac..2929c944442a 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/SetUnifiedRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java @@ -24,7 +24,7 @@ import io.airlift.airline.Command; import io.airlift.airline.Option; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; @@ -37,12 +37,12 @@ import static com.google.common.base.Preconditions.checkArgument; -@Command(name = "setunifiedrepairconfig", description = "sets the unifiedrepair configuration") -public class SetUnifiedRepairConfig extends NodeToolCmd +@Command(name = "setautorepairconfig", description = "sets the autorepair configuration") +public class SetAutoRepairConfig extends NodeToolCmd { @VisibleForTesting - @Arguments(title = " ", usage = " ", - description = "unifiedrepair param and value.\nPossible unifiedrepair parameters are as following: " + + @Arguments(title = " ", usage = " ", + description = "autorepair param and value.\nPossible autorepair parameters are as following: " + "[start_scheduler|number_of_repair_threads|number_of_subranges|min_repair_interval|sstable_upper_threshold" + "|enabled|table_max_repair_time|priority_hosts|forcerepair_hosts|ignore_dcs" + "|history_clear_delete_hosts_buffer_interval|repair_primary_token_range_only" + @@ -60,13 +60,13 @@ public class SetUnifiedRepairConfig extends NodeToolCmd @Override public void execute(NodeProbe probe) { - checkArgument(args.size() == 2, "setunifiedrepairconfig requires param-type, and value args."); + checkArgument(args.size() == 2, "setautorepairconfig requires param-type, and value args."); String paramType = args.get(0); String paramVal = args.get(1); - if (!probe.getUnifiedRepairConfig().isUnifiedRepairSchedulingEnabled() && !paramType.equalsIgnoreCase("start_scheduler")) + if (!probe.getAutoRepairConfig().isAutoRepairSchedulingEnabled() && !paramType.equalsIgnoreCase("start_scheduler")) { - out.println("Unified-repair is not enabled"); + out.println("Auto-repair is not enabled"); return; } @@ -80,13 +80,13 @@ public void execute(NodeProbe probe) } return; case "history_clear_delete_hosts_buffer_interval": - probe.setUnifiedRepairHistoryClearDeleteHostsBufferDuration(paramVal); + probe.setAutoRepairHistoryClearDeleteHostsBufferDuration(paramVal); return; case "repair_max_retries": - probe.setUnifiedRepairMaxRetriesCount(Integer.parseInt(paramVal)); + probe.setAutoRepairMaxRetriesCount(Integer.parseInt(paramVal)); return; case "repair_retry_backoff": - probe.setUnifiedRepairRetryBackoff(paramVal); + probe.setAutoRepairRetryBackoff(paramVal); return; default: // proceed to options that require --repair-type option @@ -99,7 +99,7 @@ public void execute(NodeProbe probe) switch (paramType) { case "enabled": - probe.setUnifiedRepairEnabled(repairType, Boolean.parseBoolean(paramVal)); + probe.setAutoRepairEnabled(repairType, Boolean.parseBoolean(paramVal)); break; case "number_of_repair_threads": probe.setRepairThreads(repairType, Integer.parseInt(paramVal)); @@ -114,7 +114,7 @@ public void execute(NodeProbe probe) probe.setRepairSSTableCountHigherThreshold(repairType, Integer.parseInt(paramVal)); break; case "table_max_repair_time": - probe.setUnifiedRepairTableMaxRepairTime(repairType, paramVal); + probe.setAutoRepairTableMaxRepairTime(repairType, paramVal); break; case "priority_hosts": hosts = retrieveHosts(paramVal); @@ -136,7 +136,7 @@ public void execute(NodeProbe probe) { ignoreDCs.add(dc); } - probe.setUnifiedRepairIgnoreDCs(repairType, ignoreDCs); + probe.setAutoRepairIgnoreDCs(repairType, ignoreDCs); break; case "repair_primary_token_range_only": probe.setPrimaryTokenRangeOnly(repairType, Boolean.parseBoolean(paramVal)); diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index d5332ec76953..b2dcedde48d4 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -69,7 +69,7 @@ import org.apache.cassandra.config.ParameterizedClass; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.repair.unifiedrepair.IUnifiedRepairTokenRangeSplitter; +import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -721,11 +721,11 @@ public static AbstractCryptoProvider newCryptoProvider(String className, Map parameters = parameterizedClass.parameters != null ? parameterizedClass.parameters : Collections.emptyMap(); // first attempt to initialize with Map arguments. - return (IUnifiedRepairTokenRangeSplitter) tokenRangeSplitterClass.getConstructor(Map.class).newInstance(parameters); + return (IAutoRepairTokenRangeSplitter) tokenRangeSplitterClass.getConstructor(Map.class).newInstance(parameters); } catch (NoSuchMethodException nsme) { // fall back on no argument constructor. - return (IUnifiedRepairTokenRangeSplitter) tokenRangeSplitterClass.getConstructor().newInstance(); + return (IAutoRepairTokenRangeSplitter) tokenRangeSplitterClass.getConstructor().newInstance(); } } catch (Exception ex) { - throw new ConfigurationException("Unable to create instance of IUnifiedRepairTokenRangeSplitter for " + className, ex); + throw new ConfigurationException("Unable to create instance of IAutoRepairTokenRangeSplitter for " + className, ex); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/UnifiedRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java similarity index 83% rename from test/distributed/org/apache/cassandra/distributed/test/repair/UnifiedRepairSchedulerTest.java rename to test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java index 8460ec119cee..9b583f006507 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/repair/UnifiedRepairSchedulerTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java @@ -35,14 +35,14 @@ import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.test.TestBaseImpl; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepair; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; -import org.apache.cassandra.service.UnifiedRepairService; +import org.apache.cassandra.repair.autorepair.AutoRepair; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.service.AutoRepairService; import static org.apache.cassandra.schema.SchemaConstants.DISTRIBUTED_KEYSPACE_NAME; import static org.junit.Assert.assertEquals; -public class UnifiedRepairSchedulerTest extends TestBaseImpl +public class AutoRepairSchedulerTest extends TestBaseImpl { private static Cluster cluster; @@ -57,25 +57,25 @@ public static void init() throws IOException sdf = new SimpleDateFormat(pattern); sdf.setLenient(false); cluster = Cluster.build(3).withConfig(config -> config - .set("unified_repair", + .set("auto_repair", ImmutableMap.of( "repair_type_overrides", - ImmutableMap.of(UnifiedRepairConfig.RepairType.full.toString(), - ImmutableMap.of( + ImmutableMap.of(AutoRepairConfig.RepairType.full.toString(), + ImmutableMap.of( "initial_scheduler_delay", "5s", "enabled", "true", "parallel_repair_count", "1", "parallel_repair_percentage", "0", "min_repair_interval", "1s"), - UnifiedRepairConfig.RepairType.incremental.toString(), - ImmutableMap.of( + AutoRepairConfig.RepairType.incremental.toString(), + ImmutableMap.of( "initial_scheduler_delay", "5s", "enabled", "true", "parallel_repair_count", "1", "parallel_repair_percentage", "0", "min_repair_interval", "1s")))) - .set("unified_repair.enabled", "true") - .set("unified_repair.repair_check_interval", "10s")).start(); + .set("auto_repair.enabled", "true") + .set("auto_repair.repair_check_interval", "10s")).start(); cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};"); cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (pk int, ck text, v1 int, v2 int, PRIMARY KEY (pk, ck)) WITH read_repair='NONE'")); @@ -85,7 +85,7 @@ public static void init() throws IOException public void testScheduler() throws ParseException { // ensure there was no history of previous repair runs through the scheduler - Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY), ConsistencyLevel.QUORUM); + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY), ConsistencyLevel.QUORUM); assertEquals(0, rows.length); cluster.forEach(i -> i.runOnInstance(() -> { @@ -93,9 +93,9 @@ public void testScheduler() throws ParseException { DatabaseDescriptor.setCDCOnRepairEnabled(false); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); - UnifiedRepairService.instance.setup(); + AutoRepairService.instance.setup(); DatabaseDescriptor.setCDCOnRepairEnabled(false); - UnifiedRepair.instance.setup(); + AutoRepair.instance.setup(); } catch (Exception e) { @@ -105,13 +105,13 @@ public void testScheduler() throws ParseException // wait for a couple of minutes for repair to go through on all three nodes Uninterruptibles.sleepUninterruptibly(2, TimeUnit.MINUTES); - validate(UnifiedRepairConfig.RepairType.full.toString()); - validate(UnifiedRepairConfig.RepairType.incremental.toString()); + validate(AutoRepairConfig.RepairType.full.toString()); + validate(AutoRepairConfig.RepairType.incremental.toString()); } private void validate(String repairType) throws ParseException { - Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s where repair_type='%s'", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, repairType), ConsistencyLevel.QUORUM); + Object[][] rows = cluster.coordinator(1).execute(String.format("SELECT repair_type, host_id, repair_start_ts, repair_finish_ts, repair_turn FROM %s.%s where repair_type='%s'", DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType), ConsistencyLevel.QUORUM); assertEquals(3, rows.length); for (int node = 0; node < rows.length; node++) { diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java index 71fa6c46175b..74fe87dbaf16 100644 --- a/test/unit/org/apache/cassandra/Util.java +++ b/test/unit/org/apache/cassandra/Util.java @@ -62,7 +62,7 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.config.Config; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.commons.lang3.StringUtils; import org.junit.Assume; import org.slf4j.Logger; @@ -1288,11 +1288,11 @@ public static RuntimeException testMustBeImplementedForSSTableFormat() return new UnsupportedOperationException("Test must be implemented for sstable format " + DatabaseDescriptor.getSelectedSSTableFormat().getClass().getName()); } - // Replaces the global unified-repair config with a new config where unified-repair schedulling is enabled/disabled - public static void setUnifiedRepairEnabled(boolean enabled) throws Exception + // Replaces the global auto-repair config with a new config where auto-repair schedulling is enabled/disabled + public static void setAutoRepairEnabled(boolean enabled) throws Exception { Config config = DatabaseDescriptor.getRawConfig(); - config.unified_repair = new UnifiedRepairConfig(enabled); + config.auto_repair = new AutoRepairConfig(enabled); Field configField = DatabaseDescriptor.class.getDeclaredField("conf"); configField.setAccessible(true); configField.set(null, config); diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java index 42aa930968f6..490f2534baf2 100644 --- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java +++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java @@ -103,14 +103,14 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.config.ConfigurationLoader", "org.apache.cassandra.config.Config$CorruptedTombstoneStrategy", "org.apache.cassandra.config.Config$BatchlogEndpointStrategy", - "org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig", - "org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig$Options", - "org.apache.cassandra.repair.unifiedrepair.DefaultUnifiedRepairTokenSplitter", - "org.apache.cassandra.repair.unifiedrepair.IUnifiedRepairTokenRangeSplitter", - "org.apache.cassandra.repair.unifiedrepair.FullRepairState", - "org.apache.cassandra.repair.unifiedrepair.IncrementalRepairState", - "org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig$RepairType", - "org.apache.cassandra.repair.unifiedrepair.UnifiedRepairState", + "org.apache.cassandra.repair.autorepair.AutoRepairConfig", + "org.apache.cassandra.repair.autorepair.AutoRepairConfig$Options", + "org.apache.cassandra.repair.autorepair.DefaultAutoRepairTokenSplitter", + "org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter", + "org.apache.cassandra.repair.autorepair.FullRepairState", + "org.apache.cassandra.repair.autorepair.IncrementalRepairState", + "org.apache.cassandra.repair.autorepair.AutoRepairConfig$RepairType", + "org.apache.cassandra.repair.autorepair.AutoRepairState", "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/config/YamlConfigurationLoaderTest.java b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java index 946496cfe0bf..141c762ba461 100644 --- a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java +++ b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java @@ -42,7 +42,7 @@ import static org.apache.cassandra.config.YamlConfigurationLoader.SYSTEM_PROPERTY_PREFIX; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -273,7 +273,7 @@ public void fromMapTest() Map encryptionOptions = ImmutableMap.of("cipher_suites", Collections.singletonList("FakeCipher"), "optional", false, "enabled", true); - Map unifiedRepairConfig = ImmutableMap.of("enabled", true, + Map autoRepairConfig = ImmutableMap.of("enabled", true, "global_settings", ImmutableMap.of("number_of_repair_threads", 1), "repair_type_overrides", ImmutableMap.of( @@ -287,7 +287,7 @@ public void fromMapTest() .put("internode_socket_send_buffer_size", "5B") .put("internode_socket_receive_buffer_size", "5B") .put("commitlog_sync_group_window_in_ms", "42") - .put("unified_repair", unifiedRepairConfig) + .put("auto_repair", autoRepairConfig) .build(); Config config = YamlConfigurationLoader.fromMap(map, Config.class); @@ -298,9 +298,9 @@ public void fromMapTest() assertEquals(true, config.client_encryption_options.enabled); // Check a nested object assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_send_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_receive_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) - assertEquals(true, config.unified_repair.enabled); - assertEquals(new DurationSpec.IntSecondsBound("6h"), config.unified_repair.getUnifiedRepairTableMaxRepairTime(UnifiedRepairConfig.RepairType.incremental)); - config.unified_repair.setMVRepairEnabled(UnifiedRepairConfig.RepairType.incremental, false); + assertEquals(true, config.auto_repair.enabled); + assertEquals(new DurationSpec.IntSecondsBound("6h"), config.auto_repair.getAutoRepairTableMaxRepairTime(AutoRepairConfig.RepairType.incremental)); + config.auto_repair.setMVRepairEnabled(AutoRepairConfig.RepairType.incremental, false); } @Test @@ -497,4 +497,4 @@ public static Config load(String path) } return new YamlConfigurationLoader().loadConfig(url); } -} +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java similarity index 77% rename from test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfigTest.java rename to test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java index e7ce54a1cea5..4f1dd029ef5e 100644 --- a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.util.EnumMap; import java.util.Objects; @@ -35,7 +35,7 @@ import org.apache.cassandra.config.ParameterizedClass; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.Options; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.Options; import org.apache.cassandra.utils.FBUtilities; import static org.junit.Assert.assertEquals; @@ -44,79 +44,79 @@ import static org.junit.Assert.assertTrue; @RunWith(Parameterized.class) -public class UnifiedRepairConfigTest extends CQLTester +public class AutoRepairConfigTest extends CQLTester { - private UnifiedRepairConfig config; + private AutoRepairConfig config; private Set testSet = ImmutableSet.of("dc1"); @Parameterized.Parameter - public UnifiedRepairConfig.RepairType repairType; + public AutoRepairConfig.RepairType repairType; @Parameterized.Parameters public static Object[] repairTypes() { - return UnifiedRepairConfig.RepairType.values(); + return AutoRepairConfig.RepairType.values(); } @Before public void setUp() { - config = new UnifiedRepairConfig(true); + config = new AutoRepairConfig(true); config.repair_type_overrides = null; - UnifiedRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); } @Test - public void unifiedRepairConfigDefaultsAreNotNull() + public void autoRepairConfigDefaultsAreNotNull() { - UnifiedRepairConfig config = new UnifiedRepairConfig(); + AutoRepairConfig config = new AutoRepairConfig(); assertNotNull(config.global_settings); } @Test - public void unifiedRepairConfigRepairTypesAreNotNull() + public void autoRepairConfigRepairTypesAreNotNull() { - UnifiedRepairConfig config = new UnifiedRepairConfig(); - for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) + AutoRepairConfig config = new AutoRepairConfig(); + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { assertNotNull(config.repair_type_overrides.get(repairType)); } } @Test - public void testIsUnifiedRepairEnabledReturnsTrueWhenRepairIsEnabled() + public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsEnabled() { config.global_settings.enabled = true; - assertTrue(config.isUnifiedRepairEnabled(repairType)); + assertTrue(config.isAutoRepairEnabled(repairType)); } @Test - public void testIsUnifiedRepairEnabledReturnsTrueWhenRepairIsDisabledGlobally() + public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledGlobally() { - config = new UnifiedRepairConfig(false); + config = new AutoRepairConfig(false); config.global_settings.enabled = true; - assertFalse(config.isUnifiedRepairEnabled(repairType)); + assertFalse(config.isAutoRepairEnabled(repairType)); } @Test - public void testIsUnifiedRepairEnabledReturnsTrueWhenRepairIsDisabledForRepairType() + public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledForRepairType() { config.global_settings.enabled = true; - config.repair_type_overrides = new EnumMap<>(UnifiedRepairConfig.RepairType.class); + config.repair_type_overrides = new EnumMap<>(AutoRepairConfig.RepairType.class); config.repair_type_overrides.put(repairType, new Options()); config.repair_type_overrides.get(repairType).enabled = false; - assertFalse(config.isUnifiedRepairEnabled(repairType)); + assertFalse(config.isAutoRepairEnabled(repairType)); } @Test - public void testSetUnifiedRepairEnabledNoMVOrCDC() + public void testSetAutoRepairEnabledNoMVOrCDC() { DatabaseDescriptor.setCDCEnabled(false); DatabaseDescriptor.setMaterializedViewsEnabled(false); - config.setUnifiedRepairEnabled(repairType, true); + config.setAutoRepairEnabled(repairType, true); assertTrue(config.repair_type_overrides.get(repairType).enabled); } @@ -194,19 +194,19 @@ public void testSetRepairMinFrequencyInHours() } @Test - public void testGetUnifiedRepairHistoryClearDeleteHostsBufferInSec() + public void testGetAutoRepairHistoryClearDeleteHostsBufferInSec() { config.history_clear_delete_hosts_buffer_interval = new DurationSpec.IntSecondsBound("5s"); - int result = config.getUnifiedRepairHistoryClearDeleteHostsBufferInterval().toSeconds(); + int result = config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds(); assertEquals(5, result); } @Test - public void testSetUnifiedRepairHistoryClearDeleteHostsBufferInSec() + public void testSetAutoRepairHistoryClearDeleteHostsBufferInSec() { - config.setUnifiedRepairHistoryClearDeleteHostsBufferInterval("5s"); + config.setAutoRepairHistoryClearDeleteHostsBufferInterval("5s"); assert Objects.equals(config.history_clear_delete_hosts_buffer_interval, new DurationSpec.IntSecondsBound("5s")); } @@ -230,19 +230,19 @@ public void testSetRepairSSTableCountHigherThreshold() } @Test - public void testGetUnifiedRepairTableMaxRepairTimeInSec() + public void testGetAutoRepairTableMaxRepairTimeInSec() { config.global_settings.table_max_repair_time = new DurationSpec.IntSecondsBound("5s"); - DurationSpec.IntSecondsBound result = config.getUnifiedRepairTableMaxRepairTime(repairType); + DurationSpec.IntSecondsBound result = config.getAutoRepairTableMaxRepairTime(repairType); assertEquals(5, result.toSeconds()); } @Test - public void testSetUnifiedRepairTableMaxRepairTimeInSec() + public void testSetAutoRepairTableMaxRepairTimeInSec() { - config.setUnifiedRepairTableMaxRepairTime(repairType, "5s"); + config.setAutoRepairTableMaxRepairTime(repairType, "5s"); assert config.repair_type_overrides.get(repairType).table_max_repair_time.toSeconds() == 5; } @@ -356,19 +356,19 @@ public void testGetForceRepairNewNode() } @Test - public void testIsUnifiedRepairSchedulingEnabledDefault() + public void testIsAutoRepairSchedulingEnabledDefault() { - config = new UnifiedRepairConfig(); + config = new AutoRepairConfig(); - boolean result = config.isUnifiedRepairSchedulingEnabled(); + boolean result = config.isAutoRepairSchedulingEnabled(); assertFalse(result); } @Test - public void testIsUnifiedRepairSchedulingEnabledTrue() + public void testIsAutoRepairSchedulingEnabledTrue() { - boolean result = config.isUnifiedRepairSchedulingEnabled(); + boolean result = config.isAutoRepairSchedulingEnabled(); assertTrue(result); } @@ -386,16 +386,16 @@ public void testGetDefaultOptionsTokenRangeSplitter() { Options defaultOptions = Options.getDefaultOptions(); - ParameterizedClass expectedDefault = new ParameterizedClass(DefaultUnifiedRepairTokenSplitter.class.getName(), Collections.emptyMap()); + ParameterizedClass expectedDefault = new ParameterizedClass(DefaultAutoRepairTokenSplitter.class.getName(), Collections.emptyMap()); assertEquals(expectedDefault, defaultOptions.token_range_splitter); - assertEquals(DefaultUnifiedRepairTokenSplitter.class.getName(), FBUtilities.newUnifiedRepairTokenRangeSplitter(defaultOptions.token_range_splitter).getClass().getName()); + assertEquals(DefaultAutoRepairTokenSplitter.class.getName(), FBUtilities.newAutoRepairTokenRangeSplitter(defaultOptions.token_range_splitter).getClass().getName()); } @Test(expected = ConfigurationException.class) public void testInvalidTokenRangeSplitter() { - FBUtilities.newUnifiedRepairTokenRangeSplitter(new ParameterizedClass("invalid-class", Collections.emptyMap())); + FBUtilities.newAutoRepairTokenRangeSplitter(new ParameterizedClass("invalid-class", Collections.emptyMap())); } @Test diff --git a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairDefaultTokenSplitterParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java similarity index 84% rename from test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairDefaultTokenSplitterParameterizedTest.java rename to test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java index f14f6acefeac..804d0a712b48 100644 --- a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairDefaultTokenSplitterParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.util.ArrayList; import java.util.Arrays; @@ -37,8 +37,8 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.index.sai.disk.format.Version; -import org.apache.cassandra.repair.unifiedrepair.IUnifiedRepairTokenRangeSplitter.RepairAssignment; -import org.apache.cassandra.service.UnifiedRepairService; +import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; +import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; @@ -48,7 +48,7 @@ import static org.junit.Assert.assertEquals; @RunWith(Parameterized.class) -public class UnifiedRepairDefaultTokenSplitterParameterizedTest +public class AutoRepairDefaultTokenSplitterParameterizedTest { private static final String KEYSPACE = "ks"; private static final String TABLE1 = "tbl1"; @@ -56,12 +56,12 @@ public class UnifiedRepairDefaultTokenSplitterParameterizedTest private static final String TABLE3 = "tbl3"; @Parameterized.Parameter() - public UnifiedRepairConfig.RepairType repairType; + public AutoRepairConfig.RepairType repairType; @Parameterized.Parameters(name = "repairType={0}") - public static Collection repairTypes() + public static Collection repairTypes() { - return Arrays.asList(UnifiedRepairConfig.RepairType.values()); + return Arrays.asList(AutoRepairConfig.RepairType.values()); } @BeforeClass @@ -83,7 +83,7 @@ public static void setupClass() throws Exception ServerTestUtils.registerLocal(tokens); // Ensure that the on-disk format statics are loaded before the test run Version.LATEST.onDiskFormat(); - StorageService.instance.doUnifiedRepairSetup(); + StorageService.instance.doAutoRepairSetup(); SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE)); @@ -111,7 +111,7 @@ private static void appendExpectedTokens(long left, long right, int numberOfSpli @Test public void testTokenRangesSplitByTable() { - UnifiedRepairService.instance.getUnifiedRepairConfig().setRepairByKeyspace(repairType, false); + AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(repairType, false); int totalTokenRanges = 3; Collection> tokens = StorageService.instance.getPrimaryRanges(KEYSPACE); assertEquals(totalTokenRanges, tokens.size()); @@ -125,9 +125,9 @@ public void testTokenRangesSplitByTable() appendExpectedTokens(256, 1024, numberOfSplits, expectedToken); } - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairSubRangeNum(repairType, numberOfSplits); - List assignments = new DefaultUnifiedRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); + List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); assertEquals(totalTokenRanges * numberOfSplits * tables.size(), assignments.size()); assertEquals(expectedToken.size(), assignments.size()); @@ -146,7 +146,7 @@ public void testTokenRangesSplitByTable() @Test public void testTokenRangesSplitByKeyspace() { - UnifiedRepairService.instance.getUnifiedRepairConfig().setRepairByKeyspace(repairType, true); + AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(repairType, true); int totalTokenRanges = 3; Collection> tokens = StorageService.instance.getPrimaryRanges(KEYSPACE); assertEquals(totalTokenRanges, tokens.size()); @@ -157,9 +157,9 @@ public void testTokenRangesSplitByKeyspace() appendExpectedTokens(0, 256, numberOfSplits, expectedToken); appendExpectedTokens(256, 1024, numberOfSplits, expectedToken); - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairSubRangeNum(repairType, numberOfSplits); - List assignments = new DefaultUnifiedRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); + List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); assertEquals(totalTokenRanges * numberOfSplits, assignments.size()); assertEquals(expectedToken.size(), assignments.size()); diff --git a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairKeyspaceTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java similarity index 87% rename from test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairKeyspaceTest.java rename to test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java index 748cc88b2a2e..1337cf3dd2d3 100644 --- a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairKeyspaceTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.util.HashSet; import java.util.Iterator; @@ -29,7 +29,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; -public class UnifiedRepairKeyspaceTest +public class AutoRepairKeyspaceTest { @BeforeClass public static void setupDatabaseDescriptor() @@ -38,7 +38,7 @@ public static void setupDatabaseDescriptor() } @Test - public void testEnsureUnifiedRepairTablesArePresent() + public void testEnsureAutoRepairTablesArePresent() { KeyspaceMetadata keyspaceMetadata = SystemDistributedKeyspace.metadata(); Iterator iter = keyspaceMetadata.tables.iterator(); @@ -48,7 +48,7 @@ public void testEnsureUnifiedRepairTablesArePresent() actualDistributedTablesIter.add(iter.next().name); } - Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY)); - Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY)); + Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.AUTO_REPAIR_HISTORY)); + Assert.assertTrue(actualDistributedTablesIter.contains(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY)); } } diff --git a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java similarity index 52% rename from test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairParameterizedTest.java rename to test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 4a8d375f5988..3ec202f23e19 100644 --- a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.util.ArrayList; import java.util.Arrays; @@ -37,7 +37,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.repair.RepairCoordinator; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.repair.unifiedrepair.IUnifiedRepairTokenRangeSplitter.RepairAssignment; +import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.apache.cassandra.service.StorageService; @@ -56,20 +56,20 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.metrics.UnifiedRepairMetricsManager; -import org.apache.cassandra.metrics.UnifiedRepairMetrics; +import org.apache.cassandra.metrics.AutoRepairMetricsManager; +import org.apache.cassandra.metrics.AutoRepairMetrics; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.UnifiedRepairService; +import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.utils.FBUtilities; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; -import static org.apache.cassandra.Util.setUnifiedRepairEnabled; +import static org.apache.cassandra.Util.setAutoRepairEnabled; import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.RepairTurn.NOT_MY_TURN; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -81,43 +81,43 @@ import static org.mockito.Mockito.when; @RunWith(Parameterized.class) -public class UnifiedRepairParameterizedTest extends CQLTester +public class AutoRepairParameterizedTest extends CQLTester { private static final String KEYSPACE = "ks"; private static final String TABLE = "tbl"; - private static final String TABLE_DISABLED_UNIFIED_REPAIR = "tbl_disabled_unified_repair"; + private static final String TABLE_DISABLED_AUTO_REPAIR = "tbl_disabled_auto_repair"; private static final String MV = "mv"; private static TableMetadata cfm; - private static TableMetadata cfmDisabledUnifiedRepair; + private static TableMetadata cfmDisabledAutoRepair; private static Keyspace keyspace; private static int timeFuncCalls; @Mock ScheduledExecutorPlus mockExecutor; @Mock - UnifiedRepairState unifiedRepairState; + AutoRepairState autoRepairState; @Mock RepairCoordinator repairRunnable; - private static UnifiedRepairConfig defaultConfig; + private static AutoRepairConfig defaultConfig; @Parameterized.Parameter() - public UnifiedRepairConfig.RepairType repairType; + public AutoRepairConfig.RepairType repairType; @Parameterized.Parameters(name = "repairType={0}") - public static Collection repairTypes() + public static Collection repairTypes() { - return Arrays.asList(UnifiedRepairConfig.RepairType.values()); + return Arrays.asList(AutoRepairConfig.RepairType.values()); } @BeforeClass public static void setupClass() throws Exception { SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); - UnifiedRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); - setUnifiedRepairEnabled(true); + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + setAutoRepairEnabled(true); requireNetwork(); - UnifiedRepairUtils.setup(); - StorageService.instance.doUnifiedRepairSetup(); + AutoRepairUtils.setup(); + StorageService.instance.doAutoRepairSetup(); DatabaseDescriptor.setCDCEnabled(false); } @@ -127,7 +127,7 @@ public void setup() SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE)); QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i))", KEYSPACE, TABLE)); - QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i)) WITH repair_full = {'enabled': 'false'} AND repair_incremental = {'enabled': 'false'}", KEYSPACE, TABLE_DISABLED_UNIFIED_REPAIR)); + QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i)) WITH repair_full = {'enabled': 'false'} AND repair_incremental = {'enabled': 'false'}", KEYSPACE, TABLE_DISABLED_AUTO_REPAIR)); QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); @@ -142,23 +142,23 @@ public void setup() Keyspace.open(KEYSPACE).getColumnFamilyStore(MV).truncateBlocking(); Keyspace.open(KEYSPACE).getColumnFamilyStore(MV).disableAutoCompaction(); - Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY).truncateBlocking(); - Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY).truncateBlocking(); + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY).truncateBlocking(); + Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_HISTORY).truncateBlocking(); - UnifiedRepair.instance = new UnifiedRepair(); + AutoRepair.instance = new AutoRepair(); executeCQL(); timeFuncCalls = 0; - UnifiedRepair.timeFunc = System::currentTimeMillis; + AutoRepair.timeFunc = System::currentTimeMillis; resetCounters(); resetConfig(); - UnifiedRepair.shuffleFunc = java.util.Collections::shuffle; + AutoRepair.shuffleFunc = java.util.Collections::shuffle; keyspace = Keyspace.open(KEYSPACE); cfm = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE).metadata(); - cfmDisabledUnifiedRepair = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE_DISABLED_UNIFIED_REPAIR).metadata(); + cfmDisabledAutoRepair = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE_DISABLED_AUTO_REPAIR).metadata(); DatabaseDescriptor.setCDCOnRepairEnabled(false); } @@ -170,7 +170,7 @@ public void tearDown() private void resetCounters() { - UnifiedRepairMetrics metrics = UnifiedRepairMetricsManager.getMetrics(repairType); + AutoRepairMetrics metrics = AutoRepairMetricsManager.getMetrics(repairType); Metrics.removeMatching((name, metric) -> name.startsWith("repairTurn")); metrics.repairTurnMyTurn = Metrics.counter(String.format("repairTurnMyTurn-%s", repairType)); metrics.repairTurnMyTurnForceRepair = Metrics.counter(String.format("repairTurnMyTurnForceRepair-%s", repairType)); @@ -180,15 +180,15 @@ private void resetCounters() private void resetConfig() { // prepare a fresh default config - defaultConfig = new UnifiedRepairConfig(true); - for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) + defaultConfig = new AutoRepairConfig(true); + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { - defaultConfig.setUnifiedRepairEnabled(repairType, true); + defaultConfig.setAutoRepairEnabled(repairType, true); defaultConfig.setMVRepairEnabled(repairType, false); } - // reset the UnifiedRepairService config to default - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + // reset the AutoRepairService config to default + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.repair_type_overrides = defaultConfig.repair_type_overrides; config.global_settings = defaultConfig.global_settings; config.history_clear_delete_hosts_buffer_interval = defaultConfig.history_clear_delete_hosts_buffer_interval; @@ -200,24 +200,24 @@ private void executeCQL() QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, s) VALUES ('k', 's')"); QueryProcessor.executeInternal("SELECT s FROM ks.tbl WHERE k='k'"); Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME) - .getColumnFamilyStore(SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY) + .getColumnFamilyStore(SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY) .forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); } @Test(expected = ConfigurationException.class) public void testRepairAsyncWithRepairTypeDisabled() { - UnifiedRepairService.instance.getUnifiedRepairConfig().setUnifiedRepairEnabled(repairType, false); + AutoRepairService.instance.getAutoRepairConfig().setAutoRepairEnabled(repairType, false); - UnifiedRepair.instance.repairAsync(repairType); + AutoRepair.instance.repairAsync(repairType); } @Test public void testRepairAsync() { - UnifiedRepair.instance.repairExecutors.put(repairType, mockExecutor); + AutoRepair.instance.repairExecutors.put(repairType, mockExecutor); - UnifiedRepair.instance.repairAsync(repairType); + AutoRepair.instance.repairAsync(repairType); verify(mockExecutor, Mockito.times(1)).submit(any(Runnable.class)); } @@ -226,17 +226,17 @@ public void testRepairAsync() public void testRepairTurn() { UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); - Assert.assertTrue("Expected my turn for the repair", UnifiedRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); + Assert.assertTrue("Expected my turn for the repair", AutoRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); } @Test public void testRepair() { - UnifiedRepairService.instance.getUnifiedRepairConfig().setRepairMinInterval(repairType, "0s"); - UnifiedRepair.instance.repair(repairType); - assertEquals(0, UnifiedRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); - assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); - long lastRepairTime = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); + AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repair(repairType); + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + long lastRepairTime = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); //if repair was done then lastRepairTime should be non-zero Assert.assertTrue(String.format("Expected lastRepairTime > 0, actual value lastRepairTime %d", lastRepairTime), lastRepairTime > 0); @@ -245,113 +245,113 @@ public void testRepair() @Test public void testTooFrequentRepairs() { - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); //in the first round let repair run config.setRepairMinInterval(repairType, "0s"); - UnifiedRepair.instance.repair(repairType); - long lastRepairTime1 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); - int consideredTables = UnifiedRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); + AutoRepair.instance.repair(repairType); + long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + int consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); Assert.assertNotSame(String.format("Expected total repaired tables > 0, actual value %s ", consideredTables), consideredTables, 0); //if repair was done in last 24 hours then it should not trigger another repair config.setRepairMinInterval(repairType, "24h"); - UnifiedRepair.instance.repair(repairType); - long lastRepairTime2 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); + AutoRepair.instance.repair(repairType); + long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertEquals(String.format("Expected repair time to be same, actual value lastRepairTime1 %d, " + "lastRepairTime2 %d", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); - assertEquals(0, UnifiedRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); - assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); } @Test public void testNonFrequentRepairs() { - Integer prevMetricsCount = UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); - UnifiedRepairState state = UnifiedRepair.instance.repairStates.get(repairType); + Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); long prevCount = state.getTotalMVTablesConsideredForRepair(); - UnifiedRepairService.instance.getUnifiedRepairConfig().setRepairMinInterval(repairType, "0s"); - UnifiedRepair.instance.repair(repairType); - long lastRepairTime1 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); + AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repair(repairType); + long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertTrue(String.format("Expected lastRepairTime1 > 0, actual value lastRepairTime1 %d", lastRepairTime1), lastRepairTime1 > 0); UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); Assert.assertTrue("Expected my turn for the repair", - UnifiedRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); - UnifiedRepair.instance.repair(repairType); - long lastRepairTime2 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); + AutoRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); + AutoRepair.instance.repair(repairType); + long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertNotSame(String.format("Expected repair time to be same, actual value lastRepairTime1 %d, " + "lastRepairTime2 ", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); - assertEquals(prevMetricsCount, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); } @Test public void testGetPriorityHosts() { - Integer prevMetricsCount = UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); - UnifiedRepairState state = UnifiedRepair.instance.repairStates.get(repairType); + Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); long prevCount = state.getTotalMVTablesConsideredForRepair(); - UnifiedRepairService.instance.getUnifiedRepairConfig().setRepairMinInterval(repairType, "0s"); + AutoRepairService.instance.getAutoRepairConfig().setRepairMinInterval(repairType, "0s"); Assert.assertSame(String.format("Priority host count is not same, actual value %d, expected value %d", - UnifiedRepairUtils.getPriorityHosts(repairType).size(), 0), UnifiedRepairUtils.getPriorityHosts(repairType).size(), 0); + AutoRepairUtils.getPriorityHosts(repairType).size(), 0), AutoRepairUtils.getPriorityHosts(repairType).size(), 0); UUID myId = StorageService.instance.getHostIdForEndpoint(FBUtilities.getBroadcastAddressAndPort()); - Assert.assertTrue("Expected my turn for the repair", UnifiedRepairUtils.myTurnToRunRepair(repairType, myId) != + Assert.assertTrue("Expected my turn for the repair", AutoRepairUtils.myTurnToRunRepair(repairType, myId) != NOT_MY_TURN); - UnifiedRepair.instance.repair(repairType); - UnifiedRepairUtils.addPriorityHosts(repairType, Sets.newHashSet(FBUtilities.getBroadcastAddressAndPort())); - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); + AutoRepairUtils.addPriorityHosts(repairType, Sets.newHashSet(FBUtilities.getBroadcastAddressAndPort())); + AutoRepair.instance.repair(repairType); Assert.assertSame(String.format("Priority host count is not same actual value %d, expected value %d", - UnifiedRepairUtils.getPriorityHosts(repairType).size(), 0), UnifiedRepairUtils.getPriorityHosts(repairType).size(), 0); + AutoRepairUtils.getPriorityHosts(repairType).size(), 0), AutoRepairUtils.getPriorityHosts(repairType).size(), 0); assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); - assertEquals(prevMetricsCount, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); } @Test - public void testCheckUnifiedRepairStartStop() throws Throwable + public void testCheckAutoRepairStartStop() throws Throwable { - Integer prevMetricsCount = UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); - UnifiedRepairState state = UnifiedRepair.instance.repairStates.get(repairType); - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + Integer prevMetricsCount = AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); long prevCount = state.getTotalMVTablesConsideredForRepair(); config.setRepairMinInterval(repairType, "0s"); - config.setUnifiedRepairEnabled(repairType, false); - long lastRepairTime1 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); - UnifiedRepair.instance.repair(repairType); - long lastRepairTime2 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); + config.setAutoRepairEnabled(repairType, false); + long lastRepairTime1 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); + AutoRepair.instance.repair(repairType); + long lastRepairTime2 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); //Since repair has not happened, both the last repair times should be same Assert.assertEquals(String.format("Expected lastRepairTime1 %d, and lastRepairTime2 %d to be same", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime2); - config.setUnifiedRepairEnabled(repairType, true); - UnifiedRepair.instance.repair(repairType); + config.setAutoRepairEnabled(repairType, true); + AutoRepair.instance.repair(repairType); //since repair is done now, so lastRepairTime1/lastRepairTime2 and lastRepairTime3 should not be same - long lastRepairTime3 = UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime(); + long lastRepairTime3 = AutoRepair.instance.repairStates.get(repairType).getLastRepairTime(); Assert.assertNotSame(String.format("Expected lastRepairTime1 %d, and lastRepairTime3 %d to be not same", lastRepairTime1, lastRepairTime2), lastRepairTime1, lastRepairTime3); assertEquals(prevCount, state.getTotalMVTablesConsideredForRepair()); - assertEquals(prevMetricsCount, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); + assertEquals(prevMetricsCount, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue()); } @Test public void testRepairPrimaryRangesByDefault() { Assert.assertTrue("Expected primary range repair only", - UnifiedRepairService.instance.getUnifiedRepairConfig().getRepairPrimaryTokenRangeOnly(repairType)); + AutoRepairService.instance.getAutoRepairConfig().getRepairPrimaryTokenRangeOnly(repairType)); } @Test public void testGetAllMVs() { - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setMVRepairEnabled(repairType, false); assertFalse(config.getMVRepairEnabled(repairType)); - assertEquals(0, UnifiedRepairUtils.getAllMVs(repairType, keyspace, cfm).size()); + assertEquals(0, AutoRepairUtils.getAllMVs(repairType, keyspace, cfm).size()); config.setMVRepairEnabled(repairType, true); assertTrue(config.getMVRepairEnabled(repairType)); - assertEquals(Arrays.asList(MV), UnifiedRepairUtils.getAllMVs(repairType, keyspace, cfm)); + assertEquals(Arrays.asList(MV), AutoRepairUtils.getAllMVs(repairType, keyspace, cfm)); config.setMVRepairEnabled(repairType, false); } @@ -359,32 +359,32 @@ public void testGetAllMVs() @Test public void testMVRepair() { - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setMVRepairEnabled(repairType, true); config.setRepairMinInterval(repairType, "0s"); - UnifiedRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); - UnifiedRepair.instance.repair(repairType); - assertEquals(1, UnifiedRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); - assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); + AutoRepair.instance.repair(repairType); + assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); config.setMVRepairEnabled(repairType, false); - UnifiedRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); - UnifiedRepair.instance.repair(repairType); - assertEquals(0, UnifiedRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); - assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); + AutoRepair.instance.repair(repairType); + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); config.setMVRepairEnabled(repairType, true); - UnifiedRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); - UnifiedRepair.instance.repair(repairType); - assertEquals(1, UnifiedRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); - assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(System.currentTimeMillis()); + AutoRepair.instance.repair(repairType); + assertEquals(1, AutoRepair.instance.repairStates.get(repairType).getTotalMVTablesConsideredForRepair()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); } @Test public void testSkipRepairSSTableCountHigherThreshold() { - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); - UnifiedRepairState state = UnifiedRepair.instance.repairStates.get(repairType); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + AutoRepairState state = AutoRepair.instance.repairStates.get(repairType); ColumnFamilyStore cfsBaseTable = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE); ColumnFamilyStore cfsMVTable = Keyspace.open(KEYSPACE).getColumnFamilyStore(MV); Set preBaseTable = cfsBaseTable.getLiveSSTables(); @@ -412,89 +412,89 @@ public void testSkipRepairSSTableCountHigherThreshold() config.setMVRepairEnabled(repairType, true); config.setRepairSSTableCountHigherThreshold(repairType, 9); assertEquals(0, state.getSkippedTokenRangesCount()); - assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); state.setLastRepairTime(0); - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); assertEquals(0, state.getTotalMVTablesConsideredForRepair()); - assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); // skipping both the tables - one table is due to its repair has been disabled, and another one due to high sstable count assertEquals(0, state.getSkippedTokenRangesCount()); - assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); assertEquals(2, state.getSkippedTablesCount()); - assertEquals(2, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + assertEquals(2, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); // set it to higher value, and this time, the tables should not be skipped config.setRepairSSTableCountHigherThreshold(repairType, beforeCount); state.setLastRepairTime(0); state.setSkippedTablesCount(0); state.setTotalMVTablesConsideredForRepair(0); - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); assertEquals(1, state.getTotalMVTablesConsideredForRepair()); - assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); assertEquals(0, state.getSkippedTokenRangesCount()); - assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); assertEquals(1, state.getSkippedTablesCount()); - assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).skippedTablesCount.getValue().intValue()); } @Test public void testGetRepairState() { - assertEquals(0, UnifiedRepair.instance.repairStates.get(repairType).getRepairKeyspaceCount()); + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getRepairKeyspaceCount()); - UnifiedRepairState state = UnifiedRepair.instance.getRepairState(repairType); + AutoRepairState state = AutoRepair.instance.getRepairState(repairType); state.setRepairKeyspaceCount(100); - assertEquals(100L, UnifiedRepair.instance.getRepairState(repairType).getRepairKeyspaceCount()); + assertEquals(100L, AutoRepair.instance.getRepairState(repairType).getRepairKeyspaceCount()); } @Test public void testMetrics() { - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setMVRepairEnabled(repairType, true); config.setRepairMinInterval(repairType, "0s"); - config.setUnifiedRepairTableMaxRepairTime(repairType, "0s"); - UnifiedRepair.timeFunc = () -> { + config.setAutoRepairTableMaxRepairTime(repairType, "0s"); + AutoRepair.timeFunc = () -> { timeFuncCalls++; return timeFuncCalls * 1000L; }; - UnifiedRepair.instance.repairStates.get(repairType).setLastRepairTime(1000L); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(1000L); - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); - assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); - assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).nodeRepairTimeInSec.getValue() > 0); - assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).clusterRepairTimeInSec.getValue() > 0); - assertEquals(1, UnifiedRepairMetricsManager.getMetrics(repairType).repairTurnMyTurn.getCount()); - assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue() > 0); - assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue().intValue()); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).totalMVTablesConsideredForRepair.getValue().intValue()); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).nodeRepairTimeInSec.getValue() > 0); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).clusterRepairTimeInSec.getValue() > 0); + assertEquals(1, AutoRepairMetricsManager.getMetrics(repairType).repairTurnMyTurn.getCount()); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue() > 0); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue().intValue()); - config.setUnifiedRepairTableMaxRepairTime(repairType, String.valueOf(Integer.MAX_VALUE - 1) + 's'); - UnifiedRepair.instance.repairStates.put(repairType, unifiedRepairState); - when(unifiedRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) + config.setAutoRepairTableMaxRepairTime(repairType, String.valueOf(Integer.MAX_VALUE-1) + 's'); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) .thenReturn(repairRunnable); - when(unifiedRepairState.getFailedTokenRangesCount()).thenReturn(10); - when(unifiedRepairState.getSucceededTokenRangesCount()).thenReturn(11); - when(unifiedRepairState.getLongestUnrepairedSec()).thenReturn(10); + when(autoRepairState.getFailedTokenRangesCount()).thenReturn(10); + when(autoRepairState.getSucceededTokenRangesCount()).thenReturn(11); + when(autoRepairState.getLongestUnrepairedSec()).thenReturn(10); - UnifiedRepair.instance.repair(repairType); - assertEquals(0, UnifiedRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); - assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).failedTokenRangesCount.getValue() > 0); - assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).succeededTokenRangesCount.getValue() > 0); - assertTrue(UnifiedRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue() > 0); + AutoRepair.instance.repair(repairType); + assertEquals(0, AutoRepairMetricsManager.getMetrics(repairType).skippedTokenRangesCount.getValue().intValue()); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).failedTokenRangesCount.getValue() > 0); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).succeededTokenRangesCount.getValue() > 0); + assertTrue(AutoRepairMetricsManager.getMetrics(repairType).longestUnrepairedSec.getValue() > 0); } @Test public void testRepairWaitsForRepairToFinishBeforeSchedullingNewSession() throws Exception { - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setMVRepairEnabled(repairType, false); config.setRepairRetryBackoff("0s"); - when(unifiedRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())) .thenReturn(repairRunnable); - UnifiedRepair.instance.repairStates.put(repairType, unifiedRepairState); - when(unifiedRepairState.getLastRepairTime()).thenReturn((long) 0); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + when(autoRepairState.getLastRepairTime()).thenReturn((long) 0); AtomicInteger resetWaitConditionCalls = new AtomicInteger(); AtomicInteger waitForRepairCompletedCalls = new AtomicInteger(); doAnswer(invocation -> { @@ -502,36 +502,36 @@ public void testRepairWaitsForRepairToFinishBeforeSchedullingNewSession() throws assertEquals("waitForRepairToComplete was called before resetWaitCondition", resetWaitConditionCalls.get(), waitForRepairCompletedCalls.get() + 1); return null; - }).when(unifiedRepairState).resetWaitCondition(); + }).when(autoRepairState).resetWaitCondition(); doAnswer(invocation -> { waitForRepairCompletedCalls.getAndIncrement(); assertEquals("resetWaitCondition was not called before waitForRepairToComplete", resetWaitConditionCalls.get(), waitForRepairCompletedCalls.get()); return null; - }).when(unifiedRepairState).waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); + }).when(autoRepairState).waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); - UnifiedRepair.instance.repair(repairType); - UnifiedRepair.instance.repair(repairType); - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); } @Test - public void testDisabledUnifiedRepairForATableThroughTableLevelConfiguration() + public void testDisabledAutoRepairForATableThroughTableLevelConfiguration() { - Assert.assertTrue(cfm.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full).repairEnabled()); - Assert.assertTrue(cfm.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).repairEnabled()); - Assert.assertFalse(cfmDisabledUnifiedRepair.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full).repairEnabled()); - Assert.assertFalse(cfmDisabledUnifiedRepair.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).repairEnabled()); + Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); + Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairMinInterval(repairType, "0s"); - int disabledTablesRepairCountBefore = UnifiedRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); - UnifiedRepair.instance.repair(repairType); - int consideredTables = UnifiedRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); + int disabledTablesRepairCountBefore = AutoRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); + AutoRepair.instance.repair(repairType); + int consideredTables = AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair(); Assert.assertNotSame(String.format("Expected total repaired tables > 0, actual value %s ", consideredTables), consideredTables, 0); - int disabledTablesRepairCountAfter = UnifiedRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); - Assert.assertTrue(String.format("A table %s should be skipped from unified repair, expected value: %d, actual value %d ", TABLE_DISABLED_UNIFIED_REPAIR, disabledTablesRepairCountBefore + 1, disabledTablesRepairCountAfter), + int disabledTablesRepairCountAfter = AutoRepair.instance.repairStates.get(repairType).getTotalDisabledTablesRepairCount(); + Assert.assertTrue(String.format("A table %s should be skipped from auto repair, expected value: %d, actual value %d ", TABLE_DISABLED_AUTO_REPAIR, disabledTablesRepairCountBefore + 1, disabledTablesRepairCountAfter), disabledTablesRepairCountBefore < disabledTablesRepairCountAfter); } @@ -542,7 +542,7 @@ public void testTokenRangesNoSplit() assertEquals(1, tokens.size()); List> expectedToken = new ArrayList<>(tokens); - List assignments = new DefaultUnifiedRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, Collections.singletonList(TABLE)); + List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, Collections.singletonList(TABLE)); assertEquals(1, assignments.size()); assertEquals(expectedToken.get(0).left, assignments.get(0).getTokenRange().left); assertEquals(expectedToken.get(0).right, assignments.get(0).getTokenRange().right); @@ -556,12 +556,12 @@ public void testTableAttribute() } @Test - public void testDefaultUnifiedRepair() + public void testDefaultAutomatedRepair() { - Assert.assertTrue(cfm.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full).repairEnabled()); - Assert.assertTrue(cfm.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).repairEnabled()); - Assert.assertFalse(cfmDisabledUnifiedRepair.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.full).repairEnabled()); - Assert.assertFalse(cfmDisabledUnifiedRepair.params.unifiedRepair.get(UnifiedRepairConfig.RepairType.incremental).repairEnabled()); + Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); + Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); } @Test @@ -569,7 +569,7 @@ public void testRepairShufflesKeyspacesAndTables() { AtomicInteger shuffleKeyspacesCall = new AtomicInteger(); AtomicInteger shuffleTablesCall = new AtomicInteger(); - UnifiedRepair.shuffleFunc = (List list) -> { + AutoRepair.shuffleFunc = (List list) -> { if (!list.isEmpty()) { assertTrue(list.get(0) instanceof Keyspace || list.get(0) instanceof String); @@ -585,9 +585,9 @@ else if (list.get(0) instanceof String) } }; - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairMinInterval(repairType, "0s"); - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); assertEquals(1, shuffleKeyspacesCall.get()); assertEquals(5, shuffleTablesCall.get()); @@ -596,88 +596,88 @@ else if (list.get(0) instanceof String) @Test public void testRepairTakesLastRepairTimeFromDB() { - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setMVRepairEnabled(repairType, true); long lastRepairTime = System.currentTimeMillis() - 1000; - UnifiedRepairUtils.insertNewRepairHistory(repairType, 0, lastRepairTime); - UnifiedRepair.instance.repairStates.get(repairType).setLastRepairTime(0); + AutoRepairUtils.insertNewRepairHistory(repairType, 0, lastRepairTime); + AutoRepair.instance.repairStates.get(repairType).setLastRepairTime(0); config.setRepairMinInterval(repairType, "1h"); - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); // repair scheduler should not attempt to run repair as last repair time in DB is current time - 1s - assertEquals(0, UnifiedRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair()); + assertEquals(0, AutoRepair.instance.repairStates.get(repairType).getTotalTablesConsideredForRepair()); // repair scheduler should load the repair time from the DB - assertEquals(lastRepairTime, UnifiedRepair.instance.repairStates.get(repairType).getLastRepairTime()); + assertEquals(lastRepairTime, AutoRepair.instance.repairStates.get(repairType).getLastRepairTime()); } @Test public void testRepairMaxRetries() { - when(unifiedRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); - when(unifiedRepairState.isSuccess()).thenReturn(false); - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + when(autoRepairState.isSuccess()).thenReturn(false); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); AtomicInteger sleepCalls = new AtomicInteger(); - UnifiedRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { sleepCalls.getAndIncrement(); assertEquals(TimeUnit.SECONDS, unit); assertEquals(config.getRepairRetryBackoff().toSeconds(), (long) duration); }; config.setRepairMinInterval(repairType, "0s"); - UnifiedRepair.instance.repairStates.put(repairType, unifiedRepairState); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); //system_auth.role_permissions,system_auth.network_permissions,system_auth.role_members,system_auth.roles, // system_auth.resource_role_permissons_index,system_traces.sessions,system_traces.events,ks.tbl, - // system_distributed.unified_repair_priority,system_distributed.repair_history,system_distributed.unified_repair_history, + // system_distributed.auto_repair_priority,system_distributed.repair_history,system_distributed.auto_repair_history, // system_distributed.view_build_status,system_distributed.parent_repair_history,system_distributed.partition_denylist int exptedTablesGoingThroughRepair = 18; assertEquals(config.getRepairMaxRetries()*exptedTablesGoingThroughRepair, sleepCalls.get()); - verify(unifiedRepairState, Mockito.times(1)).setSucceededTokenRangesCount(0); - verify(unifiedRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); - verify(unifiedRepairState, Mockito.times(1)).setFailedTokenRangesCount(exptedTablesGoingThroughRepair); + verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(0); + verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); + verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(exptedTablesGoingThroughRepair); } @Test public void testRepairSuccessAfterRetry() { - when(unifiedRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); - UnifiedRepairConfig config = UnifiedRepairService.instance.getUnifiedRepairConfig(); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); AtomicInteger sleepCalls = new AtomicInteger(); - UnifiedRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { sleepCalls.getAndIncrement(); assertEquals(TimeUnit.SECONDS, unit); assertEquals(config.getRepairRetryBackoff().toSeconds(), (long) duration); }; - when(unifiedRepairState.isSuccess()).then((invocationOnMock) -> { + when(autoRepairState.isSuccess()).then((invocationOnMock) -> { if (sleepCalls.get() == 0) { return false; } return true; }); config.setRepairMinInterval(repairType, "0s"); - UnifiedRepair.instance.repairStates.put(repairType, unifiedRepairState); - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + AutoRepair.instance.repair(repairType); assertEquals(1, sleepCalls.get()); - verify(unifiedRepairState, Mockito.times(1)).setSucceededTokenRangesCount(18); - verify(unifiedRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); - verify(unifiedRepairState, Mockito.times(1)).setFailedTokenRangesCount(0); + verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(18); + verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); + verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(0); } @Test public void testRepairThrowsForIRWithMVReplay() { - UnifiedRepair.instance.setup(); + AutoRepair.instance.setup(); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); - if (repairType == UnifiedRepairConfig.RepairType.incremental) + if (repairType == AutoRepairConfig.RepairType.incremental) { try { - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); fail("Expected ConfigurationException"); } catch (ConfigurationException ignored) @@ -686,7 +686,7 @@ public void testRepairThrowsForIRWithMVReplay() } else { - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); } } @@ -694,14 +694,14 @@ public void testRepairThrowsForIRWithMVReplay() @Test public void testRepairThrowsForIRWithCDCReplay() { - UnifiedRepair.instance.setup(); + AutoRepair.instance.setup(); DatabaseDescriptor.setCDCOnRepairEnabled(true); - if (repairType == UnifiedRepairConfig.RepairType.incremental) + if (repairType == AutoRepairConfig.RepairType.incremental) { try { - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); fail("Expected ConfigurationException"); } catch (ConfigurationException ignored) @@ -710,7 +710,7 @@ public void testRepairThrowsForIRWithCDCReplay() } else { - UnifiedRepair.instance.repair(repairType); + AutoRepair.instance.repair(repairType); } } } diff --git a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateFactoryTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java similarity index 76% rename from test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateFactoryTest.java rename to test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java index c1ee56e1065d..a0e5bdc45294 100644 --- a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateFactoryTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java @@ -16,24 +16,24 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import org.junit.Test; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; -public class UnifiedRepairStateFactoryTest +public class AutoRepairStateFactoryTest { @Test public void testGetRepairState() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(RepairType.full); + AutoRepairState state = RepairType.getAutoRepairState(RepairType.full); assert state instanceof FullRepairState; - state = RepairType.getUnifiedRepairState(RepairType.incremental); + state = RepairType.getAutoRepairState(RepairType.incremental); assert state instanceof IncrementalRepairState; } @@ -42,7 +42,7 @@ public void testGetRepairState() { public void testGetRepairStateSupportsAllRepairTypes() { for (RepairType repairType : RepairType.values()) { try { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); assertNotNull(state); } catch (IllegalArgumentException e) { assertNull(e); diff --git a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java similarity index 72% rename from test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateTest.java rename to test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java index df60ffc4e484..f0974dd83c1a 100644 --- a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairStateTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.util.Arrays; import java.util.Collection; @@ -33,9 +33,9 @@ import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.UnifiedRepairHistory; -import org.apache.cassandra.service.UnifiedRepairService; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; +import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.utils.concurrent.Condition; import org.apache.cassandra.utils.progress.ProgressEvent; import org.apache.cassandra.utils.progress.ProgressEventType; @@ -50,7 +50,7 @@ import static org.mockito.MockitoAnnotations.initMocks; @RunWith(Parameterized.class) -public class UnifiedRepairStateTest extends CQLTester +public class AutoRepairStateTest extends CQLTester { private static final String testTable = "test"; @@ -68,15 +68,15 @@ public static Collection repairTypes() @Before public void setUp() { - UnifiedRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); initMocks(this); createTable(String.format("CREATE TABLE IF NOT EXISTS %s.%s (pk int PRIMARY KEY, v int)", KEYSPACE, testTable)); } @Test public void testGetRepairRunnable() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); - UnifiedRepairService.setup(); + AutoRepairState state = RepairType.getAutoRepairState(repairType); + AutoRepairService.setup(); Runnable runnable = state.getRepairRunnable(KEYSPACE, ImmutableList.of(testTable), ImmutableSet.of(), false); @@ -86,7 +86,7 @@ public void testGetRepairRunnable() { @Test public void testProgressError() throws InterruptedException { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); when(progressEvent.getType()).thenReturn(ProgressEventType.ERROR); state.progress("test", progressEvent); @@ -98,7 +98,7 @@ public void testProgressError() throws InterruptedException @Test public void testProgress_progress() throws InterruptedException { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); when(progressEvent.getType()).thenReturn(ProgressEventType.PROGRESS); state.progress("test", progressEvent); @@ -111,7 +111,7 @@ public void testProgress_progress() throws InterruptedException @Test public void testProgress_complete() throws InterruptedException { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); when(progressEvent.getType()).thenReturn(ProgressEventType.COMPLETE); state.progress("test", progressEvent); @@ -123,7 +123,7 @@ public void testProgress_complete() throws InterruptedException @Test public void testWaitForRepairToComplete() throws Exception { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.condition.signalAll(); Condition finishedCondition = Condition.newOneTimeCondition(); Callable waitForRepairToComplete = () -> { @@ -139,7 +139,7 @@ public void testWaitForRepairToComplete() throws Exception @Test public void testGetLastRepairTime() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.lastRepairTimeInMs = 1; assertEquals(1, state.getLastRepairTime()); @@ -147,7 +147,7 @@ public void testGetLastRepairTime() { @Test public void testSetTotalTablesConsideredForRepair() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.setTotalTablesConsideredForRepair(1); @@ -156,7 +156,7 @@ public void testSetTotalTablesConsideredForRepair() { @Test public void testGetTotalTablesConsideredForRepair() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.totalTablesConsideredForRepair = 1; assertEquals(1, state.getTotalTablesConsideredForRepair()); @@ -164,7 +164,7 @@ public void testGetTotalTablesConsideredForRepair() { @Test public void testSetLastRepairTimeInMs() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.setLastRepairTime(1); @@ -173,7 +173,7 @@ public void testSetLastRepairTimeInMs() { @Test public void testGetClusterRepairTimeInSec() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.clusterRepairTimeInSec = 1; assertEquals(1, state.getClusterRepairTimeInSec()); @@ -181,7 +181,7 @@ public void testGetClusterRepairTimeInSec() { @Test public void testGetNodeRepairTimeInSec() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.nodeRepairTimeInSec = 1; assertEquals(1, state.getNodeRepairTimeInSec()); @@ -189,7 +189,7 @@ public void testGetNodeRepairTimeInSec() { @Test public void testSetRepairInProgress() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.setRepairInProgress(true); @@ -198,7 +198,7 @@ public void testSetRepairInProgress() { @Test public void testIsRepairInProgress() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.repairInProgress = true; assertTrue(state.isRepairInProgress()); @@ -206,7 +206,7 @@ public void testIsRepairInProgress() { @Test public void testSetSkippedTokenRangesCount() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.setSkippedTokenRangesCount(1); @@ -215,7 +215,7 @@ public void testSetSkippedTokenRangesCount() { @Test public void testGetSkippedTokenRangesCount() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.skippedTokenRangesCount = 1; assertEquals(1, state.getSkippedTokenRangesCount()); @@ -223,7 +223,7 @@ public void testGetSkippedTokenRangesCount() { @Test public void testGetLongestUnrepairedSecNull() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.longestUnrepairedNode = null; try @@ -236,10 +236,10 @@ public void testGetLongestUnrepairedSecNull() { @Test public void testGetLongestUnrepairedSec() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); - state.longestUnrepairedNode = new UnifiedRepairHistory(UUID.randomUUID(), "", 0, 1000, - null, 0, false); - UnifiedRepairState.timeFunc = () -> 2000L; + AutoRepairState state = RepairType.getAutoRepairState(repairType); + state.longestUnrepairedNode = new AutoRepairHistory(UUID.randomUUID(), "", 0, 1000, + null, 0, false); + AutoRepairState.timeFunc = () -> 2000L; try { @@ -251,7 +251,7 @@ public void testGetLongestUnrepairedSec() { @Test public void testSetTotalMVTablesConsideredForRepair() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.setTotalMVTablesConsideredForRepair(1); @@ -260,7 +260,7 @@ public void testSetTotalMVTablesConsideredForRepair() { @Test public void testGetTotalMVTablesConsideredForRepair() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.totalMVTablesConsideredForRepair = 1; assertEquals(1, state.getTotalMVTablesConsideredForRepair()); @@ -268,7 +268,7 @@ public void testGetTotalMVTablesConsideredForRepair() { @Test public void testSetNodeRepairTimeInSec() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.setNodeRepairTimeInSec(1); @@ -277,7 +277,7 @@ public void testSetNodeRepairTimeInSec() { @Test public void testSetClusterRepairTimeInSec() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.setClusterRepairTimeInSec(1); @@ -286,7 +286,7 @@ public void testSetClusterRepairTimeInSec() { @Test public void testSetRepairKeyspaceCount() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.setRepairKeyspaceCount(1); @@ -294,7 +294,7 @@ public void testSetRepairKeyspaceCount() { } @Test public void testGetRepairKeyspaceCount() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.repairKeyspaceCount = 1; assertEquals(1, state.getRepairKeyspaceCount()); @@ -302,8 +302,8 @@ public void testGetRepairKeyspaceCount() { @Test public void testSetLongestUnrepairedNode() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); - UnifiedRepairHistory history = new UnifiedRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); + AutoRepairState state = RepairType.getAutoRepairState(repairType); + AutoRepairHistory history = new AutoRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); state.setLongestUnrepairedNode(history); @@ -312,7 +312,7 @@ public void testSetLongestUnrepairedNode() { @Test public void testSetSucceededTokenRangesCount() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.setSucceededTokenRangesCount(1); @@ -321,7 +321,7 @@ public void testSetSucceededTokenRangesCount() { @Test public void testGetSucceededTokenRangesCount() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.succeededTokenRangesCount = 1; assertEquals(1, state.getSucceededTokenRangesCount()); @@ -329,7 +329,7 @@ public void testGetSucceededTokenRangesCount() { @Test public void testSetFailedTokenRangesCount() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.setFailedTokenRangesCount(1); @@ -338,7 +338,7 @@ public void testSetFailedTokenRangesCount() { @Test public void testGetFailedTokenRangesCount() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.failedTokenRangesCount = 1; assertEquals(1, state.getFailedTokenRangesCount()); @@ -346,7 +346,7 @@ public void testGetFailedTokenRangesCount() { @Test public void isSuccess() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.success = true; assertTrue(state.isSuccess()); @@ -359,7 +359,7 @@ public void isSuccess() { @Test public void testWaitForRepairToCompleteDoesNotSetSuccessWhenProgressReceivesError() throws InterruptedException { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); when(progressEvent.getType()).thenReturn(ProgressEventType.ERROR); state.progress("test", progressEvent); @@ -372,7 +372,7 @@ public void testWaitForRepairToCompleteDoesNotSetSuccessWhenProgressReceivesErro @Test public void testResetWaitCondition() { - UnifiedRepairState state = RepairType.getUnifiedRepairState(repairType); + AutoRepairState state = RepairType.getAutoRepairState(repairType); state.condition.signalAll(); assertTrue(state.condition.isSignalled()); diff --git a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java similarity index 79% rename from test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairTest.java rename to test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index 4e6e755a09f4..df9f105b615f 100644 --- a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.util.HashMap; import java.util.Map; @@ -34,38 +34,38 @@ import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.ReplicationParams; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; import org.apache.cassandra.schema.SchemaTestUtil; -import org.apache.cassandra.service.UnifiedRepairService; +import org.apache.cassandra.service.AutoRepairService; -import static org.apache.cassandra.Util.setUnifiedRepairEnabled; +import static org.apache.cassandra.Util.setAutoRepairEnabled; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -public class UnifiedRepairTest extends CQLTester +public class AutoRepairTest extends CQLTester { @BeforeClass public static void setupClass() throws Exception { - setUnifiedRepairEnabled(true); + setAutoRepairEnabled(true); requireNetwork(); } @Before public void setup() { - UnifiedRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); DatabaseDescriptor.setCDCOnRepairEnabled(false); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); - DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(RepairType.full, true); - DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(RepairType.incremental, true); - UnifiedRepairService.setup(); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.full, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + AutoRepairService.setup(); } @Test public void testSetup() { - UnifiedRepair instance = new UnifiedRepair(); + AutoRepair instance = new AutoRepair(); instance.setup(); assertEquals(RepairType.values().length, instance.repairExecutors.size()); @@ -81,7 +81,7 @@ public void testSetup() @Test public void testSafeGuardSetupCall() { - UnifiedRepair instance = new UnifiedRepair(); + AutoRepair instance = new AutoRepair(); // only one should be setup, and rest should be ignored instance.setup(); @@ -101,22 +101,22 @@ public void testSafeGuardSetupCall() @Test(expected = ConfigurationException.class) public void testSetupFailsWhenIREnabledWithCDCReplay() { - DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(RepairType.incremental, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); DatabaseDescriptor.setCDCEnabled(true); DatabaseDescriptor.setCDCOnRepairEnabled(true); - UnifiedRepair instance = new UnifiedRepair(); + AutoRepair instance = new AutoRepair(); instance.setup(); } @Test(expected = ConfigurationException.class) public void testSetupFailsWhenIREnabledWithMVReplay() { - DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(RepairType.incremental, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); DatabaseDescriptor.setCDCOnRepairEnabled(false); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); - UnifiedRepair instance = new UnifiedRepair(); + AutoRepair instance = new AutoRepair(); instance.setup(); } @@ -141,14 +141,14 @@ public void testCheckNTSreplicationNodeInsideOutsideDC() // case 1 : // node reside in "datacenter1" // keyspace has replica in "datacenter1" - Assert.assertTrue(UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + Assert.assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); } else if (ks.getName().equals(ksname2)) { // case 2 : // node reside in "datacenter1" // keyspace has replica in "datacenter2" - Assert.assertFalse(UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + Assert.assertFalse(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); } } } diff --git a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java similarity index 70% rename from test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtilsTest.java rename to test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java index 2b6bd0e14114..9a3c551c9bf6 100644 --- a/test/unit/org/apache/cassandra/repair/unifiedrepair/UnifiedRepairUtilsTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.util.List; import java.util.Set; @@ -36,9 +36,9 @@ import org.apache.cassandra.db.marshal.UUIDType; import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig.RepairType; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.UnifiedRepairHistory; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.CurrentRepairStatus; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.AutoRepairHistory; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils.CurrentRepairStatus; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.QueryProcessor; @@ -49,14 +49,14 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import static org.apache.cassandra.Util.setUnifiedRepairEnabled; +import static org.apache.cassandra.Util.setAutoRepairEnabled; import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_DELETE_HOSTS; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_FORCE_REPAIR; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_REPAIR_FINISH_TS; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_REPAIR_PRIORITY; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_REPAIR_START_TS; -import static org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils.COL_REPAIR_TURN; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_DELETE_HOSTS; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_FORCE_REPAIR; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_FINISH_TS; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_PRIORITY; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_START_TS; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.COL_REPAIR_TURN; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -64,7 +64,7 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.when; -public class UnifiedRepairUtilsTest extends CQLTester +public class AutoRepairUtilsTest extends CQLTester { static RepairType repairType = RepairType.incremental; static UUID hostId; @@ -80,12 +80,12 @@ public class UnifiedRepairUtilsTest extends CQLTester public static void setupClass() throws Exception { SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); - setUnifiedRepairEnabled(true); + setAutoRepairEnabled(true); requireNetwork(); defaultSnitch = DatabaseDescriptor.getEndpointSnitch(); localEndpoint = FBUtilities.getBroadcastAddressAndPort(); hostId = StorageService.instance.getHostIdForEndpoint(localEndpoint); - StorageService.instance.doUnifiedRepairSetup(); + StorageService.instance.doAutoRepairSetup(); } @Before @@ -95,15 +95,15 @@ public void setup() QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", "ks")); QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i))", "ks", "tbl")); - UnifiedRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); + AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); MockitoAnnotations.initMocks(this); DatabaseDescriptor.setEndpointSnitch(defaultSnitch); QueryProcessor.executeInternal(String.format( "TRUNCATE %s.%s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY)); + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY)); QueryProcessor.executeInternal(String.format( "TRUNCATE %s.%s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY)); + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY)); } @Test @@ -111,14 +111,14 @@ public void testSetForceRepair() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, force_repair) VALUES ('%s', %s, false)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); - UnifiedRepairUtils.setForceRepair(repairType, ImmutableSet.of(localEndpoint)); + AutoRepairUtils.setForceRepair(repairType, ImmutableSet.of(localEndpoint)); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT force_repair FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -128,11 +128,11 @@ public void testSetForceRepair() @Test public void testSetForceRepairNewNode() { - UnifiedRepairUtils.setForceRepairNewNode(repairType); + AutoRepairUtils.setForceRepairNewNode(repairType); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT force_repair FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -145,14 +145,14 @@ public void testClearDeleteHosts() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, delete_hosts, delete_hosts_update_time) VALUES ('%s', %s, { %s }, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId, hostId)); - UnifiedRepairUtils.clearDeleteHosts(repairType, hostId); + AutoRepairUtils.clearDeleteHosts(repairType, hostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT delete_hosts FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -161,23 +161,23 @@ public void testClearDeleteHosts() } @Test - public void testGetUnifiedRepairHistoryForLocalGroup() + public void testGetAutoRepairHistoryForLocalGroup() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, force_repair) VALUES ('%s', %s, false)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); - List history = UnifiedRepairUtils.getUnifiedRepairHistory(repairType); + List history = AutoRepairUtils.getAutoRepairHistory(repairType); assertNotNull(history); assertEquals(1, history.size()); assertEquals(hostId, history.get(0).hostId); } @Test - public void testGetUnifiedRepairHistoryForLocalGroup_empty_history() + public void testGetAutoRepairHistoryForLocalGroup_empty_history() { - List history = UnifiedRepairUtils.getUnifiedRepairHistory(repairType); + List history = AutoRepairUtils.getAutoRepairHistory(repairType); assertNull(history); } @@ -189,22 +189,22 @@ public void testGetCurrentRepairStatus() UUID regularRepair = UUID.randomUUID(); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, force_repair, repair_start_ts) VALUES ('%s', %s, true, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), forceRepair)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, repair_start_ts) VALUES ('%s', %s, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), regularRepair)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, repairType.toString(), regularRepair)); - CurrentRepairStatus status = UnifiedRepairUtils.getCurrentRepairStatus(repairType); + CurrentRepairStatus status = AutoRepairUtils.getCurrentRepairStatus(repairType); assertNotNull(status); assertEquals(1, status.historiesWithoutOnGoingRepair.size()); @@ -220,7 +220,7 @@ public void testGetCurrentRepairStatus() @Test public void testGetHostIdsInCurrentRing() { - TreeSet hosts = UnifiedRepairUtils.getHostIdsInCurrentRing(repairType); + TreeSet hosts = AutoRepairUtils.getHostIdsInCurrentRing(repairType); assertNotNull(hosts); assertEquals(1, hosts.size()); @@ -232,13 +232,13 @@ public void testGetHostIdsInCurrentRing_multiple_nodes() { InetAddressAndPort ignoredEndpoint = localEndpoint.withPort(localEndpoint.getPort() + 1); InetAddressAndPort deadEndpoint = localEndpoint.withPort(localEndpoint.getPort() + 2); - DatabaseDescriptor.getUnifiedRepairConfig().setIgnoreDCs(repairType, ImmutableSet.of("dc2")); + DatabaseDescriptor.getAutoRepairConfig().setIgnoreDCs(repairType, ImmutableSet.of("dc2")); DatabaseDescriptor.setEndpointSnitch(snitchMock); when(snitchMock.getDatacenter(localEndpoint)).thenReturn("dc1"); when(snitchMock.getDatacenter(ignoredEndpoint)).thenReturn("dc2"); when(snitchMock.getDatacenter(deadEndpoint)).thenReturn("dc1"); - TreeSet hosts = UnifiedRepairUtils.getHostIdsInCurrentRing(repairType, ImmutableSet.of(new NodeAddresses(localEndpoint), new NodeAddresses(ignoredEndpoint), new NodeAddresses(deadEndpoint))); + TreeSet hosts = AutoRepairUtils.getHostIdsInCurrentRing(repairType, ImmutableSet.of(new NodeAddresses(localEndpoint), new NodeAddresses(ignoredEndpoint), new NodeAddresses(deadEndpoint))); assertNotNull(hosts); assertEquals(1, hosts.size()); @@ -251,93 +251,93 @@ public void testGetHostWithLongestUnrepairTime() UUID otherHostId = UUID.randomUUID(); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id, repair_finish_ts) VALUES ('%s', %s, toTimestamp(now()))", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), otherHostId)); - UnifiedRepairHistory history = UnifiedRepairUtils.getHostWithLongestUnrepairTime(repairType); + AutoRepairHistory history = AutoRepairUtils.getHostWithLongestUnrepairTime(repairType); assertEquals(hostId, history.hostId); } @Test - public void testGetMaxNumberOfNodeRunUnifiedRepairInGroup_0_group_size() + public void testGetMaxNumberOfNodeRunAutoRepairInGroup_0_group_size() { - DatabaseDescriptor.getUnifiedRepairConfig().setParallelRepairCount(repairType, 2); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 2); - int count = UnifiedRepairUtils.getMaxNumberOfNodeRunUnifiedRepair(repairType, 0); + int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepair(repairType, 0); assertEquals(2, count); } @Test - public void testGetMaxNumberOfNodeRunUnifiedRepairInGroup_percentage() + public void testGetMaxNumberOfNodeRunAutoRepairInGroup_percentage() { - DatabaseDescriptor.getUnifiedRepairConfig().setParallelRepairCount(repairType, 2); - DatabaseDescriptor.getUnifiedRepairConfig().setParallelRepairPercentage(repairType, 50); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 2); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairPercentage(repairType, 50); - int count = UnifiedRepairUtils.getMaxNumberOfNodeRunUnifiedRepair(repairType, 10); + int count = AutoRepairUtils.getMaxNumberOfNodeRunAutoRepair(repairType, 10); assertEquals(5, count); } @Test - public void testDeleteUnifiedRepairHistory() + public void testDeleteAutoRepairHistory() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); - UnifiedRepairUtils.deleteUnifiedRepairHistory(repairType, hostId); + AutoRepairUtils.deleteAutoRepairHistory(repairType, hostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(0, result.size()); } @Test - public void testUpdateStartUnifiedRepairHistory() + public void testUpdateStartAutoRepairHistory() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); - UnifiedRepairUtils.updateStartUnifiedRepairHistory(repairType, hostId, 123, UnifiedRepairUtils.RepairTurn.MY_TURN); + AutoRepairUtils.updateStartAutoRepairHistory(repairType, hostId, 123, AutoRepairUtils.RepairTurn.MY_TURN); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT repair_start_ts, repair_turn FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); UntypedResultSet.Row row = result.one(); assertEquals(123, row.getLong(COL_REPAIR_START_TS, 0)); - assertEquals(UnifiedRepairUtils.RepairTurn.MY_TURN.toString(), row.getString(COL_REPAIR_TURN)); + assertEquals(AutoRepairUtils.RepairTurn.MY_TURN.toString(), row.getString(COL_REPAIR_TURN)); } @Test - public void testUpdateFinishUnifiedRepairHistory() + public void testUpdateFinishAutoRepairHistory() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); - UnifiedRepairUtils.updateFinishUnifiedRepairHistory(repairType, hostId, 123); + AutoRepairUtils.updateFinishAutoRepairHistory(repairType, hostId, 123); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT repair_finish_ts FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), hostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -350,14 +350,14 @@ public void testAddHostIdToDeleteHosts() UUID otherHostId = UUID.randomUUID(); QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, host_id) VALUES ('%s', %s)", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), otherHostId)); - UnifiedRepairUtils.addHostIdToDeleteHosts(repairType, hostId, otherHostId); + AutoRepairUtils.addHostIdToDeleteHosts(repairType, hostId, otherHostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s' AND host_id = %s", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, repairType.toString(), otherHostId)); assertNotNull(result); assertEquals(1, result.size()); @@ -370,11 +370,11 @@ public void testAddHostIdToDeleteHosts() @Test public void testAddPriorityHost() { - UnifiedRepairUtils.addPriorityHosts(repairType, ImmutableSet.of(localEndpoint)); + AutoRepairUtils.addPriorityHosts(repairType, ImmutableSet.of(localEndpoint)); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s'", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, repairType.toString())); assertNotNull(result); assertEquals(1, result.size()); @@ -389,14 +389,14 @@ public void testRemovePriorityStatus() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, repairType.toString(), hostId)); - UnifiedRepairUtils.removePriorityStatus(repairType, hostId); + AutoRepairUtils.removePriorityStatus(repairType, hostId); UntypedResultSet result = QueryProcessor.executeInternal(String.format( "SELECT * FROM %s.%s WHERE repair_type = '%s'", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, repairType.toString())); assertNotNull(result); assertEquals(1, result.size()); @@ -409,10 +409,10 @@ public void testGetPriorityHosts() { QueryProcessor.executeInternal(String.format( "INSERT INTO %s.%s (repair_type, repair_priority) VALUES ('%s', { %s })", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_PRIORITY, + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_PRIORITY, repairType.toString(), hostId)); - Set hosts = UnifiedRepairUtils.getPriorityHosts(repairType); + Set hosts = AutoRepairUtils.getPriorityHosts(repairType); assertNotNull(hosts); assertEquals(1, hosts.size()); @@ -424,29 +424,29 @@ public void testCheckNodeContainsKeyspaceReplica() { Keyspace ks = Keyspace.open("ks"); - assertTrue(UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); } @Test public void testTableMaxRepairTimeExceeded() { - DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairTableMaxRepairTime(repairType, "0s"); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairTableMaxRepairTime(repairType, "0s"); - assertTrue(UnifiedRepairUtils.tableMaxRepairTimeExceeded(repairType, 0)); + assertTrue(AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, 0)); } @Test public void testKeyspaceMaxRepairTimeExceeded() { - DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairTableMaxRepairTime(repairType, "0s"); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairTableMaxRepairTime(repairType, "0s"); - assertTrue(UnifiedRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, 0, 1)); + assertTrue(AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, 0, 1)); } @Test public void testGetLastRepairFinishTime() { - UnifiedRepairHistory history = new UnifiedRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); + AutoRepairHistory history = new AutoRepairHistory(UUID.randomUUID(), "", 0, 0, null, 0, false); assertEquals(0, history.getLastRepairFinishTime()); @@ -460,21 +460,21 @@ public void testMyTurnToRunRepairShouldReturnMyTurnWhenRepairOngoing() { UUID myID = UUID.randomUUID(); UUID otherID = UUID.randomUUID(); - DatabaseDescriptor.getUnifiedRepairConfig().setParallelRepairCount(repairType, 5); + DatabaseDescriptor.getAutoRepairConfig().setParallelRepairCount(repairType, 5); long currentMillis = System.currentTimeMillis(); // finish time less than start time means that repair is ongoing - UnifiedRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); + AutoRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); // finish time is larger than start time means that repair for other node is finished - UnifiedRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); + AutoRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); - assertEquals(UnifiedRepairUtils.RepairTurn.MY_TURN, UnifiedRepairUtils.myTurnToRunRepair(repairType, myID)); + assertEquals(AutoRepairUtils.RepairTurn.MY_TURN, AutoRepairUtils.myTurnToRunRepair(repairType, myID)); } @Test public void testLocalStrategyAndNetworkKeyspace() { - assertFalse(UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open("system"))); - assertTrue(UnifiedRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open(KEYSPACE))); + assertFalse(AutoRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open("system"))); + assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open(KEYSPACE))); } @Test @@ -483,10 +483,10 @@ public void testGetLastRepairTimeForNode() UUID myID = UUID.randomUUID(); UUID otherID = UUID.randomUUID(); long currentMillis = System.currentTimeMillis(); - UnifiedRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); - UnifiedRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); + AutoRepairUtils.insertNewRepairHistory(repairType, myID, currentMillis, currentMillis - 100); + AutoRepairUtils.insertNewRepairHistory(repairType, otherID, currentMillis, currentMillis + 100); - assertEquals(currentMillis - 100, UnifiedRepairUtils.getLastRepairTimeForNode(repairType, myID)); + assertEquals(currentMillis - 100, AutoRepairUtils.getLastRepairTimeForNode(repairType, myID)); } @Test @@ -494,6 +494,6 @@ public void testGetLastRepairTimeForNodeWhenHistoryIsEmpty() { UUID myID = UUID.randomUUID(); - assertEquals(0, UnifiedRepairUtils.getLastRepairTimeForNode(repairType, myID)); + assertEquals(0, AutoRepairUtils.getLastRepairTimeForNode(repairType, myID)); } } diff --git a/test/unit/org/apache/cassandra/repair/unifiedrepair/SSTableRepairedAtTest.java b/test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java similarity index 97% rename from test/unit/org/apache/cassandra/repair/unifiedrepair/SSTableRepairedAtTest.java rename to test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java index 14011f85896d..bd14eea805b3 100644 --- a/test/unit/org/apache/cassandra/repair/unifiedrepair/SSTableRepairedAtTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/SSTableRepairedAtTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.repair.unifiedrepair; +package org.apache.cassandra.repair.autorepair; import java.net.UnknownHostException; import java.util.Arrays; @@ -55,8 +55,8 @@ public class SSTableRepairedAtTest extends CQLTester public static void setUp() throws ConfigurationException, UnknownHostException { requireNetwork(); - UnifiedRepairUtils.setup(); - StorageService.instance.doUnifiedRepairSetup(); + AutoRepairUtils.setup(); + StorageService.instance.doAutoRepairSetup(); DatabaseDescriptor.setCDCEnabled(false); } diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java new file mode 100644 index 000000000000..054f136dad75 --- /dev/null +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.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 org.apache.cassandra.service; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; + +import static org.junit.Assert.assertEquals; + +public class AutoRepairServiceBasicTest extends CQLTester { + private static AutoRepairService autoRepairService; + private static AutoRepairConfig config; + + @Before + public void setUp() { + DatabaseDescriptor.setCDCOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); + DatabaseDescriptor.setMaterializedViewsEnabled(false); + DatabaseDescriptor.setCDCEnabled(false); + config = new AutoRepairConfig(); + autoRepairService = new AutoRepairService(); + autoRepairService.config = config; + } + + @Test + public void testSetup() { + AutoRepairService.instance.config = null; + + AutoRepairService.setup(); + + assertEquals(DatabaseDescriptor.getAutoRepairConfig(), AutoRepairService.instance.config); + } + + @Test + public void testGetAutoRepairConfigReturnsConfig() { + assertEquals(config, autoRepairService.getAutoRepairConfig()); + } + + @Test + public void testsetAutoRepairHistoryClearDeleteHostsBufferInSecV2() { + autoRepairService.setAutoRepairHistoryClearDeleteHostsBufferDuration("100s"); + + assertEquals(100, config.getAutoRepairHistoryClearDeleteHostsBufferInterval().toSeconds()); + } + + + @Test + public void testsetAutoRepairMaxRetriesCount() { + autoRepairService.setAutoRepairMaxRetriesCount(101); + + assertEquals(101, config.getRepairMaxRetries()); + } + + + @Test + public void testsetAutoRepairRetryBackoffInSec() { + autoRepairService.setAutoRepairRetryBackoff("102s"); + + assertEquals(102, config.getRepairRetryBackoff().toSeconds()); + } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsWithSchedulerDisabled() { + autoRepairService.config = new AutoRepairConfig(false); + + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsForIRWithMVReplay() { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test + public void testSetAutoRepairEnabledDoesNotThrowForIRWithMVReplayDisabled() { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setMaterializedViewsEnabled(true); + DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test(expected = ConfigurationException.class) + public void testSetAutoRepairEnabledThrowsForIRWithCDCReplay() { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setCDCOnRepairEnabled(true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } + + @Test + public void testSetAutoRepairEnabledDoesNotThrowForIRWithCDCReplayDisabled() { + autoRepairService.config = new AutoRepairConfig(true); + DatabaseDescriptor.setCDCEnabled(true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + } +} diff --git a/test/unit/org/apache/cassandra/service/UnifiedRepairServiceRepairTypeTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java similarity index 72% rename from test/unit/org/apache/cassandra/service/UnifiedRepairServiceRepairTypeTest.java rename to test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java index d91c89ee8619..7c8645149adc 100644 --- a/test/unit/org/apache/cassandra/service/UnifiedRepairServiceRepairTypeTest.java +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceRepairTypeTest.java @@ -20,8 +20,8 @@ import com.google.common.collect.ImmutableSet; import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -33,44 +33,44 @@ import java.util.Set; import java.util.UUID; -import static org.apache.cassandra.Util.setUnifiedRepairEnabled; +import static org.apache.cassandra.Util.setAutoRepairEnabled; import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; import static org.junit.Assert.assertEquals; @RunWith(Parameterized.class) -public class UnifiedRepairServiceRepairTypeTest extends CQLTester { +public class AutoRepairServiceRepairTypeTest extends CQLTester { @Parameterized.Parameter() - public UnifiedRepairConfig.RepairType repairType; + public AutoRepairConfig.RepairType repairType; private final UUID host1 = UUID.fromString("00000000-0000-0000-0000-000000000001"); private final UUID host2 = UUID.fromString("00000000-0000-0000-0000-000000000002"); - private UnifiedRepairService instance; + private AutoRepairService instance; @Parameterized.Parameters(name = "repairType={0}") - public static Collection repairTypes() { - return Arrays.asList(UnifiedRepairConfig.RepairType.values()); + public static Collection repairTypes() { + return Arrays.asList(AutoRepairConfig.RepairType.values()); } @BeforeClass public static void setupClass() throws Exception { SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); - setUnifiedRepairEnabled(true); + setAutoRepairEnabled(true); requireNetwork(); } @Before public void setUpTest() { - UnifiedRepairUtils.setup(); - instance = new UnifiedRepairService(); + AutoRepairUtils.setup(); + instance = new AutoRepairService(); } @Test public void testGetOnGoingRepairHostIdsTest() { long now = System.currentTimeMillis(); - UnifiedRepairUtils.insertNewRepairHistory(repairType, host1, now, now - 1000000); - UnifiedRepairUtils.insertNewRepairHistory(repairType, host2, now, now - 1000000); + AutoRepairUtils.insertNewRepairHistory(repairType, host1, now, now - 1000000); + AutoRepairUtils.insertNewRepairHistory(repairType, host2, now, now - 1000000); Set hosts = instance.getOnGoingRepairHostIds(repairType); diff --git a/test/unit/org/apache/cassandra/service/UnifiedRepairServiceSetterTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java similarity index 59% rename from test/unit/org/apache/cassandra/service/UnifiedRepairServiceSetterTest.java rename to test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java index 57b5500b665e..f34e1f0a7071 100644 --- a/test/unit/org/apache/cassandra/service/UnifiedRepairServiceSetterTest.java +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceSetterTest.java @@ -24,8 +24,8 @@ import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairUtils; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairUtils; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.junit.Before; @@ -43,48 +43,48 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.cassandra.Util.setUnifiedRepairEnabled; +import static org.apache.cassandra.Util.setAutoRepairEnabled; import static org.junit.Assert.assertEquals; @RunWith(Parameterized.class) -public class UnifiedRepairServiceSetterTest extends CQLTester { - private static final UnifiedRepairConfig config = new UnifiedRepairConfig(true); +public class AutoRepairServiceSetterTest extends CQLTester { + private static final AutoRepairConfig config = new AutoRepairConfig(true); @Parameterized.Parameter - public UnifiedRepairConfig.RepairType repairType; + public AutoRepairConfig.RepairType repairType; @Parameterized.Parameter(1) public T arg; @Parameterized.Parameter(2) - public BiConsumer setter; + public BiConsumer setter; @Parameterized.Parameter(3) - public Function getter; + public Function getter; @Parameterized.Parameters(name = "{index}: repairType={0}, arg={1}") public static Collection testCases() { DatabaseDescriptor.setConfig(DatabaseDescriptor.loadConfig()); return Stream.of( - forEachRepairType(true, UnifiedRepairService.instance::setUnifiedRepairEnabled, config::isUnifiedRepairEnabled), - forEachRepairType(100, UnifiedRepairService.instance::setRepairThreads, config::getRepairThreads), - forEachRepairType(200, UnifiedRepairService.instance::setRepairSubRangeNum, config::getRepairSubRangeNum), - forEachRepairType(400, UnifiedRepairService.instance::setRepairSSTableCountHigherThreshold, config::getRepairSSTableCountHigherThreshold), - forEachRepairType(ImmutableSet.of("dc1", "dc2"), UnifiedRepairService.instance::setIgnoreDCs, config::getIgnoreDCs), - forEachRepairType(true, UnifiedRepairService.instance::setPrimaryTokenRangeOnly, config::getRepairPrimaryTokenRangeOnly), - forEachRepairType(600, UnifiedRepairService.instance::setParallelRepairPercentage, config::getParallelRepairPercentage), - forEachRepairType(700, UnifiedRepairService.instance::setParallelRepairCount, config::getParallelRepairCount), - forEachRepairType(true, UnifiedRepairService.instance::setMVRepairEnabled, config::getMVRepairEnabled), - forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), UnifiedRepairService.instance::setRepairPriorityForHosts, UnifiedRepairUtils::getPriorityHosts), - forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), UnifiedRepairService.instance::setForceRepairForHosts, UnifiedRepairServiceSetterTest::isLocalHostForceRepair) + forEachRepairType(true, AutoRepairService.instance::setAutoRepairEnabled, config::isAutoRepairEnabled), + forEachRepairType(100, AutoRepairService.instance::setRepairThreads, config::getRepairThreads), + forEachRepairType(200, AutoRepairService.instance::setRepairSubRangeNum, config::getRepairSubRangeNum), + forEachRepairType(400, AutoRepairService.instance::setRepairSSTableCountHigherThreshold, config::getRepairSSTableCountHigherThreshold), + forEachRepairType(ImmutableSet.of("dc1", "dc2"), AutoRepairService.instance::setIgnoreDCs, config::getIgnoreDCs), + forEachRepairType(true, AutoRepairService.instance::setPrimaryTokenRangeOnly, config::getRepairPrimaryTokenRangeOnly), + forEachRepairType(600, AutoRepairService.instance::setParallelRepairPercentage, config::getParallelRepairPercentage), + forEachRepairType(700, AutoRepairService.instance::setParallelRepairCount, config::getParallelRepairCount), + forEachRepairType(true, AutoRepairService.instance::setMVRepairEnabled, config::getMVRepairEnabled), + forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setRepairPriorityForHosts, AutoRepairUtils::getPriorityHosts), + forEachRepairType(ImmutableSet.of(InetAddressAndPort.getLocalHost()), AutoRepairService.instance::setForceRepairForHosts, AutoRepairServiceSetterTest::isLocalHostForceRepair) ).flatMap(Function.identity()).collect(Collectors.toList()); } - private static Set isLocalHostForceRepair(UnifiedRepairConfig.RepairType type) { + private static Set isLocalHostForceRepair(AutoRepairConfig.RepairType type) { UUID hostId = StorageService.instance.getHostIdForEndpoint(InetAddressAndPort.getLocalHost()); UntypedResultSet resultSet = QueryProcessor.executeInternal(String.format( - "SELECT force_repair FROM %s.%s WHERE host_id = %s and repair_type = '%s'", - SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.UNIFIED_REPAIR_HISTORY, hostId, type)); + "SELECT force_repair FROM %s.%s WHERE host_id = %s and repair_type = '%s'", + SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, SystemDistributedKeyspace.AUTO_REPAIR_HISTORY, hostId, type)); if (!resultSet.isEmpty() && resultSet.one().getBoolean("force_repair")) { return ImmutableSet.of(InetAddressAndPort.getLocalHost()); @@ -92,9 +92,9 @@ private static Set isLocalHostForceRepair(UnifiedRepairConfi return ImmutableSet.of(); } - private static Stream forEachRepairType(T arg, BiConsumer setter, Function getter) { - Object[][] testCases = new Object[UnifiedRepairConfig.RepairType.values().length][4]; - for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) { + private static Stream forEachRepairType(T arg, BiConsumer setter, Function getter) { + Object[][] testCases = new Object[AutoRepairConfig.RepairType.values().length][4]; + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { testCases[repairType.ordinal()] = new Object[]{repairType, arg, setter, getter}; } @@ -104,22 +104,22 @@ private static Stream forEachRepairType(T arg, BiConsumer repairTypes() + public static Collection repairTypes() { - return Arrays.asList(UnifiedRepairConfig.RepairType.values()); + return Arrays.asList(AutoRepairConfig.RepairType.values()); } @Before @@ -66,13 +66,13 @@ public void setUp() throws Exception cmdOutput = new ByteArrayOutputStream(); PrintStream out = new PrintStream(cmdOutput); when(probe.output()).thenReturn(new Output(out, out)); - cmd = new UnifiedRepairStatus(); + cmd = new AutoRepairStatus(); DatabaseDescriptor.daemonInitialization(); DatabaseDescriptor.loadConfig(); - setUnifiedRepairEnabled(true); - DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(UnifiedRepairConfig.RepairType.full, true); - DatabaseDescriptor.getUnifiedRepairConfig().setUnifiedRepairEnabled(UnifiedRepairConfig.RepairType.incremental, true); - when(probe.getUnifiedRepairConfig()).thenReturn(DatabaseDescriptor.getUnifiedRepairConfig()); + setAutoRepairEnabled(true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.full, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + when(probe.getAutoRepairConfig()).thenReturn(DatabaseDescriptor.getAutoRepairConfig()); } @Test(expected = IllegalArgumentException.class) diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SetUnifiedRepairConfigTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java similarity index 79% rename from test/unit/org/apache/cassandra/tools/nodetool/SetUnifiedRepairConfigTest.java rename to test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java index 2e1dded34b54..6e2d79def026 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/SetUnifiedRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java @@ -35,7 +35,7 @@ import org.junit.runners.Suite; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.unifiedrepair.UnifiedRepairConfig; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.tools.NodeProbe; import org.mockito.Mock; import org.mockito.MockitoAnnotations; @@ -46,19 +46,19 @@ import static org.mockito.Mockito.verify; @RunWith(Suite.class) -@Suite.SuiteClasses({ SetUnifiedRepairConfigTest.NoParamTests.class, SetUnifiedRepairConfigTest.RepairTypeParamTests.class, - SetUnifiedRepairConfigTest.RepairTypeAndArgsParamsTests.class }) -public class SetUnifiedRepairConfigTest +@Suite.SuiteClasses({ SetAutoRepairConfigTest.NoParamTests.class, SetAutoRepairConfigTest.RepairTypeParamTests.class, + SetAutoRepairConfigTest.RepairTypeAndArgsParamsTests.class }) +public class SetAutoRepairConfigTest { - protected static UnifiedRepairConfig config; + protected static AutoRepairConfig config; - protected static SetUnifiedRepairConfig cmd; + protected static SetAutoRepairConfig cmd; public static void before(NodeProbe probeMock, PrintStream outMock) { - config = new UnifiedRepairConfig(true); - when(probeMock.getUnifiedRepairConfig()).thenReturn(config); - cmd = new SetUnifiedRepairConfig(); + config = new AutoRepairConfig(true); + when(probeMock.getAutoRepairConfig()).thenReturn(config); + cmd = new SetAutoRepairConfig(); cmd.out = outMock; } @@ -84,15 +84,15 @@ public void testHistoryDeleteHostsClearBufferInSec() cmd.execute(probe); - verify(probe, times(1)).setUnifiedRepairHistoryClearDeleteHostsBufferDuration("1s"); + verify(probe, times(1)).setAutoRepairHistoryClearDeleteHostsBufferDuration("1s"); - // test scenario when unified repair is disabled - when(probe.getUnifiedRepairConfig()).thenReturn(new UnifiedRepairConfig(false)); + // test scenario when auto repair is disabled + when(probe.getAutoRepairConfig()).thenReturn(new AutoRepairConfig(false)); cmd.execute(probe); - // test new calls are not made when unified repair is disabled - verify(probe, times(1)).setUnifiedRepairHistoryClearDeleteHostsBufferDuration("1s"); + // test new calls are not made when auto repair is disabled + verify(probe, times(1)).setAutoRepairHistoryClearDeleteHostsBufferDuration("1s"); } @Test @@ -102,7 +102,7 @@ public void testRepairMaxRetries() cmd.execute(probe); - verify(probe, times(1)).setUnifiedRepairMaxRetriesCount(2); + verify(probe, times(1)).setAutoRepairMaxRetriesCount(2); } @@ -113,7 +113,7 @@ public void testRetryBackoffInSec() cmd.execute(probe); - verify(probe, times(1)).setUnifiedRepairRetryBackoff("3s"); + verify(probe, times(1)).setAutoRepairRetryBackoff("3s"); } @Test @@ -143,12 +143,12 @@ public static class RepairTypeParamTests private static PrintStream out; @Parameterized.Parameter - public UnifiedRepairConfig.RepairType repairType; + public AutoRepairConfig.RepairType repairType; @Parameterized.Parameters(name = "repairType={0}") public static Object[] data() { - return UnifiedRepairConfig.RepairType.values(); + return AutoRepairConfig.RepairType.values(); } private static InetAddressAndPort localEndpoint; @@ -173,20 +173,20 @@ public void testNoArgs() @Test public void testRepairSchedulingDisabled() { - when(probe.getUnifiedRepairConfig()).thenReturn(new UnifiedRepairConfig(false)); + when(probe.getAutoRepairConfig()).thenReturn(new AutoRepairConfig(false)); cmd.repairType = repairType; cmd.args = ImmutableList.of("threads", "1"); cmd.execute(probe); - verify(out, times(1)).println("Unified-repair is not enabled"); + verify(out, times(1)).println("Auto-repair is not enabled"); verify(probe, times(0)).setRepairThreads(repairType, 1); } @Test public void testRepairTypeDisabled() { - config.setUnifiedRepairEnabled(repairType, false); + config.setAutoRepairEnabled(repairType, false); cmd.repairType = repairType; cmd.args = ImmutableList.of("number_of_repair_threads", "1"); @@ -253,7 +253,7 @@ public void testForceRepairHosts() public static class RepairTypeAndArgsParamsTests { @Parameterized.Parameter - public UnifiedRepairConfig.RepairType repairType; + public AutoRepairConfig.RepairType repairType; @Parameterized.Parameter(1) public String paramType; @@ -262,30 +262,30 @@ public static class RepairTypeAndArgsParamsTests public String paramVal; @Parameterized.Parameter(3) - public Consumer verifyFunc; + public Consumer verifyFunc; @Parameterized.Parameters(name = "repairType={0},paramType={1}") public static Collection testCases() { return Stream.of( - forEachRepairType("enabled", "true", (type) -> verify(probe, times(1)).setUnifiedRepairEnabled(type, true)), + forEachRepairType("enabled", "true", (type) -> verify(probe, times(1)).setAutoRepairEnabled(type, true)), forEachRepairType("number_of_repair_threads", "1", (type) -> verify(probe, times(1)).setRepairThreads(type, 1)), forEachRepairType("number_of_subranges", "2", (type) -> verify(probe, times(1)).setRepairSubRangeNum(type, 2)), forEachRepairType("min_repair_interval", "3h", (type) -> verify(probe, times(1)).setRepairMinInterval(type, "3h")), forEachRepairType("sstable_upper_threshold", "4", (type) -> verify(probe, times(1)).setRepairSSTableCountHigherThreshold(type, 4)), - forEachRepairType("table_max_repair_time", "5s", (type) -> verify(probe, times(1)).setUnifiedRepairTableMaxRepairTime(type, "5s")), + forEachRepairType("table_max_repair_time", "5s", (type) -> verify(probe, times(1)).setAutoRepairTableMaxRepairTime(type, "5s")), forEachRepairType("repair_primary_token_range_only", "true", (type) -> verify(probe, times(1)).setPrimaryTokenRangeOnly(type, true)), forEachRepairType("parallel_repair_count", "6", (type) -> verify(probe, times(1)).setParallelRepairCount(type, 6)), forEachRepairType("parallel_repair_percentage", "7", (type) -> verify(probe, times(1)).setParallelRepairPercentage(type, 7)), forEachRepairType("mv_repair_enabled", "true", (type) -> verify(probe, times(1)).setMVRepairEnabled(type, true)), - forEachRepairType("ignore_dcs", "dc1,dc2", (type) -> verify(probe, times(1)).setUnifiedRepairIgnoreDCs(type, ImmutableSet.of("dc1", "dc2"))) + forEachRepairType("ignore_dcs", "dc1,dc2", (type) -> verify(probe, times(1)).setAutoRepairIgnoreDCs(type, ImmutableSet.of("dc1", "dc2"))) ).flatMap(Function.identity()).collect(Collectors.toList()); } - private static Stream forEachRepairType(String paramType, String paramVal, Consumer verifyFunc) + private static Stream forEachRepairType(String paramType, String paramVal, Consumer verifyFunc) { - Object[][] testCases = new Object[UnifiedRepairConfig.RepairType.values().length][4]; - for (UnifiedRepairConfig.RepairType repairType : UnifiedRepairConfig.RepairType.values()) + Object[][] testCases = new Object[AutoRepairConfig.RepairType.values().length][4]; + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { testCases[repairType.ordinal()] = new Object[]{ repairType, paramType, paramVal, verifyFunc }; } @@ -316,12 +316,12 @@ public void test() verifyFunc.accept(repairType); - // test scenario when unified repair is disabled - when(probe.getUnifiedRepairConfig()).thenReturn(new UnifiedRepairConfig(false)); + // test scenario when auto repair is disabled + when(probe.getAutoRepairConfig()).thenReturn(new AutoRepairConfig(false)); cmd.execute(probe); - // test new calls are not made when unified repair is disabled + // test new calls are not made when auto repair is disabled verifyFunc.accept(repairType); } } From 2f52e29d5740777facf362b15f97642d0b2a7f3e Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Thu, 7 Nov 2024 12:30:28 -0800 Subject: [PATCH 059/115] Implement preview repaired metrics --- .../cassandra/metrics/KeyspaceMetrics.java | 7 +++++ .../cassandra/metrics/TableMetrics.java | 6 ++++ .../apache/cassandra/repair/RepairJob.java | 13 ++++++-- .../repair/autorepair/AutoRepairConfig.java | 5 +++- .../repair/autorepair/AutoRepairState.java | 30 +++++++++++++++++++ .../repair/consistent/SyncStatSummary.java | 14 +++++++-- .../AutoRepairStateFactoryTest.java | 4 +++ 7 files changed, 73 insertions(+), 6 deletions(-) diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java index 237fd03e2d7d..6aa0001468a3 100644 --- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java +++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java @@ -135,6 +135,11 @@ public class KeyspaceMetrics public final Counter outOfRangeTokenWrites; /** Lifetime count of paxos requests for keys outside the node's owned token ranges for this keyspace **/ public final Counter outOfRangeTokenPaxosRequests; + /** histogram over the number of desynchronized token ranges detected during preview repair */ + public final Histogram previewedDesynchronizedTokenRanges; + /** histogram over the number of desynchronized bytes detected during preview repair */ + public final Histogram previewedDesynchronizedBytes; + /* * Metrics for inconsistencies detected between repaired data sets across replicas. These @@ -258,6 +263,8 @@ public KeyspaceMetrics(final Keyspace ks) repairSyncTime = createKeyspaceTimer("RepairSyncTime"); partitionsValidated = createKeyspaceHistogram("PartitionsValidated", false); bytesValidated = createKeyspaceHistogram("BytesValidated", false); + previewedDesynchronizedTokenRanges = createKeyspaceHistogram("PreviewedDesynchronizedTokenRanges", false); + previewedDesynchronizedBytes = createKeyspaceHistogram("PreviewedDesynchronizedBytes", false); confirmedRepairedInconsistencies = createKeyspaceMeter("RepairedDataInconsistenciesConfirmed"); unconfirmedRepairedInconsistencies = createKeyspaceMeter("RepairedDataInconsistenciesUnconfirmed"); diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index ab1ab6eb73a3..2ddc1a4825b5 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -210,6 +210,10 @@ public class TableMetrics public final Counter bytesAnticompacted; /** number of bytes where the whole sstable was contained in a repairing range so that we only mutated the repair status */ public final Counter bytesMutatedAnticompaction; + /** number of desynchronized token ranges that were detected during preview repair */ + public final TableHistogram previewedDesynchronizedTokenRanges; + /** number of desynchronized bytes that were detected during preview repair */ + public final TableHistogram previewedDesynchronizedBytes; /** ratio of how much we anticompact vs how much we could mutate the repair status*/ public final Gauge mutatedAnticompactionGauge; @@ -810,6 +814,8 @@ public Long getValue() partitionsValidated = createTableHistogram("PartitionsValidated", cfs.keyspace.metric.partitionsValidated, false); bytesAnticompacted = createTableCounter("BytesAnticompacted"); bytesMutatedAnticompaction = createTableCounter("BytesMutatedAnticompaction"); + previewedDesynchronizedTokenRanges = createTableHistogram("PreviewedDesynchronizedTokenRanges", cfs.keyspace.metric.previewedDesynchronizedTokenRanges, false); + previewedDesynchronizedBytes = createTableHistogram("PreviewedDesynchronizedBytes", cfs.keyspace.metric.previewedDesynchronizedBytes, false); mutatedAnticompactionGauge = createTableGauge("MutatedAnticompactionGauge", () -> { double bytesMutated = bytesMutatedAnticompaction.getCount(); diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java index 424b69acd1c1..eca9a618bc8e 100644 --- a/src/java/org/apache/cassandra/repair/RepairJob.java +++ b/src/java/org/apache/cassandra/repair/RepairJob.java @@ -29,10 +29,11 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.*; - +import org.apache.cassandra.repair.consistent.SyncStatSummary; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.repair.state.JobState; +import org.apache.cassandra.streaming.SessionSummary; import org.apache.cassandra.utils.concurrent.AsyncFuture; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -228,7 +229,15 @@ public void onFailure(Throwable t) logger.warn("{} {}.{} sync failed", session.previewKind.logPrefix(session.getId()), desc.keyspace, desc.columnFamily); SystemDistributedKeyspace.failedRepairJob(session.getId(), desc.keyspace, desc.columnFamily, t); } - cfs.metric.repairsCompleted.inc(); + } + else + { + SyncStatSummary.Table summary = new SyncStatSummary.Table(cfs.keyspace.getName(), cfs.getTableName()); + summary.consumeStats(stats); + cfs.metric.previewedDesynchronizedTokenRanges.update(summary.getRanges()); + cfs.metric.previewedDesynchronizedBytes.update(summary.getRanges()); + } + cfs.metric.repairsCompleted.inc(); tryFailure(t instanceof NoSuchRepairSessionExceptionWrapper ? ((NoSuchRepairSessionExceptionWrapper) t).wrapped : t); diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index a80a875125a9..bffff0629e12 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -59,7 +59,8 @@ public class AutoRepairConfig implements Serializable public enum RepairType implements Serializable { full, - incremental; + incremental, + preview_repaired; public static AutoRepairState getAutoRepairState(RepairType repairType) { @@ -69,6 +70,8 @@ public static AutoRepairState getAutoRepairState(RepairType repairType) return new FullRepairState(); case incremental: return new IncrementalRepairState(); + case preview_repaired: + return new PreviewRepairedState(); } throw new IllegalArgumentException("Invalid repair type: " + repairType); diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index 84701ca78fbc..4e5061f38060 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -300,6 +300,36 @@ public void resetWaitCondition() } } +class PreviewRepairedState extends AutoRepairState +{ + public PreviewRepairedState() + { + super(RepairType.preview_repaired); + } + + @Override + public RepairRunnable getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) + { + RepairOption option = new RepairOption(RepairParallelism.PARALLEL, primaryRangeOnly, false, false, + AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), ranges, + !ranges.isEmpty(), false, false, PreviewKind.REPAIRED, false, true, false, false); + + option.getColumnFamilies().addAll(tables); + + return getRepairRunnable(keyspace, option); + } + + @Override + public void progress(String tag, ProgressEvent event) { + if (event.getType() == ProgressEventType.COMPLETE) + { + + } + + super.progress(tag, event); + } +} + class IncrementalRepairState extends AutoRepairState { public IncrementalRepairState() diff --git a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java index 855ad4bad344..035df81c99c7 100644 --- a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java +++ b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java @@ -81,7 +81,7 @@ public String toString() } } - private static class Table + public static class Table { final String keyspace; @@ -94,7 +94,7 @@ private static class Table final Map, Session> sessions = new HashMap<>(); - Table(String keyspace, String table) + public Table(String keyspace, String table) { this.keyspace = keyspace; this.table = table; @@ -119,7 +119,7 @@ void consumeStat(SyncStat stat) } } - void consumeStats(List stats) + public void consumeStats(List stats) { filter(stats, s -> s.summaries != null).forEach(this::consumeStat); } @@ -174,6 +174,14 @@ public String toString() } return output.toString(); } + + public long getBytes() { + return this.bytes; + } + + public long getRanges() { + return this.ranges; + } } private final Map, Table> summaries = new HashMap<>(); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java index a0e5bdc45294..c4d961f2a5fc 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java @@ -36,6 +36,10 @@ public void testGetRepairState() { state = RepairType.getAutoRepairState(RepairType.incremental); assert state instanceof IncrementalRepairState; + + state = RepairType.getAutoRepairState(RepairType.preview_repaired); + + assert state instanceof PreviewRepairedState; } @Test From 03c0cace97cca60ae8ad55688b393f5c58f362ef Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Thu, 21 Nov 2024 11:14:34 -0800 Subject: [PATCH 060/115] Improve preview repair metrics --- .../cassandra/metrics/KeyspaceMetrics.java | 7 ------- .../apache/cassandra/metrics/TableMetrics.java | 8 ++++---- .../cassandra/repair/PreviewRepairTask.java | 17 +++++++++++++++++ .../org/apache/cassandra/repair/RepairJob.java | 12 +----------- .../repair/autorepair/AutoRepairState.java | 2 +- .../repair/consistent/SyncStatSummary.java | 11 ++++++++--- 6 files changed, 31 insertions(+), 26 deletions(-) diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java index 6aa0001468a3..237fd03e2d7d 100644 --- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java +++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java @@ -135,11 +135,6 @@ public class KeyspaceMetrics public final Counter outOfRangeTokenWrites; /** Lifetime count of paxos requests for keys outside the node's owned token ranges for this keyspace **/ public final Counter outOfRangeTokenPaxosRequests; - /** histogram over the number of desynchronized token ranges detected during preview repair */ - public final Histogram previewedDesynchronizedTokenRanges; - /** histogram over the number of desynchronized bytes detected during preview repair */ - public final Histogram previewedDesynchronizedBytes; - /* * Metrics for inconsistencies detected between repaired data sets across replicas. These @@ -263,8 +258,6 @@ public KeyspaceMetrics(final Keyspace ks) repairSyncTime = createKeyspaceTimer("RepairSyncTime"); partitionsValidated = createKeyspaceHistogram("PartitionsValidated", false); bytesValidated = createKeyspaceHistogram("BytesValidated", false); - previewedDesynchronizedTokenRanges = createKeyspaceHistogram("PreviewedDesynchronizedTokenRanges", false); - previewedDesynchronizedBytes = createKeyspaceHistogram("PreviewedDesynchronizedBytes", false); confirmedRepairedInconsistencies = createKeyspaceMeter("RepairedDataInconsistenciesConfirmed"); unconfirmedRepairedInconsistencies = createKeyspaceMeter("RepairedDataInconsistenciesUnconfirmed"); diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index 2ddc1a4825b5..ff93cffc88b7 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -211,9 +211,9 @@ public class TableMetrics /** number of bytes where the whole sstable was contained in a repairing range so that we only mutated the repair status */ public final Counter bytesMutatedAnticompaction; /** number of desynchronized token ranges that were detected during preview repair */ - public final TableHistogram previewedDesynchronizedTokenRanges; + public final Counter previewedDesynchronizedTokenRanges; /** number of desynchronized bytes that were detected during preview repair */ - public final TableHistogram previewedDesynchronizedBytes; + public final Counter previewedDesynchronizedBytes; /** ratio of how much we anticompact vs how much we could mutate the repair status*/ public final Gauge mutatedAnticompactionGauge; @@ -814,8 +814,8 @@ public Long getValue() partitionsValidated = createTableHistogram("PartitionsValidated", cfs.keyspace.metric.partitionsValidated, false); bytesAnticompacted = createTableCounter("BytesAnticompacted"); bytesMutatedAnticompaction = createTableCounter("BytesMutatedAnticompaction"); - previewedDesynchronizedTokenRanges = createTableHistogram("PreviewedDesynchronizedTokenRanges", cfs.keyspace.metric.previewedDesynchronizedTokenRanges, false); - previewedDesynchronizedBytes = createTableHistogram("PreviewedDesynchronizedBytes", cfs.keyspace.metric.previewedDesynchronizedBytes, false); + previewedDesynchronizedTokenRanges = createTableCounter("PreviewedDesynchronizedTokenRanges"); + previewedDesynchronizedBytes = createTableCounter("PreviewedDesynchronizedBytes"); mutatedAnticompactionGauge = createTableGauge("MutatedAnticompactionGauge", () -> { double bytesMutated = bytesMutatedAnticompaction.getCount(); diff --git a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java index edee11cf2007..10e546cf5068 100644 --- a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java +++ b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java @@ -26,6 +26,7 @@ import org.apache.cassandra.concurrent.ExecutorPlus; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; @@ -89,6 +90,7 @@ public Future performUnsafe(ExecutorPlus executor, Sche if (previewKind == PreviewKind.REPAIRED) maybeSnapshotReplicas(parentSession, keyspace, result.results.get()); // we know its present as summary used it } + emitMetrics(summary); successMessage += "; " + message; coordinator.notification(message); @@ -96,6 +98,21 @@ public Future performUnsafe(ExecutorPlus executor, Sche }); } + private void emitMetrics(SyncStatSummary summary) + { + if (!summary.isEmpty()) + RepairMetrics.previewFailures.inc(); + + summary.getTotals().forEach((key, table) -> { + if (table.isCounter()) + return; + + ColumnFamilyStore cfs = Keyspace.open(key.left).getColumnFamilyStore(key.right); + cfs.metric.previewedDesynchronizedTokenRanges.inc(table.getRanges()); + cfs.metric.previewedDesynchronizedBytes.inc(table.getBytes()); + }); + } + private void maybeSnapshotReplicas(TimeUUID parentSession, String keyspace, List results) { if (!DatabaseDescriptor.snapshotOnRepairedDataMismatch()) diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java index eca9a618bc8e..98503e913059 100644 --- a/src/java/org/apache/cassandra/repair/RepairJob.java +++ b/src/java/org/apache/cassandra/repair/RepairJob.java @@ -29,11 +29,9 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.*; -import org.apache.cassandra.repair.consistent.SyncStatSummary; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.repair.state.JobState; -import org.apache.cassandra.streaming.SessionSummary; import org.apache.cassandra.utils.concurrent.AsyncFuture; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -229,15 +227,7 @@ public void onFailure(Throwable t) logger.warn("{} {}.{} sync failed", session.previewKind.logPrefix(session.getId()), desc.keyspace, desc.columnFamily); SystemDistributedKeyspace.failedRepairJob(session.getId(), desc.keyspace, desc.columnFamily, t); } - } - else - { - SyncStatSummary.Table summary = new SyncStatSummary.Table(cfs.keyspace.getName(), cfs.getTableName()); - summary.consumeStats(stats); - cfs.metric.previewedDesynchronizedTokenRanges.update(summary.getRanges()); - cfs.metric.previewedDesynchronizedBytes.update(summary.getRanges()); - } - cfs.metric.repairsCompleted.inc(); + cfs.metric.repairsCompleted.inc(); tryFailure(t instanceof NoSuchRepairSessionExceptionWrapper ? ((NoSuchRepairSessionExceptionWrapper) t).wrapped : t); diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index 4e5061f38060..73b126de74d5 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -308,7 +308,7 @@ public PreviewRepairedState() } @Override - public RepairRunnable getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) + public RepairCoordinator getRepairRunnable(String keyspace, List tables, Set> ranges, boolean primaryRangeOnly) { RepairOption option = new RepairOption(RepairParallelism.PARALLEL, primaryRangeOnly, false, false, AutoRepairService.instance.getAutoRepairConfig().getRepairThreads(repairType), ranges, diff --git a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java index 035df81c99c7..4c489e87f15e 100644 --- a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java +++ b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java @@ -119,7 +119,7 @@ void consumeStat(SyncStat stat) } } - public void consumeStats(List stats) + void consumeStats(List stats) { filter(stats, s -> s.summaries != null).forEach(this::consumeStat); } @@ -138,7 +138,7 @@ void calculateTotals() totalsCalculated = true; } - boolean isCounter() + public boolean isCounter() { TableMetadata tmd = Schema.instance.getTableMetadata(keyspace, table); return tmd != null && tmd.isCounter(); @@ -180,7 +180,7 @@ public long getBytes() { } public long getRanges() { - return this.ranges; + return this.ranges.size(); } } @@ -241,6 +241,11 @@ private void calculateTotals() totalsCalculated = true; } + public Map, Table> getTotals() { + calculateTotals(); + return summaries; + } + public String toString() { List> tables = Lists.newArrayList(summaries.keySet()); From cee8966f0b5d27c848b61120aefdf72f91abe58c Mon Sep 17 00:00:00 2001 From: kzalys Date: Mon, 25 Nov 2024 20:08:49 -0800 Subject: [PATCH 061/115] Update src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java Co-authored-by: Andrew Tolbert <6889771+tolbertam@users.noreply.github.com> From 2f336de98c3d8bba130ae5d4bbd95760cec4d468 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Tue, 3 Dec 2024 23:13:58 -0800 Subject: [PATCH 062/115] Address comments --- .../statements/schema/TableAttributes.java | 3 +++ .../cassandra/repair/PreviewRepairTask.java | 1 - .../repair/autorepair/AutoRepairState.java | 10 ---------- .../cassandra/schema/AutoRepairParams.java | 3 ++- .../cassandra/schema/SchemaKeyspace.java | 8 ++++++-- .../apache/cassandra/schema/TableParams.java | 18 ++++++++++++++++-- .../cql3/statements/DescribeStatementTest.java | 6 ++++-- .../cassandra/db/SchemaCQLHelperTest.java | 3 ++- .../AutoRepairParameterizedTest.java | 14 +++++++++----- 9 files changed, 42 insertions(+), 24 deletions(-) diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index f47b7e4ba722..e01fba85fe91 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -159,6 +159,9 @@ private TableParams build(TableParams.Builder builder) if (hasOption(Option.REPAIR_INCREMENTAL)) builder.automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, getMap(Option.REPAIR_INCREMENTAL))); + if (hasOption(Option.REPAIR_PREVIEW_REPAIRED)) + builder.automatedRepairPreviewRepaired(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.preview_repaired, getMap(Option.REPAIR_PREVIEW_REPAIRED))); + return builder.build(); } diff --git a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java index 10e546cf5068..bc794d4ff4a8 100644 --- a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java +++ b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java @@ -86,7 +86,6 @@ public Future performUnsafe(ExecutorPlus executor, Sche else { message = (previewKind == PreviewKind.REPAIRED ? "Repaired data is inconsistent\n" : "Preview complete\n") + summary; - RepairMetrics.previewFailures.inc(); if (previewKind == PreviewKind.REPAIRED) maybeSnapshotReplicas(parentSession, keyspace, result.results.get()); // we know its present as summary used it } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index 73b126de74d5..00589e56df72 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -318,16 +318,6 @@ public RepairCoordinator getRepairRunnable(String keyspace, List tables, return getRepairRunnable(keyspace, option); } - - @Override - public void progress(String tag, ProgressEvent event) { - if (event.getType() == ProgressEventType.COMPLETE) - { - - } - - super.progress(tag, event); - } } class IncrementalRepairState extends AutoRepairState diff --git a/src/java/org/apache/cassandra/schema/AutoRepairParams.java b/src/java/org/apache/cassandra/schema/AutoRepairParams.java index ea3802db93d2..aa6082ae7a51 100644 --- a/src/java/org/apache/cassandra/schema/AutoRepairParams.java +++ b/src/java/org/apache/cassandra/schema/AutoRepairParams.java @@ -46,7 +46,8 @@ public String toString() public static final Map> DEFAULT_OPTIONS = ImmutableMap.of(AutoRepairConfig.RepairType.full, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), - AutoRepairConfig.RepairType.incremental, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true))); + AutoRepairConfig.RepairType.incremental, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), + AutoRepairConfig.RepairType.preview_repaired, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true))); public final AutoRepairConfig.RepairType type; diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index fe9a35f74519..1f12d6d0da9f 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -131,6 +131,7 @@ private SchemaKeyspace() + "read_repair text," + "repair_full frozen>," + "repair_incremental frozen>," + + "repair_preview_repaired frozen>," + "PRIMARY KEY ((keyspace_name), table_name))"); private static final TableMetadata Columns = @@ -217,6 +218,7 @@ private SchemaKeyspace() + "read_repair text," + "repair_full frozen>," + "repair_incremental frozen>," + + "repair_preview_repaired frozen>," + "PRIMARY KEY ((keyspace_name), view_name))"); private static final TableMetadata Indexes = @@ -566,7 +568,8 @@ private static void addTableParamsToRowBuilder(TableParams params, Row.SimpleBui .add("read_repair", params.readRepair.toString()) .add("extensions", params.extensions) .add("repair_full", params.automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) - .add("repair_incremental", params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); + .add("repair_incremental", params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()) + .add("repair_preview_repaired", params.automatedRepair.get(AutoRepairConfig.RepairType.preview_repaired).asMap()); // Only add CDC-enabled flag to schema if it's enabled on the node. This is to work around RTE's post-8099 if a 3.8+ @@ -1052,7 +1055,8 @@ static TableParams createTableParamsFromRow(UntypedResultSet.Row row) .cdc(row.has("cdc") && row.getBoolean("cdc")) .readRepair(getReadRepairStrategy(row)) .automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, row.getFrozenTextMap("repair_full"))) - .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, row.getFrozenTextMap("repair_incremental"))); + .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, row.getFrozenTextMap("repair_incremental"))) + .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.preview_repaired, row.getFrozenTextMap("repair_preview_repaired"))); // allow_auto_snapshot column was introduced in 4.2 if (row.has("allow_auto_snapshot")) diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index 9f698ecb9114..f3f687805b1f 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -73,7 +73,9 @@ public enum Option CDC, READ_REPAIR, REPAIR_FULL, - REPAIR_INCREMENTAL; + REPAIR_INCREMENTAL, + REPAIR_PREVIEW_REPAIRED, + ; @Override public String toString() @@ -132,6 +134,7 @@ private TableParams(Builder builder) { put(AutoRepairConfig.RepairType.full, builder.automatedRepairFull); put(AutoRepairConfig.RepairType.incremental, builder.automatedRepairIncremental); + put(AutoRepairConfig.RepairType.preview_repaired, builder.automatedRepairPreviewRepaired); } }; } @@ -164,6 +167,7 @@ public static Builder builder(TableParams params) .readRepair(params.readRepair) .automatedRepairFull(params.automatedRepair.get(AutoRepairConfig.RepairType.full)) .automatedRepairIncremental(params.automatedRepair.get(AutoRepairConfig.RepairType.incremental)) + .automatedRepairPreviewRepaired(params.automatedRepair.get(AutoRepairConfig.RepairType.preview_repaired)) ; } @@ -314,6 +318,7 @@ public String toString() .add(READ_REPAIR.toString(), readRepair) .add(Option.REPAIR_FULL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.full)) .add(Option.REPAIR_INCREMENTAL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.incremental)) + .add(Option.REPAIR_PREVIEW_REPAIRED.toString(), automatedRepair.get(AutoRepairConfig.RepairType.preview_repaired)) .toString(); } @@ -369,7 +374,9 @@ public void appendCqlTo(CqlBuilder builder, boolean isView) .newLine() .append("AND repair_full = ").append(automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) .newLine() - .append("AND repair_incremental = ").append(automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()); + .append("AND repair_incremental = ").append(automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()) + .newLine() + .append("AND repair_preview_repaired = ").append(automatedRepair.get(AutoRepairConfig.RepairType.preview_repaired).asMap()); } public static final class Builder @@ -396,6 +403,7 @@ public static final class Builder private AutoRepairParams automatedRepairFull = new AutoRepairParams(AutoRepairConfig.RepairType.full); private AutoRepairParams automatedRepairIncremental = new AutoRepairParams(AutoRepairConfig.RepairType.incremental); + private AutoRepairParams automatedRepairPreviewRepaired = new AutoRepairParams(AutoRepairConfig.RepairType.preview_repaired); public Builder() { @@ -531,6 +539,12 @@ public Builder automatedRepairIncremental(AutoRepairParams val) automatedRepairIncremental = val; return this; } + + public Builder automatedRepairPreviewRepaired(AutoRepairParams val) + { + automatedRepairPreviewRepaired = val; + return this; + } } public static class Serializer implements MetadataSerializer diff --git a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java index 29b2484d9b30..569c56947ab8 100644 --- a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java +++ b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java @@ -1076,7 +1076,8 @@ private static String tableParametersCql() " AND read_repair = 'BLOCKING'\n" + " AND speculative_retry = '99p'\n" + " AND repair_full = {'enabled': 'true'}\n" + - " AND repair_incremental = {'enabled': 'true'};"; + " AND repair_incremental = {'enabled': 'true'}\n" + + " AND repair_preview_repaired = {'enabled': 'true'};"; } private static String cqlQuoted(Map map) @@ -1105,7 +1106,8 @@ private static String mvParametersCql() " AND read_repair = 'BLOCKING'\n" + " AND speculative_retry = '99p'\n" + " AND repair_full = {'enabled': 'true'}\n" + - " AND repair_incremental = {'enabled': 'true'};"; + " AND repair_incremental = {'enabled': 'true'}\n" + + " AND repair_preview_repaired = {'enabled': 'true'};"; } private static String keyspaceOutput() diff --git a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java index 3e123b36460b..422ac8b05f63 100644 --- a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java +++ b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java @@ -308,7 +308,8 @@ public void testCfmOptionsCQL() " AND read_repair = 'BLOCKING'\n" + " AND speculative_retry = 'ALWAYS'\n" + " AND repair_full = {'enabled': 'true'}\n" + - " AND repair_incremental = {'enabled': 'true'};" + " AND repair_incremental = {'enabled': 'true'}\n" + + " AND repair_preview_repaired = {'enabled': 'true'};" )); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 3ec202f23e19..01fad130e7b9 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -127,7 +127,7 @@ public void setup() SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE)); QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i))", KEYSPACE, TABLE)); - QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i)) WITH repair_full = {'enabled': 'false'} AND repair_incremental = {'enabled': 'false'}", KEYSPACE, TABLE_DISABLED_AUTO_REPAIR)); + QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i)) WITH repair_full = {'enabled': 'false'} AND repair_incremental = {'enabled': 'false'} AND repair_preview_repaired = {'enabled': 'false'}", KEYSPACE, TABLE_DISABLED_AUTO_REPAIR)); QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); @@ -553,15 +553,19 @@ public void testTableAttribute() { assertTrue(TableAttributes.validKeywords().contains("repair_full")); assertTrue(TableAttributes.validKeywords().contains("repair_incremental")); + assertTrue(TableAttributes.validKeywords().contains("repair_preview_repaired")); } @Test public void testDefaultAutomatedRepair() { - Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); - Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); - Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); - Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values() ) + { + Assert.assertTrue(String.format("expected repair type %s to be enabled on table %s", repairType, cfm.name), + cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + Assert.assertFalse(String.format("expected repair type %s to be disabled on table %s", repairType, cfmDisabledAutoRepair.name), + cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + } } @Test From 7abfa0609778a5e79c3d54012b480f6e04a16454 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Tue, 3 Dec 2024 23:31:03 -0800 Subject: [PATCH 063/115] Fix preview repair options for system_schema ks --- src/java/org/apache/cassandra/schema/SchemaKeyspace.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index 1f12d6d0da9f..328270ac0dc3 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -1056,7 +1056,7 @@ static TableParams createTableParamsFromRow(UntypedResultSet.Row row) .readRepair(getReadRepairStrategy(row)) .automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, row.getFrozenTextMap("repair_full"))) .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, row.getFrozenTextMap("repair_incremental"))) - .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.preview_repaired, row.getFrozenTextMap("repair_preview_repaired"))); + .automatedRepairPreviewRepaired(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.preview_repaired, row.getFrozenTextMap("repair_preview_repaired"))); // allow_auto_snapshot column was introduced in 4.2 if (row.has("allow_auto_snapshot")) From f6d39831b245a6dd25d42f13ebbd27031b9ed337 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Wed, 4 Dec 2024 15:23:50 -0800 Subject: [PATCH 064/115] Implement BytesPreviewed and TokenRangesPreviewed metrics --- .../db/compaction/CompactionManager.java | 4 ++-- .../cassandra/metrics/TableMetrics.java | 22 ++++++++++++------- .../cassandra/repair/PreviewRepairTask.java | 4 ++-- .../cassandra/repair/ValidationManager.java | 6 +++++ 4 files changed, 24 insertions(+), 12 deletions(-) diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java index 67b847e0efda..1fe3569bbc40 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java @@ -888,7 +888,7 @@ private static void mutateFullyContainedSSTables(ColumnFamilyStore cfs, Set fullyContainedSSTables = findSSTablesToAnticompact(sstableIterator, normalizedRanges, sessionID); - cfs.metric.bytesMutatedAnticompaction.inc(SSTableReader.getTotalBytes(fullyContainedSSTables)); + cfs.metric.bytesMutatedAnticompaction.mark(SSTableReader.getTotalBytes(fullyContainedSSTables)); cfs.getCompactionStrategyManager().mutateRepaired(fullyContainedSSTables, UNREPAIRED_SSTABLE, sessionID, isTransient); // since we're just re-writing the sstable metdata for the fully contained sstables, we don't want // them obsoleted when the anti-compaction is complete. So they're removed from the transaction here @@ -1724,7 +1724,7 @@ private void doAntiCompaction(ColumnFamilyStore cfs, // repairedAt values for these, we still avoid anti-compacting already repaired sstables, as we currently don't // make use of any actual repairedAt value and splitting up sstables just for that is not worth it at this point. Set unrepairedSSTables = sstables.stream().filter((s) -> !s.isRepaired()).collect(Collectors.toSet()); - cfs.metric.bytesAnticompacted.inc(SSTableReader.getTotalBytes(unrepairedSSTables)); + cfs.metric.bytesAnticompacted.mark(SSTableReader.getTotalBytes(unrepairedSSTables)); Collection> groupedSSTables = cfs.getCompactionStrategyManager().groupSSTablesForAntiCompaction(unrepairedSSTables); // iterate over sstables to check if the full / transient / unrepaired ranges intersect them. diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index ff93cffc88b7..3771d22ffd47 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -207,13 +207,17 @@ public class TableMetrics /** number of partitions read creating merkle trees */ public final TableHistogram partitionsValidated; /** number of bytes read while doing anticompaction */ - public final Counter bytesAnticompacted; + public final Meter bytesAnticompacted; /** number of bytes where the whole sstable was contained in a repairing range so that we only mutated the repair status */ - public final Counter bytesMutatedAnticompaction; + public final Meter bytesMutatedAnticompaction; + /** number of bytes that were scanned during preview repair */ + public final Meter bytesPreviewed; + /** number of token ranges that were scanned during preview repair */ + public final Meter tokenRangesPreviewed; /** number of desynchronized token ranges that were detected during preview repair */ - public final Counter previewedDesynchronizedTokenRanges; + public final Meter previewedDesynchronizedTokenRanges; /** number of desynchronized bytes that were detected during preview repair */ - public final Counter previewedDesynchronizedBytes; + public final Meter previewedDesynchronizedBytes; /** ratio of how much we anticompact vs how much we could mutate the repair status*/ public final Gauge mutatedAnticompactionGauge; @@ -812,10 +816,12 @@ public Long getValue() bytesValidated = createTableHistogram("BytesValidated", cfs.keyspace.metric.bytesValidated, false); partitionsValidated = createTableHistogram("PartitionsValidated", cfs.keyspace.metric.partitionsValidated, false); - bytesAnticompacted = createTableCounter("BytesAnticompacted"); - bytesMutatedAnticompaction = createTableCounter("BytesMutatedAnticompaction"); - previewedDesynchronizedTokenRanges = createTableCounter("PreviewedDesynchronizedTokenRanges"); - previewedDesynchronizedBytes = createTableCounter("PreviewedDesynchronizedBytes"); + bytesAnticompacted = createTableMeter("BytesAnticompacted"); + bytesMutatedAnticompaction = createTableMeter("BytesMutatedAnticompaction"); + bytesPreviewed = createTableMeter("BytesPreviewed"); + tokenRangesPreviewed = createTableMeter("TokenRangesPreviewed"); + previewedDesynchronizedTokenRanges = createTableMeter("PreviewedDesynchronizedTokenRanges"); + previewedDesynchronizedBytes = createTableMeter("PreviewedDesynchronizedBytes"); mutatedAnticompactionGauge = createTableGauge("MutatedAnticompactionGauge", () -> { double bytesMutated = bytesMutatedAnticompaction.getCount(); diff --git a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java index bc794d4ff4a8..f3c3943408eb 100644 --- a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java +++ b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java @@ -107,8 +107,8 @@ private void emitMetrics(SyncStatSummary summary) return; ColumnFamilyStore cfs = Keyspace.open(key.left).getColumnFamilyStore(key.right); - cfs.metric.previewedDesynchronizedTokenRanges.inc(table.getRanges()); - cfs.metric.previewedDesynchronizedBytes.inc(table.getBytes()); + cfs.metric.previewedDesynchronizedTokenRanges.mark(table.getRanges()); + cfs.metric.previewedDesynchronizedBytes.mark(table.getBytes()); }); } diff --git a/src/java/org/apache/cassandra/repair/ValidationManager.java b/src/java/org/apache/cassandra/repair/ValidationManager.java index e3598cd38f87..b2a4091646e7 100644 --- a/src/java/org/apache/cassandra/repair/ValidationManager.java +++ b/src/java/org/apache/cassandra/repair/ValidationManager.java @@ -38,6 +38,7 @@ import org.apache.cassandra.metrics.TopPartitionTracker; import org.apache.cassandra.repair.state.ValidationState; import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MerkleTree; import org.apache.cassandra.utils.MerkleTrees; @@ -143,6 +144,11 @@ public static void doValidation(ColumnFamilyStore cfs, Validator validator) thro { cfs.metric.bytesValidated.update(state.estimatedTotalBytes); cfs.metric.partitionsValidated.update(state.partitionsProcessed); + if (validator.getPreviewKind() != PreviewKind.NONE) + { + cfs.metric.tokenRangesPreviewed.mark(validator.desc.ranges.size()); + cfs.metric.bytesPreviewed.mark(state.estimatedTotalBytes); + } if (topPartitionCollector != null) cfs.topPartitions.merge(topPartitionCollector); } From 8043b8399964e0211a5191f1bbf1918a38da9ca3 Mon Sep 17 00:00:00 2001 From: kzalys Date: Fri, 6 Dec 2024 11:17:37 -0800 Subject: [PATCH 065/115] Update test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java Co-authored-by: Andrew Tolbert <6889771+tolbertam@users.noreply.github.com> --- .../repair/autorepair/AutoRepairParameterizedTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 01fad130e7b9..ab0d83549fb7 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -559,7 +559,7 @@ public void testTableAttribute() @Test public void testDefaultAutomatedRepair() { - for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values() ) + for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { Assert.assertTrue(String.format("expected repair type %s to be enabled on table %s", repairType, cfm.name), cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); From 6a1436375a3462e9c98e3a5dbf2385e2a5ae2ac1 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Fri, 6 Dec 2024 13:13:12 -0800 Subject: [PATCH 066/115] Address comments --- src/java/org/apache/cassandra/metrics/TableMetrics.java | 8 ++++---- .../org/apache/cassandra/repair/PreviewRepairTask.java | 4 ++-- .../cassandra/repair/consistent/SyncStatSummary.java | 9 ++++++--- .../repair/autorepair/AutoRepairStateFactoryTest.java | 7 ++++--- 4 files changed, 16 insertions(+), 12 deletions(-) diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index 3771d22ffd47..c3eb80df29bf 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -215,9 +215,9 @@ public class TableMetrics /** number of token ranges that were scanned during preview repair */ public final Meter tokenRangesPreviewed; /** number of desynchronized token ranges that were detected during preview repair */ - public final Meter previewedDesynchronizedTokenRanges; + public final Meter tokenRangesPreviewedDesynchronized; /** number of desynchronized bytes that were detected during preview repair */ - public final Meter previewedDesynchronizedBytes; + public final Meter bytesPreviewedDesynchronized; /** ratio of how much we anticompact vs how much we could mutate the repair status*/ public final Gauge mutatedAnticompactionGauge; @@ -820,8 +820,8 @@ public Long getValue() bytesMutatedAnticompaction = createTableMeter("BytesMutatedAnticompaction"); bytesPreviewed = createTableMeter("BytesPreviewed"); tokenRangesPreviewed = createTableMeter("TokenRangesPreviewed"); - previewedDesynchronizedTokenRanges = createTableMeter("PreviewedDesynchronizedTokenRanges"); - previewedDesynchronizedBytes = createTableMeter("PreviewedDesynchronizedBytes"); + tokenRangesPreviewedDesynchronized = createTableMeter("TokenRangesPreviewedDesynchronized"); + bytesPreviewedDesynchronized = createTableMeter("BytesPreviewedDesynchronized"); mutatedAnticompactionGauge = createTableGauge("MutatedAnticompactionGauge", () -> { double bytesMutated = bytesMutatedAnticompaction.getCount(); diff --git a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java index f3c3943408eb..600e64610add 100644 --- a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java +++ b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java @@ -107,8 +107,8 @@ private void emitMetrics(SyncStatSummary summary) return; ColumnFamilyStore cfs = Keyspace.open(key.left).getColumnFamilyStore(key.right); - cfs.metric.previewedDesynchronizedTokenRanges.mark(table.getRanges()); - cfs.metric.previewedDesynchronizedBytes.mark(table.getBytes()); + cfs.metric.tokenRangesPreviewedDesynchronized.mark(table.getRanges()); + cfs.metric.bytesPreviewedDesynchronized.mark(table.getBytes()); }); } diff --git a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java index 4c489e87f15e..820c6b011ba6 100644 --- a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java +++ b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java @@ -175,11 +175,13 @@ public String toString() return output.toString(); } - public long getBytes() { + public long getBytes() + { return this.bytes; } - public long getRanges() { + public long getRanges() + { return this.ranges.size(); } } @@ -241,7 +243,8 @@ private void calculateTotals() totalsCalculated = true; } - public Map, Table> getTotals() { + public Map, Table> getTotals() + { calculateTotals(); return summaries; } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java index c4d961f2a5fc..3acdd313e52d 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; public class AutoRepairStateFactoryTest { @@ -31,15 +32,15 @@ public class AutoRepairStateFactoryTest public void testGetRepairState() { AutoRepairState state = RepairType.getAutoRepairState(RepairType.full); - assert state instanceof FullRepairState; + assertTrue(state instanceof FullRepairState); state = RepairType.getAutoRepairState(RepairType.incremental); - assert state instanceof IncrementalRepairState; + assertTrue(state instanceof IncrementalRepairState); state = RepairType.getAutoRepairState(RepairType.preview_repaired); - assert state instanceof PreviewRepairedState; + assertTrue(state instanceof PreviewRepairedState); } @Test From 497563f82162b247b627987c282307182999e82e Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Fri, 6 Dec 2024 13:20:20 -0800 Subject: [PATCH 067/115] Update metrics docs --- .../cassandra/pages/managing/operating/metrics.adoc | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/doc/modules/cassandra/pages/managing/operating/metrics.adoc b/doc/modules/cassandra/pages/managing/operating/metrics.adoc index 2abeaefa6d39..d69fcb21c4bc 100644 --- a/doc/modules/cassandra/pages/managing/operating/metrics.adoc +++ b/doc/modules/cassandra/pages/managing/operating/metrics.adoc @@ -246,12 +246,20 @@ during validation. |PartitionsValidated |Histogram |Histogram over the number of partitions read during validation. -|BytesAnticompacted |Counter |How many bytes we anticompacted. +|BytesAnticompacted |Meter |How many bytes we anticompacted. -|BytesMutatedAnticompaction |Counter |How many bytes we avoided +|BytesMutatedAnticompaction |Meter |How many bytes we avoided anticompacting because the sstable was fully contained in the repaired range. +|BytesPreviewed |Meter |How many bytes we scanned during preview repair. + +|TokenRangesPreviewed |Meter |How many token ranges we scanned during preview repair. + +|BytesPreviewedDesynchronized |Meter |How many desynchronized bytes we found during preview repair. + +|TokenRangesPreviewedDesynchronized |Meter |How many desynchronized token ranges we found during preview repair. + |MutatedAnticompactionGauge |Gauge |Ratio of bytes mutated vs total bytes repaired. |=== From a27ea6f68f7f10ba07a4921ef2eac395bb7cd9ea Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Fri, 6 Dec 2024 15:01:37 -0800 Subject: [PATCH 068/115] Update metrics --- .../cassandra/pages/managing/operating/metrics.adoc | 9 ++++----- src/java/org/apache/cassandra/metrics/TableMetrics.java | 3 --- src/java/org/apache/cassandra/repair/RepairJob.java | 1 + .../org/apache/cassandra/repair/ValidationManager.java | 1 - 4 files changed, 5 insertions(+), 9 deletions(-) diff --git a/doc/modules/cassandra/pages/managing/operating/metrics.adoc b/doc/modules/cassandra/pages/managing/operating/metrics.adoc index d69fcb21c4bc..ca3dfc4b57d3 100644 --- a/doc/modules/cassandra/pages/managing/operating/metrics.adoc +++ b/doc/modules/cassandra/pages/managing/operating/metrics.adoc @@ -252,13 +252,12 @@ read during validation. anticompacting because the sstable was fully contained in the repaired range. -|BytesPreviewed |Meter |How many bytes we scanned during preview repair. +|BytesPreviewed |Meter |Estimated number of bytes that were scanned for local replica during preview repair -|TokenRangesPreviewed |Meter |How many token ranges we scanned during preview repair. +|BytesPreviewedDesynchronized |Meter |Number of desynchronized bytes that were detected among all replicas during preview repair -|BytesPreviewedDesynchronized |Meter |How many desynchronized bytes we found during preview repair. - -|TokenRangesPreviewedDesynchronized |Meter |How many desynchronized token ranges we found during preview repair. +|TokenRangesPreviewedDesynchronized |Meter |Number of token ranges among all replicas where desynchronization was found +during preview repair. These ranges would need to be streamed during subsequent repair. |MutatedAnticompactionGauge |Gauge |Ratio of bytes mutated vs total bytes repaired. diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index c3eb80df29bf..5e5890311025 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -212,8 +212,6 @@ public class TableMetrics public final Meter bytesMutatedAnticompaction; /** number of bytes that were scanned during preview repair */ public final Meter bytesPreviewed; - /** number of token ranges that were scanned during preview repair */ - public final Meter tokenRangesPreviewed; /** number of desynchronized token ranges that were detected during preview repair */ public final Meter tokenRangesPreviewedDesynchronized; /** number of desynchronized bytes that were detected during preview repair */ @@ -819,7 +817,6 @@ public Long getValue() bytesAnticompacted = createTableMeter("BytesAnticompacted"); bytesMutatedAnticompaction = createTableMeter("BytesMutatedAnticompaction"); bytesPreviewed = createTableMeter("BytesPreviewed"); - tokenRangesPreviewed = createTableMeter("TokenRangesPreviewed"); tokenRangesPreviewedDesynchronized = createTableMeter("TokenRangesPreviewedDesynchronized"); bytesPreviewedDesynchronized = createTableMeter("BytesPreviewedDesynchronized"); mutatedAnticompactionGauge = createTableGauge("MutatedAnticompactionGauge", () -> diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java index 98503e913059..424b69acd1c1 100644 --- a/src/java/org/apache/cassandra/repair/RepairJob.java +++ b/src/java/org/apache/cassandra/repair/RepairJob.java @@ -29,6 +29,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.*; + import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.repair.state.JobState; diff --git a/src/java/org/apache/cassandra/repair/ValidationManager.java b/src/java/org/apache/cassandra/repair/ValidationManager.java index b2a4091646e7..f43d5f2f5917 100644 --- a/src/java/org/apache/cassandra/repair/ValidationManager.java +++ b/src/java/org/apache/cassandra/repair/ValidationManager.java @@ -146,7 +146,6 @@ public static void doValidation(ColumnFamilyStore cfs, Validator validator) thro cfs.metric.partitionsValidated.update(state.partitionsProcessed); if (validator.getPreviewKind() != PreviewKind.NONE) { - cfs.metric.tokenRangesPreviewed.mark(validator.desc.ranges.size()); cfs.metric.bytesPreviewed.mark(state.estimatedTotalBytes); } if (topPartitionCollector != null) From daf4cfe2b76d42ea87f9342d1a334b14efaf9d05 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sun, 3 Nov 2024 13:10:20 -0600 Subject: [PATCH 069/115] UnrepairedBytesBasedTokenRangeSplitter prototype Adds prototype UnrepairedBytesBasedTokenRangeSplitter that demonstrates consuming splitter-based properties to limit the repair assignments returned to be bound to the requested amount of work. --- .../io/sstable/format/SSTableScanner.java | 2 +- .../sstable/format/big/BigTableScanner.java | 45 ++- ...nrepairedBytesBasedTokenRangeSplitter.java | 337 ++++++++++++++++++ 3 files changed, 378 insertions(+), 6 deletions(-) create mode 100644 src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java 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..1d0e5c1596e0 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java @@ -86,7 +86,7 @@ protected SSTableScanner(S sstable, this.listener = listener; } - protected static List> makeBounds(SSTableReader sstable, Collection> tokenRanges) + public static List> makeBounds(SSTableReader sstable, Collection> tokenRanges) { List> boundsList = new ArrayList<>(tokenRanges.size()); for (Range range : Range.normalize(tokenRanges)) 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..c09e7d0b6bde 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 @@ -21,6 +21,8 @@ import java.util.Collection; import java.util.Iterator; +import javax.annotation.Nullable; + import com.google.common.collect.Iterators; import org.apache.cassandra.db.DataRange; @@ -85,9 +87,12 @@ private BigTableScanner(BigTableReader sstable, this.rowIndexEntrySerializer = new RowIndexEntry.Serializer(sstable.descriptor.version, sstable.header, sstable.owner().map(SSTable.Owner::getMetrics).orElse(null)); } - private void seekToCurrentRangeStart() + // Helper method to seek to the index for the given position or range and optionally in the data file + private long seekAndProcess(@Nullable PartitionPosition position, + @Nullable AbstractBounds range, + boolean seekDataFile) throws CorruptSSTableException { - long indexPosition = sstable.getIndexScanPosition(currentRange.left); + long indexPosition = sstable.getIndexScanPosition(position); ifile.seek(indexPosition); try { @@ -96,13 +101,21 @@ private void seekToCurrentRangeStart() { indexPosition = ifile.getFilePointer(); DecoratedKey indexDecoratedKey = sstable.decorateKey(ByteBufferUtil.readWithShortLength(ifile)); - if (indexDecoratedKey.compareTo(currentRange.left) > 0 || currentRange.contains(indexDecoratedKey)) + + // Whether the position or range is present in the SSTable. + boolean isFound = (range == null && indexDecoratedKey.compareTo(position) > 0) + || (range != null && (indexDecoratedKey.compareTo(range.left) > 0 || range.contains(indexDecoratedKey))); + if (isFound) { // Found, just read the dataPosition and seek into index and data files long dataPosition = RowIndexEntry.Serializer.readPosition(ifile); + // seek the index file position as we will presumably seek further when going to the next position. ifile.seek(indexPosition); - dfile.seek(dataPosition); - break; + if (seekDataFile) + { + dfile.seek(dataPosition); + } + return dataPosition; } else { @@ -115,6 +128,28 @@ private void seekToCurrentRangeStart() sstable.markSuspect(); throw new CorruptSSTableException(e, sstable.getFilename()); } + // If for whatever reason we don't find position in file, just return 0 + return 0L; + } + + /** + * Seeks to the start of the current range and updates both the index and data file positions. + */ + private void seekToCurrentRangeStart() throws CorruptSSTableException + { + seekAndProcess(currentRange.left, currentRange, true); + } + + /** + * Gets the position in the data file, but does not seek to it. This does seek the index to find the data position + * but does not actually seek the data file. + * @param position position to find in data file. + * @return offset in data file where position exists. + * @throws CorruptSSTableException if SSTable was malformed + */ + public long getDataPosition(PartitionPosition position) throws CorruptSSTableException + { + return seekAndProcess(position, null, false); } protected void doClose() throws IOException diff --git a/src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java new file mode 100644 index 000000000000..953fa57faf99 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java @@ -0,0 +1,337 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DataStorageSpec; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.lifecycle.SSTableIntervalTree; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.db.lifecycle.View; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.big.BigTableReader; +import org.apache.cassandra.io.sstable.format.big.BigTableScanner; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.concurrent.Refs; + +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.splitEvenly; + +public class UnrepairedBytesBasedTokenRangeSplitter implements IAutoRepairTokenRangeSplitter +{ + private static final Logger logger = LoggerFactory.getLogger(UnrepairedBytesBasedTokenRangeSplitter.class); + + static final String SUBRANGE_SIZE = "subrange_size"; + static final String MAX_BYTES_PER_SCHEDULE = "max_bytes_per_schedule"; + + // target bytes per subrange + private final DataStorageSpec.LongBytesBound subrangeSize; + + // maximum target bytes to repair + private final DataStorageSpec.LongBytesBound maxBytesPerSchedule; + + private final long subrangeBytes; + + private final long maxBytesPerScheduleBytes; + + private static final DataStorageSpec.LongBytesBound DEFAULT_SUBRANGE_SIZE = new DataStorageSpec.LongBytesBound("100GiB"); + private static final DataStorageSpec.LongBytesBound DEFAULT_MAX_BYTES_PER_SCHEDULE = new DataStorageSpec.LongBytesBound("500GiB"); + + public UnrepairedBytesBasedTokenRangeSplitter(Map parameters) + { + // Demonstrates parameterizing a range splitter so we can have splitter specific options. + if (parameters.containsKey(SUBRANGE_SIZE)) + { + subrangeSize = new DataStorageSpec.LongBytesBound(parameters.get(SUBRANGE_SIZE)); + } + else + { + subrangeSize = DEFAULT_SUBRANGE_SIZE; + } + subrangeBytes = subrangeSize.toBytes(); + + if (parameters.containsKey(MAX_BYTES_PER_SCHEDULE)) + { + maxBytesPerSchedule = new DataStorageSpec.LongBytesBound(parameters.get(MAX_BYTES_PER_SCHEDULE)); + } + else + { + maxBytesPerSchedule = DEFAULT_MAX_BYTES_PER_SCHEDULE; + } + maxBytesPerScheduleBytes = maxBytesPerSchedule.toBytes(); + + logger.info("Configured {} with {}={}, {}={}", UnrepairedBytesBasedTokenRangeSplitter.class.getName(), + SUBRANGE_SIZE, subrangeSize, MAX_BYTES_PER_SCHEDULE, maxBytesPerSchedule); + } + + @Override + public List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) + { + List repairAssignments = new ArrayList<>(); + + logger.info("Calculating token range splits for repairType={} primaryRangeOnly={} keyspaceName={} tableNames={}", repairType, primaryRangeOnly, keyspaceName, tableNames); + if (repairType != AutoRepairConfig.RepairType.incremental) + { + throw new IllegalArgumentException(this.getClass().getName() + " only supports " + AutoRepairConfig.RepairType.incremental + " repair"); + } + + // TODO: create a custom repair assignment that indicates number of bytes in repair and join tables by byte size. + Collection> tokenRanges = getTokenRanges(primaryRangeOnly, keyspaceName); + for (String tableName : tableNames) + { + repairAssignments.addAll(getRepairAssignmentsForTable(keyspaceName, tableName, tokenRanges)); + } + return repairAssignments; + } + + public List getRepairAssignmentsForTable(String keyspaceName, String tableName, Collection> tokenRanges) + { + List repairAssignments = new ArrayList<>(); + + long targetBytesSoFar = 0; + + for (Range tokenRange : tokenRanges) + { + logger.info("Calculating unrepaired bytes for {}.{} for range {}", keyspaceName, tableName, tokenRange); + // Capture the amount of unrepaired bytes for range + long approximateUnrepairedBytesForRange = 0L; + // Capture the total bytes in read sstables, this will be useful for calculating the ratio + // of data in SSTables including this range and also useful to know how much anticompaction there will be. + long totalBytesInUnrepairedSSTables = 0L; + try (Refs refs = getSSTableReaderRefs(keyspaceName, tableName, tokenRange)) + { + for (SSTableReader reader : refs) + { + // Only evaluate unrepaired SSTables. + if (!reader.isRepaired()) + { + long sstableSize = reader.bytesOnDisk(); + totalBytesInUnrepairedSSTables += sstableSize; + // get the bounds of the sstable for this range using the index file but do not actually read it. + List> bounds = BigTableScanner.makeBounds(reader, Collections.singleton(tokenRange)); + try (BigTableScanner scanner = (BigTableScanner) BigTableScanner.getScanner((BigTableReader) reader, Collections.singleton(tokenRange))) + { + assert bounds.size() == 1; + + AbstractBounds bound = bounds.get(0); + long startPosition = scanner.getDataPosition(bound.left); + long endPosition = scanner.getDataPosition(bound.right); + // If end position is 0 we can assume the sstable ended before that token, bound at size of file + if (endPosition == 0) + { + endPosition = sstableSize; + } + + long approximateRangeBytesInSSTable = Math.max(0, endPosition - startPosition); + // get the fraction of the sstable belonging to the range. + approximateUnrepairedBytesForRange += Math.min(approximateRangeBytesInSSTable, sstableSize); + double ratio = approximateRangeBytesInSSTable / (double) sstableSize; + logger.info("Calculations for {}.{} {}: sstableSize={}, rangeBytesInSSTable={}, startPosition={}, endPosition={}, ratio={}", + keyspaceName, tableName, reader.descriptor.baseFile().name(), + FileUtils.stringifyFileSize(sstableSize), FileUtils.stringifyFileSize(approximateRangeBytesInSSTable), startPosition, endPosition, ratio); + } + } + else + { + logger.info("Skipping over {}.{} {} ({}) because it is repaired", keyspaceName, tableName, reader.descriptor.baseFile().name(), FileUtils.stringifyFileSize(reader.bytesOnDisk())); + } + } + } + + // Only consider token range if it had unrepaired sstables or live data in memtables. + if (totalBytesInUnrepairedSSTables > 0L) + { + // TODO: Possibly some anticompaction configuration we want here, where if we detect a large amount of anticompaction we want to reduce the work we do. + double ratio = approximateUnrepairedBytesForRange / (double) totalBytesInUnrepairedSSTables; + logger.info("Calculated unrepaired bytes for {}.{} for range {}: sstableSize={}, rangeBytesInSSTables={}, ratio={}", keyspaceName, tableName, tokenRange, + FileUtils.stringifyFileSize(totalBytesInUnrepairedSSTables), FileUtils.stringifyFileSize(approximateUnrepairedBytesForRange), ratio); + + // TODO: split on byte size here, this is currently a bit naive in assuming that data is evenly distributed among the range which may not be the + // right assumption. May want to consider when splitting on these ranges to reevaluate how much data is in the range, but for this + // exists as a demonstration. + if (approximateUnrepairedBytesForRange < subrangeBytes) + { + // accept range as is if less than bytes. + logger.info("Using 1 repair assignment for {}.{} for range {} as {} is less than {}", keyspaceName, tableName, tokenRange, + FileUtils.stringifyFileSize(approximateUnrepairedBytesForRange), subrangeSize); + // TODO: this is a bit repetitive see if can reduce more. + RepairAssignment assignment = new BytesBasedRepairAssignment(tokenRange, keyspaceName, Collections.singletonList(tableName), approximateUnrepairedBytesForRange); + if (canAddAssignment(assignment, targetBytesSoFar, approximateUnrepairedBytesForRange)) + { + repairAssignments.add(assignment); + targetBytesSoFar += approximateUnrepairedBytesForRange; + } + else + return repairAssignments; + } + else + { + long targetRanges = approximateUnrepairedBytesForRange / subrangeBytes; + // TODO: approximation per range, this is a bit lossy since targetRanges rounds down. + long approximateBytesPerSplit = approximateUnrepairedBytesForRange / targetRanges; + logger.info("Splitting {}.{} for range {} into {} sub ranges, approximateBytesPerSplit={}", keyspaceName, tableName, tokenRange, targetRanges, FileUtils.stringifyFileSize(approximateBytesPerSplit)); + List> splitRanges = splitEvenly(tokenRange, (int) targetRanges); + int splitRangeCount = 0; + for (Range splitRange : splitRanges) + { + RepairAssignment assignment = new BytesBasedRepairAssignment(splitRange, keyspaceName, Collections.singletonList(tableName), approximateBytesPerSplit); + if (canAddAssignment(assignment, targetBytesSoFar, approximateBytesPerSplit)) + { + logger.info("Added repair assignment for {}.{} for subrange {} (#{}/{}) with approximateBytes={}", + keyspaceName, tableName, splitRange, ++splitRangeCount, splitRanges.size(), FileUtils.stringifyFileSize(approximateBytesPerSplit)); + repairAssignments.add(assignment); + targetBytesSoFar += approximateBytesPerSplit; + } + else + return repairAssignments; + } + } + } + else + { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); + long memtableSize = cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize(); + if (memtableSize > 0L) + { + logger.info("Included {}.{} range {}, had no unrepaired SSTables, but memtableSize={}, adding single repair assignment", keyspaceName, tableName, tokenRange, memtableSize); + RepairAssignment assignment = new BytesBasedRepairAssignment(tokenRange, keyspaceName, Collections.singletonList(tableName), memtableSize); + if (targetBytesSoFar >= maxBytesPerScheduleBytes) + { + return repairAssignments; + } + repairAssignments.add(assignment); + targetBytesSoFar += memtableSize; + } + else + { + logger.info("Skipping {}.{} for range {} because it had no unrepaired SSTables and no memtable data", keyspaceName, tableName, tokenRange); + } + } + } + return repairAssignments; + } + + private boolean canAddAssignment(RepairAssignment repairAssignment, long targetBytesSoFar, long bytesToBeAdded) + { + if (targetBytesSoFar + bytesToBeAdded < maxBytesPerScheduleBytes) + { + return true; + } + logger.warn("Refusing to add {} with a target size of {} because it would increase total repair bytes to {} which is greater than {}={}", + repairAssignment, FileUtils.stringifyFileSize(bytesToBeAdded), FileUtils.stringifyFileSize(targetBytesSoFar + bytesToBeAdded), MAX_BYTES_PER_SCHEDULE, maxBytesPerSchedule); + return false; + } + + public Collection> getTokenRanges(boolean primaryRangeOnly, String keyspaceName) + { + // Collect all applicable token ranges + Collection> wrappedRanges; + if (primaryRangeOnly) + { + wrappedRanges = StorageService.instance.getPrimaryRanges(keyspaceName); + } + else + { + wrappedRanges = StorageService.instance.getLocalRanges(keyspaceName); + } + + // Unwrap each range as we need to account for ranges that overlap the ring + Collection> ranges = new ArrayList<>(); + for (Range wrappedRange : wrappedRanges) + { + ranges.addAll(wrappedRange.unwrap()); + } + + return ranges; + } + + public Refs getSSTableReaderRefs(String keyspaceName, String tableName, Range tokenRange) + { + final ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); + + if (cfs == null) + { + throw new IllegalArgumentException(String.format("Could not resolve ColumnFamilyStore from %s.%s", keyspaceName, tableName)); + } + + Iterable sstables = cfs.getTracker().getView().select(SSTableSet.CANONICAL); + SSTableIntervalTree tree = SSTableIntervalTree.build(sstables); + Range r = Range.makeRowRange(tokenRange); + Iterable canonicalSSTables = View.sstablesInBounds(r.left, r.right, tree); + + // TODO: may need to reason about this not working. + return Refs.ref(canonicalSSTables); + } + + public static class BytesBasedRepairAssignment extends RepairAssignment + { + private final long approximateBytes; + + public BytesBasedRepairAssignment(Range tokenRange, String keyspaceName, List tableNames, long approximateBytes) + { + super(tokenRange, keyspaceName, tableNames); + this.approximateBytes = approximateBytes; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + BytesBasedRepairAssignment that = (BytesBasedRepairAssignment) o; + return approximateBytes == that.approximateBytes; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), approximateBytes); + } + + @Override + public String toString() + { + return "BytesBasedRepairAssignment{" + + "keyspaceName='" + keyspaceName + '\'' + + ", approximateBytes=" + approximateBytes + + ", tokenRange=" + tokenRange + + ", tableNames=" + tableNames + + '}'; + } + + public long getApproximateBytes() + { + return approximateBytes; + } + } +} From 0bcf9d7197d80278fe0a806446b2ed4020bdd98f Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Thu, 7 Nov 2024 15:38:16 -0600 Subject: [PATCH 070/115] Add Repair range splitter --- .../statements/schema/TableAttributes.java | 6 +- .../repair/autorepair/AutoRepairConfig.java | 12 +- .../repair/autorepair/AutoRepairState.java | 6 +- .../repair/autorepair/AutoRepairUtils.java | 36 +- .../DefaultAutoRepairTokenSplitter.java | 5 +- .../autorepair/RepairRangeSplitter.java | 396 ++++++++++++++++++ ...nrepairedBytesBasedTokenRangeSplitter.java | 8 +- .../cassandra/schema/AutoRepairParams.java | 6 +- .../cassandra/schema/SchemaKeyspace.java | 12 +- .../apache/cassandra/schema/TableParams.java | 30 +- .../cassandra/service/AutoRepairService.java | 2 +- .../test/repair/AutoRepairSchedulerTest.java | 8 +- .../config/YamlConfigurationLoaderTest.java | 4 +- .../AutoRepairParameterizedTest.java | 20 +- .../AutoRepairStateFactoryTest.java | 6 +- .../repair/autorepair/AutoRepairTest.java | 8 +- .../autorepair/AutoRepairUtilsTest.java | 2 +- .../autorepair/RepairRangeSplitterTest.java | 239 +++++++++++ .../service/AutoRepairServiceBasicTest.java | 10 +- .../tools/nodetool/AutoRepairStatusTest.java | 4 +- 20 files changed, 726 insertions(+), 94 deletions(-) create mode 100644 src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java create mode 100644 test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index e01fba85fe91..491a0f6b3da0 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -154,13 +154,13 @@ private TableParams build(TableParams.Builder builder) builder.readRepair(ReadRepairStrategy.fromString(getString(READ_REPAIR))); if (hasOption(Option.REPAIR_FULL)) - builder.automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, getMap(Option.REPAIR_FULL))); + builder.automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.FULL, getMap(Option.REPAIR_FULL))); if (hasOption(Option.REPAIR_INCREMENTAL)) - builder.automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, getMap(Option.REPAIR_INCREMENTAL))); + builder.automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.INCREMENTAL, getMap(Option.REPAIR_INCREMENTAL))); if (hasOption(Option.REPAIR_PREVIEW_REPAIRED)) - builder.automatedRepairPreviewRepaired(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.preview_repaired, getMap(Option.REPAIR_PREVIEW_REPAIRED))); + builder.automatedRepairPreviewRepaired(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.PREVIEW_REPAIRED, getMap(Option.REPAIR_PREVIEW_REPAIRED))); return builder.build(); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index bffff0629e12..4a06d78c6eb6 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -58,19 +58,19 @@ public class AutoRepairConfig implements Serializable public enum RepairType implements Serializable { - full, - incremental, - preview_repaired; + FULL, + INCREMENTAL, + PREVIEW_REPAIRED; public static AutoRepairState getAutoRepairState(RepairType repairType) { switch (repairType) { - case full: + case FULL: return new FullRepairState(); - case incremental: + case INCREMENTAL: return new IncrementalRepairState(); - case preview_repaired: + case PREVIEW_REPAIRED: return new PreviewRepairedState(); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java index 00589e56df72..8d2d18cab001 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairState.java @@ -304,7 +304,7 @@ class PreviewRepairedState extends AutoRepairState { public PreviewRepairedState() { - super(RepairType.preview_repaired); + super(RepairType.PREVIEW_REPAIRED); } @Override @@ -324,7 +324,7 @@ class IncrementalRepairState extends AutoRepairState { public IncrementalRepairState() { - super(RepairType.incremental); + super(RepairType.INCREMENTAL); } @Override @@ -369,7 +369,7 @@ class FullRepairState extends AutoRepairState { public FullRepairState() { - super(RepairType.full); + super(RepairType.FULL); } @Override diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index 8849b7847aef..77bdfbfb2e5b 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -19,10 +19,12 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.TreeSet; import java.util.UUID; @@ -33,6 +35,7 @@ import com.google.common.collect.Lists; import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Splitter; import org.apache.cassandra.dht.Token; import org.apache.cassandra.locator.LocalStrategy; @@ -73,6 +76,7 @@ import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.utils.NoSpamLogger; import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN; import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.MY_TURN_DUE_TO_PRIORITY; @@ -848,29 +852,19 @@ public static void runRepairOnNewlyBootstrappedNodeIfEnabled() } } - public static List> splitEvenly(Range tokenRange, int numberOfSplits) + public static Collection> split(Range tokenRange, int numberOfSplits) { - List> splitRanges = new ArrayList<>(); - long left = (Long) tokenRange.left.getTokenValue(); - long right = (Long) tokenRange.right.getTokenValue(); - long repairTokenWidth = (right - left) / numberOfSplits; - for (int i = 0; i < numberOfSplits; i++) + Collection> ranges; + Optional splitter = DatabaseDescriptor.getPartitioner().splitter(); + if (splitter.isEmpty()) { - long curLeft = left + (i * repairTokenWidth); - long curRight = curLeft + repairTokenWidth; - - if ((i + 1) == numberOfSplits) - { - curRight = right; - } - - Token childStartToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + curLeft); - Token childEndToken = ClusterMetadata.current().partitioner.getTokenFactory().fromString("" + curRight); - logger.debug("Current Token Left side {}, right side {}", childStartToken - .toString(), childEndToken.toString()); - Range splitRange = new Range<>(childStartToken, childEndToken); - splitRanges.add(splitRange); + NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 30, TimeUnit.MINUTES, "Partitioner {} does not support splitting, falling back to splitting by token range", DatabaseDescriptor.getPartitioner()); + ranges = Collections.singleton(tokenRange); + } + else + { + ranges = splitter.get().split(Collections.singleton(tokenRange), numberOfSplits); } - return splitRanges; + return ranges; } } diff --git a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java index 9a884f61c581..7c118b99ac7d 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java @@ -29,7 +29,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.service.StorageService; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.splitEvenly; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.split; public class DefaultAutoRepairTokenSplitter implements IAutoRepairTokenRangeSplitter { @@ -48,12 +48,11 @@ public List getRepairAssignments(AutoRepairConfig.RepairType r int numberOfSubranges = config.getRepairSubRangeNum(repairType); boolean byKeyspace = config.getRepairByKeyspace(repairType); - // collect all token ranges. List> allRanges = new ArrayList<>(); for (Range token : tokens) { - allRanges.addAll(splitEvenly(token, numberOfSubranges)); + allRanges.addAll(split(token, numberOfSubranges)); } if (byKeyspace) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java new file mode 100644 index 000000000000..09a5234b6fb0 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -0,0 +1,396 @@ +package org.apache.cassandra.repair.autorepair; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.clearspring.analytics.stream.cardinality.CardinalityMergeException; +import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus; +import com.clearspring.analytics.stream.cardinality.ICardinality; +import org.apache.cassandra.config.DataStorageSpec; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.lifecycle.SSTableIntervalTree; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.db.lifecycle.View; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.big.BigTableReader; +import org.apache.cassandra.io.sstable.format.big.BigTableScanner; +import org.apache.cassandra.io.sstable.metadata.CompactionMetadata; +import org.apache.cassandra.io.sstable.metadata.MetadataType; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.concurrent.Refs; + +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.split; + +/** + * In Apache Cassandra, tuning the repair ranges has three main goals: + *

+ * 1. **Create smaller, consistent repair times**: Long repairs, such as those lasting 15 hours, can be problematic. + * If a node fails 14 hours into the repair, the entire process must be restarted. The goal is to reduce the impact + * of disturbances or failures. However, making the repairs too short can lead to overhead from repair orchestration + * becoming the main bottleneck. + *

+ * 2. **Minimize the impact on hosts**: Repairs should not heavily affect the host systems. For incremental repairs, + * this might involve anti-compaction work. In full repairs, streaming large amounts of data—especially with wide + * partitions—can lead to issues with disk usage and higher compaction costs. + *

+ * 3. **Reduce overstreaming**: The Merkle tree, which represents data within each partition and range, has a maximum size. + * If a repair covers too many partitions, the tree’s leaves represent larger data ranges. Even a small change in a leaf + * can trigger excessive data streaming, making the process inefficient. + *

+ * Additionally, if there are many small tables, it's beneficial to batch these tables together under a single parent repair. + * This prevents the repair overhead from becoming a bottleneck, especially when dealing with hundreds of tables. Running + * individual repairs for each table can significantly impact performance and efficiency. + *

+ * To manage these issues, the strategy involves estimating the size and number of partitions within a range and splitting + * it accordingly to bound the size of the range splits. + */ + +public class RepairRangeSplitter implements IAutoRepairTokenRangeSplitter +{ + private static final Logger logger = LoggerFactory.getLogger(RepairRangeSplitter.class); + + static final String SUBRANGE_SIZE = "bytes_per_assignment"; + static final String PARTITION_COUNT = "partitions_per_assignment"; + static final String TABLE_BATCH_LIMIT = "max_tables_per_assignment"; + static final String MAX_BYTES_PER_SCHEDULE = "max_bytes_per_schedule"; + + private final int tablesPerAssignmentLimit; + private final long maxBytesPerScheduleBytes; + private final long bytesPerSubrange; + private final long partitionsPerSubrange; + + private static final DataStorageSpec.LongBytesBound DEFAULT_SUBRANGE_SIZE = new DataStorageSpec.LongBytesBound("100GiB"); + private static final long DEFAULT_MAX_BYTES_PER_SCHEDULE = Long.MAX_VALUE; + private static final long DEFAULT_PARTITION_LIMIT = (long) Math.pow(2, DatabaseDescriptor.getRepairSessionMaxTreeDepth()); + private static final int DEFAULT_TABLE_BATCH_LIMIT = 64; + + public RepairRangeSplitter(Map parameters) + { + // Demonstrates parameterizing a range splitter so we can have splitter specific options. + DataStorageSpec.LongBytesBound subrangeSize; + if (parameters.containsKey(SUBRANGE_SIZE)) + { + subrangeSize = new DataStorageSpec.LongBytesBound(parameters.get(SUBRANGE_SIZE)); + } + else + { + subrangeSize = DEFAULT_SUBRANGE_SIZE; + } + bytesPerSubrange = subrangeSize.toBytes(); + + if (parameters.containsKey(MAX_BYTES_PER_SCHEDULE)) + { + maxBytesPerScheduleBytes = new DataStorageSpec.LongBytesBound(parameters.get(MAX_BYTES_PER_SCHEDULE)).toBytes(); + } + else + { + maxBytesPerScheduleBytes = DEFAULT_MAX_BYTES_PER_SCHEDULE; + } + + if (parameters.containsKey(PARTITION_COUNT)) + { + partitionsPerSubrange = Long.parseLong(parameters.get(PARTITION_COUNT)); + } + else + { + partitionsPerSubrange = DEFAULT_PARTITION_LIMIT; + } + + if (parameters.containsKey(TABLE_BATCH_LIMIT)) + { + tablesPerAssignmentLimit = Integer.parseInt(parameters.get(TABLE_BATCH_LIMIT)); + } + else + { + tablesPerAssignmentLimit = DEFAULT_TABLE_BATCH_LIMIT; + } + + logger.info("Configured {} with {}={}, {}={}, {}={}, {}={}", RepairRangeSplitter.class.getName(), + SUBRANGE_SIZE, subrangeSize, MAX_BYTES_PER_SCHEDULE, maxBytesPerScheduleBytes, + PARTITION_COUNT, partitionsPerSubrange, TABLE_BATCH_LIMIT, tablesPerAssignmentLimit); + } + + @Override + public List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) + { + logger.debug("Calculating token range splits for repairType={} primaryRangeOnly={} keyspaceName={} tableNames={}", repairType, primaryRangeOnly, keyspaceName, tableNames); + Collection> tokenRanges = getTokenRanges(primaryRangeOnly, keyspaceName); + return getRepairAssignments(repairType, keyspaceName, tableNames, tokenRanges); + } + + @VisibleForTesting + List getRepairAssignments(AutoRepairConfig.RepairType repairType, String keyspaceName, List tableNames, Collection> tokenRanges) + { + List repairAssignments = new ArrayList<>(); + // this is used for batching minimal single assignment tables together + List currentAssignments = new ArrayList<>(); + + for (String tableName : tableNames) + { + List tableAssignments = getRepairAssignmentsForTable(repairType, keyspaceName, tableName, tokenRanges); + + if (tableAssignments.isEmpty()) + continue; + + // If the table assignments are for the same token range and we have room to add more tables to the current assignment + if (tableAssignments.size() == 1 && currentAssignments.size() < tablesPerAssignmentLimit && + (currentAssignments.isEmpty() || currentAssignments.get(0).getTokenRange().equals(tableAssignments.get(0).getTokenRange()))) + { + currentAssignments.addAll(tableAssignments); + } + else + { + if (!currentAssignments.isEmpty()) + { + repairAssignments.add(merge(currentAssignments)); + currentAssignments.clear(); + } + repairAssignments.addAll(tableAssignments); + } + } + if (!currentAssignments.isEmpty()) + repairAssignments.add(merge(currentAssignments)); + return repairAssignments; + } + + @VisibleForTesting + static RepairAssignment merge(List assignments) + { + if (assignments.isEmpty()) + throw new IllegalStateException("Cannot merge empty assignments"); + + Set mergedTableNames = new HashSet<>(); + Range referenceTokenRange = assignments.get(0).getTokenRange(); + String referenceKeyspaceName = assignments.get(0).getKeyspaceName(); + + for (RepairAssignment assignment : assignments) + { + // These checks _should_ be unnecessary but are here to ensure that the assignments are consistent + if (!assignment.getTokenRange().equals(referenceTokenRange)) + throw new IllegalStateException("All assignments must have the same token range"); + if (!assignment.getKeyspaceName().equals(referenceKeyspaceName)) + throw new IllegalStateException("All assignments must have the same keyspace name"); + + mergedTableNames.addAll(assignment.getTableNames()); + } + + return new RepairAssignment(referenceTokenRange, referenceKeyspaceName, new ArrayList<>(mergedTableNames)); + } + + public List getRepairAssignmentsForTable(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName, Collection> tokenRanges) + { + List repairAssignments = new ArrayList<>(); + + long targetBytesSoFar = 0; + + List sizeEstimates = getRangeSizeEstimate(repairType, keyspaceName, tableName, tokenRanges); + // since its possible for us to hit maxBytesPerScheduleBytes before seeing all ranges, shuffle so there is chance + // at least of hitting all the ranges _eventually_ for the worst case scenarios + Collections.shuffle(sizeEstimates); + for (SizeEstimate estimate : sizeEstimates) + { + if (estimate.sizeInRange == 0) + { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); + long memtableSize = cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize(); + if (memtableSize > 0L) + { + logger.debug("Included {}.{} range {}, had no unrepaired SSTables, but memtableSize={}, adding single repair assignment", keyspaceName, tableName, estimate.tokenRange, memtableSize); + RepairAssignment assignment = new RepairAssignment(estimate.tokenRange, keyspaceName, Collections.singletonList(tableName)); + repairAssignments.add(assignment); + } + else + { + logger.debug("Skipping {}.{} for range {} because it had no unrepaired SSTables and no memtable data", keyspaceName, tableName, estimate.tokenRange); + } + } + else if (targetBytesSoFar + estimate.sizeInRange < maxBytesPerScheduleBytes) + { + targetBytesSoFar += estimate.sizeInRange; + // Check if the estimate needs splitting based on the criteria + boolean needsSplitting = estimate.sizeInRange > bytesPerSubrange || estimate.partitions > partitionsPerSubrange; + + if (needsSplitting) + { + // Calculate the number of splits needed for size and partitions + int splitsForSize = (int) Math.ceil((double) estimate.sizeInRange / bytesPerSubrange); + int splitsForPartitions = (int) Math.ceil((double) estimate.partitions / partitionsPerSubrange); + + // Choose the larger of the two as the number of splits + int numberOfSplits = Math.max(splitsForSize, splitsForPartitions); + + // Split the token range into subranges + Collection> subranges = split(estimate.tokenRange, numberOfSplits); + for (Range subrange : subranges) + { + RepairAssignment assignment = new RepairAssignment(subrange, keyspaceName, Collections.singletonList(tableName)); + repairAssignments.add(assignment); + } + } + else + { + // No splitting needed, repair the entire range as-is + RepairAssignment assignment = new RepairAssignment(estimate.tokenRange, keyspaceName, Collections.singletonList(tableName)); + repairAssignments.add(assignment); + } + } + else + { + // Really this is "Ok" but it does mean we are relying on randomness to cover the other ranges + logger.info("Skipping range {} for {}.{} as it would exceed maxBytesPerScheduleBytes, consider increasing maxBytesPerScheduleBytes, reducing node density or monitoring to ensure all ranges do get repaired within gc_grace_seconds", estimate.tokenRange, keyspaceName, tableName); + } + } + return repairAssignments; + } + + public Collection> getTokenRanges(boolean primaryRangeOnly, String keyspaceName) + { + // Collect all applicable token ranges + Collection> wrappedRanges; + if (primaryRangeOnly) + { + wrappedRanges = StorageService.instance.getPrimaryRanges(keyspaceName); + } + else + { + wrappedRanges = StorageService.instance.getLocalRanges(keyspaceName); + } + + // Unwrap each range as we need to account for ranges that overlap the ring + List> ranges = new ArrayList<>(); + for (Range wrappedRange : wrappedRanges) + { + ranges.addAll(wrappedRange.unwrap()); + } + return ranges; + } + + private List getRangeSizeEstimate(AutoRepairConfig.RepairType repairType, String keyspace, String table, Collection> tokenRanges) + { + List sizeEstimates = new ArrayList<>(); + for (Range tokenRange : tokenRanges) + { + logger.debug("Calculating size estimate for {}.{} for range {}", keyspace, table, tokenRange); + try (Refs refs = getSSTableReaderRefs(repairType, keyspace, table, tokenRange)) + { + SizeEstimate estimate = getSizesForRangeOfSSTables(keyspace, table, tokenRange, refs); + logger.debug("Size estimate for {}.{} for range {} is {}", keyspace, table, tokenRange, estimate); + sizeEstimates.add(estimate); + } + } + return sizeEstimates; + } + + private static SizeEstimate getSizesForRangeOfSSTables(String keyspace, String table, Range tokenRange, Refs refs) + { + ICardinality cardinality = new HyperLogLogPlus(13, 25); + long approxBytesInRange = 0L; + long totalBytes = 0L; + + for (SSTableReader reader : refs) + { + try + { + if (reader.openReason == SSTableReader.OpenReason.EARLY) + continue; + CompactionMetadata metadata = (CompactionMetadata) reader.descriptor.getMetadataSerializer().deserialize(reader.descriptor, MetadataType.COMPACTION); + if (metadata != null) + cardinality = cardinality.merge(metadata.cardinalityEstimator); + + long sstableSize = reader.bytesOnDisk(); + totalBytes += sstableSize; + // TODO since reading the index file anyway may be able to get more accurate ratio for partition count, + // still better to use the cardinality estimator then the index since it wont count duplicates. + // get the bounds of the sstable for this range using the index file but do not actually read it. + List> bounds = BigTableScanner.makeBounds(reader, Collections.singleton(tokenRange)); + try (BigTableScanner scanner = (BigTableScanner) BigTableScanner.getScanner((BigTableReader) reader, Collections.singleton(tokenRange))) + { + assert bounds.size() == 1; + + AbstractBounds bound = bounds.get(0); + long startPosition = scanner.getDataPosition(bound.left); + long endPosition = scanner.getDataPosition(bound.right); + // If end position is 0 we can assume the sstable ended before that token, bound at size of file + if (endPosition == 0) + { + endPosition = sstableSize; + } + + long approximateRangeBytesInSSTable = Math.max(0, endPosition - startPosition); + approxBytesInRange += Math.min(approximateRangeBytesInSSTable, sstableSize); + } + } + catch (IOException | CardinalityMergeException e) + { + logger.error("Error calculating size estimate for {}.{} for range {} on {}", keyspace, table, tokenRange, reader, e); + } + } + double ratio = approxBytesInRange / (double) totalBytes; + // use the ratio from size to estimate the partitions in the range as well + long partitions = (long) Math.max(1, Math.ceil(cardinality.cardinality() * ratio)); + return new SizeEstimate(keyspace, table, tokenRange, partitions, approxBytesInRange, totalBytes); + } + + private static Refs getSSTableReaderRefs(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName, Range tokenRange) + { + final ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); + + if (cfs == null) + { + throw new IllegalArgumentException(String.format("Could not resolve ColumnFamilyStore from %s.%s", keyspaceName, tableName)); + } + + Refs refs = null; + + while (refs == null) + { + Iterable sstables = cfs.getTracker().getView().select(SSTableSet.CANONICAL); + SSTableIntervalTree tree = SSTableIntervalTree.build(sstables); + Range r = Range.makeRowRange(tokenRange); + List canonicalSSTables = View.sstablesInBounds(r.left, r.right, tree); + if (repairType == AutoRepairConfig.RepairType.INCREMENTAL) + { + canonicalSSTables = canonicalSSTables.stream().filter(SSTableReader::isRepaired).collect(Collectors.toList()); + } + refs = Refs.tryRef(canonicalSSTables); + } + return refs; + } + + @VisibleForTesting + protected static class SizeEstimate + { + public final String keyspace; + public final String table; + public final Range tokenRange; + public final long partitions; + public final long sizeInRange; + public final long totalSize; + + public SizeEstimate(String keyspace, String table, Range tokenRange, long partitions, long sizeInRange, long totalSize) + { + this.keyspace = keyspace; + this.table = table; + this.tokenRange = tokenRange; + this.partitions = partitions; + this.sizeInRange = sizeInRange; + this.totalSize = totalSize; + } + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java index 953fa57faf99..91d5783abe8e 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java @@ -44,7 +44,7 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.concurrent.Refs; -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.splitEvenly; +import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.split; public class UnrepairedBytesBasedTokenRangeSplitter implements IAutoRepairTokenRangeSplitter { @@ -99,9 +99,9 @@ public List getRepairAssignments(AutoRepairConfig.RepairType r List repairAssignments = new ArrayList<>(); logger.info("Calculating token range splits for repairType={} primaryRangeOnly={} keyspaceName={} tableNames={}", repairType, primaryRangeOnly, keyspaceName, tableNames); - if (repairType != AutoRepairConfig.RepairType.incremental) + if (repairType != AutoRepairConfig.RepairType.INCREMENTAL) { - throw new IllegalArgumentException(this.getClass().getName() + " only supports " + AutoRepairConfig.RepairType.incremental + " repair"); + throw new IllegalArgumentException(this.getClass().getName() + " only supports " + AutoRepairConfig.RepairType.INCREMENTAL + " repair"); } // TODO: create a custom repair assignment that indicates number of bytes in repair and join tables by byte size. @@ -199,7 +199,7 @@ public List getRepairAssignmentsForTable(String keyspaceName, // TODO: approximation per range, this is a bit lossy since targetRanges rounds down. long approximateBytesPerSplit = approximateUnrepairedBytesForRange / targetRanges; logger.info("Splitting {}.{} for range {} into {} sub ranges, approximateBytesPerSplit={}", keyspaceName, tableName, tokenRange, targetRanges, FileUtils.stringifyFileSize(approximateBytesPerSplit)); - List> splitRanges = splitEvenly(tokenRange, (int) targetRanges); + Collection> splitRanges = split(tokenRange, (int) targetRanges); int splitRangeCount = 0; for (Range splitRange : splitRanges) { diff --git a/src/java/org/apache/cassandra/schema/AutoRepairParams.java b/src/java/org/apache/cassandra/schema/AutoRepairParams.java index aa6082ae7a51..7a27f0293f39 100644 --- a/src/java/org/apache/cassandra/schema/AutoRepairParams.java +++ b/src/java/org/apache/cassandra/schema/AutoRepairParams.java @@ -45,9 +45,9 @@ public String toString() } public static final Map> DEFAULT_OPTIONS = - ImmutableMap.of(AutoRepairConfig.RepairType.full, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), - AutoRepairConfig.RepairType.incremental, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), - AutoRepairConfig.RepairType.preview_repaired, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true))); + ImmutableMap.of(AutoRepairConfig.RepairType.FULL, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), + AutoRepairConfig.RepairType.INCREMENTAL, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), + AutoRepairConfig.RepairType.PREVIEW_REPAIRED, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true))); public final AutoRepairConfig.RepairType type; diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index 328270ac0dc3..c17f777feac7 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -567,9 +567,9 @@ private static void addTableParamsToRowBuilder(TableParams params, Row.SimpleBui .add("compression", params.compression.asMap()) .add("read_repair", params.readRepair.toString()) .add("extensions", params.extensions) - .add("repair_full", params.automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) - .add("repair_incremental", params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()) - .add("repair_preview_repaired", params.automatedRepair.get(AutoRepairConfig.RepairType.preview_repaired).asMap()); + .add("repair_full", params.automatedRepair.get(AutoRepairConfig.RepairType.FULL).asMap()) + .add("repair_incremental", params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).asMap()) + .add("repair_preview_repaired", params.automatedRepair.get(AutoRepairConfig.RepairType.PREVIEW_REPAIRED).asMap()); // Only add CDC-enabled flag to schema if it's enabled on the node. This is to work around RTE's post-8099 if a 3.8+ @@ -1054,9 +1054,9 @@ static TableParams createTableParamsFromRow(UntypedResultSet.Row row) SpeculativeRetryPolicy.fromString("99PERCENTILE")) .cdc(row.has("cdc") && row.getBoolean("cdc")) .readRepair(getReadRepairStrategy(row)) - .automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.full, row.getFrozenTextMap("repair_full"))) - .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.incremental, row.getFrozenTextMap("repair_incremental"))) - .automatedRepairPreviewRepaired(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.preview_repaired, row.getFrozenTextMap("repair_preview_repaired"))); + .automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.FULL, row.getFrozenTextMap("repair_full"))) + .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.INCREMENTAL, row.getFrozenTextMap("repair_incremental"))) + .automatedRepairPreviewRepaired(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.PREVIEW_REPAIRED, row.getFrozenTextMap("repair_preview_repaired"))); // allow_auto_snapshot column was introduced in 4.2 if (row.has("allow_auto_snapshot")) diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index f3f687805b1f..fe2681f6c10c 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -132,9 +132,9 @@ private TableParams(Builder builder) automatedRepair = new EnumMap(AutoRepairConfig.RepairType.class) { { - put(AutoRepairConfig.RepairType.full, builder.automatedRepairFull); - put(AutoRepairConfig.RepairType.incremental, builder.automatedRepairIncremental); - put(AutoRepairConfig.RepairType.preview_repaired, builder.automatedRepairPreviewRepaired); + put(AutoRepairConfig.RepairType.FULL, builder.automatedRepairFull); + put(AutoRepairConfig.RepairType.INCREMENTAL, builder.automatedRepairIncremental); + put(AutoRepairConfig.RepairType.PREVIEW_REPAIRED, builder.automatedRepairPreviewRepaired); } }; } @@ -165,9 +165,9 @@ public static Builder builder(TableParams params) .extensions(params.extensions) .cdc(params.cdc) .readRepair(params.readRepair) - .automatedRepairFull(params.automatedRepair.get(AutoRepairConfig.RepairType.full)) - .automatedRepairIncremental(params.automatedRepair.get(AutoRepairConfig.RepairType.incremental)) - .automatedRepairPreviewRepaired(params.automatedRepair.get(AutoRepairConfig.RepairType.preview_repaired)) + .automatedRepairFull(params.automatedRepair.get(AutoRepairConfig.RepairType.FULL)) + .automatedRepairIncremental(params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL)) + .automatedRepairPreviewRepaired(params.automatedRepair.get(AutoRepairConfig.RepairType.PREVIEW_REPAIRED)) ; } @@ -316,9 +316,9 @@ public String toString() .add(EXTENSIONS.toString(), extensions) .add(CDC.toString(), cdc) .add(READ_REPAIR.toString(), readRepair) - .add(Option.REPAIR_FULL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.full)) - .add(Option.REPAIR_INCREMENTAL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.incremental)) - .add(Option.REPAIR_PREVIEW_REPAIRED.toString(), automatedRepair.get(AutoRepairConfig.RepairType.preview_repaired)) + .add(Option.REPAIR_FULL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.FULL)) + .add(Option.REPAIR_INCREMENTAL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL)) + .add(Option.REPAIR_PREVIEW_REPAIRED.toString(), automatedRepair.get(AutoRepairConfig.RepairType.PREVIEW_REPAIRED)) .toString(); } @@ -372,11 +372,11 @@ public void appendCqlTo(CqlBuilder builder, boolean isView) .newLine() .append("AND speculative_retry = ").appendWithSingleQuotes(speculativeRetry.toString()) .newLine() - .append("AND repair_full = ").append(automatedRepair.get(AutoRepairConfig.RepairType.full).asMap()) + .append("AND repair_full = ").append(automatedRepair.get(AutoRepairConfig.RepairType.FULL).asMap()) .newLine() - .append("AND repair_incremental = ").append(automatedRepair.get(AutoRepairConfig.RepairType.incremental).asMap()) + .append("AND repair_incremental = ").append(automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).asMap()) .newLine() - .append("AND repair_preview_repaired = ").append(automatedRepair.get(AutoRepairConfig.RepairType.preview_repaired).asMap()); + .append("AND repair_preview_repaired = ").append(automatedRepair.get(AutoRepairConfig.RepairType.PREVIEW_REPAIRED).asMap()); } public static final class Builder @@ -401,9 +401,9 @@ public static final class Builder private boolean cdc; private ReadRepairStrategy readRepair = ReadRepairStrategy.BLOCKING; - private AutoRepairParams automatedRepairFull = new AutoRepairParams(AutoRepairConfig.RepairType.full); - private AutoRepairParams automatedRepairIncremental = new AutoRepairParams(AutoRepairConfig.RepairType.incremental); - private AutoRepairParams automatedRepairPreviewRepaired = new AutoRepairParams(AutoRepairConfig.RepairType.preview_repaired); + private AutoRepairParams automatedRepairFull = new AutoRepairParams(AutoRepairConfig.RepairType.FULL); + private AutoRepairParams automatedRepairIncremental = new AutoRepairParams(AutoRepairConfig.RepairType.INCREMENTAL); + private AutoRepairParams automatedRepairPreviewRepaired = new AutoRepairParams(AutoRepairConfig.RepairType.PREVIEW_REPAIRED); public Builder() { diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index 7a8342c2a7a4..d8fd68253596 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -61,7 +61,7 @@ public void checkCanRun(RepairType repairType) if (!config.isAutoRepairSchedulingEnabled()) throw new ConfigurationException("Auto-repair scheduller is disabled."); - if (repairType != RepairType.incremental) + if (repairType != RepairType.INCREMENTAL) return; if (DatabaseDescriptor.isMaterializedViewsOnRepairEnabled()) diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java index 9b583f006507..3c180dbdd613 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java @@ -60,14 +60,14 @@ public static void init() throws IOException .set("auto_repair", ImmutableMap.of( "repair_type_overrides", - ImmutableMap.of(AutoRepairConfig.RepairType.full.toString(), + ImmutableMap.of(AutoRepairConfig.RepairType.FULL.toString(), ImmutableMap.of( "initial_scheduler_delay", "5s", "enabled", "true", "parallel_repair_count", "1", "parallel_repair_percentage", "0", "min_repair_interval", "1s"), - AutoRepairConfig.RepairType.incremental.toString(), + AutoRepairConfig.RepairType.INCREMENTAL.toString(), ImmutableMap.of( "initial_scheduler_delay", "5s", "enabled", "true", @@ -105,8 +105,8 @@ public void testScheduler() throws ParseException // wait for a couple of minutes for repair to go through on all three nodes Uninterruptibles.sleepUninterruptibly(2, TimeUnit.MINUTES); - validate(AutoRepairConfig.RepairType.full.toString()); - validate(AutoRepairConfig.RepairType.incremental.toString()); + validate(AutoRepairConfig.RepairType.FULL.toString()); + validate(AutoRepairConfig.RepairType.INCREMENTAL.toString()); } private void validate(String repairType) throws ParseException diff --git a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java index 141c762ba461..8d1ccc39b06e 100644 --- a/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java +++ b/test/unit/org/apache/cassandra/config/YamlConfigurationLoaderTest.java @@ -299,8 +299,8 @@ public void fromMapTest() assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_send_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) assertEquals(new DataStorageSpec.IntBytesBound("5B"), config.internode_socket_receive_buffer_size); // Check names backward compatibility (CASSANDRA-17141 and CASSANDRA-15234) assertEquals(true, config.auto_repair.enabled); - assertEquals(new DurationSpec.IntSecondsBound("6h"), config.auto_repair.getAutoRepairTableMaxRepairTime(AutoRepairConfig.RepairType.incremental)); - config.auto_repair.setMVRepairEnabled(AutoRepairConfig.RepairType.incremental, false); + assertEquals(new DurationSpec.IntSecondsBound("6h"), config.auto_repair.getAutoRepairTableMaxRepairTime(AutoRepairConfig.RepairType.INCREMENTAL)); + config.auto_repair.setMVRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL, false); } @Test diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index ab0d83549fb7..977de921e7d5 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -518,10 +518,10 @@ public void testRepairWaitsForRepairToFinishBeforeSchedullingNewSession() throws @Test public void testDisabledAutoRepairForATableThroughTableLevelConfiguration() { - Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); - Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); - Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); - Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.incremental).repairEnabled()); + Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.FULL).repairEnabled()); + Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).repairEnabled()); + Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.FULL).repairEnabled()); + Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).repairEnabled()); AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairMinInterval(repairType, "0s"); @@ -562,9 +562,13 @@ public void testDefaultAutomatedRepair() for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { Assert.assertTrue(String.format("expected repair type %s to be enabled on table %s", repairType, cfm.name), - cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.FULL).repairEnabled()); + Assert.assertTrue(String.format("expected repair type %s to be enabled on table %s", repairType, cfm.name), + cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).repairEnabled()); + Assert.assertFalse(String.format("expected repair type %s to be disabled on table %s", repairType, cfmDisabledAutoRepair.name), + cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.FULL).repairEnabled()); Assert.assertFalse(String.format("expected repair type %s to be disabled on table %s", repairType, cfmDisabledAutoRepair.name), - cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.full).repairEnabled()); + cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).repairEnabled()); } } @@ -677,7 +681,7 @@ public void testRepairThrowsForIRWithMVReplay() AutoRepair.instance.setup(); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); - if (repairType == AutoRepairConfig.RepairType.incremental) + if (repairType == AutoRepairConfig.RepairType.INCREMENTAL) { try { @@ -701,7 +705,7 @@ public void testRepairThrowsForIRWithCDCReplay() AutoRepair.instance.setup(); DatabaseDescriptor.setCDCOnRepairEnabled(true); - if (repairType == AutoRepairConfig.RepairType.incremental) + if (repairType == AutoRepairConfig.RepairType.INCREMENTAL) { try { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java index 3acdd313e52d..a45583b9bf87 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairStateFactoryTest.java @@ -30,15 +30,15 @@ public class AutoRepairStateFactoryTest { @Test public void testGetRepairState() { - AutoRepairState state = RepairType.getAutoRepairState(RepairType.full); + AutoRepairState state = RepairType.getAutoRepairState(RepairType.FULL); assertTrue(state instanceof FullRepairState); - state = RepairType.getAutoRepairState(RepairType.incremental); + state = RepairType.getAutoRepairState(RepairType.INCREMENTAL); assertTrue(state instanceof IncrementalRepairState); - state = RepairType.getAutoRepairState(RepairType.preview_repaired); + state = RepairType.getAutoRepairState(RepairType.PREVIEW_REPAIRED); assertTrue(state instanceof PreviewRepairedState); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index df9f105b615f..425dc9b31f37 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -57,8 +57,8 @@ public void setup() AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); DatabaseDescriptor.setCDCOnRepairEnabled(false); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.full, true); - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.FULL, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.INCREMENTAL, true); AutoRepairService.setup(); } @@ -101,7 +101,7 @@ public void testSafeGuardSetupCall() @Test(expected = ConfigurationException.class) public void testSetupFailsWhenIREnabledWithCDCReplay() { - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.INCREMENTAL, true); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); DatabaseDescriptor.setCDCEnabled(true); DatabaseDescriptor.setCDCOnRepairEnabled(true); @@ -113,7 +113,7 @@ public void testSetupFailsWhenIREnabledWithCDCReplay() @Test(expected = ConfigurationException.class) public void testSetupFailsWhenIREnabledWithMVReplay() { - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.incremental, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(RepairType.INCREMENTAL, true); DatabaseDescriptor.setCDCOnRepairEnabled(false); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); AutoRepair instance = new AutoRepair(); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java index 9a3c551c9bf6..c831bc625586 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -66,7 +66,7 @@ public class AutoRepairUtilsTest extends CQLTester { - static RepairType repairType = RepairType.incremental; + static RepairType repairType = RepairType.INCREMENTAL; static UUID hostId; static InetAddressAndPort localEndpoint; diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java new file mode 100644 index 000000000000..f1367aad10f1 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.repair.autorepair; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; + +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.cql3.CQLTester; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; + +import static org.apache.cassandra.repair.autorepair.RepairRangeSplitter.TABLE_BATCH_LIMIT; +import static org.junit.Assert.assertEquals; + +public class RepairRangeSplitterTest extends CQLTester +{ + private RepairRangeSplitter repairRangeSplitter; + private String tableName; + private static Range fullRange; + + @BeforeClass + public static void setUpClass() + { + CQLTester.setUpClass(); + fullRange = new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), DatabaseDescriptor.getPartitioner().getMaximumToken()); + } + + @Before + public void setUp() { + repairRangeSplitter = new RepairRangeSplitter(Collections.emptyMap()); + tableName = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); + } + + @Test + public void testGetRepairAssignmentsForTable_NoSSTables() { + Collection> ranges = Collections.singleton(new Range<>(Murmur3Partitioner.instance.getMinimumToken(), Murmur3Partitioner.instance.getMaximumToken())); + List assignments = repairRangeSplitter.getRepairAssignmentsForTable(AutoRepairConfig.RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); + assertEquals(0, assignments.size()); + } + + @Test + public void testGetRepairAssignmentsForTable_Single() throws Throwable { + Collection> ranges = Collections.singleton(new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), DatabaseDescriptor.getPartitioner().getMaximumToken())); + insertAndFlushSingleTable(tableName); + List assignments = repairRangeSplitter.getRepairAssignmentsForTable(AutoRepairConfig.RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); + assertEquals(1, assignments.size()); + } + + @Test + public void testGetRepairAssignmentsForTable_BatchingTables() throws Throwable { + repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "2")); + Collection> ranges = Collections.singleton(fullRange); + + List tableNames = createAndInsertTables(3); + List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + + // We expect two assignments, one with table1 and table2 batched, and one with table3 + assertEquals(2, assignments.size()); + assertEquals(2, assignments.get(0).getTableNames().size()); + assertEquals(1, assignments.get(1).getTableNames().size()); + } + + @Test + public void testGetRepairAssignmentsForTable_BatchSize() throws Throwable { + repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "2")); + Collection> ranges = Collections.singleton(fullRange); + + List tableNames = createAndInsertTables(2); + List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + + // We expect one assignment, with two tables batched + assertEquals(1, assignments.size()); + assertEquals(2, assignments.get(0).getTableNames().size()); + } + + @Test + public void testGetRepairAssignmentsForTable_NoBatching() throws Throwable { + repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "1")); + Collection> ranges = Collections.singleton(fullRange); + + List tableNames = createAndInsertTables(3); + List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + + assertEquals(3, assignments.size()); + } + + @Test + public void testGetRepairAssignmentsForTable_AllBatched() throws Throwable { + repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "100")); + Collection> ranges = Collections.singleton(fullRange); + + List tableNames = createAndInsertTables(5); + List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + + assertEquals(1, assignments.size()); + } + + + + @Test(expected = IllegalStateException.class) + public void testMergeEmptyAssignments() { + // Test when the list of assignments is empty + List emptyAssignments = Collections.emptyList(); + RepairRangeSplitter.merge(emptyAssignments); + } + + @Test + public void testMergeSingleAssignment() { + // Test when there is only one assignment in the list + String keyspaceName = "testKeyspace"; + List tableNames = Arrays.asList("table1", "table2"); + + RepairAssignment assignment = new RepairAssignment(fullRange, keyspaceName, tableNames); + List assignments = Collections.singletonList(assignment); + + RepairAssignment result = RepairRangeSplitter.merge(assignments); + + assertEquals(fullRange, result.getTokenRange()); + assertEquals(keyspaceName, result.getKeyspaceName()); + assertEquals(new HashSet<>(tableNames), new HashSet<>(result.getTableNames())); + } + + @Test + public void testMergeMultipleAssignmentsWithSameTokenRangeAndKeyspace() { + // Test merging multiple assignments with the same token range and keyspace + String keyspaceName = "testKeyspace"; + List tableNames1 = Arrays.asList("table1", "table2"); + List tableNames2 = Arrays.asList("table2", "table3"); + + RepairAssignment assignment1 = new RepairAssignment(fullRange, keyspaceName, tableNames1); + RepairAssignment assignment2 = new RepairAssignment(fullRange, keyspaceName, tableNames2); + List assignments = Arrays.asList(assignment1, assignment2); + + RepairAssignment result = RepairRangeSplitter.merge(assignments); + + assertEquals(fullRange, result.getTokenRange()); + assertEquals(keyspaceName, result.getKeyspaceName()); + assertEquals(new HashSet<>(Arrays.asList("table1", "table2", "table3")), new HashSet<>(result.getTableNames())); + } + + @Test(expected = IllegalStateException.class) + public void testMergeDifferentTokenRange() { + // Test merging assignments with different token ranges + Iterator> range = AutoRepairUtils.split(fullRange, 2).iterator(); // Split the full range into two ranges ie (0-100, 100-200 + Range tokenRange1 = range.next(); + Range tokenRange2 = range.next(); + Assert.assertFalse(range.hasNext()); + + String keyspaceName = "testKeyspace"; + List tableNames = Arrays.asList("table1", "table2"); + + RepairAssignment assignment1 = new RepairAssignment(tokenRange1, keyspaceName, tableNames); + RepairAssignment assignment2 = new RepairAssignment(tokenRange2, keyspaceName, tableNames); + List assignments = Arrays.asList(assignment1, assignment2); + + RepairRangeSplitter.merge(assignments); // Should throw IllegalStateException + } + + @Test(expected = IllegalStateException.class) + public void testMergeDifferentKeyspaceName() { + // Test merging assignments with different keyspace names + List tableNames = Arrays.asList("table1", "table2"); + + RepairAssignment assignment1 = new RepairAssignment(fullRange, "keyspace1", tableNames); + RepairAssignment assignment2 = new RepairAssignment(fullRange, "keyspace2", tableNames); + List assignments = Arrays.asList(assignment1, assignment2); + + RepairRangeSplitter.merge(assignments); // Should throw IllegalStateException + } + + @Test + public void testMergeWithDuplicateTables() { + // Test merging assignments with duplicate table names + String keyspaceName = "testKeyspace"; + List tableNames1 = Arrays.asList("table1", "table2"); + List tableNames2 = Arrays.asList("table2", "table3"); + + RepairAssignment assignment1 = new RepairAssignment(fullRange, keyspaceName, tableNames1); + RepairAssignment assignment2 = new RepairAssignment(fullRange, keyspaceName, tableNames2); + List assignments = Arrays.asList(assignment1, assignment2); + + RepairAssignment result = RepairRangeSplitter.merge(assignments); + + // The merged result should contain all unique table names + assertEquals(new HashSet<>(Arrays.asList("table1", "table2", "table3")), new HashSet<>(result.getTableNames())); + } + + private void insertAndFlushSingleTable(String tableName) throws Throwable { + execute("INSERT INTO %s (k, v) values (?, ?)", 1, 1); + flush(); + } + + private List createAndInsertTables(int count) throws Throwable { + List tableNames = new ArrayList<>(); + for (int i = 0; i < count; i++) { + String tableName = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); + tableNames.add(tableName); + insertAndFlushTable(tableName); + } + return tableNames; + } + + private void insertAndFlushTable(String tableName) throws Throwable { + executeFormattedQuery("INSERT INTO " + KEYSPACE + '.' + tableName + " (k, v) values (?, ?)", 1, 1); + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, tableName); + cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + assertEquals(1, cfs.getLiveSSTables().size()); + } +} diff --git a/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java index 054f136dad75..f8911acbd3ee 100644 --- a/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java +++ b/test/unit/org/apache/cassandra/service/AutoRepairServiceBasicTest.java @@ -84,14 +84,14 @@ public void testsetAutoRepairRetryBackoffInSec() { public void testSetAutoRepairEnabledThrowsWithSchedulerDisabled() { autoRepairService.config = new AutoRepairConfig(false); - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL, true); } @Test(expected = ConfigurationException.class) public void testSetAutoRepairEnabledThrowsForIRWithMVReplay() { autoRepairService.config = new AutoRepairConfig(true); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(true); - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL, true); } @Test @@ -99,20 +99,20 @@ public void testSetAutoRepairEnabledDoesNotThrowForIRWithMVReplayDisabled() { autoRepairService.config = new AutoRepairConfig(true); DatabaseDescriptor.setMaterializedViewsEnabled(true); DatabaseDescriptor.setMaterializedViewsOnRepairEnabled(false); - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL, true); } @Test(expected = ConfigurationException.class) public void testSetAutoRepairEnabledThrowsForIRWithCDCReplay() { autoRepairService.config = new AutoRepairConfig(true); DatabaseDescriptor.setCDCOnRepairEnabled(true); - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL, true); } @Test public void testSetAutoRepairEnabledDoesNotThrowForIRWithCDCReplayDisabled() { autoRepairService.config = new AutoRepairConfig(true); DatabaseDescriptor.setCDCEnabled(true); - autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + autoRepairService.setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL, true); } } diff --git a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java index 16b12ee2350e..f1faabc0fc03 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/AutoRepairStatusTest.java @@ -70,8 +70,8 @@ public void setUp() throws Exception DatabaseDescriptor.daemonInitialization(); DatabaseDescriptor.loadConfig(); setAutoRepairEnabled(true); - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.full, true); - DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.incremental, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.FULL, true); + DatabaseDescriptor.getAutoRepairConfig().setAutoRepairEnabled(AutoRepairConfig.RepairType.INCREMENTAL, true); when(probe.getAutoRepairConfig()).thenReturn(DatabaseDescriptor.getAutoRepairConfig()); } From 7eb0059a03c26871fac02a62cf6e0dea60b06722 Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Thu, 7 Nov 2024 21:01:45 -0600 Subject: [PATCH 071/115] add assignment priorirty to table options --- .../IAutoRepairTokenRangeSplitter.java | 33 +++++++ .../autorepair/RepairRangeSplitter.java | 13 +++ .../cassandra/schema/AutoRepairParams.java | 28 ++++-- .../autorepair/RepairRangeSplitterTest.java | 91 +++++++++++++++++++ 4 files changed, 159 insertions(+), 6 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java index 169600eca405..a4760bc3f94f 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java @@ -18,11 +18,17 @@ package org.apache.cassandra.repair.autorepair; +import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.Objects; +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.schema.AutoRepairParams; public interface IAutoRepairTokenRangeSplitter { @@ -38,6 +44,21 @@ public interface IAutoRepairTokenRangeSplitter */ List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames); + /** + * Reorders the list of {@link RepairAssignment} objects based on their priority for a given repair type. + * The list is sorted in descending order, so higher priority assignments appear first. + * If two assignments have the same priority for the specified repair type, their original order is preserved. + * + * @param repairAssignments A list of {@link RepairAssignment} objects to be reordered. + * @param repairType The {@link AutoRepairConfig.RepairType} used to determine the priority of each assignment. + * The priority is determined using the {@link RepairAssignment#getPriority(AutoRepairConfig.RepairType)} method. + */ + @VisibleForTesting + default void reorderByPriority(List repairAssignments, AutoRepairConfig.RepairType repairType) + { + repairAssignments.sort(Comparator.comparingInt(a -> ((RepairAssignment) a).getPriority(repairType)).reversed()); + } + /** * Defines a repair assignment to be issued by the autorepair framework. */ @@ -71,6 +92,18 @@ public List getTableNames() return tableNames; } + public int getPriority(AutoRepairConfig.RepairType type) + { + int max = 0; + for (String table : tableNames) + { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, table); + if (cfs != null) + max = Math.max(max, cfs.metadata().params.automatedRepair.get(type).priority()); + } + return max; + } + @Override public boolean equals(Object o) { diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index 09a5234b6fb0..a13fa9748bb5 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -4,6 +4,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -141,6 +142,16 @@ List getRepairAssignments(AutoRepairConfig.RepairType repairTy // this is used for batching minimal single assignment tables together List currentAssignments = new ArrayList<>(); + // sort the tables by size so can batch the smallest ones together + tableNames.sort((t1, t2) -> { + ColumnFamilyStore cfs1 = ColumnFamilyStore.getIfExists(keyspaceName, t1); + ColumnFamilyStore cfs2 = ColumnFamilyStore.getIfExists(keyspaceName, t2); + if (cfs1 == null || cfs2 == null) + { + throw new IllegalArgumentException(String.format("Could not resolve ColumnFamilyStore from %s.%s", keyspaceName, t1)); + } + return Long.compare(cfs1.metric.totalDiskSpaceUsed.getCount(), cfs2.metric.totalDiskSpaceUsed.getCount()); + }); for (String tableName : tableNames) { List tableAssignments = getRepairAssignmentsForTable(repairType, keyspaceName, tableName, tokenRanges); @@ -166,6 +177,8 @@ List getRepairAssignments(AutoRepairConfig.RepairType repairTy } if (!currentAssignments.isEmpty()) repairAssignments.add(merge(currentAssignments)); + + reorderByPriority(repairAssignments, repairType); return repairAssignments; } diff --git a/src/java/org/apache/cassandra/schema/AutoRepairParams.java b/src/java/org/apache/cassandra/schema/AutoRepairParams.java index 7a27f0293f39..feadbe4e227e 100644 --- a/src/java/org/apache/cassandra/schema/AutoRepairParams.java +++ b/src/java/org/apache/cassandra/schema/AutoRepairParams.java @@ -17,6 +17,8 @@ */ package org.apache.cassandra.schema; +import java.util.Arrays; +import java.util.EnumMap; import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -35,7 +37,8 @@ public final class AutoRepairParams { public enum Option { - ENABLED; + ENABLED, + PRIORITY; @Override public String toString() @@ -44,10 +47,15 @@ public String toString() } } + public static final Map DEFAULT_SUB_OPTIONS = ImmutableMap.of( + Option.ENABLED.toString(), Boolean.toString(true), + Option.PRIORITY.toString(), "0" + ); + public static final Map> DEFAULT_OPTIONS = - ImmutableMap.of(AutoRepairConfig.RepairType.FULL, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), - AutoRepairConfig.RepairType.INCREMENTAL, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true)), - AutoRepairConfig.RepairType.PREVIEW_REPAIRED, ImmutableMap.of(Option.ENABLED.toString(), Boolean.toString(true))); + ImmutableMap.of(AutoRepairConfig.RepairType.FULL, DEFAULT_SUB_OPTIONS, + AutoRepairConfig.RepairType.INCREMENTAL, DEFAULT_SUB_OPTIONS, + AutoRepairConfig.RepairType.PREVIEW_REPAIRED, DEFAULT_SUB_OPTIONS); public final AutoRepairConfig.RepairType type; @@ -60,7 +68,7 @@ public String toString() public static AutoRepairParams create(AutoRepairConfig.RepairType repairType, Map options) { - Map> optionsMap = new HashMap<>(); + Map> optionsMap = new EnumMap<>(AutoRepairConfig.RepairType.class); for (Map.Entry> entry : DEFAULT_OPTIONS.entrySet()) { optionsMap.put(entry.getKey(), new HashMap<>(entry.getValue())); @@ -69,7 +77,7 @@ public static AutoRepairParams create(AutoRepairConfig.RepairType repairType, Ma { for (Map.Entry entry : options.entrySet()) { - if (!Option.ENABLED.toString().equals(toLowerCaseLocalized(entry.getKey()))) + if (Arrays.stream(Option.values()).noneMatch(option -> option.toString().equalsIgnoreCase(entry.getKey()))) { throw new ConfigurationException(format("Unknown property '%s'", entry.getKey())); } @@ -89,6 +97,14 @@ public boolean repairEnabled() : Boolean.parseBoolean(enabled); } + public int priority() + { + String priority = options.get(type).get(Option.PRIORITY.toString()); + return priority == null + ? Integer.parseInt(DEFAULT_OPTIONS.get(type).get(Option.PRIORITY.toString())) + : Integer.parseInt(priority); + } + public void validate() { String enabled = options.get(type).get(Option.ENABLED.toString()); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java index f1367aad10f1..3413c5171a74 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java @@ -41,6 +41,7 @@ import static org.apache.cassandra.repair.autorepair.RepairRangeSplitter.TABLE_BATCH_LIMIT; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class RepairRangeSplitterTest extends CQLTester { @@ -61,6 +62,95 @@ public void setUp() { tableName = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); } + @Test + public void testReorderByPriorityWithDifferentPriorities() { + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '3'}"); + String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); + + // Test reordering assignments with different priorities + RepairAssignment assignment1 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table1)); + RepairAssignment assignment2 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table2)); + RepairAssignment assignment3 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table3)); + + // Assume these priorities based on the repair type + List assignments = new ArrayList<>(Arrays.asList(assignment1, assignment2, assignment3)); + + repairRangeSplitter.reorderByPriority(assignments, AutoRepairConfig.RepairType.FULL); + + // Verify the order is by descending priority + assertEquals(assignment2, assignments.get(0)); + assertEquals(assignment1, assignments.get(1)); + assertEquals(assignment3, assignments.get(2)); + } + + @Test + public void testReorderByPriorityWithSamePriority() { + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + + // Test reordering assignments with the same priority + RepairAssignment assignment1 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table1)); + RepairAssignment assignment2 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table2)); + RepairAssignment assignment3 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table3)); + + List assignments = new ArrayList<>(Arrays.asList(assignment1, assignment2, assignment3)); + + repairRangeSplitter.reorderByPriority(assignments, AutoRepairConfig.RepairType.FULL); + + // Verify the original order is preserved as all priorities are the same + assertEquals(assignment1, assignments.get(0)); + assertEquals(assignment2, assignments.get(1)); + assertEquals(assignment3, assignments.get(2)); + } + + @Test + public void testReorderByPriorityWithMixedPriorities() { + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '3'}"); + String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table4 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); + + // Test reordering assignments with mixed priorities + RepairAssignment assignment1 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table1)); + RepairAssignment assignment2 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table2)); + RepairAssignment assignment3 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table3)); + RepairAssignment assignment4 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table4)); + + List assignments = new ArrayList<>(Arrays.asList(assignment1, assignment2, assignment3, assignment4)); + + repairRangeSplitter.reorderByPriority(assignments, AutoRepairConfig.RepairType.FULL); + + // Verify the order: highest priority first, then preserved order for same priority + assertEquals(assignment2, assignments.get(0)); // Priority 3 + assertEquals(assignment1, assignments.get(1)); // Priority 2 + assertEquals(assignment3, assignments.get(2)); // Priority 2 + assertEquals(assignment4, assignments.get(3)); // Priority 1 + } + + @Test + public void testReorderByPriorityWithEmptyList() { + // Test with an empty list (should remain empty) + List assignments = new ArrayList<>(); + repairRangeSplitter.reorderByPriority(assignments, AutoRepairConfig.RepairType.FULL); + assertTrue(assignments.isEmpty()); + } + + @Test + public void testReorderByPriorityWithOneElement() { + // Test with a single element (should remain unchanged) + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '5'}"); + + RepairAssignment assignment1 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table1)); + + List assignments = new ArrayList<>(Collections.singletonList(assignment1)); + + repairRangeSplitter.reorderByPriority(assignments, AutoRepairConfig.RepairType.FULL); + + assertEquals(assignment1, assignments.get(0)); // Single element should remain in place + } + @Test public void testGetRepairAssignmentsForTable_NoSSTables() { Collection> ranges = Collections.singleton(new Range<>(Murmur3Partitioner.instance.getMinimumToken(), Murmur3Partitioner.instance.getMaximumToken())); @@ -215,6 +305,7 @@ public void testMergeWithDuplicateTables() { assertEquals(new HashSet<>(Arrays.asList("table1", "table2", "table3")), new HashSet<>(result.getTableNames())); } + private void insertAndFlushSingleTable(String tableName) throws Throwable { execute("INSERT INTO %s (k, v) values (?, ?)", 1, 1); flush(); From ba729c3405596eacf39f19a2d07c0957beb8ed24 Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Thu, 7 Nov 2024 21:03:05 -0600 Subject: [PATCH 072/115] reorder with DefaultAutoRepairTokenSplitter --- .../repair/autorepair/DefaultAutoRepairTokenSplitter.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java index 7c118b99ac7d..c0ddc6b9aa7e 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java @@ -74,6 +74,7 @@ public List getRepairAssignments(AutoRepairConfig.RepairType r } } } + reorderByPriority(repairAssignments, repairType); return repairAssignments; } } \ No newline at end of file From dca85bad1c21448aea730f1c54915dc392538256 Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Thu, 7 Nov 2024 22:48:30 -0600 Subject: [PATCH 073/115] add tests for partition count estimation --- .../autorepair/RepairRangeSplitter.java | 8 +- .../autorepair/RepairRangeSplitterTest.java | 98 +++++++++++++------ 2 files changed, 71 insertions(+), 35 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index a13fa9748bb5..ffb104a53686 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -147,9 +147,7 @@ List getRepairAssignments(AutoRepairConfig.RepairType repairTy ColumnFamilyStore cfs1 = ColumnFamilyStore.getIfExists(keyspaceName, t1); ColumnFamilyStore cfs2 = ColumnFamilyStore.getIfExists(keyspaceName, t2); if (cfs1 == null || cfs2 == null) - { throw new IllegalArgumentException(String.format("Could not resolve ColumnFamilyStore from %s.%s", keyspaceName, t1)); - } return Long.compare(cfs1.metric.totalDiskSpaceUsed.getCount(), cfs2.metric.totalDiskSpaceUsed.getCount()); }); for (String tableName : tableNames) @@ -310,7 +308,8 @@ private List getRangeSizeEstimate(AutoRepairConfig.RepairType repa return sizeEstimates; } - private static SizeEstimate getSizesForRangeOfSSTables(String keyspace, String table, Range tokenRange, Refs refs) + @VisibleForTesting + static SizeEstimate getSizesForRangeOfSSTables(String keyspace, String table, Range tokenRange, Refs refs) { ICardinality cardinality = new HyperLogLogPlus(13, 25); long approxBytesInRange = 0L; @@ -360,7 +359,8 @@ private static SizeEstimate getSizesForRangeOfSSTables(String keyspace, String t return new SizeEstimate(keyspace, table, tokenRange, partitions, approxBytesInRange, totalBytes); } - private static Refs getSSTableReaderRefs(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName, Range tokenRange) + @VisibleForTesting + static Refs getSSTableReaderRefs(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName, Range tokenRange) { final ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java index 3413c5171a74..aa81e75300cf 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java @@ -37,7 +37,9 @@ import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; +import org.apache.cassandra.utils.concurrent.Refs; import static org.apache.cassandra.repair.autorepair.RepairRangeSplitter.TABLE_BATCH_LIMIT; import static org.junit.Assert.assertEquals; @@ -47,13 +49,13 @@ public class RepairRangeSplitterTest extends CQLTester { private RepairRangeSplitter repairRangeSplitter; private String tableName; - private static Range fullRange; + private static Range FULL_RANGE; @BeforeClass public static void setUpClass() { CQLTester.setUpClass(); - fullRange = new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), DatabaseDescriptor.getPartitioner().getMaximumToken()); + FULL_RANGE = new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), DatabaseDescriptor.getPartitioner().getMaximumToken()); } @Before @@ -62,6 +64,36 @@ public void setUp() { tableName = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); } + @Test + public void testSizePartitionCount() throws Throwable + { + insertAndFlushTable(tableName, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + Refs sstables = RepairRangeSplitter.getSSTableReaderRefs(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, FULL_RANGE); + assertEquals(10, sstables.iterator().next().getEstimatedPartitionSize().count()); + RepairRangeSplitter.SizeEstimate sizes = RepairRangeSplitter.getSizesForRangeOfSSTables(KEYSPACE, tableName, FULL_RANGE, sstables); + assertEquals(10, sizes.partitions); + } + + @Test + public void testSizePartitionCountSplit() throws Throwable + { + int[] values = new int[10000]; + for (int i = 0; i < values.length; i++) + values[i] = i + 1; + insertAndFlushTable(tableName, values); + Iterator> range = AutoRepairUtils.split(FULL_RANGE, 2).iterator(); + Range tokenRange1 = range.next(); + Range tokenRange2 = range.next(); + Assert.assertFalse(range.hasNext()); + + Refs sstables1 = RepairRangeSplitter.getSSTableReaderRefs(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, tokenRange1); + Refs sstables2 = RepairRangeSplitter.getSSTableReaderRefs(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, tokenRange2); + RepairRangeSplitter.SizeEstimate sizes1 = RepairRangeSplitter.getSizesForRangeOfSSTables(KEYSPACE, tableName, tokenRange1, sstables1); + RepairRangeSplitter.SizeEstimate sizes2 = RepairRangeSplitter.getSizesForRangeOfSSTables(KEYSPACE, tableName, tokenRange2, sstables2); + // +-5% because HLL merge and the applying of range size approx ratio causes estimation errors + assertTrue(Math.abs(10000 - (sizes1.partitions + sizes2.partitions)) <= 60); + } + @Test public void testReorderByPriorityWithDifferentPriorities() { String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); @@ -69,9 +101,9 @@ public void testReorderByPriorityWithDifferentPriorities() { String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); // Test reordering assignments with different priorities - RepairAssignment assignment1 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table1)); - RepairAssignment assignment2 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table2)); - RepairAssignment assignment3 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table3)); + RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table1)); + RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table2)); + RepairAssignment assignment3 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table3)); // Assume these priorities based on the repair type List assignments = new ArrayList<>(Arrays.asList(assignment1, assignment2, assignment3)); @@ -91,9 +123,9 @@ public void testReorderByPriorityWithSamePriority() { String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); // Test reordering assignments with the same priority - RepairAssignment assignment1 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table1)); - RepairAssignment assignment2 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table2)); - RepairAssignment assignment3 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table3)); + RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table1)); + RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table2)); + RepairAssignment assignment3 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table3)); List assignments = new ArrayList<>(Arrays.asList(assignment1, assignment2, assignment3)); @@ -113,10 +145,10 @@ public void testReorderByPriorityWithMixedPriorities() { String table4 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); // Test reordering assignments with mixed priorities - RepairAssignment assignment1 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table1)); - RepairAssignment assignment2 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table2)); - RepairAssignment assignment3 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table3)); - RepairAssignment assignment4 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table4)); + RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table1)); + RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table2)); + RepairAssignment assignment3 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table3)); + RepairAssignment assignment4 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table4)); List assignments = new ArrayList<>(Arrays.asList(assignment1, assignment2, assignment3, assignment4)); @@ -142,7 +174,7 @@ public void testReorderByPriorityWithOneElement() { // Test with a single element (should remain unchanged) String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '5'}"); - RepairAssignment assignment1 = new RepairAssignment(fullRange, KEYSPACE, Collections.singletonList(table1)); + RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table1)); List assignments = new ArrayList<>(Collections.singletonList(assignment1)); @@ -169,7 +201,7 @@ public void testGetRepairAssignmentsForTable_Single() throws Throwable { @Test public void testGetRepairAssignmentsForTable_BatchingTables() throws Throwable { repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "2")); - Collection> ranges = Collections.singleton(fullRange); + Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(3); List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); @@ -183,7 +215,7 @@ public void testGetRepairAssignmentsForTable_BatchingTables() throws Throwable { @Test public void testGetRepairAssignmentsForTable_BatchSize() throws Throwable { repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "2")); - Collection> ranges = Collections.singleton(fullRange); + Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(2); List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); @@ -196,7 +228,7 @@ public void testGetRepairAssignmentsForTable_BatchSize() throws Throwable { @Test public void testGetRepairAssignmentsForTable_NoBatching() throws Throwable { repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "1")); - Collection> ranges = Collections.singleton(fullRange); + Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(3); List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); @@ -207,7 +239,7 @@ public void testGetRepairAssignmentsForTable_NoBatching() throws Throwable { @Test public void testGetRepairAssignmentsForTable_AllBatched() throws Throwable { repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "100")); - Collection> ranges = Collections.singleton(fullRange); + Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(5); List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); @@ -215,8 +247,6 @@ public void testGetRepairAssignmentsForTable_AllBatched() throws Throwable { assertEquals(1, assignments.size()); } - - @Test(expected = IllegalStateException.class) public void testMergeEmptyAssignments() { // Test when the list of assignments is empty @@ -230,12 +260,12 @@ public void testMergeSingleAssignment() { String keyspaceName = "testKeyspace"; List tableNames = Arrays.asList("table1", "table2"); - RepairAssignment assignment = new RepairAssignment(fullRange, keyspaceName, tableNames); + RepairAssignment assignment = new RepairAssignment(FULL_RANGE, keyspaceName, tableNames); List assignments = Collections.singletonList(assignment); RepairAssignment result = RepairRangeSplitter.merge(assignments); - assertEquals(fullRange, result.getTokenRange()); + assertEquals(FULL_RANGE, result.getTokenRange()); assertEquals(keyspaceName, result.getKeyspaceName()); assertEquals(new HashSet<>(tableNames), new HashSet<>(result.getTableNames())); } @@ -247,13 +277,13 @@ public void testMergeMultipleAssignmentsWithSameTokenRangeAndKeyspace() { List tableNames1 = Arrays.asList("table1", "table2"); List tableNames2 = Arrays.asList("table2", "table3"); - RepairAssignment assignment1 = new RepairAssignment(fullRange, keyspaceName, tableNames1); - RepairAssignment assignment2 = new RepairAssignment(fullRange, keyspaceName, tableNames2); + RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, keyspaceName, tableNames1); + RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, keyspaceName, tableNames2); List assignments = Arrays.asList(assignment1, assignment2); RepairAssignment result = RepairRangeSplitter.merge(assignments); - assertEquals(fullRange, result.getTokenRange()); + assertEquals(FULL_RANGE, result.getTokenRange()); assertEquals(keyspaceName, result.getKeyspaceName()); assertEquals(new HashSet<>(Arrays.asList("table1", "table2", "table3")), new HashSet<>(result.getTableNames())); } @@ -261,7 +291,7 @@ public void testMergeMultipleAssignmentsWithSameTokenRangeAndKeyspace() { @Test(expected = IllegalStateException.class) public void testMergeDifferentTokenRange() { // Test merging assignments with different token ranges - Iterator> range = AutoRepairUtils.split(fullRange, 2).iterator(); // Split the full range into two ranges ie (0-100, 100-200 + Iterator> range = AutoRepairUtils.split(FULL_RANGE, 2).iterator(); // Split the full range into two ranges ie (0-100, 100-200 Range tokenRange1 = range.next(); Range tokenRange2 = range.next(); Assert.assertFalse(range.hasNext()); @@ -281,8 +311,8 @@ public void testMergeDifferentKeyspaceName() { // Test merging assignments with different keyspace names List tableNames = Arrays.asList("table1", "table2"); - RepairAssignment assignment1 = new RepairAssignment(fullRange, "keyspace1", tableNames); - RepairAssignment assignment2 = new RepairAssignment(fullRange, "keyspace2", tableNames); + RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, "keyspace1", tableNames); + RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, "keyspace2", tableNames); List assignments = Arrays.asList(assignment1, assignment2); RepairRangeSplitter.merge(assignments); // Should throw IllegalStateException @@ -295,8 +325,8 @@ public void testMergeWithDuplicateTables() { List tableNames1 = Arrays.asList("table1", "table2"); List tableNames2 = Arrays.asList("table2", "table3"); - RepairAssignment assignment1 = new RepairAssignment(fullRange, keyspaceName, tableNames1); - RepairAssignment assignment2 = new RepairAssignment(fullRange, keyspaceName, tableNames2); + RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, keyspaceName, tableNames1); + RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, keyspaceName, tableNames2); List assignments = Arrays.asList(assignment1, assignment2); RepairAssignment result = RepairRangeSplitter.merge(assignments); @@ -322,9 +352,15 @@ private List createAndInsertTables(int count) throws Throwable { } private void insertAndFlushTable(String tableName) throws Throwable { - executeFormattedQuery("INSERT INTO " + KEYSPACE + '.' + tableName + " (k, v) values (?, ?)", 1, 1); + insertAndFlushTable(tableName, 1); + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, tableName); + } + private void insertAndFlushTable(String tableName, int... vals) throws Throwable { + for (int i : vals) + { + executeFormattedQuery("INSERT INTO " + KEYSPACE + '.' + tableName + " (k, v) values (?, ?)", i, i); + } ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, tableName); cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); - assertEquals(1, cfs.getLiveSSTables().size()); } } From eb083105918913a1786e3164c1af679fdc8ef21c Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sun, 10 Nov 2024 14:29:56 -0600 Subject: [PATCH 074/115] Add license header to RepairRangeSplitter --- .../repair/autorepair/RepairRangeSplitter.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index ffb104a53686..8b0f2f8e3dd9 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -1,3 +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. + */ + package org.apache.cassandra.repair.autorepair; import java.io.IOException; From 0c14b850e0e85f2b041f400dd808ea61e87d91d1 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sun, 10 Nov 2024 15:19:24 -0600 Subject: [PATCH 075/115] Simplify access of repair_type_overrides, key by String A lot of care existed to ensure each RepairType was populated in repair_type_overrides (using ensureOverrides). This could be simplified quite a bit by protecting access to repair_type_overrides by enclosing it in a getOptions method that ensures the provided RepairType has a populated entry. Additionally, change the key type to String and use ConcurrentMap; otherwise the change to make the enum names all caps does not work. --- .../repair/autorepair/AutoRepairConfig.java | 126 ++++++++++-------- .../autorepair/AutoRepairConfigTest.java | 40 +++--- 2 files changed, 85 insertions(+), 81 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index 4a06d78c6eb6..b84ee5f2f652 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -19,17 +19,16 @@ package org.apache.cassandra.repair.autorepair; import java.io.Serializable; -import java.util.ArrayList; import java.util.Collections; -import java.util.EnumMap; import java.util.HashSet; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; import java.util.Set; +import java.util.concurrent.ConcurrentMap; import java.util.function.Function; +import javax.annotation.Nonnull; + import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Maps; import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.config.ParameterizedClass; @@ -62,6 +61,27 @@ public enum RepairType implements Serializable INCREMENTAL, PREVIEW_REPAIRED; + private final String configName; + + RepairType() + { + this.configName = name().toLowerCase(); + } + + RepairType(String configName) + { + this.configName = configName; + } + + /** + * @return Format of the repair type as it should be represented in configuration. + * Canonically this is the enum name in lowerCase. + */ + public String getConfigName() + { + return configName; + } + public static AutoRepairState getAutoRepairState(RepairType repairType) { switch (repairType) @@ -78,8 +98,9 @@ public static AutoRepairState getAutoRepairState(RepairType repairType) } } - // repair_type_overrides overrides the global_settings for a specific repair type - public volatile Map repair_type_overrides = new EnumMap<>(RepairType.class); + // repair_type_overrides overrides the global_settings for a specific repair type. String used as key instead + // of enum to allow lower case key in yaml. + public volatile ConcurrentMap repair_type_overrides = Maps.newConcurrentMap(); public AutoRepairConfig() { @@ -90,10 +111,6 @@ public AutoRepairConfig(boolean enabled) { this.enabled = enabled; global_settings = Options.getDefaultOptions(); - for (RepairType type : RepairType.values()) - { - repair_type_overrides.put(type, new Options()); - } } public DurationSpec.IntSecondsBound getRepairCheckInterval() @@ -149,14 +166,12 @@ public boolean isAutoRepairEnabled(RepairType repairType) public void setAutoRepairEnabled(RepairType repairType, boolean enabled) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).enabled = enabled; + getOptions(repairType).enabled = enabled; } public void setRepairByKeyspace(RepairType repairType, boolean repairByKeyspace) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).repair_by_keyspace = repairByKeyspace; + getOptions(repairType).repair_by_keyspace = repairByKeyspace; } public boolean getRepairByKeyspace(RepairType repairType) @@ -171,8 +186,7 @@ public int getRepairThreads(RepairType repairType) public void setRepairThreads(RepairType repairType, int repairThreads) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).number_of_repair_threads = repairThreads; + getOptions(repairType).number_of_repair_threads = repairThreads; } public int getRepairSubRangeNum(RepairType repairType) @@ -182,8 +196,7 @@ public int getRepairSubRangeNum(RepairType repairType) public void setRepairSubRangeNum(RepairType repairType, int repairSubRanges) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).number_of_subranges = repairSubRanges; + getOptions(repairType).number_of_subranges = repairSubRanges; } public DurationSpec.IntSecondsBound getRepairMinInterval(RepairType repairType) @@ -193,8 +206,7 @@ public DurationSpec.IntSecondsBound getRepairMinInterval(RepairType repairType) public void setRepairMinInterval(RepairType repairType, String minRepairInterval) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).min_repair_interval = new DurationSpec.IntSecondsBound(minRepairInterval); + getOptions(repairType).min_repair_interval = new DurationSpec.IntSecondsBound(minRepairInterval); } public int getRepairSSTableCountHigherThreshold(RepairType repairType) @@ -204,8 +216,7 @@ public int getRepairSSTableCountHigherThreshold(RepairType repairType) public void setRepairSSTableCountHigherThreshold(RepairType repairType, int sstableHigherThreshold) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).sstable_upper_threshold = sstableHigherThreshold; + getOptions(repairType).sstable_upper_threshold = sstableHigherThreshold; } public DurationSpec.IntSecondsBound getAutoRepairTableMaxRepairTime(RepairType repairType) @@ -215,8 +226,7 @@ public DurationSpec.IntSecondsBound getAutoRepairTableMaxRepairTime(RepairType r public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).table_max_repair_time = new DurationSpec.IntSecondsBound(autoRepairTableMaxRepairTime); + getOptions(repairType).table_max_repair_time = new DurationSpec.IntSecondsBound(autoRepairTableMaxRepairTime); } public Set getIgnoreDCs(RepairType repairType) @@ -226,8 +236,7 @@ public Set getIgnoreDCs(RepairType repairType) public void setIgnoreDCs(RepairType repairType, Set ignoreDCs) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).ignore_dcs = ignoreDCs; + getOptions(repairType).ignore_dcs = ignoreDCs; } public boolean getRepairPrimaryTokenRangeOnly(RepairType repairType) @@ -237,8 +246,7 @@ public boolean getRepairPrimaryTokenRangeOnly(RepairType repairType) public void setRepairPrimaryTokenRangeOnly(RepairType repairType, boolean primaryTokenRangeOnly) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).repair_primary_token_range_only = primaryTokenRangeOnly; + getOptions(repairType).repair_primary_token_range_only = primaryTokenRangeOnly; } public int getParallelRepairPercentage(RepairType repairType) @@ -248,8 +256,7 @@ public int getParallelRepairPercentage(RepairType repairType) public void setParallelRepairPercentage(RepairType repairType, int percentage) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).parallel_repair_percentage = percentage; + getOptions(repairType).parallel_repair_percentage = percentage; } public int getParallelRepairCount(RepairType repairType) @@ -259,8 +266,7 @@ public int getParallelRepairCount(RepairType repairType) public void setParallelRepairCount(RepairType repairType, int count) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).parallel_repair_count = count; + getOptions(repairType).parallel_repair_count = count; } public boolean getMVRepairEnabled(RepairType repairType) @@ -270,14 +276,12 @@ public boolean getMVRepairEnabled(RepairType repairType) public void setMVRepairEnabled(RepairType repairType, boolean enabled) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).mv_repair_enabled = enabled; + getOptions(repairType).mv_repair_enabled = enabled; } public void setForceRepairNewNode(RepairType repairType, boolean forceRepairNewNode) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).force_repair_new_node = forceRepairNewNode; + getOptions(repairType).force_repair_new_node = forceRepairNewNode; } public boolean getForceRepairNewNode(RepairType repairType) @@ -292,8 +296,7 @@ public ParameterizedClass getTokenRangeSplitter(RepairType repairType) public void setInitialSchedulerDelay(RepairType repairType, String initialSchedulerDelay) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).initial_scheduler_delay = new DurationSpec.IntSecondsBound(initialSchedulerDelay); + getOptions(repairType).initial_scheduler_delay = new DurationSpec.IntSecondsBound(initialSchedulerDelay); } public DurationSpec.IntSecondsBound getInitialSchedulerDelay(RepairType repairType) @@ -308,8 +311,7 @@ public DurationSpec.IntSecondsBound getRepairSessionTimeout(RepairType repairTyp public void setRepairSessionTimeout(RepairType repairType, String repairSessionTimeout) { - ensureOverrides(repairType); - repair_type_overrides.get(repairType).repair_session_timeout = new DurationSpec.IntSecondsBound(repairSessionTimeout); + getOptions(repairType).repair_session_timeout = new DurationSpec.IntSecondsBound(repairSessionTimeout); } // Options configures auto-repair behavior for a given repair type. @@ -438,31 +440,37 @@ public String toString() } } - @VisibleForTesting - protected T applyOverrides(RepairType repairType, Function optionSupplier) + @Nonnull + protected Options getOptions(RepairType repairType) { - ArrayList optsProviders = new ArrayList<>(); - if (repair_type_overrides != null) - { - optsProviders.add(repair_type_overrides.get(repairType)); - } - optsProviders.add(global_settings); - optsProviders.add(Options.defaultOptions); + return repair_type_overrides.computeIfAbsent(repairType.getConfigName(), k -> new Options()); + } - return optsProviders.stream() - .map(opt -> Optional.ofNullable(opt).map(optionSupplier).orElse(null)) - .filter(Objects::nonNull) - .findFirst() - .orElse(null); + private static T getOverride(Options options, Function optionSupplier) + { + return options != null ? optionSupplier.apply(options) : null; } - protected void ensureOverrides(RepairType repairType) + @VisibleForTesting + protected T applyOverrides(RepairType repairType, Function optionSupplier) { - if (repair_type_overrides == null) + // Check option by repair type first + Options repairTypeOverrides = getOptions(repairType); + T val = optionSupplier.apply(repairTypeOverrides); + + if (val != null) + return val; + + // Check option in global settings + if (global_settings != null) { - repair_type_overrides = new EnumMap<>(RepairType.class); + val = getOverride(global_settings, optionSupplier); + + if (val != null) + return val; } - repair_type_overrides.computeIfAbsent(repairType, k -> new Options()); + // Otherwise check defaults + return getOverride(Options.defaultOptions, optionSupplier); } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java index 4f1dd029ef5e..08ae9631a192 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -18,7 +18,6 @@ package org.apache.cassandra.repair.autorepair; -import java.util.EnumMap; import java.util.Objects; import java.util.Collections; import java.util.Set; @@ -63,7 +62,6 @@ public static Object[] repairTypes() public void setUp() { config = new AutoRepairConfig(true); - config.repair_type_overrides = null; AutoRepair.SLEEP_IF_REPAIR_FINISHES_QUICKLY = new DurationSpec.IntSecondsBound("0s"); } @@ -80,7 +78,7 @@ public void autoRepairConfigRepairTypesAreNotNull() AutoRepairConfig config = new AutoRepairConfig(); for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { - assertNotNull(config.repair_type_overrides.get(repairType)); + assertNotNull(config.getOptions(repairType)); } } @@ -105,10 +103,8 @@ public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledGlobally() public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledForRepairType() { config.global_settings.enabled = true; - config.repair_type_overrides = new EnumMap<>(AutoRepairConfig.RepairType.class); - config.repair_type_overrides.put(repairType, new Options()); - config.repair_type_overrides.get(repairType).enabled = false; - assertFalse(config.isAutoRepairEnabled(repairType)); + config.setAutoRepairEnabled(repairType, false); + assertFalse(config.getOptions(repairType).enabled); } @Test @@ -118,7 +114,7 @@ public void testSetAutoRepairEnabledNoMVOrCDC() DatabaseDescriptor.setMaterializedViewsEnabled(false); config.setAutoRepairEnabled(repairType, true); - assertTrue(config.repair_type_overrides.get(repairType).enabled); + assertTrue(config.getOptions(repairType).enabled); } @Test @@ -126,7 +122,7 @@ public void testSetRepairByKeyspace() { config.setRepairByKeyspace(repairType, true); - assertTrue(config.repair_type_overrides.get(repairType).repair_by_keyspace); + assertTrue(config.getOptions(repairType).repair_by_keyspace); } @Test @@ -144,7 +140,7 @@ public void testSetRepairThreads() { config.setRepairThreads(repairType, 5); - assert config.repair_type_overrides.get(repairType).number_of_repair_threads == 5; + assert config.getOptions(repairType).number_of_repair_threads == 5; } @Test @@ -172,7 +168,7 @@ public void testSetRepairSubRangeNum() { config.setRepairSubRangeNum(repairType, 5); - assert config.repair_type_overrides.get(repairType).number_of_subranges == 5; + assert config.getOptions(repairType).number_of_subranges == 5; } @Test @@ -190,7 +186,7 @@ public void testSetRepairMinFrequencyInHours() { config.setRepairMinInterval(repairType, "5s"); - assert config.repair_type_overrides.get(repairType).min_repair_interval.toSeconds() == 5; + assert config.getOptions(repairType).min_repair_interval.toSeconds() == 5; } @Test @@ -226,7 +222,7 @@ public void testSetRepairSSTableCountHigherThreshold() { config.setRepairSSTableCountHigherThreshold(repairType, 5); - assert config.repair_type_overrides.get(repairType).sstable_upper_threshold == 5; + assert config.getOptions(repairType).sstable_upper_threshold == 5; } @Test @@ -244,7 +240,7 @@ public void testSetAutoRepairTableMaxRepairTimeInSec() { config.setAutoRepairTableMaxRepairTime(repairType, "5s"); - assert config.repair_type_overrides.get(repairType).table_max_repair_time.toSeconds() == 5; + assert config.getOptions(repairType).table_max_repair_time.toSeconds() == 5; } @Test @@ -262,7 +258,7 @@ public void testSetIgnoreDCs() { config.setIgnoreDCs(repairType, testSet); - assertEquals(config.repair_type_overrides.get(repairType).ignore_dcs, testSet); + assertEquals(config.getOptions(repairType).ignore_dcs, testSet); } @Test @@ -280,7 +276,7 @@ public void testSetRepairPrimaryTokenRangeOnly() { config.setRepairPrimaryTokenRangeOnly(repairType, true); - assertTrue(config.repair_type_overrides.get(repairType).repair_primary_token_range_only); + assertTrue(config.getOptions(repairType).repair_primary_token_range_only); } @Test @@ -298,7 +294,7 @@ public void testSetParallelRepairPercentageInGroup() { config.setParallelRepairPercentage(repairType, 5); - assert config.repair_type_overrides.get(repairType).parallel_repair_percentage == 5; + assert config.getOptions(repairType).parallel_repair_percentage == 5; } @Test @@ -316,7 +312,7 @@ public void testSetParallelRepairCountInGroup() { config.setParallelRepairCount(repairType, 5); - assert config.repair_type_overrides.get(repairType).parallel_repair_count == 5; + assert config.getOptions(repairType).parallel_repair_count == 5; } @Test @@ -334,7 +330,7 @@ public void testSetMVRepairEnabled() { config.setMVRepairEnabled(repairType, true); - assertTrue(config.repair_type_overrides.get(repairType).mv_repair_enabled); + assertTrue(config.getOptions(repairType).mv_repair_enabled); } @Test @@ -342,7 +338,7 @@ public void testSetForceRepairNewNode() { config.setForceRepairNewNode(repairType, true); - assertTrue(config.repair_type_overrides.get(repairType).force_repair_new_node); + assertTrue(config.getOptions(repairType).force_repair_new_node); } @Test @@ -403,7 +399,7 @@ public void testSetInitialSchedulerDelay() { config.setInitialSchedulerDelay(repairType, "5s"); - assert config.repair_type_overrides.get(repairType).initial_scheduler_delay.toSeconds() == 5; + assert config.getOptions(repairType).initial_scheduler_delay.toSeconds() == 5; } @Test @@ -421,7 +417,7 @@ public void testSetRepairSessionTimeout() { config.setRepairSessionTimeout(repairType, "1h"); - assert config.repair_type_overrides.get(repairType).repair_session_timeout.toSeconds() == 3600; + assert config.getOptions(repairType).repair_session_timeout.toSeconds() == 3600; } } From f53ae5cee93fa0a26fcd2e24c13881c901dc1cb1 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sun, 10 Nov 2024 22:39:07 -0600 Subject: [PATCH 076/115] Allow partial repair of range, factor in total bytes for IR * Rename maxBytesPerScheduleBytes to maxBytesPerSchedule * Allow partial repair of range if the smallest subrange we can repair is less than the remaining bytes we can repair. * Consider totalSize for IR (because of anticompaction) sizeinRange for full repair. * Increased logging while debugging. --- .../autorepair/RepairRangeSplitter.java | 104 ++++++++++++++---- .../autorepair/RepairRangeSplitterTest.java | 6 +- 2 files changed, 83 insertions(+), 27 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index 8b0f2f8e3dd9..a6b138a85996 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -51,6 +50,7 @@ import org.apache.cassandra.io.sstable.format.big.BigTableScanner; import org.apache.cassandra.io.sstable.metadata.CompactionMetadata; import org.apache.cassandra.io.sstable.metadata.MetadataType; +import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.concurrent.Refs; @@ -90,7 +90,7 @@ public class RepairRangeSplitter implements IAutoRepairTokenRangeSplitter static final String MAX_BYTES_PER_SCHEDULE = "max_bytes_per_schedule"; private final int tablesPerAssignmentLimit; - private final long maxBytesPerScheduleBytes; + private final long maxBytesPerSchedule; private final long bytesPerSubrange; private final long partitionsPerSubrange; @@ -115,11 +115,11 @@ public RepairRangeSplitter(Map parameters) if (parameters.containsKey(MAX_BYTES_PER_SCHEDULE)) { - maxBytesPerScheduleBytes = new DataStorageSpec.LongBytesBound(parameters.get(MAX_BYTES_PER_SCHEDULE)).toBytes(); + maxBytesPerSchedule = new DataStorageSpec.LongBytesBound(parameters.get(MAX_BYTES_PER_SCHEDULE)).toBytes(); } else { - maxBytesPerScheduleBytes = DEFAULT_MAX_BYTES_PER_SCHEDULE; + maxBytesPerSchedule = DEFAULT_MAX_BYTES_PER_SCHEDULE; } if (parameters.containsKey(PARTITION_COUNT)) @@ -141,7 +141,7 @@ public RepairRangeSplitter(Map parameters) } logger.info("Configured {} with {}={}, {}={}, {}={}, {}={}", RepairRangeSplitter.class.getName(), - SUBRANGE_SIZE, subrangeSize, MAX_BYTES_PER_SCHEDULE, maxBytesPerScheduleBytes, + SUBRANGE_SIZE, subrangeSize, MAX_BYTES_PER_SCHEDULE, maxBytesPerSchedule, PARTITION_COUNT, partitionsPerSubrange, TABLE_BATCH_LIMIT, tablesPerAssignmentLimit); } @@ -234,7 +234,10 @@ public List getRepairAssignmentsForTable(AutoRepairConfig.Repa Collections.shuffle(sizeEstimates); for (SizeEstimate estimate : sizeEstimates) { - if (estimate.sizeInRange == 0) + // Calculate the smallest subrange if we were to split, this can be used to determine if we can do any + // work if we have a non-zero estimate. + long smallestSubrange = Math.min(estimate.sizeForRepair, bytesPerSubrange); + if (estimate.sizeForRepair == 0) { ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); long memtableSize = cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize(); @@ -249,31 +252,40 @@ public List getRepairAssignmentsForTable(AutoRepairConfig.Repa logger.debug("Skipping {}.{} for range {} because it had no unrepaired SSTables and no memtable data", keyspaceName, tableName, estimate.tokenRange); } } - else if (targetBytesSoFar + estimate.sizeInRange < maxBytesPerScheduleBytes) + else if (targetBytesSoFar + smallestSubrange < maxBytesPerSchedule) { - targetBytesSoFar += estimate.sizeInRange; // Check if the estimate needs splitting based on the criteria - boolean needsSplitting = estimate.sizeInRange > bytesPerSubrange || estimate.partitions > partitionsPerSubrange; + boolean needsSplitting = estimate.sizeForRepair > bytesPerSubrange || estimate.partitions > partitionsPerSubrange; if (needsSplitting) { - // Calculate the number of splits needed for size and partitions - int splitsForSize = (int) Math.ceil((double) estimate.sizeInRange / bytesPerSubrange); - int splitsForPartitions = (int) Math.ceil((double) estimate.partitions / partitionsPerSubrange); - - // Choose the larger of the two as the number of splits - int numberOfSplits = Math.max(splitsForSize, splitsForPartitions); - - // Split the token range into subranges + int numberOfSplits = calculateNumberOfSplits(estimate); + long approximateBytesPerSplit = estimate.sizeForRepair / numberOfSplits; Collection> subranges = split(estimate.tokenRange, numberOfSplits); for (Range subrange : subranges) { + if (targetBytesSoFar + approximateBytesPerSplit > maxBytesPerSchedule) + { + logger.warn("Refusing to add repair assignment for {}.{} for subrange {} with a approximateBytesPerSplit={} because it would increase total repair bytes to {} which is greater than {}={}", + keyspaceName, tableName, subrange, FileUtils.stringifyFileSize(approximateBytesPerSplit), FileUtils.stringifyFileSize(targetBytesSoFar + approximateBytesPerSplit), + MAX_BYTES_PER_SCHEDULE, FileUtils.stringifyFileSize(maxBytesPerSchedule)); + break; + } + logger.info("Added repair assignment for {}.{} for subrange {} (#{}/{})", + keyspaceName, tableName, subrange, repairAssignments.size() + 1, numberOfSplits); RepairAssignment assignment = new RepairAssignment(subrange, keyspaceName, Collections.singletonList(tableName)); repairAssignments.add(assignment); + targetBytesSoFar += approximateBytesPerSplit; } } else { + logger.info("Using 1 repair assignment for {}.{} for range {} as rangeBytes={} is less than {}={} and partitionEstimate={} is less than {}={}", + keyspaceName, tableName, estimate.tokenRange, + FileUtils.stringifyFileSize(estimate.sizeForRepair), + SUBRANGE_SIZE, FileUtils.stringifyFileSize(bytesPerSubrange), + estimate.partitions, + PARTITION_COUNT, partitionsPerSubrange); // No splitting needed, repair the entire range as-is RepairAssignment assignment = new RepairAssignment(estimate.tokenRange, keyspaceName, Collections.singletonList(tableName)); repairAssignments.add(assignment); @@ -281,13 +293,45 @@ else if (targetBytesSoFar + estimate.sizeInRange < maxBytesPerScheduleBytes) } else { + long calculatedSize = targetBytesSoFar + estimate.sizeForRepair; // Really this is "Ok" but it does mean we are relying on randomness to cover the other ranges - logger.info("Skipping range {} for {}.{} as it would exceed maxBytesPerScheduleBytes, consider increasing maxBytesPerScheduleBytes, reducing node density or monitoring to ensure all ranges do get repaired within gc_grace_seconds", estimate.tokenRange, keyspaceName, tableName); + logger.info("Skipping range {} for {}.{} as {} would exceed {}={}, " + + "consider increasing {}, reducing node denity or monitoring to ensure all ranges do get repaired within gc_grace_seconds", + estimate.tokenRange, + keyspaceName, tableName, + FileUtils.stringifyFileSize(calculatedSize), + MAX_BYTES_PER_SCHEDULE, FileUtils.stringifyFileSize(calculatedSize), MAX_BYTES_PER_SCHEDULE); } } return repairAssignments; } + private int calculateNumberOfSplits(SizeEstimate estimate) + { + // Calculate the number of splits needed for size and partitions + int splitsForSize = (int) Math.ceil((double) estimate.sizeForRepair / bytesPerSubrange); + int splitsForPartitions = (int) Math.ceil((double) estimate.partitions / partitionsPerSubrange); + + // Split the token range into subranges based on whichever (partitions, bytes) would generate the most splits. + boolean splitBySize = splitsForSize > splitsForPartitions; + int splits = splitBySize ? splitsForSize : splitsForPartitions; + + // calculate approximation for logging purposes + long approximateBytesPerSplit = estimate.sizeForRepair / splits; + long approximatePartitionsPerSplit = estimate.partitions / splits; + + logger.info("Splitting {}.{} for range {} into {} sub ranges by {} (splitsForSize={}, splitsForPartitions={}, " + + "approximateBytesInRange={}, approximatePartitionsInRange={}, " + + "approximateBytesPerSplit={}, approximatePartitionsPerSplit={})", + estimate.keyspace, estimate.table, estimate.tokenRange, + splits, splitBySize ? "size" : "partitions", + splitsForSize, splitsForPartitions, + FileUtils.stringifyFileSize(estimate.sizeForRepair), estimate.partitions, + FileUtils.stringifyFileSize(approximateBytesPerSplit), approximatePartitionsPerSplit + ); + return splits; + } + public Collection> getTokenRanges(boolean primaryRangeOnly, String keyspaceName) { // Collect all applicable token ranges @@ -318,7 +362,7 @@ private List getRangeSizeEstimate(AutoRepairConfig.RepairType repa logger.debug("Calculating size estimate for {}.{} for range {}", keyspace, table, tokenRange); try (Refs refs = getSSTableReaderRefs(repairType, keyspace, table, tokenRange)) { - SizeEstimate estimate = getSizesForRangeOfSSTables(keyspace, table, tokenRange, refs); + SizeEstimate estimate = getSizesForRangeOfSSTables(repairType, keyspace, table, tokenRange, refs); logger.debug("Size estimate for {}.{} for range {} is {}", keyspace, table, tokenRange, estimate); sizeEstimates.add(estimate); } @@ -327,7 +371,7 @@ private List getRangeSizeEstimate(AutoRepairConfig.RepairType repa } @VisibleForTesting - static SizeEstimate getSizesForRangeOfSSTables(String keyspace, String table, Range tokenRange, Refs refs) + static SizeEstimate getSizesForRangeOfSSTables(AutoRepairConfig.RepairType repairType, String keyspace, String table, Range tokenRange, Refs refs) { ICardinality cardinality = new HyperLogLogPlus(13, 25); long approxBytesInRange = 0L; @@ -374,7 +418,7 @@ static SizeEstimate getSizesForRangeOfSSTables(String keyspace, String table, Ra double ratio = approxBytesInRange / (double) totalBytes; // use the ratio from size to estimate the partitions in the range as well long partitions = (long) Math.max(1, Math.ceil(cardinality.cardinality() * ratio)); - return new SizeEstimate(keyspace, table, tokenRange, partitions, approxBytesInRange, totalBytes); + return new SizeEstimate(repairType, keyspace, table, tokenRange, partitions, approxBytesInRange, totalBytes); } @VisibleForTesting @@ -397,7 +441,7 @@ static Refs getSSTableReaderRefs(AutoRepairConfig.RepairType repa List canonicalSSTables = View.sstablesInBounds(r.left, r.right, tree); if (repairType == AutoRepairConfig.RepairType.INCREMENTAL) { - canonicalSSTables = canonicalSSTables.stream().filter(SSTableReader::isRepaired).collect(Collectors.toList()); + canonicalSSTables = canonicalSSTables.stream().filter((sstable) -> !sstable.isRepaired()).collect(Collectors.toList()); } refs = Refs.tryRef(canonicalSSTables); } @@ -407,21 +451,33 @@ static Refs getSSTableReaderRefs(AutoRepairConfig.RepairType repa @VisibleForTesting protected static class SizeEstimate { + public final AutoRepairConfig.RepairType repairType; public final String keyspace; public final String table; public final Range tokenRange; public final long partitions; public final long sizeInRange; public final long totalSize; - - public SizeEstimate(String keyspace, String table, Range tokenRange, long partitions, long sizeInRange, long totalSize) + /** + * Size to consider in the repair. For incremental repair, we want to consider the total size + * of the estimate as we have to factor in anticompacting the entire SSTable. + * For full repair, just use the size containing the range. + */ + public final long sizeForRepair; + + public SizeEstimate(AutoRepairConfig.RepairType repairType, + String keyspace, String table, Range tokenRange, + long partitions, long sizeInRange, long totalSize) { + this.repairType = repairType; this.keyspace = keyspace; this.table = table; this.tokenRange = tokenRange; this.partitions = partitions; this.sizeInRange = sizeInRange; this.totalSize = totalSize; + + this.sizeForRepair = repairType == AutoRepairConfig.RepairType.INCREMENTAL ? totalSize : sizeInRange; } } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java index aa81e75300cf..71b60a92232e 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java @@ -70,7 +70,7 @@ public void testSizePartitionCount() throws Throwable insertAndFlushTable(tableName, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); Refs sstables = RepairRangeSplitter.getSSTableReaderRefs(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, FULL_RANGE); assertEquals(10, sstables.iterator().next().getEstimatedPartitionSize().count()); - RepairRangeSplitter.SizeEstimate sizes = RepairRangeSplitter.getSizesForRangeOfSSTables(KEYSPACE, tableName, FULL_RANGE, sstables); + RepairRangeSplitter.SizeEstimate sizes = RepairRangeSplitter.getSizesForRangeOfSSTables(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, FULL_RANGE, sstables); assertEquals(10, sizes.partitions); } @@ -88,8 +88,8 @@ public void testSizePartitionCountSplit() throws Throwable Refs sstables1 = RepairRangeSplitter.getSSTableReaderRefs(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, tokenRange1); Refs sstables2 = RepairRangeSplitter.getSSTableReaderRefs(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, tokenRange2); - RepairRangeSplitter.SizeEstimate sizes1 = RepairRangeSplitter.getSizesForRangeOfSSTables(KEYSPACE, tableName, tokenRange1, sstables1); - RepairRangeSplitter.SizeEstimate sizes2 = RepairRangeSplitter.getSizesForRangeOfSSTables(KEYSPACE, tableName, tokenRange2, sstables2); + RepairRangeSplitter.SizeEstimate sizes1 = RepairRangeSplitter.getSizesForRangeOfSSTables(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, tokenRange1, sstables1); + RepairRangeSplitter.SizeEstimate sizes2 = RepairRangeSplitter.getSizesForRangeOfSSTables(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, tokenRange2, sstables2); // +-5% because HLL merge and the applying of range size approx ratio causes estimation errors assertTrue(Math.abs(10000 - (sizes1.partitions + sizes2.partitions)) <= 60); } From 3a8643b13a4aaf5e676cf392b2fd59f075e8eba6 Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Thu, 14 Nov 2024 11:41:51 -0600 Subject: [PATCH 077/115] update warnings based on repair type --- .../autorepair/RepairRangeSplitter.java | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index a6b138a85996..3fbc2239f0c8 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -101,7 +101,6 @@ public class RepairRangeSplitter implements IAutoRepairTokenRangeSplitter public RepairRangeSplitter(Map parameters) { - // Demonstrates parameterizing a range splitter so we can have splitter specific options. DataStorageSpec.LongBytesBound subrangeSize; if (parameters.containsKey(SUBRANGE_SIZE)) { @@ -175,7 +174,7 @@ List getRepairAssignments(AutoRepairConfig.RepairType repairTy if (tableAssignments.isEmpty()) continue; - // If the table assignments are for the same token range and we have room to add more tables to the current assignment + // If the table assignments are for the same token range, and we have room to add more tables to the current assignment if (tableAssignments.size() == 1 && currentAssignments.size() < tablesPerAssignmentLimit && (currentAssignments.isEmpty() || currentAssignments.get(0).getTokenRange().equals(tableAssignments.get(0).getTokenRange()))) { @@ -229,7 +228,7 @@ public List getRepairAssignmentsForTable(AutoRepairConfig.Repa long targetBytesSoFar = 0; List sizeEstimates = getRangeSizeEstimate(repairType, keyspaceName, tableName, tokenRanges); - // since its possible for us to hit maxBytesPerScheduleBytes before seeing all ranges, shuffle so there is chance + // since its possible for us to hit maxBytesPerSchedule before seeing all ranges, shuffle so there is chance // at least of hitting all the ranges _eventually_ for the worst case scenarios Collections.shuffle(sizeEstimates); for (SizeEstimate estimate : sizeEstimates) @@ -266,9 +265,7 @@ else if (targetBytesSoFar + smallestSubrange < maxBytesPerSchedule) { if (targetBytesSoFar + approximateBytesPerSplit > maxBytesPerSchedule) { - logger.warn("Refusing to add repair assignment for {}.{} for subrange {} with a approximateBytesPerSplit={} because it would increase total repair bytes to {} which is greater than {}={}", - keyspaceName, tableName, subrange, FileUtils.stringifyFileSize(approximateBytesPerSplit), FileUtils.stringifyFileSize(targetBytesSoFar + approximateBytesPerSplit), - MAX_BYTES_PER_SCHEDULE, FileUtils.stringifyFileSize(maxBytesPerSchedule)); + warnMaxBytesPerSchedule(repairType, keyspaceName, tableName); break; } logger.info("Added repair assignment for {}.{} for subrange {} (#{}/{})", @@ -293,19 +290,23 @@ else if (targetBytesSoFar + smallestSubrange < maxBytesPerSchedule) } else { - long calculatedSize = targetBytesSoFar + estimate.sizeForRepair; // Really this is "Ok" but it does mean we are relying on randomness to cover the other ranges - logger.info("Skipping range {} for {}.{} as {} would exceed {}={}, " + - "consider increasing {}, reducing node denity or monitoring to ensure all ranges do get repaired within gc_grace_seconds", - estimate.tokenRange, - keyspaceName, tableName, - FileUtils.stringifyFileSize(calculatedSize), - MAX_BYTES_PER_SCHEDULE, FileUtils.stringifyFileSize(calculatedSize), MAX_BYTES_PER_SCHEDULE); + warnMaxBytesPerSchedule(repairType, keyspaceName, tableName); } } return repairAssignments; } + private void warnMaxBytesPerSchedule(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName) + { + String warning = "Refusing to add repair assignment for {}.{} because it would increase total repair bytes greater {}"; + if (repairType == AutoRepairConfig.RepairType.FULL) + { + warning = warning + ", everything will not be repaired this schedule. Consider increasing maxBytesPerSchedule, reducing node density or monitoring to ensure all ranges do get repaired within gc_grace_seconds"; + } + logger.warn(warning, keyspaceName, tableName, FileUtils.stringifyFileSize(maxBytesPerSchedule)); + } + private int calculateNumberOfSplits(SizeEstimate estimate) { // Calculate the number of splits needed for size and partitions From fe799c52215b29a8046a58e47d15dd7cbc13c006 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Tue, 26 Nov 2024 22:27:11 -0600 Subject: [PATCH 078/115] Move size comparison up so priority can be factored in Rather than skipping assignments based on size as they are generated, generate all assignments. This way we can use table priorities as a guide for which assignments to exclude. There is still some more work to do here, as we are still splitting by max bytes, where we should really split by remaining bytes. Some further refactoring can be done here but I think this will put more emphasis on table priority when selecting repair assignments and the size of the repair is greater than max schedule bytes. --- .../IAutoRepairTokenRangeSplitter.java | 2 +- .../autorepair/RepairRangeSplitter.java | 176 ++++++++++++++---- .../autorepair/RepairRangeSplitterTest.java | 39 ++-- 3 files changed, 158 insertions(+), 59 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java index a4760bc3f94f..d43a9f562594 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java @@ -54,7 +54,7 @@ public interface IAutoRepairTokenRangeSplitter * The priority is determined using the {@link RepairAssignment#getPriority(AutoRepairConfig.RepairType)} method. */ @VisibleForTesting - default void reorderByPriority(List repairAssignments, AutoRepairConfig.RepairType repairType) + default void reorderByPriority(List repairAssignments, AutoRepairConfig.RepairType repairType) { repairAssignments.sort(Comparator.comparingInt(a -> ((RepairAssignment) a).getPriority(repairType)).reversed()); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index 3fbc2239f0c8..e4822adcfb13 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -25,6 +25,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -139,6 +140,12 @@ public RepairRangeSplitter(Map parameters) tablesPerAssignmentLimit = DEFAULT_TABLE_BATCH_LIMIT; } + if (bytesPerSubrange > maxBytesPerSchedule) + { + throw new RuntimeException(String.format("bytesPerSubrange '%s' cannot be greater than maxBytesPerSchedule '%s'", + FileUtils.stringifyFileSize(bytesPerSubrange), FileUtils.stringifyFileSize(maxBytesPerSchedule))); + } + logger.info("Configured {} with {}={}, {}={}, {}={}, {}={}", RepairRangeSplitter.class.getName(), SUBRANGE_SIZE, subrangeSize, MAX_BYTES_PER_SCHEDULE, maxBytesPerSchedule, PARTITION_COUNT, partitionsPerSubrange, TABLE_BATCH_LIMIT, tablesPerAssignmentLimit); @@ -155,9 +162,9 @@ public List getRepairAssignments(AutoRepairConfig.RepairType r @VisibleForTesting List getRepairAssignments(AutoRepairConfig.RepairType repairType, String keyspaceName, List tableNames, Collection> tokenRanges) { - List repairAssignments = new ArrayList<>(); + List repairAssignments = new ArrayList<>(); // this is used for batching minimal single assignment tables together - List currentAssignments = new ArrayList<>(); + List currentAssignments = new ArrayList<>(); // sort the tables by size so can batch the smallest ones together tableNames.sort((t1, t2) -> { @@ -167,18 +174,30 @@ List getRepairAssignments(AutoRepairConfig.RepairType repairTy throw new IllegalArgumentException(String.format("Could not resolve ColumnFamilyStore from %s.%s", keyspaceName, t1)); return Long.compare(cfs1.metric.totalDiskSpaceUsed.getCount(), cfs2.metric.totalDiskSpaceUsed.getCount()); }); + for (String tableName : tableNames) { - List tableAssignments = getRepairAssignmentsForTable(repairType, keyspaceName, tableName, tokenRanges); + List tableAssignments = getRepairAssignmentsForTable(repairType, keyspaceName, tableName, tokenRanges); if (tableAssignments.isEmpty()) continue; // If the table assignments are for the same token range, and we have room to add more tables to the current assignment - if (tableAssignments.size() == 1 && currentAssignments.size() < tablesPerAssignmentLimit && + if (tableAssignments.size() == 1 && + currentAssignments.size() < tablesPerAssignmentLimit && (currentAssignments.isEmpty() || currentAssignments.get(0).getTokenRange().equals(tableAssignments.get(0).getTokenRange()))) { - currentAssignments.addAll(tableAssignments); + long currentAssignmentsBytes = getBytesInAssignments(currentAssignments); + long tableAssignmentsBytes = getBytesInAssignments(tableAssignments); + // only add assignments together if they don't exceed max bytes per schedule. + if (currentAssignmentsBytes + tableAssignmentsBytes < maxBytesPerSchedule) { + currentAssignments.addAll(tableAssignments); + } + else + { + // add table assignments by themselves + repairAssignments.addAll(tableAssignments); + } } else { @@ -193,12 +212,55 @@ List getRepairAssignments(AutoRepairConfig.RepairType repairTy if (!currentAssignments.isEmpty()) repairAssignments.add(merge(currentAssignments)); + return filterRepairAssignments(repairType, repairAssignments); + } + + /** + * Given a repair type and list of sized-based repair assignments order them by priority of the + * assignments' underlying tables and confine them by maxBytesPerSchedule. + * @param repairType used to determine underyling table priorities + * @param repairAssignments the assignments to filter. + * @return A list of repair assignments ordered by priority and confined by maxBytesPerSchedule. + */ + protected List filterRepairAssignments(AutoRepairConfig.RepairType repairType, List repairAssignments) + { + // Reorder the repair assignments reorderByPriority(repairAssignments, repairType); - return repairAssignments; + + // Confine repair assignemnts by maxBytesPer Schedule if greater than maxBytesPerSchedule. + long assignmentBytes = getBytesInAssignments(repairAssignments); + if (assignmentBytes < maxBytesPerSchedule) + return repairAssignments.stream().map((a) -> (RepairAssignment)a).collect(Collectors.toList()); + else + { + long bytesSoFar = 0L; + List assignmentsToReturn = new ArrayList<>(repairAssignments.size()); + for (SizedRepairAssignment repairAssignment : repairAssignments) { + // skip any repair assignments that would accumulate us past the maxBytesPerSchedule + if (bytesSoFar + repairAssignment.getEstimatedSizeInBytes() > maxBytesPerSchedule) + { + // log that repair assignment was skipped. + warnMaxBytesPerSchedule(repairType, repairAssignment); + } + else + { + assignmentsToReturn.add(repairAssignment); + bytesSoFar += repairAssignment.getEstimatedSizeInBytes(); + } + } + return assignmentsToReturn; + } + } + + @VisibleForTesting + protected static long getBytesInAssignments(List repairAssignments) { + return repairAssignments + .stream() + .mapToLong(SizedRepairAssignment::getEstimatedSizeInBytes).sum(); } @VisibleForTesting - static RepairAssignment merge(List assignments) + static SizedRepairAssignment merge(List assignments) { if (assignments.isEmpty()) throw new IllegalStateException("Cannot merge empty assignments"); @@ -207,7 +269,7 @@ static RepairAssignment merge(List assignments) Range referenceTokenRange = assignments.get(0).getTokenRange(); String referenceKeyspaceName = assignments.get(0).getKeyspaceName(); - for (RepairAssignment assignment : assignments) + for (SizedRepairAssignment assignment : assignments) { // These checks _should_ be unnecessary but are here to ensure that the assignments are consistent if (!assignment.getTokenRange().equals(referenceTokenRange)) @@ -218,40 +280,43 @@ static RepairAssignment merge(List assignments) mergedTableNames.addAll(assignment.getTableNames()); } - return new RepairAssignment(referenceTokenRange, referenceKeyspaceName, new ArrayList<>(mergedTableNames)); + long sizeForAssignment = getBytesInAssignments(assignments); + return new SizedRepairAssignment(referenceTokenRange, referenceKeyspaceName, new ArrayList<>(mergedTableNames), sizeForAssignment); } - public List getRepairAssignmentsForTable(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName, Collection> tokenRanges) + @VisibleForTesting + protected List getRepairAssignmentsForTable(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName, Collection> tokenRanges) { - List repairAssignments = new ArrayList<>(); + List sizeEstimates = getRangeSizeEstimate(repairType, keyspaceName, tableName, tokenRanges); + return getRepairAssignments(sizeEstimates); + } - long targetBytesSoFar = 0; + @VisibleForTesting + protected List getRepairAssignments(List sizeEstimates) + { + List repairAssignments = new ArrayList<>(); - List sizeEstimates = getRangeSizeEstimate(repairType, keyspaceName, tableName, tokenRanges); // since its possible for us to hit maxBytesPerSchedule before seeing all ranges, shuffle so there is chance // at least of hitting all the ranges _eventually_ for the worst case scenarios Collections.shuffle(sizeEstimates); for (SizeEstimate estimate : sizeEstimates) { - // Calculate the smallest subrange if we were to split, this can be used to determine if we can do any - // work if we have a non-zero estimate. - long smallestSubrange = Math.min(estimate.sizeForRepair, bytesPerSubrange); if (estimate.sizeForRepair == 0) { - ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(estimate.keyspace, estimate.table); long memtableSize = cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize(); if (memtableSize > 0L) { - logger.debug("Included {}.{} range {}, had no unrepaired SSTables, but memtableSize={}, adding single repair assignment", keyspaceName, tableName, estimate.tokenRange, memtableSize); - RepairAssignment assignment = new RepairAssignment(estimate.tokenRange, keyspaceName, Collections.singletonList(tableName)); + logger.debug("Included {}.{} range {}, had no unrepaired SSTables, but memtableSize={}, adding single repair assignment", estimate.keyspace, estimate.table, estimate.tokenRange, memtableSize); + SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, Collections.singletonList(estimate.table), estimate.sizeInRange); repairAssignments.add(assignment); } else { - logger.debug("Skipping {}.{} for range {} because it had no unrepaired SSTables and no memtable data", keyspaceName, tableName, estimate.tokenRange); + logger.debug("Skipping {}.{} for range {} because it had no unrepaired SSTables and no memtable data", estimate.keyspace, estimate.table, estimate.tokenRange); } } - else if (targetBytesSoFar + smallestSubrange < maxBytesPerSchedule) + else { // Check if the estimate needs splitting based on the criteria boolean needsSplitting = estimate.sizeForRepair > bytesPerSubrange || estimate.partitions > partitionsPerSubrange; @@ -263,48 +328,37 @@ else if (targetBytesSoFar + smallestSubrange < maxBytesPerSchedule) Collection> subranges = split(estimate.tokenRange, numberOfSplits); for (Range subrange : subranges) { - if (targetBytesSoFar + approximateBytesPerSplit > maxBytesPerSchedule) - { - warnMaxBytesPerSchedule(repairType, keyspaceName, tableName); - break; - } logger.info("Added repair assignment for {}.{} for subrange {} (#{}/{})", - keyspaceName, tableName, subrange, repairAssignments.size() + 1, numberOfSplits); - RepairAssignment assignment = new RepairAssignment(subrange, keyspaceName, Collections.singletonList(tableName)); + estimate.keyspace, estimate.table, subrange, repairAssignments.size() + 1, numberOfSplits); + SizedRepairAssignment assignment = new SizedRepairAssignment(subrange, estimate.keyspace, Collections.singletonList(estimate.table), approximateBytesPerSplit); repairAssignments.add(assignment); - targetBytesSoFar += approximateBytesPerSplit; } } else { logger.info("Using 1 repair assignment for {}.{} for range {} as rangeBytes={} is less than {}={} and partitionEstimate={} is less than {}={}", - keyspaceName, tableName, estimate.tokenRange, + estimate.keyspace, estimate.table, estimate.tokenRange, FileUtils.stringifyFileSize(estimate.sizeForRepair), SUBRANGE_SIZE, FileUtils.stringifyFileSize(bytesPerSubrange), estimate.partitions, PARTITION_COUNT, partitionsPerSubrange); // No splitting needed, repair the entire range as-is - RepairAssignment assignment = new RepairAssignment(estimate.tokenRange, keyspaceName, Collections.singletonList(tableName)); + SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, Collections.singletonList(estimate.table), estimate.sizeForRepair); repairAssignments.add(assignment); } } - else - { - // Really this is "Ok" but it does mean we are relying on randomness to cover the other ranges - warnMaxBytesPerSchedule(repairType, keyspaceName, tableName); - } } return repairAssignments; } - private void warnMaxBytesPerSchedule(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName) + private void warnMaxBytesPerSchedule(AutoRepairConfig.RepairType repairType, SizedRepairAssignment repairAssignment) { - String warning = "Refusing to add repair assignment for {}.{} because it would increase total repair bytes greater {}"; + String warning = "Refusing to add repair assignment of size {} for {}.{} because it would increase total repair bytes to a value greater than {}"; if (repairType == AutoRepairConfig.RepairType.FULL) { warning = warning + ", everything will not be repaired this schedule. Consider increasing maxBytesPerSchedule, reducing node density or monitoring to ensure all ranges do get repaired within gc_grace_seconds"; } - logger.warn(warning, keyspaceName, tableName, FileUtils.stringifyFileSize(maxBytesPerSchedule)); + logger.warn(warning, repairAssignment.getEstimatedSizeInBytes(), repairAssignment.keyspaceName, repairAssignment.tableNames, FileUtils.stringifyFileSize(maxBytesPerSchedule)); } private int calculateNumberOfSplits(SizeEstimate estimate) @@ -481,4 +535,48 @@ public SizeEstimate(AutoRepairConfig.RepairType repairType, this.sizeForRepair = repairType == AutoRepairConfig.RepairType.INCREMENTAL ? totalSize : sizeInRange; } } + + @VisibleForTesting + protected static class SizedRepairAssignment extends RepairAssignment { + + final long estimatedSizedInBytes; + + public SizedRepairAssignment(Range tokenRange, String keyspaceName, List tableNames, long estimatedSizeInBytes) + { + super(tokenRange, keyspaceName, tableNames); + this.estimatedSizedInBytes = estimatedSizeInBytes; + } + + public long getEstimatedSizeInBytes() { + return estimatedSizedInBytes; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + SizedRepairAssignment that = (SizedRepairAssignment) o; + return estimatedSizedInBytes == that.estimatedSizedInBytes; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), estimatedSizedInBytes); + } + + @Override + public String toString() + { + return "SizedRepairAssignement{" + + "estimatedSizedInBytes=" + FileUtils.stringifyFileSize(estimatedSizedInBytes) + + ", keyspaceName='" + keyspaceName + '\'' + + ", tokenRange=" + tokenRange + + ", tableNames=" + tableNames + + '}'; + } + + } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java index 71b60a92232e..2873b2566b0f 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java @@ -39,6 +39,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; +import org.apache.cassandra.repair.autorepair.RepairRangeSplitter.SizedRepairAssignment; import org.apache.cassandra.utils.concurrent.Refs; import static org.apache.cassandra.repair.autorepair.RepairRangeSplitter.TABLE_BATCH_LIMIT; @@ -186,7 +187,7 @@ public void testReorderByPriorityWithOneElement() { @Test public void testGetRepairAssignmentsForTable_NoSSTables() { Collection> ranges = Collections.singleton(new Range<>(Murmur3Partitioner.instance.getMinimumToken(), Murmur3Partitioner.instance.getMaximumToken())); - List assignments = repairRangeSplitter.getRepairAssignmentsForTable(AutoRepairConfig.RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForTable(AutoRepairConfig.RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); assertEquals(0, assignments.size()); } @@ -194,7 +195,7 @@ public void testGetRepairAssignmentsForTable_NoSSTables() { public void testGetRepairAssignmentsForTable_Single() throws Throwable { Collection> ranges = Collections.singleton(new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), DatabaseDescriptor.getPartitioner().getMaximumToken())); insertAndFlushSingleTable(tableName); - List assignments = repairRangeSplitter.getRepairAssignmentsForTable(AutoRepairConfig.RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForTable(AutoRepairConfig.RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); assertEquals(1, assignments.size()); } @@ -250,7 +251,7 @@ public void testGetRepairAssignmentsForTable_AllBatched() throws Throwable { @Test(expected = IllegalStateException.class) public void testMergeEmptyAssignments() { // Test when the list of assignments is empty - List emptyAssignments = Collections.emptyList(); + List emptyAssignments = Collections.emptyList(); RepairRangeSplitter.merge(emptyAssignments); } @@ -260,10 +261,10 @@ public void testMergeSingleAssignment() { String keyspaceName = "testKeyspace"; List tableNames = Arrays.asList("table1", "table2"); - RepairAssignment assignment = new RepairAssignment(FULL_RANGE, keyspaceName, tableNames); - List assignments = Collections.singletonList(assignment); + SizedRepairAssignment assignment = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames, 0L); + List assignments = Collections.singletonList(assignment); - RepairAssignment result = RepairRangeSplitter.merge(assignments); + SizedRepairAssignment result = RepairRangeSplitter.merge(assignments); assertEquals(FULL_RANGE, result.getTokenRange()); assertEquals(keyspaceName, result.getKeyspaceName()); @@ -277,11 +278,11 @@ public void testMergeMultipleAssignmentsWithSameTokenRangeAndKeyspace() { List tableNames1 = Arrays.asList("table1", "table2"); List tableNames2 = Arrays.asList("table2", "table3"); - RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, keyspaceName, tableNames1); - RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, keyspaceName, tableNames2); - List assignments = Arrays.asList(assignment1, assignment2); + SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames1, 0L); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames2, 0L); + List assignments = Arrays.asList(assignment1, assignment2); - RepairAssignment result = RepairRangeSplitter.merge(assignments); + SizedRepairAssignment result = RepairRangeSplitter.merge(assignments); assertEquals(FULL_RANGE, result.getTokenRange()); assertEquals(keyspaceName, result.getKeyspaceName()); @@ -299,9 +300,9 @@ public void testMergeDifferentTokenRange() { String keyspaceName = "testKeyspace"; List tableNames = Arrays.asList("table1", "table2"); - RepairAssignment assignment1 = new RepairAssignment(tokenRange1, keyspaceName, tableNames); - RepairAssignment assignment2 = new RepairAssignment(tokenRange2, keyspaceName, tableNames); - List assignments = Arrays.asList(assignment1, assignment2); + SizedRepairAssignment assignment1 = new SizedRepairAssignment(tokenRange1, keyspaceName, tableNames, 0L); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(tokenRange2, keyspaceName, tableNames, 0L); + List assignments = Arrays.asList(assignment1, assignment2); RepairRangeSplitter.merge(assignments); // Should throw IllegalStateException } @@ -311,9 +312,9 @@ public void testMergeDifferentKeyspaceName() { // Test merging assignments with different keyspace names List tableNames = Arrays.asList("table1", "table2"); - RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, "keyspace1", tableNames); - RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, "keyspace2", tableNames); - List assignments = Arrays.asList(assignment1, assignment2); + SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, "keyspace1", tableNames, 0L); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, "keyspace2", tableNames, 0L); + List assignments = Arrays.asList(assignment1, assignment2); RepairRangeSplitter.merge(assignments); // Should throw IllegalStateException } @@ -325,9 +326,9 @@ public void testMergeWithDuplicateTables() { List tableNames1 = Arrays.asList("table1", "table2"); List tableNames2 = Arrays.asList("table2", "table3"); - RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, keyspaceName, tableNames1); - RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, keyspaceName, tableNames2); - List assignments = Arrays.asList(assignment1, assignment2); + SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames1, 0L); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames2, 0L); + List assignments = Arrays.asList(assignment1, assignment2); RepairAssignment result = RepairRangeSplitter.merge(assignments); From f909af5f94167c65569e29f5fd37ec97d6997cf3 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Wed, 27 Nov 2024 08:24:06 -0600 Subject: [PATCH 079/115] Various cleanup --- .../autorepair/RepairRangeSplitter.java | 70 +++++++++++++------ 1 file changed, 48 insertions(+), 22 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index e4822adcfb13..510edc36f719 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -142,8 +142,11 @@ public RepairRangeSplitter(Map parameters) if (bytesPerSubrange > maxBytesPerSchedule) { - throw new RuntimeException(String.format("bytesPerSubrange '%s' cannot be greater than maxBytesPerSchedule '%s'", - FileUtils.stringifyFileSize(bytesPerSubrange), FileUtils.stringifyFileSize(maxBytesPerSchedule))); + throw new IllegalArgumentException(String.format("%s='%s' cannot be greater than %s='%s'", + SUBRANGE_SIZE, + FileUtils.stringifyFileSize(bytesPerSubrange), + MAX_BYTES_PER_SCHEDULE, + FileUtils.stringifyFileSize(maxBytesPerSchedule))); } logger.info("Configured {} with {}={}, {}={}, {}={}, {}={}", RepairRangeSplitter.class.getName(), @@ -187,8 +190,8 @@ List getRepairAssignments(AutoRepairConfig.RepairType repairTy currentAssignments.size() < tablesPerAssignmentLimit && (currentAssignments.isEmpty() || currentAssignments.get(0).getTokenRange().equals(tableAssignments.get(0).getTokenRange()))) { - long currentAssignmentsBytes = getBytesInAssignments(currentAssignments); - long tableAssignmentsBytes = getBytesInAssignments(tableAssignments); + long currentAssignmentsBytes = getEstimatedBytes(currentAssignments); + long tableAssignmentsBytes = getEstimatedBytes(tableAssignments); // only add assignments together if they don't exceed max bytes per schedule. if (currentAssignmentsBytes + tableAssignmentsBytes < maxBytesPerSchedule) { currentAssignments.addAll(tableAssignments); @@ -228,7 +231,7 @@ protected List filterRepairAssignments(AutoRepairConfig.Repair reorderByPriority(repairAssignments, repairType); // Confine repair assignemnts by maxBytesPer Schedule if greater than maxBytesPerSchedule. - long assignmentBytes = getBytesInAssignments(repairAssignments); + long assignmentBytes = getEstimatedBytes(repairAssignments); if (assignmentBytes < maxBytesPerSchedule) return repairAssignments.stream().map((a) -> (RepairAssignment)a).collect(Collectors.toList()); else @@ -237,7 +240,7 @@ protected List filterRepairAssignments(AutoRepairConfig.Repair List assignmentsToReturn = new ArrayList<>(repairAssignments.size()); for (SizedRepairAssignment repairAssignment : repairAssignments) { // skip any repair assignments that would accumulate us past the maxBytesPerSchedule - if (bytesSoFar + repairAssignment.getEstimatedSizeInBytes() > maxBytesPerSchedule) + if (bytesSoFar + repairAssignment.getEstimatedBytes() > maxBytesPerSchedule) { // log that repair assignment was skipped. warnMaxBytesPerSchedule(repairType, repairAssignment); @@ -245,18 +248,25 @@ protected List filterRepairAssignments(AutoRepairConfig.Repair else { assignmentsToReturn.add(repairAssignment); - bytesSoFar += repairAssignment.getEstimatedSizeInBytes(); + bytesSoFar += repairAssignment.getEstimatedBytes(); } } return assignmentsToReturn; } } + /** + * @return The sum of {@link SizedRepairAssignment#getEstimatedBytes()} of all given + * repairAssignments. + * @param repairAssignments The assignments to sum + */ @VisibleForTesting - protected static long getBytesInAssignments(List repairAssignments) { + protected static long getEstimatedBytes(List repairAssignments) + { return repairAssignments .stream() - .mapToLong(SizedRepairAssignment::getEstimatedSizeInBytes).sum(); + .mapToLong(SizedRepairAssignment::getEstimatedBytes) + .sum(); } @VisibleForTesting @@ -280,7 +290,7 @@ static SizedRepairAssignment merge(List assignments) mergedTableNames.addAll(assignment.getTableNames()); } - long sizeForAssignment = getBytesInAssignments(assignments); + long sizeForAssignment = getEstimatedBytes(assignments); return new SizedRepairAssignment(referenceTokenRange, referenceKeyspaceName, new ArrayList<>(mergedTableNames), sizeForAssignment); } @@ -353,12 +363,20 @@ protected List getRepairAssignments(List si private void warnMaxBytesPerSchedule(AutoRepairConfig.RepairType repairType, SizedRepairAssignment repairAssignment) { - String warning = "Refusing to add repair assignment of size {} for {}.{} because it would increase total repair bytes to a value greater than {}"; + String warning = "Refusing to add repair assignment of size {} for {}.{} because it would increase total repair bytes to a value greater than {} ({})"; if (repairType == AutoRepairConfig.RepairType.FULL) { warning = warning + ", everything will not be repaired this schedule. Consider increasing maxBytesPerSchedule, reducing node density or monitoring to ensure all ranges do get repaired within gc_grace_seconds"; } - logger.warn(warning, repairAssignment.getEstimatedSizeInBytes(), repairAssignment.keyspaceName, repairAssignment.tableNames, FileUtils.stringifyFileSize(maxBytesPerSchedule)); + // TODO: Add two metrics: + // Meter: RepairRangeSplitter.RepairType.SkippedAssignments + // Meter: RepairRangeSplitter.RepairType.SkippedBytes + logger.warn(warning, + repairAssignment.getEstimatedBytes(), + repairAssignment.keyspaceName, + repairAssignment.tableNames, + FileUtils.stringifyFileSize(maxBytesPerSchedule), + FileUtils.stringifyFileSize(maxBytesPerSchedule + repairAssignment.getEstimatedBytes())); } private int calculateNumberOfSplits(SizeEstimate estimate) @@ -387,7 +405,7 @@ private int calculateNumberOfSplits(SizeEstimate estimate) return splits; } - public Collection> getTokenRanges(boolean primaryRangeOnly, String keyspaceName) + private Collection> getTokenRanges(boolean primaryRangeOnly, String keyspaceName) { // Collect all applicable token ranges Collection> wrappedRanges; @@ -536,19 +554,28 @@ public SizeEstimate(AutoRepairConfig.RepairType repairType, } } + /** + * Implementation of RepairAssignment that also assigns an estimation of bytes involved + * in the repair. + */ @VisibleForTesting protected static class SizedRepairAssignment extends RepairAssignment { - final long estimatedSizedInBytes; + final long estimatedBytes; - public SizedRepairAssignment(Range tokenRange, String keyspaceName, List tableNames, long estimatedSizeInBytes) + public SizedRepairAssignment(Range tokenRange, String keyspaceName, List tableNames, long estimatedBytes) { super(tokenRange, keyspaceName, tableNames); - this.estimatedSizedInBytes = estimatedSizeInBytes; + this.estimatedBytes = estimatedBytes; } - public long getEstimatedSizeInBytes() { - return estimatedSizedInBytes; + /** + * Estimated bytes involved in the assignment. Typically Derived from {@link SizeEstimate#sizeForRepair}. + * @return estimated bytes involved in the assignment. + */ + public long getEstimatedBytes() + { + return estimatedBytes; } @Override @@ -558,25 +585,24 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) return false; if (!super.equals(o)) return false; SizedRepairAssignment that = (SizedRepairAssignment) o; - return estimatedSizedInBytes == that.estimatedSizedInBytes; + return estimatedBytes == that.estimatedBytes; } @Override public int hashCode() { - return Objects.hash(super.hashCode(), estimatedSizedInBytes); + return Objects.hash(super.hashCode(), estimatedBytes); } @Override public String toString() { return "SizedRepairAssignement{" + - "estimatedSizedInBytes=" + FileUtils.stringifyFileSize(estimatedSizedInBytes) + + "estimatedSizedInBytes=" + FileUtils.stringifyFileSize(estimatedBytes) + ", keyspaceName='" + keyspaceName + '\'' + ", tokenRange=" + tokenRange + ", tableNames=" + tableNames + '}'; } - } } From 451a58d5125de78277bd70cc6f11dd69b374c50c Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Wed, 27 Nov 2024 09:12:58 -0600 Subject: [PATCH 080/115] Handle EmpySSTableScanner being returned from getScanner If no SSTables cover a requested range an EmptySSTableScanner will be returned, handle this. --- .../autorepair/RepairRangeSplitter.java | 37 ++++++++++++------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index 510edc36f719..3af982214e5b 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -46,6 +46,7 @@ import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.ISSTableScanner; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.big.BigTableReader; import org.apache.cassandra.io.sstable.format.big.BigTableScanner; @@ -317,13 +318,13 @@ protected List getRepairAssignments(List si long memtableSize = cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize(); if (memtableSize > 0L) { - logger.debug("Included {}.{} range {}, had no unrepaired SSTables, but memtableSize={}, adding single repair assignment", estimate.keyspace, estimate.table, estimate.tokenRange, memtableSize); + logger.info("Included {}.{} range {}, had no unrepaired SSTables, but memtableSize={}, adding single repair assignment", estimate.keyspace, estimate.table, estimate.tokenRange, memtableSize); SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, Collections.singletonList(estimate.table), estimate.sizeInRange); repairAssignments.add(assignment); } else { - logger.debug("Skipping {}.{} for range {} because it had no unrepaired SSTables and no memtable data", estimate.keyspace, estimate.table, estimate.tokenRange); + logger.info("Skipping {}.{} for range {} because it had no unrepaired SSTables and no memtable data", estimate.keyspace, estimate.table, estimate.tokenRange); } } else @@ -466,22 +467,30 @@ static SizeEstimate getSizesForRangeOfSSTables(AutoRepairConfig.RepairType repai // still better to use the cardinality estimator then the index since it wont count duplicates. // get the bounds of the sstable for this range using the index file but do not actually read it. List> bounds = BigTableScanner.makeBounds(reader, Collections.singleton(tokenRange)); - try (BigTableScanner scanner = (BigTableScanner) BigTableScanner.getScanner((BigTableReader) reader, Collections.singleton(tokenRange))) - { - assert bounds.size() == 1; - AbstractBounds bound = bounds.get(0); - long startPosition = scanner.getDataPosition(bound.left); - long endPosition = scanner.getDataPosition(bound.right); - // If end position is 0 we can assume the sstable ended before that token, bound at size of file - if (endPosition == 0) + ISSTableScanner rangeScanner = BigTableScanner.getScanner((BigTableReader) reader, Collections.singleton(tokenRange)); + // Type check scanner returned as it may be an EmptySSTableScanner if the range is not covered in the + // SSTable, in this case we will avoid incrementing approxBytesInRange. + if (rangeScanner instanceof BigTableScanner) + { + try (BigTableScanner scanner = (BigTableScanner) rangeScanner) { - endPosition = sstableSize; + assert bounds.size() == 1; + + AbstractBounds bound = bounds.get(0); + long startPosition = scanner.getDataPosition(bound.left); + long endPosition = scanner.getDataPosition(bound.right); + // If end position is 0 we can assume the sstable ended before that token, bound at size of file + if (endPosition == 0) + { + endPosition = sstableSize; + } + + long approximateRangeBytesInSSTable = Math.max(0, endPosition - startPosition); + approxBytesInRange += Math.min(approximateRangeBytesInSSTable, sstableSize); } - - long approximateRangeBytesInSSTable = Math.max(0, endPosition - startPosition); - approxBytesInRange += Math.min(approximateRangeBytesInSSTable, sstableSize); } + } catch (IOException | CardinalityMergeException e) { From 104094aed042d6e50404e4776d6ebd85934ae91c Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Wed, 27 Nov 2024 11:47:16 -0600 Subject: [PATCH 081/115] Minor refactoring around bytesPerSubrange, logging Avoid intermediate variable by making DEFAULT_SUBRANGE_SIZE a long. Log bytes using stringifyFileSize --- .../repair/autorepair/RepairRangeSplitter.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index 3af982214e5b..82c66d4bb6aa 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -96,23 +96,21 @@ public class RepairRangeSplitter implements IAutoRepairTokenRangeSplitter private final long bytesPerSubrange; private final long partitionsPerSubrange; - private static final DataStorageSpec.LongBytesBound DEFAULT_SUBRANGE_SIZE = new DataStorageSpec.LongBytesBound("100GiB"); + private static final long DEFAULT_SUBRANGE_SIZE = new DataStorageSpec.LongBytesBound("100GiB").toBytes(); private static final long DEFAULT_MAX_BYTES_PER_SCHEDULE = Long.MAX_VALUE; private static final long DEFAULT_PARTITION_LIMIT = (long) Math.pow(2, DatabaseDescriptor.getRepairSessionMaxTreeDepth()); private static final int DEFAULT_TABLE_BATCH_LIMIT = 64; public RepairRangeSplitter(Map parameters) { - DataStorageSpec.LongBytesBound subrangeSize; if (parameters.containsKey(SUBRANGE_SIZE)) { - subrangeSize = new DataStorageSpec.LongBytesBound(parameters.get(SUBRANGE_SIZE)); + bytesPerSubrange = new DataStorageSpec.LongBytesBound(parameters.get(SUBRANGE_SIZE)).toBytes(); } else { - subrangeSize = DEFAULT_SUBRANGE_SIZE; + bytesPerSubrange = DEFAULT_SUBRANGE_SIZE; } - bytesPerSubrange = subrangeSize.toBytes(); if (parameters.containsKey(MAX_BYTES_PER_SCHEDULE)) { @@ -151,7 +149,8 @@ public RepairRangeSplitter(Map parameters) } logger.info("Configured {} with {}={}, {}={}, {}={}, {}={}", RepairRangeSplitter.class.getName(), - SUBRANGE_SIZE, subrangeSize, MAX_BYTES_PER_SCHEDULE, maxBytesPerSchedule, + SUBRANGE_SIZE, FileUtils.stringifyFileSize(bytesPerSubrange), + MAX_BYTES_PER_SCHEDULE, FileUtils.stringifyFileSize(maxBytesPerSchedule), PARTITION_COUNT, partitionsPerSubrange, TABLE_BATCH_LIMIT, tablesPerAssignmentLimit); } @@ -373,7 +372,7 @@ private void warnMaxBytesPerSchedule(AutoRepairConfig.RepairType repairType, Siz // Meter: RepairRangeSplitter.RepairType.SkippedAssignments // Meter: RepairRangeSplitter.RepairType.SkippedBytes logger.warn(warning, - repairAssignment.getEstimatedBytes(), + FileUtils.stringifyFileSize(repairAssignment.getEstimatedBytes()), repairAssignment.keyspaceName, repairAssignment.tableNames, FileUtils.stringifyFileSize(maxBytesPerSchedule), From 06de4a3c04f5d14950e63d4a0a6309ce9c76cc10 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Wed, 27 Nov 2024 15:47:27 -0600 Subject: [PATCH 082/115] Cleanup logging of repair assignments --- .../autorepair/RepairRangeSplitter.java | 163 ++++++++++++------ .../autorepair/RepairRangeSplitterTest.java | 18 +- 2 files changed, 124 insertions(+), 57 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index 82c66d4bb6aa..93e7431a15c1 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -212,10 +212,11 @@ List getRepairAssignments(AutoRepairConfig.RepairType repairTy repairAssignments.addAll(tableAssignments); } } + if (!currentAssignments.isEmpty()) repairAssignments.add(merge(currentAssignments)); - return filterRepairAssignments(repairType, repairAssignments); + return filterRepairAssignments(repairType, keyspaceName, repairAssignments); } /** @@ -225,34 +226,74 @@ List getRepairAssignments(AutoRepairConfig.RepairType repairTy * @param repairAssignments the assignments to filter. * @return A list of repair assignments ordered by priority and confined by maxBytesPerSchedule. */ - protected List filterRepairAssignments(AutoRepairConfig.RepairType repairType, List repairAssignments) + protected List filterRepairAssignments(AutoRepairConfig.RepairType repairType, String keyspaceName, List repairAssignments) { + if (repairAssignments.isEmpty()) + return Collections.emptyList(); + // Reorder the repair assignments reorderByPriority(repairAssignments, repairType); - // Confine repair assignemnts by maxBytesPer Schedule if greater than maxBytesPerSchedule. - long assignmentBytes = getEstimatedBytes(repairAssignments); - if (assignmentBytes < maxBytesPerSchedule) - return repairAssignments.stream().map((a) -> (RepairAssignment)a).collect(Collectors.toList()); - else + // Confine repair assignments by maxBytesPerSchedule. + long bytesSoFar = 0L; + boolean isIncremental = false; + long bytesNotRepaired = 0L; + int assignmentsNotRepaired = 0; + List assignmentsToReturn = new ArrayList<>(repairAssignments.size()); + for (SizedRepairAssignment repairAssignment : repairAssignments) { - long bytesSoFar = 0L; - List assignmentsToReturn = new ArrayList<>(repairAssignments.size()); - for (SizedRepairAssignment repairAssignment : repairAssignments) { - // skip any repair assignments that would accumulate us past the maxBytesPerSchedule - if (bytesSoFar + repairAssignment.getEstimatedBytes() > maxBytesPerSchedule) - { - // log that repair assignment was skipped. - warnMaxBytesPerSchedule(repairType, repairAssignment); - } - else - { - assignmentsToReturn.add(repairAssignment); - bytesSoFar += repairAssignment.getEstimatedBytes(); - } + // skip any repair assignments that would accumulate us past the maxBytesPerSchedule + if (bytesSoFar + repairAssignment.getEstimatedBytes() > maxBytesPerSchedule) + { + // log that repair assignment was skipped. + bytesNotRepaired += repairAssignment.getEstimatedBytes(); + assignmentsNotRepaired++; + logger.warn("Skipping {} because it would increase total repair bytes to {}", + repairAssignment, + getBytesOfMaxBytesPerSchedule(bytesSoFar + repairAssignment.getEstimatedBytes())); + } + else + { + bytesSoFar += repairAssignment.getEstimatedBytes(); + logger.info("Adding {}, increasing repair bytes to {}", + repairAssignment, + getBytesOfMaxBytesPerSchedule(bytesSoFar)); + assignmentsToReturn.add(repairAssignment); } - return assignmentsToReturn; } + + String message = "Returning {} assignment(s) for {}, totaling {}"; + if (assignmentsNotRepaired != 0) + { + message += ". Skipping {} of {} assignment(s), totaling {}"; + if (repairType != AutoRepairConfig.RepairType.INCREMENTAL) + { + message += ". The entire primary range will not be repaired this schedule. " + + "Consider increasing maxBytesPerSchedule, reducing node density or monitoring to ensure " + + "all ranges do get repaired within gc_grace_seconds"; + logger.warn(message, assignmentsToReturn.size(), keyspaceName, getBytesOfMaxBytesPerSchedule(bytesSoFar), + assignmentsNotRepaired, repairAssignments.size(), + FileUtils.stringifyFileSize(bytesNotRepaired)); + } + else + { + logger.info(message, assignmentsToReturn.size(), keyspaceName, getBytesOfMaxBytesPerSchedule(bytesSoFar), + assignmentsNotRepaired, repairAssignments.size(), + FileUtils.stringifyFileSize(bytesNotRepaired)); + } + } + else + { + logger.info(message, assignmentsToReturn.size(), keyspaceName, getBytesOfMaxBytesPerSchedule(bytesSoFar)); + } + return assignmentsToReturn; + } + + private String getBytesOfMaxBytesPerSchedule(long bytes) { + if (maxBytesPerSchedule == Long.MAX_VALUE) + return FileUtils.stringifyFileSize(bytes); + else + return String.format("%s of %s", FileUtils.stringifyFileSize(bytes), FileUtils.stringifyFileSize(maxBytesPerSchedule)); } /** @@ -291,7 +332,8 @@ static SizedRepairAssignment merge(List assignments) } long sizeForAssignment = getEstimatedBytes(assignments); - return new SizedRepairAssignment(referenceTokenRange, referenceKeyspaceName, new ArrayList<>(mergedTableNames), sizeForAssignment); + return new SizedRepairAssignment(referenceTokenRange, referenceKeyspaceName, new ArrayList<>(mergedTableNames), + "full primary range for " + mergedTableNames.size() + " tables", sizeForAssignment); } @VisibleForTesting @@ -317,20 +359,19 @@ protected List getRepairAssignments(List si long memtableSize = cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize(); if (memtableSize > 0L) { - logger.info("Included {}.{} range {}, had no unrepaired SSTables, but memtableSize={}, adding single repair assignment", estimate.keyspace, estimate.table, estimate.tokenRange, memtableSize); - SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, Collections.singletonList(estimate.table), estimate.sizeInRange); + logger.debug("Included {}.{} range {}, had no unrepaired SSTables, but memtableSize={}, adding single repair assignment", estimate.keyspace, estimate.table, estimate.tokenRange, memtableSize); + SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, Collections.singletonList(estimate.table), "memtable only", estimate.sizeInRange); repairAssignments.add(assignment); } else { - logger.info("Skipping {}.{} for range {} because it had no unrepaired SSTables and no memtable data", estimate.keyspace, estimate.table, estimate.tokenRange); + logger.debug("Skipping {}.{} for range {} because it had no unrepaired SSTables and no memtable data", estimate.keyspace, estimate.table, estimate.tokenRange); } } else { // Check if the estimate needs splitting based on the criteria boolean needsSplitting = estimate.sizeForRepair > bytesPerSubrange || estimate.partitions > partitionsPerSubrange; - if (needsSplitting) { int numberOfSplits = calculateNumberOfSplits(estimate); @@ -338,22 +379,18 @@ protected List getRepairAssignments(List si Collection> subranges = split(estimate.tokenRange, numberOfSplits); for (Range subrange : subranges) { - logger.info("Added repair assignment for {}.{} for subrange {} (#{}/{})", - estimate.keyspace, estimate.table, subrange, repairAssignments.size() + 1, numberOfSplits); - SizedRepairAssignment assignment = new SizedRepairAssignment(subrange, estimate.keyspace, Collections.singletonList(estimate.table), approximateBytesPerSplit); + SizedRepairAssignment assignment = new SizedRepairAssignment(subrange, estimate.keyspace, Collections.singletonList(estimate.table), + String.format("subrange %d of %d", repairAssignments.size()+1, numberOfSplits), + approximateBytesPerSplit); repairAssignments.add(assignment); } } else { - logger.info("Using 1 repair assignment for {}.{} for range {} as rangeBytes={} is less than {}={} and partitionEstimate={} is less than {}={}", - estimate.keyspace, estimate.table, estimate.tokenRange, - FileUtils.stringifyFileSize(estimate.sizeForRepair), - SUBRANGE_SIZE, FileUtils.stringifyFileSize(bytesPerSubrange), - estimate.partitions, - PARTITION_COUNT, partitionsPerSubrange); // No splitting needed, repair the entire range as-is - SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, Collections.singletonList(estimate.table), estimate.sizeForRepair); + SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, + Collections.singletonList(estimate.table), + "full primary range for table", estimate.sizeForRepair); repairAssignments.add(assignment); } } @@ -363,7 +400,7 @@ protected List getRepairAssignments(List si private void warnMaxBytesPerSchedule(AutoRepairConfig.RepairType repairType, SizedRepairAssignment repairAssignment) { - String warning = "Refusing to add repair assignment of size {} for {}.{} because it would increase total repair bytes to a value greater than {} ({})"; + String warning = "Refusing to add repair assignment {} because it would increase total repair bytes to a value greater than {} ({})"; if (repairType == AutoRepairConfig.RepairType.FULL) { warning = warning + ", everything will not be repaired this schedule. Consider increasing maxBytesPerSchedule, reducing node density or monitoring to ensure all ranges do get repaired within gc_grace_seconds"; @@ -372,9 +409,7 @@ private void warnMaxBytesPerSchedule(AutoRepairConfig.RepairType repairType, Siz // Meter: RepairRangeSplitter.RepairType.SkippedAssignments // Meter: RepairRangeSplitter.RepairType.SkippedBytes logger.warn(warning, - FileUtils.stringifyFileSize(repairAssignment.getEstimatedBytes()), - repairAssignment.keyspaceName, - repairAssignment.tableNames, + repairAssignment, FileUtils.stringifyFileSize(maxBytesPerSchedule), FileUtils.stringifyFileSize(maxBytesPerSchedule + repairAssignment.getEstimatedBytes())); } @@ -436,7 +471,7 @@ private List getRangeSizeEstimate(AutoRepairConfig.RepairType repa try (Refs refs = getSSTableReaderRefs(repairType, keyspace, table, tokenRange)) { SizeEstimate estimate = getSizesForRangeOfSSTables(repairType, keyspace, table, tokenRange, refs); - logger.debug("Size estimate for {}.{} for range {} is {}", keyspace, table, tokenRange, estimate); + logger.debug("Generated size estimate {}", estimate); sizeEstimates.add(estimate); } } @@ -560,6 +595,21 @@ public SizeEstimate(AutoRepairConfig.RepairType repairType, this.sizeForRepair = repairType == AutoRepairConfig.RepairType.INCREMENTAL ? totalSize : sizeInRange; } + + @Override + public String toString() + { + return "SizeEstimate{" + + "repairType=" + repairType + + ", keyspace='" + keyspace + '\'' + + ", table='" + table + '\'' + + ", tokenRange=" + tokenRange + + ", partitions=" + partitions + + ", sizeInRange=" + sizeInRange + + ", totalSize=" + totalSize + + ", sizeForRepair=" + sizeForRepair + + '}'; + } } /** @@ -569,14 +619,30 @@ public SizeEstimate(AutoRepairConfig.RepairType repairType, @VisibleForTesting protected static class SizedRepairAssignment extends RepairAssignment { + final String description; final long estimatedBytes; - public SizedRepairAssignment(Range tokenRange, String keyspaceName, List tableNames, long estimatedBytes) + public SizedRepairAssignment(Range tokenRange, String keyspaceName, List tableNames) + { + this(tokenRange, keyspaceName, tableNames, "", 0L); + } + + public SizedRepairAssignment(Range tokenRange, String keyspaceName, List tableNames, + String description, + long estimatedBytes) { super(tokenRange, keyspaceName, tableNames); + this.description = description; this.estimatedBytes = estimatedBytes; } + /** + * @return Additional metadata about the repair assignment. + */ + public String getDescription() { + return description; + } + /** * Estimated bytes involved in the assignment. Typically Derived from {@link SizeEstimate#sizeForRepair}. * @return estimated bytes involved in the assignment. @@ -593,23 +659,24 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) return false; if (!super.equals(o)) return false; SizedRepairAssignment that = (SizedRepairAssignment) o; - return estimatedBytes == that.estimatedBytes; + return estimatedBytes == that.estimatedBytes && Objects.equals(description, that.description); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), estimatedBytes); + return Objects.hash(super.hashCode(), description, estimatedBytes); } @Override public String toString() { - return "SizedRepairAssignement{" + - "estimatedSizedInBytes=" + FileUtils.stringifyFileSize(estimatedBytes) + - ", keyspaceName='" + keyspaceName + '\'' + + return "SizedRepairAssignment{" + + "description='" + description + '\'' + ", tokenRange=" + tokenRange + + ", keyspaceName='" + keyspaceName + '\'' + ", tableNames=" + tableNames + + ", estimatedBytes=" + FileUtils.stringifyFileSize(estimatedBytes) + '}'; } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java index 2873b2566b0f..6c91fbd539be 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java @@ -261,7 +261,7 @@ public void testMergeSingleAssignment() { String keyspaceName = "testKeyspace"; List tableNames = Arrays.asList("table1", "table2"); - SizedRepairAssignment assignment = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames, 0L); + SizedRepairAssignment assignment = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames); List assignments = Collections.singletonList(assignment); SizedRepairAssignment result = RepairRangeSplitter.merge(assignments); @@ -278,8 +278,8 @@ public void testMergeMultipleAssignmentsWithSameTokenRangeAndKeyspace() { List tableNames1 = Arrays.asList("table1", "table2"); List tableNames2 = Arrays.asList("table2", "table3"); - SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames1, 0L); - SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames2, 0L); + SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames1); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames2); List assignments = Arrays.asList(assignment1, assignment2); SizedRepairAssignment result = RepairRangeSplitter.merge(assignments); @@ -300,8 +300,8 @@ public void testMergeDifferentTokenRange() { String keyspaceName = "testKeyspace"; List tableNames = Arrays.asList("table1", "table2"); - SizedRepairAssignment assignment1 = new SizedRepairAssignment(tokenRange1, keyspaceName, tableNames, 0L); - SizedRepairAssignment assignment2 = new SizedRepairAssignment(tokenRange2, keyspaceName, tableNames, 0L); + SizedRepairAssignment assignment1 = new SizedRepairAssignment(tokenRange1, keyspaceName, tableNames); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(tokenRange2, keyspaceName, tableNames); List assignments = Arrays.asList(assignment1, assignment2); RepairRangeSplitter.merge(assignments); // Should throw IllegalStateException @@ -312,8 +312,8 @@ public void testMergeDifferentKeyspaceName() { // Test merging assignments with different keyspace names List tableNames = Arrays.asList("table1", "table2"); - SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, "keyspace1", tableNames, 0L); - SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, "keyspace2", tableNames, 0L); + SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, "keyspace1", tableNames); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, "keyspace2", tableNames); List assignments = Arrays.asList(assignment1, assignment2); RepairRangeSplitter.merge(assignments); // Should throw IllegalStateException @@ -326,8 +326,8 @@ public void testMergeWithDuplicateTables() { List tableNames1 = Arrays.asList("table1", "table2"); List tableNames2 = Arrays.asList("table2", "table3"); - SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames1, 0L); - SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames2, 0L); + SizedRepairAssignment assignment1 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames1); + SizedRepairAssignment assignment2 = new SizedRepairAssignment(FULL_RANGE, keyspaceName, tableNames2); List assignments = Arrays.asList(assignment1, assignment2); RepairAssignment result = RepairRangeSplitter.merge(assignments); From c72e8678e265fcc2166b57a367a43fb2a88a8426 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sat, 30 Nov 2024 12:04:09 -0600 Subject: [PATCH 083/115] Remove UnrepairedBytesBasedTokenRangeSplitter --- ...nrepairedBytesBasedTokenRangeSplitter.java | 337 ------------------ 1 file changed, 337 deletions(-) delete mode 100644 src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java diff --git a/src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java deleted file mode 100644 index 91d5783abe8e..000000000000 --- a/src/java/org/apache/cassandra/repair/autorepair/UnrepairedBytesBasedTokenRangeSplitter.java +++ /dev/null @@ -1,337 +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.repair.autorepair; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.config.DataStorageSpec; -import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.PartitionPosition; -import org.apache.cassandra.db.lifecycle.SSTableIntervalTree; -import org.apache.cassandra.db.lifecycle.SSTableSet; -import org.apache.cassandra.db.lifecycle.View; -import org.apache.cassandra.dht.AbstractBounds; -import org.apache.cassandra.dht.Range; -import org.apache.cassandra.dht.Token; -import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.io.sstable.format.big.BigTableReader; -import org.apache.cassandra.io.sstable.format.big.BigTableScanner; -import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.concurrent.Refs; - -import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.split; - -public class UnrepairedBytesBasedTokenRangeSplitter implements IAutoRepairTokenRangeSplitter -{ - private static final Logger logger = LoggerFactory.getLogger(UnrepairedBytesBasedTokenRangeSplitter.class); - - static final String SUBRANGE_SIZE = "subrange_size"; - static final String MAX_BYTES_PER_SCHEDULE = "max_bytes_per_schedule"; - - // target bytes per subrange - private final DataStorageSpec.LongBytesBound subrangeSize; - - // maximum target bytes to repair - private final DataStorageSpec.LongBytesBound maxBytesPerSchedule; - - private final long subrangeBytes; - - private final long maxBytesPerScheduleBytes; - - private static final DataStorageSpec.LongBytesBound DEFAULT_SUBRANGE_SIZE = new DataStorageSpec.LongBytesBound("100GiB"); - private static final DataStorageSpec.LongBytesBound DEFAULT_MAX_BYTES_PER_SCHEDULE = new DataStorageSpec.LongBytesBound("500GiB"); - - public UnrepairedBytesBasedTokenRangeSplitter(Map parameters) - { - // Demonstrates parameterizing a range splitter so we can have splitter specific options. - if (parameters.containsKey(SUBRANGE_SIZE)) - { - subrangeSize = new DataStorageSpec.LongBytesBound(parameters.get(SUBRANGE_SIZE)); - } - else - { - subrangeSize = DEFAULT_SUBRANGE_SIZE; - } - subrangeBytes = subrangeSize.toBytes(); - - if (parameters.containsKey(MAX_BYTES_PER_SCHEDULE)) - { - maxBytesPerSchedule = new DataStorageSpec.LongBytesBound(parameters.get(MAX_BYTES_PER_SCHEDULE)); - } - else - { - maxBytesPerSchedule = DEFAULT_MAX_BYTES_PER_SCHEDULE; - } - maxBytesPerScheduleBytes = maxBytesPerSchedule.toBytes(); - - logger.info("Configured {} with {}={}, {}={}", UnrepairedBytesBasedTokenRangeSplitter.class.getName(), - SUBRANGE_SIZE, subrangeSize, MAX_BYTES_PER_SCHEDULE, maxBytesPerSchedule); - } - - @Override - public List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) - { - List repairAssignments = new ArrayList<>(); - - logger.info("Calculating token range splits for repairType={} primaryRangeOnly={} keyspaceName={} tableNames={}", repairType, primaryRangeOnly, keyspaceName, tableNames); - if (repairType != AutoRepairConfig.RepairType.INCREMENTAL) - { - throw new IllegalArgumentException(this.getClass().getName() + " only supports " + AutoRepairConfig.RepairType.INCREMENTAL + " repair"); - } - - // TODO: create a custom repair assignment that indicates number of bytes in repair and join tables by byte size. - Collection> tokenRanges = getTokenRanges(primaryRangeOnly, keyspaceName); - for (String tableName : tableNames) - { - repairAssignments.addAll(getRepairAssignmentsForTable(keyspaceName, tableName, tokenRanges)); - } - return repairAssignments; - } - - public List getRepairAssignmentsForTable(String keyspaceName, String tableName, Collection> tokenRanges) - { - List repairAssignments = new ArrayList<>(); - - long targetBytesSoFar = 0; - - for (Range tokenRange : tokenRanges) - { - logger.info("Calculating unrepaired bytes for {}.{} for range {}", keyspaceName, tableName, tokenRange); - // Capture the amount of unrepaired bytes for range - long approximateUnrepairedBytesForRange = 0L; - // Capture the total bytes in read sstables, this will be useful for calculating the ratio - // of data in SSTables including this range and also useful to know how much anticompaction there will be. - long totalBytesInUnrepairedSSTables = 0L; - try (Refs refs = getSSTableReaderRefs(keyspaceName, tableName, tokenRange)) - { - for (SSTableReader reader : refs) - { - // Only evaluate unrepaired SSTables. - if (!reader.isRepaired()) - { - long sstableSize = reader.bytesOnDisk(); - totalBytesInUnrepairedSSTables += sstableSize; - // get the bounds of the sstable for this range using the index file but do not actually read it. - List> bounds = BigTableScanner.makeBounds(reader, Collections.singleton(tokenRange)); - try (BigTableScanner scanner = (BigTableScanner) BigTableScanner.getScanner((BigTableReader) reader, Collections.singleton(tokenRange))) - { - assert bounds.size() == 1; - - AbstractBounds bound = bounds.get(0); - long startPosition = scanner.getDataPosition(bound.left); - long endPosition = scanner.getDataPosition(bound.right); - // If end position is 0 we can assume the sstable ended before that token, bound at size of file - if (endPosition == 0) - { - endPosition = sstableSize; - } - - long approximateRangeBytesInSSTable = Math.max(0, endPosition - startPosition); - // get the fraction of the sstable belonging to the range. - approximateUnrepairedBytesForRange += Math.min(approximateRangeBytesInSSTable, sstableSize); - double ratio = approximateRangeBytesInSSTable / (double) sstableSize; - logger.info("Calculations for {}.{} {}: sstableSize={}, rangeBytesInSSTable={}, startPosition={}, endPosition={}, ratio={}", - keyspaceName, tableName, reader.descriptor.baseFile().name(), - FileUtils.stringifyFileSize(sstableSize), FileUtils.stringifyFileSize(approximateRangeBytesInSSTable), startPosition, endPosition, ratio); - } - } - else - { - logger.info("Skipping over {}.{} {} ({}) because it is repaired", keyspaceName, tableName, reader.descriptor.baseFile().name(), FileUtils.stringifyFileSize(reader.bytesOnDisk())); - } - } - } - - // Only consider token range if it had unrepaired sstables or live data in memtables. - if (totalBytesInUnrepairedSSTables > 0L) - { - // TODO: Possibly some anticompaction configuration we want here, where if we detect a large amount of anticompaction we want to reduce the work we do. - double ratio = approximateUnrepairedBytesForRange / (double) totalBytesInUnrepairedSSTables; - logger.info("Calculated unrepaired bytes for {}.{} for range {}: sstableSize={}, rangeBytesInSSTables={}, ratio={}", keyspaceName, tableName, tokenRange, - FileUtils.stringifyFileSize(totalBytesInUnrepairedSSTables), FileUtils.stringifyFileSize(approximateUnrepairedBytesForRange), ratio); - - // TODO: split on byte size here, this is currently a bit naive in assuming that data is evenly distributed among the range which may not be the - // right assumption. May want to consider when splitting on these ranges to reevaluate how much data is in the range, but for this - // exists as a demonstration. - if (approximateUnrepairedBytesForRange < subrangeBytes) - { - // accept range as is if less than bytes. - logger.info("Using 1 repair assignment for {}.{} for range {} as {} is less than {}", keyspaceName, tableName, tokenRange, - FileUtils.stringifyFileSize(approximateUnrepairedBytesForRange), subrangeSize); - // TODO: this is a bit repetitive see if can reduce more. - RepairAssignment assignment = new BytesBasedRepairAssignment(tokenRange, keyspaceName, Collections.singletonList(tableName), approximateUnrepairedBytesForRange); - if (canAddAssignment(assignment, targetBytesSoFar, approximateUnrepairedBytesForRange)) - { - repairAssignments.add(assignment); - targetBytesSoFar += approximateUnrepairedBytesForRange; - } - else - return repairAssignments; - } - else - { - long targetRanges = approximateUnrepairedBytesForRange / subrangeBytes; - // TODO: approximation per range, this is a bit lossy since targetRanges rounds down. - long approximateBytesPerSplit = approximateUnrepairedBytesForRange / targetRanges; - logger.info("Splitting {}.{} for range {} into {} sub ranges, approximateBytesPerSplit={}", keyspaceName, tableName, tokenRange, targetRanges, FileUtils.stringifyFileSize(approximateBytesPerSplit)); - Collection> splitRanges = split(tokenRange, (int) targetRanges); - int splitRangeCount = 0; - for (Range splitRange : splitRanges) - { - RepairAssignment assignment = new BytesBasedRepairAssignment(splitRange, keyspaceName, Collections.singletonList(tableName), approximateBytesPerSplit); - if (canAddAssignment(assignment, targetBytesSoFar, approximateBytesPerSplit)) - { - logger.info("Added repair assignment for {}.{} for subrange {} (#{}/{}) with approximateBytes={}", - keyspaceName, tableName, splitRange, ++splitRangeCount, splitRanges.size(), FileUtils.stringifyFileSize(approximateBytesPerSplit)); - repairAssignments.add(assignment); - targetBytesSoFar += approximateBytesPerSplit; - } - else - return repairAssignments; - } - } - } - else - { - ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); - long memtableSize = cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize(); - if (memtableSize > 0L) - { - logger.info("Included {}.{} range {}, had no unrepaired SSTables, but memtableSize={}, adding single repair assignment", keyspaceName, tableName, tokenRange, memtableSize); - RepairAssignment assignment = new BytesBasedRepairAssignment(tokenRange, keyspaceName, Collections.singletonList(tableName), memtableSize); - if (targetBytesSoFar >= maxBytesPerScheduleBytes) - { - return repairAssignments; - } - repairAssignments.add(assignment); - targetBytesSoFar += memtableSize; - } - else - { - logger.info("Skipping {}.{} for range {} because it had no unrepaired SSTables and no memtable data", keyspaceName, tableName, tokenRange); - } - } - } - return repairAssignments; - } - - private boolean canAddAssignment(RepairAssignment repairAssignment, long targetBytesSoFar, long bytesToBeAdded) - { - if (targetBytesSoFar + bytesToBeAdded < maxBytesPerScheduleBytes) - { - return true; - } - logger.warn("Refusing to add {} with a target size of {} because it would increase total repair bytes to {} which is greater than {}={}", - repairAssignment, FileUtils.stringifyFileSize(bytesToBeAdded), FileUtils.stringifyFileSize(targetBytesSoFar + bytesToBeAdded), MAX_BYTES_PER_SCHEDULE, maxBytesPerSchedule); - return false; - } - - public Collection> getTokenRanges(boolean primaryRangeOnly, String keyspaceName) - { - // Collect all applicable token ranges - Collection> wrappedRanges; - if (primaryRangeOnly) - { - wrappedRanges = StorageService.instance.getPrimaryRanges(keyspaceName); - } - else - { - wrappedRanges = StorageService.instance.getLocalRanges(keyspaceName); - } - - // Unwrap each range as we need to account for ranges that overlap the ring - Collection> ranges = new ArrayList<>(); - for (Range wrappedRange : wrappedRanges) - { - ranges.addAll(wrappedRange.unwrap()); - } - - return ranges; - } - - public Refs getSSTableReaderRefs(String keyspaceName, String tableName, Range tokenRange) - { - final ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); - - if (cfs == null) - { - throw new IllegalArgumentException(String.format("Could not resolve ColumnFamilyStore from %s.%s", keyspaceName, tableName)); - } - - Iterable sstables = cfs.getTracker().getView().select(SSTableSet.CANONICAL); - SSTableIntervalTree tree = SSTableIntervalTree.build(sstables); - Range r = Range.makeRowRange(tokenRange); - Iterable canonicalSSTables = View.sstablesInBounds(r.left, r.right, tree); - - // TODO: may need to reason about this not working. - return Refs.ref(canonicalSSTables); - } - - public static class BytesBasedRepairAssignment extends RepairAssignment - { - private final long approximateBytes; - - public BytesBasedRepairAssignment(Range tokenRange, String keyspaceName, List tableNames, long approximateBytes) - { - super(tokenRange, keyspaceName, tableNames); - this.approximateBytes = approximateBytes; - } - - @Override - public boolean equals(Object o) - { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - BytesBasedRepairAssignment that = (BytesBasedRepairAssignment) o; - return approximateBytes == that.approximateBytes; - } - - @Override - public int hashCode() - { - return Objects.hash(super.hashCode(), approximateBytes); - } - - @Override - public String toString() - { - return "BytesBasedRepairAssignment{" + - "keyspaceName='" + keyspaceName + '\'' + - ", approximateBytes=" + approximateBytes + - ", tokenRange=" + tokenRange + - ", tableNames=" + tableNames + - '}'; - } - - public long getApproximateBytes() - { - return approximateBytes; - } - } -} From 1c1b8aa9f5e8c628042db340ace4e3c226a5597f Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sat, 30 Nov 2024 15:10:02 -0600 Subject: [PATCH 084/115] Refactor RepairRangeSplitter interface to accept all keyspaces This enables us to evaluate all RepairAssignments in the schedule at once which enhances the ability to order and restrict the RepairAssignments returned based on the splitter implementation. For example, RepairRangeSplitter's max_bytes_per_schedule would be more useful if it was evaluated for all keyspaces collectively instead of per keyspace. Also refactors AutoRepair to order keyspaces by highest table priority. This may be enhanced later to bucket repair submissions by priority instead of ordering keyspaces by highest table priority. --- .../repair/autorepair/AutoRepair.java | 54 ++++++- .../DefaultAutoRepairTokenSplitter.java | 15 +- .../IAutoRepairTokenRangeSplitter.java | 6 +- .../autorepair/RepairRangeSplitter.java | 150 ++++++++++-------- ...DefaultTokenSplitterParameterizedTest.java | 76 +++++---- .../AutoRepairParameterizedTest.java | 15 +- .../autorepair/RepairRangeSplitterTest.java | 12 +- 7 files changed, 214 insertions(+), 114 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 1abb71d69cac..2ec92276ec2f 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -22,6 +22,7 @@ import java.util.EnumMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -147,6 +148,37 @@ public void repairAsync(AutoRepairConfig.RepairType repairType) repairExecutors.get(repairType).submit(() -> repair(repairType)); } + /** + * @return The priority of the given table if defined, otherwise 0. + */ + private int getPriority(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName) + { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); + return cfs != null ? cfs.metadata().params.automatedRepair.get(repairType).priority() : 0; + } + + /** + * @return A new map of keyspaces to tables sorted by the keyspace's underlying highest table priority. + */ + private Map> sortKeyspaceMapByTablePriority(AutoRepairConfig.RepairType repairType, Map> keyspacesAndTablesToRepair) + { + // Sort assignments by keyspace with tables with the highest priority. + List>> entriesOrderedByTablePriority = new ArrayList<>(keyspacesAndTablesToRepair.entrySet()); + entriesOrderedByTablePriority.sort((a, b) -> { + int aMaxPriority = a.getValue().stream().mapToInt((tableName) -> getPriority(repairType, a.getKey(), tableName)).max().orElse(0); + int bMaxPriority = b.getValue().stream().mapToInt((tableName) -> getPriority(repairType, b.getKey(), tableName)).max().orElse(0); + return bMaxPriority - aMaxPriority; + }); + + Map> sortedKeyspacesToRepair = new LinkedHashMap<>(); + for (Map.Entry> sortedEntries : entriesOrderedByTablePriority) + { + sortedKeyspacesToRepair.put(sortedEntries.getKey(), sortedEntries.getValue()); + } + + return sortedKeyspacesToRepair; + } + // repair runs a repair session of the given type synchronously. public void repair(AutoRepairConfig.RepairType repairType) { @@ -205,18 +237,30 @@ public void repair(AutoRepairConfig.RepairType repairType) // sessions on overlapping datasets at the same time. Shuffling keyspaces reduces the likelihood of this happening. shuffleFunc.accept(keyspaces); + // Filter out keyspaces and tables to repair and group into a map by keyspace. + Map> keyspacesAndTablesToRepair = new LinkedHashMap<>(); for (Keyspace keyspace : keyspaces) { if (!AutoRepairUtils.checkNodeContainsKeyspaceReplica(keyspace)) { continue; } - - repairState.setRepairKeyspaceCount(repairState.getRepairKeyspaceCount() + 1); List tablesToBeRepairedList = retrieveTablesToBeRepaired(keyspace, config, repairType, repairState, collectectedRepairStats); shuffleFunc.accept(tablesToBeRepairedList); - String keyspaceName = keyspace.getName(); - List repairAssignments = tokenRangeSplitters.get(repairType).getRepairAssignments(repairType, primaryRangeOnly, keyspaceName, tablesToBeRepairedList); + keyspacesAndTablesToRepair.put(keyspace.getName(), tablesToBeRepairedList); + } + // sort keyspaces to repair by table priority + Map> sortedKeyspacesToRepair = sortKeyspaceMapByTablePriority(repairType, keyspacesAndTablesToRepair); + + // calculate the repair assignments for each kesypace. + Map> repairAssignmentsByKeyspace = tokenRangeSplitters.get(repairType).getRepairAssignments(repairType, primaryRangeOnly, sortedKeyspacesToRepair); + + // evaluate over each keyspace's repair assignments. + for (Map.Entry> keyspaceAssignments : repairAssignmentsByKeyspace.entrySet()) + { + String keyspaceName = keyspaceAssignments.getKey(); + List repairAssignments = keyspaceAssignments.getValue(); + repairState.setRepairKeyspaceCount(repairState.getRepairKeyspaceCount() + 1); int totalRepairAssignments = repairAssignments.size(); long keyspaceStartTime = timeFunc.get(); @@ -243,7 +287,7 @@ public void repair(AutoRepairConfig.RepairType repairType) repairState.setRepairInProgress(false); return; } - if (AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, keyspaceStartTime, tablesToBeRepairedList.size())) + if (AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, keyspaceStartTime, sortedKeyspacesToRepair.get(keyspaceName).size())) { collectectedRepairStats.skippedTokenRanges += totalRepairAssignments - totalProcessedAssignments; logger.info("Keyspace took too much time to repair hence skipping it {}", diff --git a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java index c0ddc6b9aa7e..7a106411373e 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java @@ -21,7 +21,9 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import org.apache.cassandra.service.AutoRepairService; @@ -34,7 +36,18 @@ public class DefaultAutoRepairTokenSplitter implements IAutoRepairTokenRangeSplitter { @Override - public List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) + public Map> getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, Map> keyspacesAndTablesToRepair) + { + Map> repairAssignmentsByKeyspace = new LinkedHashMap<>(); + for (Map.Entry> keyspace : keyspacesAndTablesToRepair.entrySet()) + { + repairAssignmentsByKeyspace.put(keyspace.getKey(), getRepairAssignmentsByKeyspace(repairType, primaryRangeOnly, keyspace.getKey(), keyspace.getValue())); + } + + return repairAssignmentsByKeyspace; + } + + private List getRepairAssignmentsByKeyspace(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) { AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); List repairAssignments = new ArrayList<>(); diff --git a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java index d43a9f562594..d64916dcbd58 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java @@ -28,7 +28,6 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.schema.AutoRepairParams; public interface IAutoRepairTokenRangeSplitter { @@ -38,11 +37,10 @@ public interface IAutoRepairTokenRangeSplitter * The autorepair framework will repair the list of returned subrange in a sequence. * @param repairType The type of repair being executed * @param primaryRangeOnly Whether to repair only this node's primary ranges or all of its ranges. - * @param keyspaceName The keyspace being repaired - * @param tableNames The tables to repair + * @param keyspacesAndTablesToRepair A map keyed by keyspace name and valued by tables to generate assignments for. * @return repair assignments broken up by range, keyspace and tables. */ - List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames); + Map> getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, Map> keyspacesAndTablesToRepair); /** * Reorders the list of {@link RepairAssignment} objects based on their priority for a given repair type. diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index 93e7431a15c1..00a7a9dc9e95 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -23,6 +23,7 @@ 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; @@ -53,6 +54,7 @@ import org.apache.cassandra.io.sstable.metadata.CompactionMetadata; import org.apache.cassandra.io.sstable.metadata.MetadataType; import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.concurrent.Refs; @@ -155,28 +157,40 @@ public RepairRangeSplitter(Map parameters) } @Override - public List getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) + public Map> getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, Map> keyspacesAndTablesToRepair) { - logger.debug("Calculating token range splits for repairType={} primaryRangeOnly={} keyspaceName={} tableNames={}", repairType, primaryRangeOnly, keyspaceName, tableNames); - Collection> tokenRanges = getTokenRanges(primaryRangeOnly, keyspaceName); - return getRepairAssignments(repairType, keyspaceName, tableNames, tokenRanges); + Map> repairAssignmentsByKeyspace = new LinkedHashMap<>(); + for (Map.Entry> keyspace : keyspacesAndTablesToRepair.entrySet()) + { + logger.debug("Calculating token range splits for repairType={} primaryRangeOnly={} keyspaceName={} tableNames={}", repairType, primaryRangeOnly, keyspace.getKey(), keyspace.getValue()); + Collection> tokenRanges = getTokenRanges(primaryRangeOnly, keyspace.getKey()); + repairAssignmentsByKeyspace.put(keyspace.getKey(), getRepairAssignments(repairType, keyspace.getKey(), keyspace.getValue(), tokenRanges)); + } + + return filterAndOrderRepairAssignments(repairType, repairAssignmentsByKeyspace); } @VisibleForTesting - List getRepairAssignments(AutoRepairConfig.RepairType repairType, String keyspaceName, List tableNames, Collection> tokenRanges) + List getRepairAssignments(AutoRepairConfig.RepairType repairType, String keyspaceName, List tableNames, Collection> tokenRanges) { List repairAssignments = new ArrayList<>(); // this is used for batching minimal single assignment tables together List currentAssignments = new ArrayList<>(); - // sort the tables by size so can batch the smallest ones together - tableNames.sort((t1, t2) -> { - ColumnFamilyStore cfs1 = ColumnFamilyStore.getIfExists(keyspaceName, t1); - ColumnFamilyStore cfs2 = ColumnFamilyStore.getIfExists(keyspaceName, t2); - if (cfs1 == null || cfs2 == null) - throw new IllegalArgumentException(String.format("Could not resolve ColumnFamilyStore from %s.%s", keyspaceName, t1)); - return Long.compare(cfs1.metric.totalDiskSpaceUsed.getCount(), cfs2.metric.totalDiskSpaceUsed.getCount()); - }); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + + // If we can repair by keyspace, sort the tables by size so can batch the smallest ones together + boolean repairByKeyspace = config.getRepairByKeyspace(repairType); + if (repairByKeyspace) + { + tableNames.sort((t1, t2) -> { + ColumnFamilyStore cfs1 = ColumnFamilyStore.getIfExists(keyspaceName, t1); + ColumnFamilyStore cfs2 = ColumnFamilyStore.getIfExists(keyspaceName, t2); + if (cfs1 == null || cfs2 == null) + throw new IllegalArgumentException(String.format("Could not resolve ColumnFamilyStore from %s.%s", keyspaceName, t1)); + return Long.compare(cfs1.metric.totalDiskSpaceUsed.getCount(), cfs2.metric.totalDiskSpaceUsed.getCount()); + }); + } for (String tableName : tableNames) { @@ -185,8 +199,13 @@ List getRepairAssignments(AutoRepairConfig.RepairType repairTy if (tableAssignments.isEmpty()) continue; + // if not repairing by keyspace don't attempt to batch them with others. + if (!repairByKeyspace) + { + repairAssignments.addAll(tableAssignments); + } // If the table assignments are for the same token range, and we have room to add more tables to the current assignment - if (tableAssignments.size() == 1 && + else if (tableAssignments.size() == 1 && currentAssignments.size() < tablesPerAssignmentLimit && (currentAssignments.isEmpty() || currentAssignments.get(0).getTokenRange().equals(tableAssignments.get(0).getTokenRange()))) { @@ -216,77 +235,92 @@ List getRepairAssignments(AutoRepairConfig.RepairType repairTy if (!currentAssignments.isEmpty()) repairAssignments.add(merge(currentAssignments)); - return filterRepairAssignments(repairType, keyspaceName, repairAssignments); + return repairAssignments; } /** - * Given a repair type and list of sized-based repair assignments order them by priority of the + * Given a repair type and map of sized-based repair assignments by keyspace order them by priority of the * assignments' underlying tables and confine them by maxBytesPerSchedule. * @param repairType used to determine underyling table priorities - * @param repairAssignments the assignments to filter. + * @param repairAssignmentsByKeyspace the assignments to filter. * @return A list of repair assignments ordered by priority and confined by maxBytesPerSchedule. */ - protected List filterRepairAssignments(AutoRepairConfig.RepairType repairType, String keyspaceName, List repairAssignments) + @VisibleForTesting + Map> filterAndOrderRepairAssignments(AutoRepairConfig.RepairType repairType, Map> repairAssignmentsByKeyspace) { - if (repairAssignments.isEmpty()) - return Collections.emptyList(); - - // Reorder the repair assignments - reorderByPriority(repairAssignments, repairType); - // Confine repair assignments by maxBytesPerSchedule. long bytesSoFar = 0L; - boolean isIncremental = false; long bytesNotRepaired = 0L; int assignmentsNotRepaired = 0; - List assignmentsToReturn = new ArrayList<>(repairAssignments.size()); - for (SizedRepairAssignment repairAssignment : repairAssignments) + int assignmentsToRepair = 0; + int totalAssignments = 0; + + Map> filteredRepairAssignmentsByKeyspace = new LinkedHashMap<>(); + for (Map.Entry> keyspaceAssignments : repairAssignmentsByKeyspace.entrySet()) { - // skip any repair assignments that would accumulate us past the maxBytesPerSchedule - if (bytesSoFar + repairAssignment.getEstimatedBytes() > maxBytesPerSchedule) + String keyspace = keyspaceAssignments.getKey(); + List repairAssignments = keyspaceAssignments.getValue(); + + // Reorder the repair assignments + reorderByPriority(repairAssignments, repairType); + + List assignmentsToReturn = new ArrayList<>(repairAssignments.size()); + for (SizedRepairAssignment repairAssignment : repairAssignments) { - // log that repair assignment was skipped. - bytesNotRepaired += repairAssignment.getEstimatedBytes(); - assignmentsNotRepaired++; - logger.warn("Skipping {} because it would increase total repair bytes to {}", - repairAssignment, - getBytesOfMaxBytesPerSchedule(bytesSoFar + repairAssignment.getEstimatedBytes())); + totalAssignments++; + // skip any repair assignments that would accumulate us past the maxBytesPerSchedule + if (bytesSoFar + repairAssignment.getEstimatedBytes() > maxBytesPerSchedule) + { + // log that repair assignment was skipped. + bytesNotRepaired += repairAssignment.getEstimatedBytes(); + assignmentsNotRepaired++; + logger.warn("Skipping {} because it would increase total repair bytes to {}", + repairAssignment, + getBytesOfMaxBytesPerSchedule(bytesSoFar + repairAssignment.getEstimatedBytes())); + } + else + { + bytesSoFar += repairAssignment.getEstimatedBytes(); + assignmentsToRepair++; + logger.info("Adding {}, increasing repair bytes to {}", + repairAssignment, + getBytesOfMaxBytesPerSchedule(bytesSoFar)); + assignmentsToReturn.add(repairAssignment); + } } - else + + if (!assignmentsToReturn.isEmpty()) { - bytesSoFar += repairAssignment.getEstimatedBytes(); - logger.info("Adding {}, increasing repair bytes to {}", - repairAssignment, - getBytesOfMaxBytesPerSchedule(bytesSoFar)); - assignmentsToReturn.add(repairAssignment); + filteredRepairAssignmentsByKeyspace.put(keyspace, assignmentsToReturn); } } - String message = "Returning {} assignment(s) for {}, totaling {}"; + String message = "Returning {} assignment(s), totaling {}"; if (assignmentsNotRepaired != 0) { message += ". Skipping {} of {} assignment(s), totaling {}"; if (repairType != AutoRepairConfig.RepairType.INCREMENTAL) { message += ". The entire primary range will not be repaired this schedule. " + - "Consider increasing maxBytesPerSchedule, reducing node density or monitoring to ensure " + - "all ranges do get repaired within gc_grace_seconds"; - logger.warn(message, assignmentsToReturn.size(), keyspaceName, getBytesOfMaxBytesPerSchedule(bytesSoFar), - assignmentsNotRepaired, repairAssignments.size(), + "Consider increasing maxBytesPerSchedule, reducing node density or monitoring to ensure " + + "all ranges do get repaired within gc_grace_seconds"; + logger.warn(message, assignmentsToRepair, getBytesOfMaxBytesPerSchedule(bytesSoFar), + assignmentsNotRepaired, totalAssignments, FileUtils.stringifyFileSize(bytesNotRepaired)); } else { - logger.info(message, assignmentsToReturn.size(), keyspaceName, getBytesOfMaxBytesPerSchedule(bytesSoFar), - assignmentsNotRepaired, repairAssignments.size(), + logger.info(message, assignmentsToRepair, getBytesOfMaxBytesPerSchedule(bytesSoFar), + assignmentsNotRepaired, totalAssignments, FileUtils.stringifyFileSize(bytesNotRepaired)); } } else { - logger.info(message, assignmentsToReturn.size(), keyspaceName, getBytesOfMaxBytesPerSchedule(bytesSoFar)); + logger.info(message, assignmentsToRepair, getBytesOfMaxBytesPerSchedule(bytesSoFar)); } - return assignmentsToReturn; + + return filteredRepairAssignmentsByKeyspace; } private String getBytesOfMaxBytesPerSchedule(long bytes) { @@ -398,22 +432,6 @@ protected List getRepairAssignments(List si return repairAssignments; } - private void warnMaxBytesPerSchedule(AutoRepairConfig.RepairType repairType, SizedRepairAssignment repairAssignment) - { - String warning = "Refusing to add repair assignment {} because it would increase total repair bytes to a value greater than {} ({})"; - if (repairType == AutoRepairConfig.RepairType.FULL) - { - warning = warning + ", everything will not be repaired this schedule. Consider increasing maxBytesPerSchedule, reducing node density or monitoring to ensure all ranges do get repaired within gc_grace_seconds"; - } - // TODO: Add two metrics: - // Meter: RepairRangeSplitter.RepairType.SkippedAssignments - // Meter: RepairRangeSplitter.RepairType.SkippedBytes - logger.warn(warning, - repairAssignment, - FileUtils.stringifyFileSize(maxBytesPerSchedule), - FileUtils.stringifyFileSize(maxBytesPerSchedule + repairAssignment.getEstimatedBytes())); - } - private int calculateNumberOfSplits(SizeEstimate estimate) { // Calculate the number of splits needed for size and partitions diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java index 804d0a712b48..8cee64f5a238 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java @@ -24,6 +24,8 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.LinkedHashMap; +import java.util.Map; import org.junit.BeforeClass; import org.junit.Test; @@ -40,12 +42,12 @@ import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.tcm.ClusterMetadata; import static org.apache.cassandra.config.CassandraRelevantProperties.SYSTEM_DISTRIBUTED_DEFAULT_RF; import static org.apache.cassandra.cql3.CQLTester.Fuzzed.setupSeed; import static org.apache.cassandra.cql3.CQLTester.Fuzzed.updateConfigs; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; @RunWith(Parameterized.class) public class AutoRepairDefaultTokenSplitterParameterizedTest @@ -72,9 +74,9 @@ public static void setupClass() throws Exception DatabaseDescriptor.setPartitioner("org.apache.cassandra.dht.Murmur3Partitioner"); ServerTestUtils.prepareServerNoRegister(); - Token t1 = new Murmur3Partitioner.LongToken(0); - Token t2 = new Murmur3Partitioner.LongToken(256); - Token t3 = new Murmur3Partitioner.LongToken(1024); + Token t1 = new Murmur3Partitioner.LongToken(-9223372036854775808L); + Token t2 = new Murmur3Partitioner.LongToken(-3074457345618258603L); + Token t3 = new Murmur3Partitioner.LongToken(3074457345618258602L); Set tokens = new HashSet<>(); tokens.add(t1); tokens.add(t2); @@ -89,25 +91,6 @@ public static void setupClass() throws Exception QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE)); } - private static void appendExpectedTokens(long left, long right, int numberOfSplits, List> expectedToken) - { - long repairTokenWidth = (right - left) / numberOfSplits; - for (int i = 0; i < numberOfSplits; i++) - { - long curLeft = left + (i * repairTokenWidth); - long curRight = curLeft + repairTokenWidth; - if ((i + 1) == numberOfSplits) - { - curRight = right; - } - Token childStartToken = ClusterMetadata.current() - .partitioner.getTokenFactory().fromString("" + curLeft); - Token childEndToken = ClusterMetadata.current() - .partitioner.getTokenFactory().fromString("" + curRight); - expectedToken.add(new Range<>(childStartToken, childEndToken)); - } - } - @Test public void testTokenRangesSplitByTable() { @@ -120,15 +103,24 @@ public void testTokenRangesSplitByTable() List> expectedToken = new ArrayList<>(); for (int i = 0; i < tables.size(); i++) { - appendExpectedTokens(1024, 0, numberOfSplits, expectedToken); - appendExpectedTokens(0, 256, numberOfSplits, expectedToken); - appendExpectedTokens(256, 1024, numberOfSplits, expectedToken); + for (Range range : tokens) + { + expectedToken.addAll(AutoRepairUtils.split(range, numberOfSplits)); + } } AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairSubRangeNum(repairType, numberOfSplits); - List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); - assertEquals(totalTokenRanges * numberOfSplits * tables.size(), assignments.size()); + Map> keyspaceToTables = new LinkedHashMap<>(); + keyspaceToTables.put(KEYSPACE, tables); + Map> assignmentsByKeyspace = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, keyspaceToTables); + + // should be 1 entry for the keyspace. + assertEquals(1, assignmentsByKeyspace.size()); + List assignments = assignmentsByKeyspace.get(KEYSPACE); + assertNotNull(assignments); + + assertEquals(totalTokenRanges*numberOfSplits*tables.size(), assignments.size()); assertEquals(expectedToken.size(), assignments.size()); int expectedTableIndex = -1; @@ -138,6 +130,15 @@ public void testTokenRangesSplitByTable() { expectedTableIndex++; } + } + + expectedTableIndex = -1; + for (int i = 0; i tables = Arrays.asList(TABLE1, TABLE2, TABLE3); List> expectedToken = new ArrayList<>(); - appendExpectedTokens(1024, 0, numberOfSplits, expectedToken); - appendExpectedTokens(0, 256, numberOfSplits, expectedToken); - appendExpectedTokens(256, 1024, numberOfSplits, expectedToken); + for (Range range : tokens) + { + expectedToken.addAll(AutoRepairUtils.split(range, numberOfSplits)); + } AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairSubRangeNum(repairType, numberOfSplits); - List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, tables); - assertEquals(totalTokenRanges * numberOfSplits, assignments.size()); + Map> keyspaceToTables = new LinkedHashMap<>(); + keyspaceToTables.put(KEYSPACE, tables); + Map> assignmentsByKeyspace = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, keyspaceToTables); + + // should be 1 entry for the keyspace. + assertEquals(1, assignmentsByKeyspace.size()); + List assignments = assignmentsByKeyspace.get(KEYSPACE); + assertNotNull(assignments); + + assertEquals(totalTokenRanges*numberOfSplits, assignments.size()); assertEquals(expectedToken.size(), assignments.size()); for (int i = 0; i < totalTokenRanges * numberOfSplits; i++) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 977de921e7d5..3ce7f39d580b 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -23,7 +23,9 @@ 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.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -72,6 +74,7 @@ import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; 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.mockito.ArgumentMatchers.any; @@ -454,6 +457,7 @@ public void testMetrics() AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setMVRepairEnabled(repairType, true); config.setRepairMinInterval(repairType, "0s"); + config.setRepairRetryBackoff("0s"); config.setAutoRepairTableMaxRepairTime(repairType, "0s"); AutoRepair.timeFunc = () -> { timeFuncCalls++; @@ -542,7 +546,16 @@ public void testTokenRangesNoSplit() assertEquals(1, tokens.size()); List> expectedToken = new ArrayList<>(tokens); - List assignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, KEYSPACE, Collections.singletonList(TABLE)); + Map> keyspaceToTables = new LinkedHashMap<>(); + keyspaceToTables.put(KEYSPACE, Collections.singletonList(TABLE)); + Map> assignmentsByKeyspace = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, keyspaceToTables); + + // should be 1 entry for the keyspace. + assertEquals(1, assignmentsByKeyspace.size()); + List assignments = assignmentsByKeyspace.get(KEYSPACE); + assertNotNull(assignments); + + // should be 1 entry for the table which covers the full range. assertEquals(1, assignments.size()); assertEquals(expectedToken.get(0).left, assignments.get(0).getTokenRange().left); assertEquals(expectedToken.get(0).right, assignments.get(0).getTokenRange().right); diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java index 6c91fbd539be..0f18d82eec3f 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java @@ -40,6 +40,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; import org.apache.cassandra.repair.autorepair.RepairRangeSplitter.SizedRepairAssignment; +import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.utils.concurrent.Refs; import static org.apache.cassandra.repair.autorepair.RepairRangeSplitter.TABLE_BATCH_LIMIT; @@ -56,6 +57,9 @@ public class RepairRangeSplitterTest extends CQLTester public static void setUpClass() { CQLTester.setUpClass(); + AutoRepairService.setup(); + // TODO: For now, always repair by keyspace to exercise priority tests + AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(AutoRepairConfig.RepairType.FULL, true); FULL_RANGE = new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), DatabaseDescriptor.getPartitioner().getMaximumToken()); } @@ -205,7 +209,7 @@ public void testGetRepairAssignmentsForTable_BatchingTables() throws Throwable { Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(3); - List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); // We expect two assignments, one with table1 and table2 batched, and one with table3 assertEquals(2, assignments.size()); @@ -219,7 +223,7 @@ public void testGetRepairAssignmentsForTable_BatchSize() throws Throwable { Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(2); - List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); // We expect one assignment, with two tables batched assertEquals(1, assignments.size()); @@ -232,7 +236,7 @@ public void testGetRepairAssignmentsForTable_NoBatching() throws Throwable { Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(3); - List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); assertEquals(3, assignments.size()); } @@ -243,7 +247,7 @@ public void testGetRepairAssignmentsForTable_AllBatched() throws Throwable { Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(5); - List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); assertEquals(1, assignments.size()); } From 0fc29852f9432f450ef5d2fcc687a4084e807fbb Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sat, 30 Nov 2024 15:17:46 -0600 Subject: [PATCH 085/115] Naming consistency getRepairAssignmentsByKeyspace -> getRepairAssignmentsForKeyspace getRepairAssignments -> getRepairAssignmentsForKeyspace --- .../repair/autorepair/DefaultAutoRepairTokenSplitter.java | 4 ++-- .../cassandra/repair/autorepair/RepairRangeSplitter.java | 4 ++-- .../repair/autorepair/RepairRangeSplitterTest.java | 8 ++++---- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java index 7a106411373e..05e6bafa3173 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java @@ -41,13 +41,13 @@ public Map> getRepairAssignments(AutoRepairConfig Map> repairAssignmentsByKeyspace = new LinkedHashMap<>(); for (Map.Entry> keyspace : keyspacesAndTablesToRepair.entrySet()) { - repairAssignmentsByKeyspace.put(keyspace.getKey(), getRepairAssignmentsByKeyspace(repairType, primaryRangeOnly, keyspace.getKey(), keyspace.getValue())); + repairAssignmentsByKeyspace.put(keyspace.getKey(), getRepairAssignmentsForKeyspace(repairType, primaryRangeOnly, keyspace.getKey(), keyspace.getValue())); } return repairAssignmentsByKeyspace; } - private List getRepairAssignmentsByKeyspace(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) + private List getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) { AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); List repairAssignments = new ArrayList<>(); diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index 00a7a9dc9e95..838e4ce407af 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -164,14 +164,14 @@ public Map> getRepairAssignments(AutoRepairConfig { logger.debug("Calculating token range splits for repairType={} primaryRangeOnly={} keyspaceName={} tableNames={}", repairType, primaryRangeOnly, keyspace.getKey(), keyspace.getValue()); Collection> tokenRanges = getTokenRanges(primaryRangeOnly, keyspace.getKey()); - repairAssignmentsByKeyspace.put(keyspace.getKey(), getRepairAssignments(repairType, keyspace.getKey(), keyspace.getValue(), tokenRanges)); + repairAssignmentsByKeyspace.put(keyspace.getKey(), getRepairAssignmentsForKeyspace(repairType, keyspace.getKey(), keyspace.getValue(), tokenRanges)); } return filterAndOrderRepairAssignments(repairType, repairAssignmentsByKeyspace); } @VisibleForTesting - List getRepairAssignments(AutoRepairConfig.RepairType repairType, String keyspaceName, List tableNames, Collection> tokenRanges) + List getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType repairType, String keyspaceName, List tableNames, Collection> tokenRanges) { List repairAssignments = new ArrayList<>(); // this is used for batching minimal single assignment tables together diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java index 0f18d82eec3f..606737817ac4 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java @@ -209,7 +209,7 @@ public void testGetRepairAssignmentsForTable_BatchingTables() throws Throwable { Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(3); - List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); // We expect two assignments, one with table1 and table2 batched, and one with table3 assertEquals(2, assignments.size()); @@ -223,7 +223,7 @@ public void testGetRepairAssignmentsForTable_BatchSize() throws Throwable { Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(2); - List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); // We expect one assignment, with two tables batched assertEquals(1, assignments.size()); @@ -236,7 +236,7 @@ public void testGetRepairAssignmentsForTable_NoBatching() throws Throwable { Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(3); - List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); assertEquals(3, assignments.size()); } @@ -247,7 +247,7 @@ public void testGetRepairAssignmentsForTable_AllBatched() throws Throwable { Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(5); - List assignments = repairRangeSplitter.getRepairAssignments(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); assertEquals(1, assignments.size()); } From 9ac5fa86b993c1e36fbe09d7dcf26c0efd12acbb Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sat, 30 Nov 2024 15:47:17 -0600 Subject: [PATCH 086/115] Use memtable size for memtable only --- .../apache/cassandra/repair/autorepair/RepairRangeSplitter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index 838e4ce407af..7dd2b5e1ff5f 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -394,7 +394,7 @@ protected List getRepairAssignments(List si if (memtableSize > 0L) { logger.debug("Included {}.{} range {}, had no unrepaired SSTables, but memtableSize={}, adding single repair assignment", estimate.keyspace, estimate.table, estimate.tokenRange, memtableSize); - SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, Collections.singletonList(estimate.table), "memtable only", estimate.sizeInRange); + SizedRepairAssignment assignment = new SizedRepairAssignment(estimate.tokenRange, estimate.keyspace, Collections.singletonList(estimate.table), "memtable only", memtableSize); repairAssignments.add(assignment); } else From 9a7037cb1e652263a8508a75731aaad3b8a2e648 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sun, 1 Dec 2024 18:48:43 -0600 Subject: [PATCH 087/115] Fix dtest --- .../distributed/test/repair/AutoRepairSchedulerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java index 3c180dbdd613..caf1ad7c3ea0 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/AutoRepairSchedulerTest.java @@ -60,14 +60,14 @@ public static void init() throws IOException .set("auto_repair", ImmutableMap.of( "repair_type_overrides", - ImmutableMap.of(AutoRepairConfig.RepairType.FULL.toString(), + ImmutableMap.of(AutoRepairConfig.RepairType.FULL.getConfigName(), ImmutableMap.of( "initial_scheduler_delay", "5s", "enabled", "true", "parallel_repair_count", "1", "parallel_repair_percentage", "0", "min_repair_interval", "1s"), - AutoRepairConfig.RepairType.INCREMENTAL.toString(), + AutoRepairConfig.RepairType.INCREMENTAL.getConfigName(), ImmutableMap.of( "initial_scheduler_delay", "5s", "enabled", "true", From def87fcd1928685f36f1058db246dcb4e2db88e2 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sun, 1 Dec 2024 19:37:52 -0600 Subject: [PATCH 088/115] Refactor IAutoRepairTokenRangeSplitter to be Iterator based Refactors IAutoRepairTokenRangeSplitter interface to be iterator based. This makes generation of RepairAssignments for priority buckets and keyspaces lazy instead of generating the RepairAssignments all at once. The main value here is that repairs can take some time, and it is beneficial to calculate repair assignments as close to when repairs would be submitted for a keyspace as possible. Additionally, introduced "PrioritizedRepairPlan" and "KeyspaceRepairPlan" types. Rather than passing around multiple-depth Maps where the contents aren't clear, create specialized types. Introduces PrioritizedRepairPlan.build for organizing a desired repair plan by grouping tables together that share the same priority. Moves RepairAssignment up to its own class to simplify the IAutoRepairTokenRangeSplitter. Since grouping tables by priority is now done in AutoRepair, the splitter implementations no longer have to reason with priority so all of this logic has been removed from the splitter code. Instead, they will now be given a List that is ordered by priority. --- .../repair/autorepair/AutoRepair.java | 302 +++++++++--------- .../DefaultAutoRepairTokenSplitter.java | 69 +++- .../IAutoRepairTokenRangeSplitter.java | 103 +----- .../autorepair/KeyspaceRepairAssignments.java | 53 +++ .../repair/autorepair/KeyspaceRepairPlan.java | 71 ++++ .../autorepair/PrioritizedRepairPlan.java | 160 ++++++++++ .../repair/autorepair/RepairAssignment.java | 85 +++++ .../autorepair/RepairRangeSplitter.java | 160 +++++++--- ...DefaultTokenSplitterParameterizedTest.java | 42 ++- .../AutoRepairParameterizedTest.java | 20 +- .../autorepair/PrioritizedRepairPlanTest.java | 125 ++++++++ .../autorepair/RepairRangeSplitterTest.java | 142 ++------ 12 files changed, 883 insertions(+), 449 deletions(-) create mode 100644 src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairAssignments.java create mode 100644 src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairPlan.java create mode 100644 src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java create mode 100644 src/java/org/apache/cassandra/repair/autorepair/RepairAssignment.java create mode 100644 test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 2ec92276ec2f..1a6ad2c7db27 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -41,7 +41,6 @@ import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.utils.Clock; -import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -157,28 +156,6 @@ private int getPriority(AutoRepairConfig.RepairType repairType, String keyspaceN return cfs != null ? cfs.metadata().params.automatedRepair.get(repairType).priority() : 0; } - /** - * @return A new map of keyspaces to tables sorted by the keyspace's underlying highest table priority. - */ - private Map> sortKeyspaceMapByTablePriority(AutoRepairConfig.RepairType repairType, Map> keyspacesAndTablesToRepair) - { - // Sort assignments by keyspace with tables with the highest priority. - List>> entriesOrderedByTablePriority = new ArrayList<>(keyspacesAndTablesToRepair.entrySet()); - entriesOrderedByTablePriority.sort((a, b) -> { - int aMaxPriority = a.getValue().stream().mapToInt((tableName) -> getPriority(repairType, a.getKey(), tableName)).max().orElse(0); - int bMaxPriority = b.getValue().stream().mapToInt((tableName) -> getPriority(repairType, b.getKey(), tableName)).max().orElse(0); - return bMaxPriority - aMaxPriority; - }); - - Map> sortedKeyspacesToRepair = new LinkedHashMap<>(); - for (Map.Entry> sortedEntries : entriesOrderedByTablePriority) - { - sortedKeyspacesToRepair.put(sortedEntries.getKey(), sortedEntries.getValue()); - } - - return sortedKeyspacesToRepair; - } - // repair runs a repair session of the given type synchronously. public void repair(AutoRepairConfig.RepairType repairType) { @@ -229,7 +206,7 @@ public void repair(AutoRepairConfig.RepairType repairType) repairState.setTotalTablesConsideredForRepair(0); repairState.setTotalMVTablesConsideredForRepair(0); - CollectectedRepairStats collectectedRepairStats = new CollectectedRepairStats(); + CollectedRepairStats collectedRepairStats = new CollectedRepairStats(); List keyspaces = new ArrayList<>(); Keyspace.all().forEach(keyspaces::add); @@ -245,136 +222,32 @@ public void repair(AutoRepairConfig.RepairType repairType) { continue; } - List tablesToBeRepairedList = retrieveTablesToBeRepaired(keyspace, config, repairType, repairState, collectectedRepairStats); + List tablesToBeRepairedList = retrieveTablesToBeRepaired(keyspace, config, repairType, repairState, collectedRepairStats); shuffleFunc.accept(tablesToBeRepairedList); keyspacesAndTablesToRepair.put(keyspace.getName(), tablesToBeRepairedList); } - // sort keyspaces to repair by table priority - Map> sortedKeyspacesToRepair = sortKeyspaceMapByTablePriority(repairType, keyspacesAndTablesToRepair); - // calculate the repair assignments for each kesypace. - Map> repairAssignmentsByKeyspace = tokenRangeSplitters.get(repairType).getRepairAssignments(repairType, primaryRangeOnly, sortedKeyspacesToRepair); + // Separate out the keyspaces and tables to repair based on their priority, with each repair plan representing a uniquely occuring priority. + List repairPlans = PrioritizedRepairPlan.build(keyspacesAndTablesToRepair, repairType, shuffleFunc); - // evaluate over each keyspace's repair assignments. - for (Map.Entry> keyspaceAssignments : repairAssignmentsByKeyspace.entrySet()) + // calculate the repair assignments for each priority:keyspace. + Iterator repairAssignmentsIterator = tokenRangeSplitters.get(repairType).getRepairAssignments(repairType, primaryRangeOnly, repairPlans); + + while (repairAssignmentsIterator.hasNext()) { - String keyspaceName = keyspaceAssignments.getKey(); - List repairAssignments = keyspaceAssignments.getValue(); - repairState.setRepairKeyspaceCount(repairState.getRepairKeyspaceCount() + 1); - - int totalRepairAssignments = repairAssignments.size(); - long keyspaceStartTime = timeFunc.get(); - RepairAssignment previousAssignment = null; - long tableStartTime = timeFunc.get(); - int totalProcessedAssignments = 0; - Set> ranges = new HashSet<>(); - for (RepairAssignment curRepairAssignment : repairAssignments) + KeyspaceRepairAssignments repairAssignments = repairAssignmentsIterator.next(); + List assignments = repairAssignments.getRepairAssignments(); + if (assignments.isEmpty()) { - try - { - totalProcessedAssignments++; - boolean repairOneTableAtATime = !config.getRepairByKeyspace(repairType); - if (previousAssignment != null && repairOneTableAtATime && !previousAssignment.tableNames.equals(curRepairAssignment.tableNames)) - { - // In the repair assignment, all the tables are appended sequnetially. - // Check if we have a different table, and if so, we should reset the table start time. - tableStartTime = timeFunc.get(); - } - previousAssignment = curRepairAssignment; - if (!config.isAutoRepairEnabled(repairType)) - { - logger.error("Auto-repair for type {} is disabled hence not running repair", repairType); - repairState.setRepairInProgress(false); - return; - } - if (AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, keyspaceStartTime, sortedKeyspacesToRepair.get(keyspaceName).size())) - { - collectectedRepairStats.skippedTokenRanges += totalRepairAssignments - totalProcessedAssignments; - logger.info("Keyspace took too much time to repair hence skipping it {}", - keyspaceName); - break; - } - if (repairOneTableAtATime && AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, tableStartTime)) - { - collectectedRepairStats.skippedTokenRanges += 1; - logger.info("Table took too much time to repair hence skipping it table name {}.{}, token range {}", - keyspaceName, curRepairAssignment.tableNames, curRepairAssignment.tokenRange); - continue; - } - - Range tokenRange = curRepairAssignment.getTokenRange(); - logger.debug("Current Token Left side {}, right side {}", - tokenRange.left.toString(), - tokenRange.right.toString()); - - ranges.add(curRepairAssignment.getTokenRange()); - if ((totalProcessedAssignments % config.getRepairThreads(repairType) == 0) || - (totalProcessedAssignments == totalRepairAssignments)) - { - int retryCount = 0; - Future f = null; - while (retryCount <= config.getRepairMaxRetries()) - { - RepairCoordinator task = repairState.getRepairRunnable(keyspaceName, - Lists.newArrayList(curRepairAssignment.getTableNames()), - ranges, primaryRangeOnly); - repairState.resetWaitCondition(); - f = repairRunnableExecutors.get(repairType).submit(task); - try - { - repairState.waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); - } - catch (InterruptedException e) - { - logger.error("Exception in cond await:", e); - } - if (repairState.isSuccess()) - { - break; - } - else if (retryCount < config.getRepairMaxRetries()) - { - boolean cancellationStatus = f.cancel(true); - logger.warn("Repair failed for range {}-{} for {} tables {} with cancellationStatus: {} retrying after {} seconds...", - tokenRange.left, tokenRange.right, - keyspaceName, curRepairAssignment.getTableNames(), - cancellationStatus, config.getRepairRetryBackoff().toSeconds()); - sleepFunc.accept(config.getRepairRetryBackoff().toSeconds(), TimeUnit.SECONDS); - } - retryCount++; - } - //check repair status - if (repairState.isSuccess()) - { - logger.info("Repair completed for range {}-{} for {} tables {}, total assignments: {}," + - "processed assignments: {}", tokenRange.left, tokenRange.right, - keyspaceName, curRepairAssignment.getTableNames(), totalRepairAssignments, totalProcessedAssignments); - collectectedRepairStats.succeededTokenRanges += ranges.size(); - } - else - { - boolean cancellationStatus = true; - if (f != null) - { - cancellationStatus = f.cancel(true); - } - //in the future we can add retry, etc. - logger.error("Repair failed for range {}-{} for {} tables {} after {} retries, total assignments: {}," + - "processed assignments: {}, cancellationStatus: {}", tokenRange.left, tokenRange.right, keyspaceName, - curRepairAssignment.getTableNames(), retryCount, totalRepairAssignments, totalProcessedAssignments, cancellationStatus); - collectectedRepairStats.failedTokenRanges += ranges.size(); - } - ranges.clear(); - } - logger.info("Repair completed for {} tables {}, range {}", keyspaceName, curRepairAssignment.getTableNames(), curRepairAssignment.getTokenRange()); - } - catch (Exception e) - { - logger.error("Exception while repairing keyspace {}:", keyspaceName, e); - } + logger.info("Skipping repairs for priorityBucket={} for keyspace={} since it yielded no assignments", repairAssignments.getPriority(), repairAssignments.getKeyspaceName()); + continue; } + + logger.info("Submitting repairs for priorityBucket={} for keyspace={} with assignmentCount={}", repairAssignments.getPriority(), repairAssignments.getKeyspaceName(), repairAssignments.getRepairAssignments().size()); + repairKeyspace(repairType, primaryRangeOnly, repairAssignments.getKeyspaceName(), repairAssignments.getRepairAssignments(), collectedRepairStats); } - cleanupAndUpdateStats(turn, repairType, repairState, myId, startTime, collectectedRepairStats); + + cleanupAndUpdateStats(turn, repairType, repairState, myId, startTime, collectedRepairStats); } else { @@ -387,6 +260,127 @@ else if (retryCount < config.getRepairMaxRetries()) } } + private void repairKeyspace(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List repairAssignments, CollectedRepairStats collectedRepairStats) + { + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + AutoRepairState repairState = repairStates.get(repairType); + + // evaluate over each keyspace's repair assignments. + repairState.setRepairKeyspaceCount(repairState.getRepairKeyspaceCount() + 1); + + int totalRepairAssignments = repairAssignments.size(); + long keyspaceStartTime = timeFunc.get(); + RepairAssignment previousAssignment = null; + long tableStartTime = timeFunc.get(); + int totalProcessedAssignments = 0; + Set> ranges = new HashSet<>(); + for (RepairAssignment curRepairAssignment : repairAssignments) + { + try + { + totalProcessedAssignments++; + boolean repairOneTableAtATime = !config.getRepairByKeyspace(repairType); + if (previousAssignment != null && repairOneTableAtATime && !previousAssignment.tableNames.equals(curRepairAssignment.tableNames)) + { + // In the repair assignment, all the tables are appended sequnetially. + // Check if we have a different table, and if so, we should reset the table start time. + tableStartTime = timeFunc.get(); + } + previousAssignment = curRepairAssignment; + if (!config.isAutoRepairEnabled(repairType)) + { + logger.error("Auto-repair for type {} is disabled hence not running repair", repairType); + repairState.setRepairInProgress(false); + return; + } + if (AutoRepairUtils.keyspaceMaxRepairTimeExceeded(repairType, keyspaceStartTime, repairAssignments.size())) + { + collectedRepairStats.skippedTokenRanges += totalRepairAssignments - totalProcessedAssignments; + logger.info("Keyspace took too much time to repair hence skipping it {}", + keyspaceName); + break; + } + if (repairOneTableAtATime && AutoRepairUtils.tableMaxRepairTimeExceeded(repairType, tableStartTime)) + { + collectedRepairStats.skippedTokenRanges += 1; + logger.info("Table took too much time to repair hence skipping it table name {}.{}, token range {}", + keyspaceName, curRepairAssignment.tableNames, curRepairAssignment.tokenRange); + continue; + } + + Range tokenRange = curRepairAssignment.getTokenRange(); + logger.debug("Current Token Left side {}, right side {}", + tokenRange.left.toString(), + tokenRange.right.toString()); + + ranges.add(curRepairAssignment.getTokenRange()); + if ((totalProcessedAssignments % config.getRepairThreads(repairType) == 0) || + (totalProcessedAssignments == totalRepairAssignments)) + { + int retryCount = 0; + Future f = null; + while (retryCount <= config.getRepairMaxRetries()) + { + RepairCoordinator task = repairState.getRepairRunnable(keyspaceName, + Lists.newArrayList(curRepairAssignment.getTableNames()), + ranges, primaryRangeOnly); + repairState.resetWaitCondition(); + f = repairRunnableExecutors.get(repairType).submit(task); + try + { + repairState.waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); + } + catch (InterruptedException e) + { + logger.error("Exception in cond await:", e); + } + if (repairState.isSuccess()) + { + break; + } + else if (retryCount < config.getRepairMaxRetries()) + { + boolean cancellationStatus = f.cancel(true); + logger.warn("Repair failed for range {}-{} for {} tables {} with cancellationStatus: {} retrying after {} seconds...", + tokenRange.left, tokenRange.right, + keyspaceName, curRepairAssignment.getTableNames(), + cancellationStatus, config.getRepairRetryBackoff().toSeconds()); + sleepFunc.accept(config.getRepairRetryBackoff().toSeconds(), TimeUnit.SECONDS); + } + retryCount++; + } + //check repair status + if (repairState.isSuccess()) + { + logger.info("Repair completed for range {}-{} for {} tables {}, total assignments: {}," + + "processed assignments: {}", tokenRange.left, tokenRange.right, + keyspaceName, curRepairAssignment.getTableNames(), totalRepairAssignments, totalProcessedAssignments); + collectedRepairStats.succeededTokenRanges += ranges.size(); + } + else + { + boolean cancellationStatus = true; + if (f != null) + { + cancellationStatus = f.cancel(true); + } + //in the future we can add retry, etc. + logger.error("Repair failed for range {}-{} for {} tables {} after {} retries, total assignments: {}," + + "processed assignments: {}, cancellationStatus: {}", tokenRange.left, tokenRange.right, keyspaceName, + curRepairAssignment.getTableNames(), retryCount, totalRepairAssignments, totalProcessedAssignments, cancellationStatus); + collectedRepairStats.failedTokenRanges += ranges.size(); + } + ranges.clear(); + } + logger.info("Repair completed for {} tables {}, range {}", keyspaceName, curRepairAssignment.getTableNames(), curRepairAssignment.getTokenRange()); + } + catch (Exception e) + { + logger.error("Exception while repairing keyspace {}:", keyspaceName, e); + } + } + } + private boolean tooSoonToRunRepair(AutoRepairConfig.RepairType repairType, AutoRepairState repairState, AutoRepairConfig config, UUID myId) { if (repairState.getLastRepairTime() == 0) @@ -409,7 +403,7 @@ private boolean tooSoonToRunRepair(AutoRepairConfig.RepairType repairType, AutoR return false; } - private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairConfig config, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, CollectectedRepairStats collectectedRepairStats) + private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairConfig config, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, CollectedRepairStats collectedRepairStats) { Tables tables = keyspace.getMetadata().tables; List tablesToBeRepaired = new ArrayList<>(); @@ -425,7 +419,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon { logger.info("Repair is disabled for keyspace {} for tables: {}", keyspace.getName(), tableName); repairState.setTotalDisabledTablesRepairCount(repairState.getTotalDisabledTablesRepairCount() + 1); - collectectedRepairStats.skippedTables++; + collectedRepairStats.skippedTables++; continue; } @@ -436,7 +430,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon { logger.info("Too many SSTables for repair for table {}.{}" + "totalSSTables {}", keyspace.getName(), tableName, totalSSTables); - collectectedRepairStats.skippedTables++; + collectedRepairStats.skippedTables++; continue; } @@ -454,7 +448,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon } private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType repairType, AutoRepairState repairState, UUID myId, - long startTime, CollectectedRepairStats collectectedRepairStats) throws InterruptedException + long startTime, CollectedRepairStats collectedRepairStats) throws InterruptedException { //if it was due to priority then remove it now if (turn == MY_TURN_DUE_TO_PRIORITY) @@ -463,10 +457,10 @@ private void cleanupAndUpdateStats(RepairTurn turn, AutoRepairConfig.RepairType AutoRepairUtils.removePriorityStatus(repairType, myId); } - repairState.setFailedTokenRangesCount(collectectedRepairStats.failedTokenRanges); - repairState.setSucceededTokenRangesCount(collectectedRepairStats.succeededTokenRanges); - repairState.setSkippedTokenRangesCount(collectectedRepairStats.skippedTokenRanges); - repairState.setSkippedTablesCount(collectectedRepairStats.skippedTables); + repairState.setFailedTokenRangesCount(collectedRepairStats.failedTokenRanges); + repairState.setSucceededTokenRangesCount(collectedRepairStats.succeededTokenRanges); + repairState.setSkippedTokenRangesCount(collectedRepairStats.skippedTokenRanges); + repairState.setSkippedTablesCount(collectedRepairStats.skippedTables); repairState.setNodeRepairTimeInSec((int) TimeUnit.MILLISECONDS.toSeconds(timeFunc.get() - startTime)); long timeInHours = TimeUnit.SECONDS.toHours(repairState.getNodeRepairTimeInSec()); logger.info("Local {} repair time {} hour(s), stats: repairKeyspaceCount {}, " + @@ -498,7 +492,7 @@ public AutoRepairState getRepairState(AutoRepairConfig.RepairType repairType) return repairStates.get(repairType); } - static class CollectectedRepairStats + static class CollectedRepairStats { int failedTokenRanges = 0; int succeededTokenRanges = 0; diff --git a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java index 05e6bafa3173..7bb1042b41c8 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/DefaultAutoRepairTokenSplitter.java @@ -21,9 +21,9 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.LinkedHashMap; +import java.util.Iterator; import java.util.List; -import java.util.Map; +import java.util.NoSuchElementException; import org.apache.cassandra.service.AutoRepairService; @@ -36,21 +36,70 @@ public class DefaultAutoRepairTokenSplitter implements IAutoRepairTokenRangeSplitter { @Override - public Map> getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, Map> keyspacesAndTablesToRepair) + public Iterator getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, List repairPlans) { - Map> repairAssignmentsByKeyspace = new LinkedHashMap<>(); - for (Map.Entry> keyspace : keyspacesAndTablesToRepair.entrySet()) + return new RepairAssignmentIterator(repairType, primaryRangeOnly, repairPlans); + } + + private class RepairAssignmentIterator implements Iterator + { + + private final AutoRepairConfig.RepairType repairType; + private final boolean primaryRangeOnly; + + private final Iterator repairPlanIterator; + + private Iterator currentIterator = null; + private PrioritizedRepairPlan currentPlan = null; + + RepairAssignmentIterator(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, List repairPlans) + { + this.repairType = repairType; + this.primaryRangeOnly = primaryRangeOnly; + this.repairPlanIterator = repairPlans.iterator(); + } + + private synchronized Iterator currentIterator() + { + if (currentIterator == null || !currentIterator.hasNext()) + { + // Advance the repair plan iterator if the current repair plan is exhausted, but only + // if there are more repair plans. + if (repairPlanIterator.hasNext()) + { + currentPlan = repairPlanIterator.next(); + currentIterator = currentPlan.getKeyspaceRepairPlans().iterator(); + } + } + + return currentIterator; + } + + @Override + public boolean hasNext() { - repairAssignmentsByKeyspace.put(keyspace.getKey(), getRepairAssignmentsForKeyspace(repairType, primaryRangeOnly, keyspace.getKey(), keyspace.getValue())); + return currentIterator().hasNext(); } - return repairAssignmentsByKeyspace; + @Override + public KeyspaceRepairAssignments next() + { + if (!currentIterator.hasNext()) + { + throw new NoSuchElementException("No remaining repair plans"); + } + + final KeyspaceRepairPlan repairPlan = currentIterator().next(); + return getRepairAssignmentsForKeyspace(repairType, primaryRangeOnly, currentPlan.getPriority(), repairPlan); + } } - private List getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, String keyspaceName, List tableNames) + private KeyspaceRepairAssignments getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, int priority, KeyspaceRepairPlan repairPlan) { AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); List repairAssignments = new ArrayList<>(); + String keyspaceName = repairPlan.getKeyspaceName(); + List tableNames = repairPlan.getTableNames(); Collection> tokens = StorageService.instance.getPrimaryRanges(keyspaceName); if (!primaryRangeOnly) @@ -87,7 +136,7 @@ private List getRepairAssignmentsForKeyspace(AutoRepairConfig. } } } - reorderByPriority(repairAssignments, repairType); - return repairAssignments; + + return new KeyspaceRepairAssignments(priority, keyspaceName, repairAssignments); } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java index d64916dcbd58..b51fb19ab460 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/IAutoRepairTokenRangeSplitter.java @@ -17,17 +17,8 @@ */ package org.apache.cassandra.repair.autorepair; - -import java.util.Comparator; +import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.Objects; - -import com.google.common.annotations.VisibleForTesting; - -import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.dht.Range; -import org.apache.cassandra.dht.Token; public interface IAutoRepairTokenRangeSplitter { @@ -37,94 +28,8 @@ public interface IAutoRepairTokenRangeSplitter * The autorepair framework will repair the list of returned subrange in a sequence. * @param repairType The type of repair being executed * @param primaryRangeOnly Whether to repair only this node's primary ranges or all of its ranges. - * @param keyspacesAndTablesToRepair A map keyed by keyspace name and valued by tables to generate assignments for. - * @return repair assignments broken up by range, keyspace and tables. - */ - Map> getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, Map> keyspacesAndTablesToRepair); - - /** - * Reorders the list of {@link RepairAssignment} objects based on their priority for a given repair type. - * The list is sorted in descending order, so higher priority assignments appear first. - * If two assignments have the same priority for the specified repair type, their original order is preserved. - * - * @param repairAssignments A list of {@link RepairAssignment} objects to be reordered. - * @param repairType The {@link AutoRepairConfig.RepairType} used to determine the priority of each assignment. - * The priority is determined using the {@link RepairAssignment#getPriority(AutoRepairConfig.RepairType)} method. + * @param repairPlans A list of ordered prioritized repair plans to generate assignments for in order. + * @return Iterator of repair assignments, with each element representing a grouping of repair assignments for a given keyspace. */ - @VisibleForTesting - default void reorderByPriority(List repairAssignments, AutoRepairConfig.RepairType repairType) - { - repairAssignments.sort(Comparator.comparingInt(a -> ((RepairAssignment) a).getPriority(repairType)).reversed()); - } - - /** - * Defines a repair assignment to be issued by the autorepair framework. - */ - class RepairAssignment - { - final Range tokenRange; - - final String keyspaceName; - - final List tableNames; - - public RepairAssignment(Range tokenRange, String keyspaceName, List tableNames) - { - this.tokenRange = tokenRange; - this.keyspaceName = keyspaceName; - this.tableNames = tableNames; - } - - public Range getTokenRange() - { - return tokenRange; - } - - public String getKeyspaceName() - { - return keyspaceName; - } - - public List getTableNames() - { - return tableNames; - } - - public int getPriority(AutoRepairConfig.RepairType type) - { - int max = 0; - for (String table : tableNames) - { - ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, table); - if (cfs != null) - max = Math.max(max, cfs.metadata().params.automatedRepair.get(type).priority()); - } - return max; - } - - @Override - public boolean equals(Object o) - { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - RepairAssignment that = (RepairAssignment) o; - return Objects.equals(tokenRange, that.tokenRange) && Objects.equals(keyspaceName, that.keyspaceName) && Objects.equals(tableNames, that.tableNames); - } - - @Override - public int hashCode() - { - return Objects.hash(tokenRange, keyspaceName, tableNames); - } - - @Override - public String toString() - { - return "RepairAssignment{" + - "tokenRange=" + tokenRange + - ", keyspaceName='" + keyspaceName + '\'' + - ", tableNames=" + tableNames + - '}'; - } - } + Iterator getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, List repairPlans); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairAssignments.java b/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairAssignments.java new file mode 100644 index 000000000000..3ea91e9922f9 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairAssignments.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.repair.autorepair; + +import java.util.List; + +/** + * A grouping of repair assignments that were generated for a particular keyspace for a given priority. + */ +public class KeyspaceRepairAssignments +{ + private final int priority; + private final String keyspaceName; + private final List repairAssignments; + + public KeyspaceRepairAssignments(int priority, String keyspaceName, List repairAssignments) + { + this.priority = priority; + this.keyspaceName = keyspaceName; + this.repairAssignments = repairAssignments; + } + + public int getPriority() + { + return priority; + } + + public String getKeyspaceName() + { + return keyspaceName; + } + + public List getRepairAssignments() + { + return repairAssignments; + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairPlan.java b/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairPlan.java new file mode 100644 index 000000000000..3c13e3d80d08 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/KeyspaceRepairPlan.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.repair.autorepair; + +import java.util.List; +import java.util.Objects; + +/** + * Encapsulates an intent to repair the given keyspace's tables + */ +public class KeyspaceRepairPlan +{ + private final String keyspaceName; + + private final List tableNames; + + public KeyspaceRepairPlan(String keyspaceName, List tableNames) + { + this.keyspaceName = keyspaceName; + this.tableNames = tableNames; + } + + public String getKeyspaceName() + { + return keyspaceName; + } + + public List getTableNames() + { + return tableNames; + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + KeyspaceRepairPlan that = (KeyspaceRepairPlan) o; + return Objects.equals(keyspaceName, that.keyspaceName) && Objects.equals(tableNames, that.tableNames); + } + + @Override + public int hashCode() + { + return Objects.hash(keyspaceName, tableNames); + } + + @Override + public String toString() + { + return "KeyspaceRepairPlan{" + + "keyspaceName='" + keyspaceName + '\'' + + ", tableNames=" + tableNames + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java b/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java new file mode 100644 index 000000000000..98cdd95421b1 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.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.repair.autorepair; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeSet; +import java.util.function.Consumer; + +import org.apache.cassandra.db.ColumnFamilyStore; + +/** + * Encapsulates a devised plan to repair tables, grouped by their keyspace and a given priority. This is used + * by {@link AutoRepair} to pass in an organized plan to + * {@link IAutoRepairTokenRangeSplitter#getRepairAssignments(AutoRepairConfig.RepairType, boolean, List)} which + * can iterate over this plan in order to generate {@link RepairAssignment}s. + */ +public class PrioritizedRepairPlan +{ + private final int priority; + + private final List keyspaceRepairPlans; + + public PrioritizedRepairPlan(int priority, List keyspaceRepairPlans) + { + this.priority = priority; + this.keyspaceRepairPlans = keyspaceRepairPlans; + } + + public int getPriority() + { + return priority; + } + + public List getKeyspaceRepairPlans() + { + return keyspaceRepairPlans; + } + + @Override + public boolean equals(Object o) + { + if (o == null || getClass() != o.getClass()) return false; + PrioritizedRepairPlan that = (PrioritizedRepairPlan) o; + return priority == that.priority && Objects.equals(keyspaceRepairPlans, that.keyspaceRepairPlans); + } + + @Override + public int hashCode() + { + return Objects.hash(priority, keyspaceRepairPlans); + } + + @Override + public String toString() + { + return "PrioritizedRepairPlan{" + + "priority=" + priority + + ", keyspaceRepairPlans=" + keyspaceRepairPlans + + '}'; + } + + /** + * Builds a list of {@link PrioritizedRepairPlan}s for the given keyspace and table map, ordered by priority from + * highest to lowest, where priority is derived from table schema's defined priority for the given repair type. + *

+ * If a keyspace has tables with differing priorities, those tables will be included in the PrioritizedRepairPlan + * for their given priority. + * + * @param keyspacesToTableNames A mapping keyspace to table names + * @param repairType The repair type that is being executed + * @param orderFunc A function to order keyspace and tables in the returned plan. + * @return Ordered list of plan's by table priorities. + */ + public static List build(Map> keyspacesToTableNames, AutoRepairConfig.RepairType repairType, Consumer> orderFunc) + { + // Build a map of priority -> (keyspace -> tables) + Map>> plans = new HashMap<>(); + for (Map.Entry> keyspaceToTableNames : keyspacesToTableNames.entrySet()) + { + String keyspaceName = keyspaceToTableNames.getKey(); + for (String tableName : keyspaceToTableNames.getValue()) + { + int priority = getPriority(repairType, keyspaceName, tableName); + Map> keyspacesForPriority = plans.computeIfAbsent(priority, p -> new HashMap<>()); + List tableNamesAtPriority = keyspacesForPriority.computeIfAbsent(keyspaceName, k -> new ArrayList<>()); + tableNamesAtPriority.add(tableName); + } + } + + // Extract map into a List ordered by priority from highest to lowest. + List planList = new ArrayList<>(plans.size()); + TreeSet priorities = new TreeSet<>(Comparator.reverseOrder()); + priorities.addAll(plans.keySet()); + for (int priority : priorities) + { + Map> keyspacesAndTables = plans.get(priority); + List keyspaceRepairPlans = new ArrayList<>(keyspacesAndTables.size()); + planList.add(new PrioritizedRepairPlan(priority, keyspaceRepairPlans)); + + // Order keyspace and table names based on the input function (typically, this would shuffle the keyspace + // and table names randomly). + List keyspaceNames = new ArrayList<>(keyspacesAndTables.keySet()); + orderFunc.accept(keyspaceNames); + + for(String keyspaceName : keyspaceNames) + { + List tableNames = keyspacesAndTables.get(keyspaceName); + orderFunc.accept(tableNames); + KeyspaceRepairPlan keyspaceRepairPlan = new KeyspaceRepairPlan(keyspaceName, new ArrayList<>(tableNames)); + keyspaceRepairPlans.add(keyspaceRepairPlan); + } + } + + return planList; + } + + /** + * Convenience method to build a repair plan for a single keyspace with tables. Primarily useful in testing. + * @param keyspaceName Keyspace to repair + * @param tableNames tables to repair for the given keyspace. + * @return Single repair plan. + */ + static List buildSingleKeyspacePlan(AutoRepairConfig.RepairType repairType, String keyspaceName, String ... tableNames) + { + Map> keyspaceMap = new HashMap<>(); + keyspaceMap.put(keyspaceName, Arrays.asList(tableNames)); + return build(keyspaceMap, repairType, (l) -> {}); + } + + /** + * @return The priority of the given table if defined, otherwise 0. + */ + private static int getPriority(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName) + { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); + return cfs != null ? cfs.metadata().params.automatedRepair.get(repairType).priority() : 0; + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairAssignment.java b/src/java/org/apache/cassandra/repair/autorepair/RepairAssignment.java new file mode 100644 index 000000000000..c06d58e71fa2 --- /dev/null +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairAssignment.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.repair.autorepair; + +import java.util.List; +import java.util.Objects; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; + +/** + * Defines a repair assignment to be issued by the autorepair framework. + */ +public class RepairAssignment +{ + final Range tokenRange; + + final String keyspaceName; + + final List tableNames; + + public RepairAssignment(Range tokenRange, String keyspaceName, List tableNames) + { + this.tokenRange = tokenRange; + this.keyspaceName = keyspaceName; + this.tableNames = tableNames; + } + + public Range getTokenRange() + { + return tokenRange; + } + + public String getKeyspaceName() + { + return keyspaceName; + } + + public List getTableNames() + { + return tableNames; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RepairAssignment that = (RepairAssignment) o; + return Objects.equals(tokenRange, that.tokenRange) && Objects.equals(keyspaceName, that.keyspaceName) && Objects.equals(tableNames, that.tableNames); + } + + @Override + public int hashCode() + { + return Objects.hash(tokenRange, keyspaceName, tableNames); + } + + @Override + public String toString() + { + return "RepairAssignment{" + + "tokenRange=" + tokenRange + + ", keyspaceName='" + keyspaceName + '\'' + + ", tableNames=" + tableNames + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index 7dd2b5e1ff5f..eb70389ba147 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -23,9 +23,10 @@ import java.util.Collection; import java.util.Collections; import java.util.HashSet; -import java.util.LinkedHashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -157,17 +158,68 @@ public RepairRangeSplitter(Map parameters) } @Override - public Map> getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, Map> keyspacesAndTablesToRepair) + public Iterator getRepairAssignments(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, List repairPlans) { - Map> repairAssignmentsByKeyspace = new LinkedHashMap<>(); - for (Map.Entry> keyspace : keyspacesAndTablesToRepair.entrySet()) + return new RepairAssignmentIterator(repairType, primaryRangeOnly, repairPlans); + } + + private class RepairAssignmentIterator implements Iterator + { + + private final AutoRepairConfig.RepairType repairType; + private final boolean primaryRangeOnly; + + private final Iterator repairPlanIterator; + + private Iterator currentIterator = null; + private PrioritizedRepairPlan currentPlan = null; + private long bytesSoFar = 0; + + RepairAssignmentIterator(AutoRepairConfig.RepairType repairType, boolean primaryRangeOnly, List repairPlans) { - logger.debug("Calculating token range splits for repairType={} primaryRangeOnly={} keyspaceName={} tableNames={}", repairType, primaryRangeOnly, keyspace.getKey(), keyspace.getValue()); - Collection> tokenRanges = getTokenRanges(primaryRangeOnly, keyspace.getKey()); - repairAssignmentsByKeyspace.put(keyspace.getKey(), getRepairAssignmentsForKeyspace(repairType, keyspace.getKey(), keyspace.getValue(), tokenRanges)); + this.repairType = repairType; + this.primaryRangeOnly = primaryRangeOnly; + this.repairPlanIterator = repairPlans.iterator(); } - return filterAndOrderRepairAssignments(repairType, repairAssignmentsByKeyspace); + private synchronized Iterator currentIterator() + { + if (currentIterator == null || !currentIterator.hasNext()) + { + // Advance the repair plan iterator if the current repair plan is exhausted, but only + // if there are more repair plans. + if (repairPlanIterator.hasNext()) + { + currentPlan = repairPlanIterator.next(); + currentIterator = currentPlan.getKeyspaceRepairPlans().iterator(); + } + } + + return currentIterator; + } + + @Override + public boolean hasNext() + { + return currentIterator().hasNext(); + } + + @Override + public KeyspaceRepairAssignments next() + { + // Should not happen unless violating the contract of iterator of checking hasNext first. + if (!currentIterator.hasNext()) + { + throw new NoSuchElementException("No remaining repair plans"); + } + + final KeyspaceRepairPlan repairPlan = currentIterator().next(); + Collection> tokenRanges = getTokenRanges(primaryRangeOnly, repairPlan.getKeyspaceName()); + List repairAssignments = getRepairAssignmentsForKeyspace(repairType, repairPlan.getKeyspaceName(), repairPlan.getTableNames(), tokenRanges); + FilteredRepairAssignments filteredRepairAssignments = filterRepairAssignments(repairType, currentPlan.getPriority(), repairPlan.getKeyspaceName(), repairAssignments, bytesSoFar); + bytesSoFar = filteredRepairAssignments.newBytesSoFar; + return new KeyspaceRepairAssignments(currentPlan.getPriority(), repairPlan.getKeyspaceName(), filteredRepairAssignments.repairAssignments); + } } @VisibleForTesting @@ -239,63 +291,49 @@ else if (tableAssignments.size() == 1 && } /** - * Given a repair type and map of sized-based repair assignments by keyspace order them by priority of the - * assignments' underlying tables and confine them by maxBytesPerSchedule. + * Given a repair type and map of sized-based repair assignments, confine them by maxBytesPerSchedule. * @param repairType used to determine underyling table priorities - * @param repairAssignmentsByKeyspace the assignments to filter. - * @return A list of repair assignments ordered by priority and confined by maxBytesPerSchedule. + * @param repairAssignments the assignments to filter. + * @param bytesSoFar repair assignment bytes accumulated so far. + * @return A list of repair assignments confined by maxBytesPerSchedule. */ @VisibleForTesting - Map> filterAndOrderRepairAssignments(AutoRepairConfig.RepairType repairType, Map> repairAssignmentsByKeyspace) + FilteredRepairAssignments filterRepairAssignments(AutoRepairConfig.RepairType repairType, int priority, String keyspaceName, List repairAssignments, long bytesSoFar) { // Confine repair assignments by maxBytesPerSchedule. - long bytesSoFar = 0L; + long bytesSoFarThisIteration = 0L; long bytesNotRepaired = 0L; int assignmentsNotRepaired = 0; int assignmentsToRepair = 0; int totalAssignments = 0; - Map> filteredRepairAssignmentsByKeyspace = new LinkedHashMap<>(); - for (Map.Entry> keyspaceAssignments : repairAssignmentsByKeyspace.entrySet()) + List assignmentsToReturn = new ArrayList<>(repairAssignments.size()); + for (SizedRepairAssignment repairAssignment : repairAssignments) { - String keyspace = keyspaceAssignments.getKey(); - List repairAssignments = keyspaceAssignments.getValue(); - - // Reorder the repair assignments - reorderByPriority(repairAssignments, repairType); - - List assignmentsToReturn = new ArrayList<>(repairAssignments.size()); - for (SizedRepairAssignment repairAssignment : repairAssignments) + totalAssignments++; + // skip any repair assignments that would accumulate us past the maxBytesPerSchedule + if (bytesSoFar + repairAssignment.getEstimatedBytes() > maxBytesPerSchedule) { - totalAssignments++; - // skip any repair assignments that would accumulate us past the maxBytesPerSchedule - if (bytesSoFar + repairAssignment.getEstimatedBytes() > maxBytesPerSchedule) - { - // log that repair assignment was skipped. - bytesNotRepaired += repairAssignment.getEstimatedBytes(); - assignmentsNotRepaired++; - logger.warn("Skipping {} because it would increase total repair bytes to {}", - repairAssignment, - getBytesOfMaxBytesPerSchedule(bytesSoFar + repairAssignment.getEstimatedBytes())); - } - else - { - bytesSoFar += repairAssignment.getEstimatedBytes(); - assignmentsToRepair++; - logger.info("Adding {}, increasing repair bytes to {}", - repairAssignment, - getBytesOfMaxBytesPerSchedule(bytesSoFar)); - assignmentsToReturn.add(repairAssignment); - } + // log that repair assignment was skipped. + bytesNotRepaired += repairAssignment.getEstimatedBytes(); + assignmentsNotRepaired++; + logger.warn("Skipping {} because it would increase total repair bytes to {}", + repairAssignment, + getBytesOfMaxBytesPerSchedule(bytesSoFar + repairAssignment.getEstimatedBytes())); } - - if (!assignmentsToReturn.isEmpty()) + else { - filteredRepairAssignmentsByKeyspace.put(keyspace, assignmentsToReturn); + bytesSoFar += repairAssignment.getEstimatedBytes(); + bytesSoFarThisIteration += repairAssignment.getEstimatedBytes(); + assignmentsToRepair++; + logger.info("Adding {}, increasing repair bytes to {}", + repairAssignment, + getBytesOfMaxBytesPerSchedule(bytesSoFar)); + assignmentsToReturn.add(repairAssignment); } } - String message = "Returning {} assignment(s), totaling {}"; + String message = "Returning {} assignment(s) for priorityBucket {} and keyspace {}, totaling {} ({} overall)"; if (assignmentsNotRepaired != 0) { message += ". Skipping {} of {} assignment(s), totaling {}"; @@ -304,23 +342,41 @@ Map> filterAndOrderRepairAssignments(AutoRepairCo message += ". The entire primary range will not be repaired this schedule. " + "Consider increasing maxBytesPerSchedule, reducing node density or monitoring to ensure " + "all ranges do get repaired within gc_grace_seconds"; - logger.warn(message, assignmentsToRepair, getBytesOfMaxBytesPerSchedule(bytesSoFar), + logger.warn(message, assignmentsToRepair, priority, keyspaceName, + FileUtils.stringifyFileSize(bytesSoFarThisIteration), + getBytesOfMaxBytesPerSchedule(bytesSoFar), assignmentsNotRepaired, totalAssignments, FileUtils.stringifyFileSize(bytesNotRepaired)); } else { - logger.info(message, assignmentsToRepair, getBytesOfMaxBytesPerSchedule(bytesSoFar), + logger.info(message, assignmentsToRepair, priority, keyspaceName, + FileUtils.stringifyFileSize(bytesSoFarThisIteration), + getBytesOfMaxBytesPerSchedule(bytesSoFar), assignmentsNotRepaired, totalAssignments, FileUtils.stringifyFileSize(bytesNotRepaired)); } } else { - logger.info(message, assignmentsToRepair, getBytesOfMaxBytesPerSchedule(bytesSoFar)); + logger.info(message, assignmentsToRepair, priority, keyspaceName, + FileUtils.stringifyFileSize(bytesSoFarThisIteration), + getBytesOfMaxBytesPerSchedule(bytesSoFar)); } - return filteredRepairAssignmentsByKeyspace; + return new FilteredRepairAssignments(assignmentsToReturn, bytesSoFar); + } + + private static class FilteredRepairAssignments + { + private final List repairAssignments; + private final long newBytesSoFar; + + private FilteredRepairAssignments(List repairAssignments, long newBytesSoFar) + { + this.repairAssignments = repairAssignments; + this.newBytesSoFar = newBytesSoFar; + } } private String getBytesOfMaxBytesPerSchedule(long bytes) { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java index 8cee64f5a238..086d8c114897 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairDefaultTokenSplitterParameterizedTest.java @@ -24,8 +24,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.LinkedHashMap; -import java.util.Map; +import java.util.Iterator; import org.junit.BeforeClass; import org.junit.Test; @@ -39,7 +38,6 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.index.sai.disk.format.Version; -import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.service.StorageService; @@ -47,7 +45,9 @@ import static org.apache.cassandra.cql3.CQLTester.Fuzzed.setupSeed; import static org.apache.cassandra.cql3.CQLTester.Fuzzed.updateConfigs; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; @RunWith(Parameterized.class) public class AutoRepairDefaultTokenSplitterParameterizedTest @@ -111,15 +111,17 @@ public void testTokenRangesSplitByTable() AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairSubRangeNum(repairType, numberOfSplits); - Map> keyspaceToTables = new LinkedHashMap<>(); - keyspaceToTables.put(KEYSPACE, tables); - Map> assignmentsByKeyspace = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, keyspaceToTables); - // should be 1 entry for the keyspace. - assertEquals(1, assignmentsByKeyspace.size()); - List assignments = assignmentsByKeyspace.get(KEYSPACE); - assertNotNull(assignments); + List plan = PrioritizedRepairPlan.buildSingleKeyspacePlan(repairType, KEYSPACE, TABLE1, TABLE2, TABLE3); + + Iterator keyspaceAssignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, plan); + + // should be only 1 entry for the keyspace. + assertTrue(keyspaceAssignments.hasNext()); + KeyspaceRepairAssignments keyspace = keyspaceAssignments.next(); + assertFalse(keyspaceAssignments.hasNext()); + List assignments = keyspace.getRepairAssignments(); assertEquals(totalTokenRanges*numberOfSplits*tables.size(), assignments.size()); assertEquals(expectedToken.size(), assignments.size()); @@ -133,6 +135,7 @@ public void testTokenRangesSplitByTable() } expectedTableIndex = -1; + // should be a set of ranges for each table. for (int i = 0; i> keyspaceToTables = new LinkedHashMap<>(); - keyspaceToTables.put(KEYSPACE, tables); - Map> assignmentsByKeyspace = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, keyspaceToTables); - // should be 1 entry for the keyspace. - assertEquals(1, assignmentsByKeyspace.size()); - List assignments = assignmentsByKeyspace.get(KEYSPACE); + List plan = PrioritizedRepairPlan.buildSingleKeyspacePlan(repairType, KEYSPACE, TABLE1, TABLE2, TABLE3); + + Iterator keyspaceAssignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, plan); + + // should be only 1 entry for the keyspace. + assertTrue(keyspaceAssignments.hasNext()); + KeyspaceRepairAssignments keyspace = keyspaceAssignments.next(); + assertFalse(keyspaceAssignments.hasNext()); + + List assignments = keyspace.getRepairAssignments(); assertNotNull(assignments); assertEquals(totalTokenRanges*numberOfSplits, assignments.size()); assertEquals(expectedToken.size(), assignments.size()); - for (int i = 0; i < totalTokenRanges * numberOfSplits; i++) + // should only be one set of ranges for the entire keyspace. + for (int i = 0; i> expectedToken = new ArrayList<>(tokens); - Map> keyspaceToTables = new LinkedHashMap<>(); - keyspaceToTables.put(KEYSPACE, Collections.singletonList(TABLE)); - Map> assignmentsByKeyspace = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, keyspaceToTables); + List plan = PrioritizedRepairPlan.buildSingleKeyspacePlan(repairType, KEYSPACE, TABLE); - // should be 1 entry for the keyspace. - assertEquals(1, assignmentsByKeyspace.size()); - List assignments = assignmentsByKeyspace.get(KEYSPACE); + Iterator keyspaceAssignments = new DefaultAutoRepairTokenSplitter().getRepairAssignments(repairType, true, plan); + + // should be only 1 entry for the keyspace. + assertTrue(keyspaceAssignments.hasNext()); + KeyspaceRepairAssignments keyspace = keyspaceAssignments.next(); + assertFalse(keyspaceAssignments.hasNext()); + + List assignments = keyspace.getRepairAssignments(); assertNotNull(assignments); // should be 1 entry for the table which covers the full range. diff --git a/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java b/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java new file mode 100644 index 000000000000..507d26437e36 --- /dev/null +++ b/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.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.repair.autorepair; + +import java.util.List; + +import org.junit.Test; + +import org.apache.cassandra.cql3.CQLTester; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class PrioritizedRepairPlanTest extends CQLTester +{ + + @Test + public void testBuildWithDifferentPriorities() + { + // Test reordering assignments with different priorities + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '3'}"); + String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); + + List prioritizedRepairPlans = PrioritizedRepairPlan.buildSingleKeyspacePlan(AutoRepairConfig.RepairType.FULL, KEYSPACE, table1, table2, table3); + assertEquals(3, prioritizedRepairPlans.size()); + + // Verify the order is by descending priority and matches the expected tables + assertEquals(3, prioritizedRepairPlans.get(0).getPriority()); + assertEquals(table2, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + + assertEquals(2, prioritizedRepairPlans.get(1).getPriority()); + assertEquals(table1, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + + assertEquals(1, prioritizedRepairPlans.get(2).getPriority()); + assertEquals(table3, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + } + + @Test + public void testBuildWithSamePriority() + { + // Test reordering assignments with the same priority + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + + // Expect only 1 plan since all tables share the same priority + List prioritizedRepairPlans = PrioritizedRepairPlan.buildSingleKeyspacePlan(AutoRepairConfig.RepairType.FULL, KEYSPACE, table1, table2, table3); + assertEquals(1, prioritizedRepairPlans.size()); + + // Verify all tables present in the plan + assertEquals(1, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().size()); + KeyspaceRepairPlan keyspaceRepairPlan = prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().get(0); + + List tableNames = keyspaceRepairPlan.getTableNames(); + assertEquals(3, tableNames.size()); + assertEquals(table1, tableNames.get(0)); + assertEquals(table2, tableNames.get(1)); + assertEquals(table3, tableNames.get(2)); + } + + @Test + public void testBuildWithMixedPriorities() + { + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '3'}"); + String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table4 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); + + // Expect only 3 plans + List prioritizedRepairPlans = PrioritizedRepairPlan.buildSingleKeyspacePlan(AutoRepairConfig.RepairType.FULL, KEYSPACE, table1, table2, table3, table4); + assertEquals(3, prioritizedRepairPlans.size()); + + // Verify the order is by descending priority and matches the expected tables + assertEquals(3, prioritizedRepairPlans.get(0).getPriority()); + assertEquals(table2, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + + assertEquals(2, prioritizedRepairPlans.get(1).getPriority()); + assertEquals(table1, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + assertEquals(table3, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().get(0).getTableNames().get(1)); + + assertEquals(1, prioritizedRepairPlans.get(2).getPriority()); + assertEquals(table4, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + } + + @Test + public void testBuildWithEmptyTableList() + { + // Test with an empty table list (should remain empty) + List prioritizedRepairPlans = PrioritizedRepairPlan.buildSingleKeyspacePlan(AutoRepairConfig.RepairType.FULL, KEYSPACE); + assertTrue(prioritizedRepairPlans.isEmpty()); + } + + @Test + public void testBuildWithOneTable() + { + // Test with a single element (should remain unchanged) + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '5'}"); + + // Expect only 1 plans + List prioritizedRepairPlans = PrioritizedRepairPlan.buildSingleKeyspacePlan(AutoRepairConfig.RepairType.FULL, KEYSPACE, table1); + assertEquals(1, prioritizedRepairPlans.size()); + + // Verify the order is by descending priority and matches the expected tables + assertEquals(5, prioritizedRepairPlans.get(0).getPriority()); + assertEquals(table1, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + } + +} diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java index 606737817ac4..b41da38ed641 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java @@ -38,7 +38,6 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.repair.autorepair.IAutoRepairTokenRangeSplitter.RepairAssignment; import org.apache.cassandra.repair.autorepair.RepairRangeSplitter.SizedRepairAssignment; import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.utils.concurrent.Refs; @@ -64,7 +63,8 @@ public static void setUpClass() } @Before - public void setUp() { + public void setUp() + { repairRangeSplitter = new RepairRangeSplitter(Collections.emptyMap()); tableName = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); } @@ -100,103 +100,16 @@ public void testSizePartitionCountSplit() throws Throwable } @Test - public void testReorderByPriorityWithDifferentPriorities() { - String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '3'}"); - String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); - - // Test reordering assignments with different priorities - RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table1)); - RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table2)); - RepairAssignment assignment3 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table3)); - - // Assume these priorities based on the repair type - List assignments = new ArrayList<>(Arrays.asList(assignment1, assignment2, assignment3)); - - repairRangeSplitter.reorderByPriority(assignments, AutoRepairConfig.RepairType.FULL); - - // Verify the order is by descending priority - assertEquals(assignment2, assignments.get(0)); - assertEquals(assignment1, assignments.get(1)); - assertEquals(assignment3, assignments.get(2)); - } - - @Test - public void testReorderByPriorityWithSamePriority() { - String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - - // Test reordering assignments with the same priority - RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table1)); - RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table2)); - RepairAssignment assignment3 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table3)); - - List assignments = new ArrayList<>(Arrays.asList(assignment1, assignment2, assignment3)); - - repairRangeSplitter.reorderByPriority(assignments, AutoRepairConfig.RepairType.FULL); - - // Verify the original order is preserved as all priorities are the same - assertEquals(assignment1, assignments.get(0)); - assertEquals(assignment2, assignments.get(1)); - assertEquals(assignment3, assignments.get(2)); - } - - @Test - public void testReorderByPriorityWithMixedPriorities() { - String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '3'}"); - String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table4 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); - - // Test reordering assignments with mixed priorities - RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table1)); - RepairAssignment assignment2 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table2)); - RepairAssignment assignment3 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table3)); - RepairAssignment assignment4 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table4)); - - List assignments = new ArrayList<>(Arrays.asList(assignment1, assignment2, assignment3, assignment4)); - - repairRangeSplitter.reorderByPriority(assignments, AutoRepairConfig.RepairType.FULL); - - // Verify the order: highest priority first, then preserved order for same priority - assertEquals(assignment2, assignments.get(0)); // Priority 3 - assertEquals(assignment1, assignments.get(1)); // Priority 2 - assertEquals(assignment3, assignments.get(2)); // Priority 2 - assertEquals(assignment4, assignments.get(3)); // Priority 1 - } - - @Test - public void testReorderByPriorityWithEmptyList() { - // Test with an empty list (should remain empty) - List assignments = new ArrayList<>(); - repairRangeSplitter.reorderByPriority(assignments, AutoRepairConfig.RepairType.FULL); - assertTrue(assignments.isEmpty()); - } - - @Test - public void testReorderByPriorityWithOneElement() { - // Test with a single element (should remain unchanged) - String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '5'}"); - - RepairAssignment assignment1 = new RepairAssignment(FULL_RANGE, KEYSPACE, Collections.singletonList(table1)); - - List assignments = new ArrayList<>(Collections.singletonList(assignment1)); - - repairRangeSplitter.reorderByPriority(assignments, AutoRepairConfig.RepairType.FULL); - - assertEquals(assignment1, assignments.get(0)); // Single element should remain in place - } - - @Test - public void testGetRepairAssignmentsForTable_NoSSTables() { + public void testGetRepairAssignmentsForTable_NoSSTables() + { Collection> ranges = Collections.singleton(new Range<>(Murmur3Partitioner.instance.getMinimumToken(), Murmur3Partitioner.instance.getMaximumToken())); List assignments = repairRangeSplitter.getRepairAssignmentsForTable(AutoRepairConfig.RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); assertEquals(0, assignments.size()); } @Test - public void testGetRepairAssignmentsForTable_Single() throws Throwable { + public void testGetRepairAssignmentsForTable_Single() throws Throwable + { Collection> ranges = Collections.singleton(new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), DatabaseDescriptor.getPartitioner().getMaximumToken())); insertAndFlushSingleTable(tableName); List assignments = repairRangeSplitter.getRepairAssignmentsForTable(AutoRepairConfig.RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); @@ -204,7 +117,8 @@ public void testGetRepairAssignmentsForTable_Single() throws Throwable { } @Test - public void testGetRepairAssignmentsForTable_BatchingTables() throws Throwable { + public void testGetRepairAssignmentsForTable_BatchingTables() throws Throwable + { repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "2")); Collection> ranges = Collections.singleton(FULL_RANGE); @@ -218,7 +132,8 @@ public void testGetRepairAssignmentsForTable_BatchingTables() throws Throwable { } @Test - public void testGetRepairAssignmentsForTable_BatchSize() throws Throwable { + public void testGetRepairAssignmentsForTable_BatchSize() throws Throwable + { repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "2")); Collection> ranges = Collections.singleton(FULL_RANGE); @@ -231,7 +146,8 @@ public void testGetRepairAssignmentsForTable_BatchSize() throws Throwable { } @Test - public void testGetRepairAssignmentsForTable_NoBatching() throws Throwable { + public void testGetRepairAssignmentsForTable_NoBatching() throws Throwable + { repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "1")); Collection> ranges = Collections.singleton(FULL_RANGE); @@ -242,7 +158,8 @@ public void testGetRepairAssignmentsForTable_NoBatching() throws Throwable { } @Test - public void testGetRepairAssignmentsForTable_AllBatched() throws Throwable { + public void testGetRepairAssignmentsForTable_AllBatched() throws Throwable + { repairRangeSplitter = new RepairRangeSplitter(Collections.singletonMap(TABLE_BATCH_LIMIT, "100")); Collection> ranges = Collections.singleton(FULL_RANGE); @@ -253,14 +170,16 @@ public void testGetRepairAssignmentsForTable_AllBatched() throws Throwable { } @Test(expected = IllegalStateException.class) - public void testMergeEmptyAssignments() { + public void testMergeEmptyAssignments() + { // Test when the list of assignments is empty List emptyAssignments = Collections.emptyList(); RepairRangeSplitter.merge(emptyAssignments); } @Test - public void testMergeSingleAssignment() { + public void testMergeSingleAssignment() + { // Test when there is only one assignment in the list String keyspaceName = "testKeyspace"; List tableNames = Arrays.asList("table1", "table2"); @@ -276,7 +195,8 @@ public void testMergeSingleAssignment() { } @Test - public void testMergeMultipleAssignmentsWithSameTokenRangeAndKeyspace() { + public void testMergeMultipleAssignmentsWithSameTokenRangeAndKeyspace() + { // Test merging multiple assignments with the same token range and keyspace String keyspaceName = "testKeyspace"; List tableNames1 = Arrays.asList("table1", "table2"); @@ -294,7 +214,8 @@ public void testMergeMultipleAssignmentsWithSameTokenRangeAndKeyspace() { } @Test(expected = IllegalStateException.class) - public void testMergeDifferentTokenRange() { + public void testMergeDifferentTokenRange() + { // Test merging assignments with different token ranges Iterator> range = AutoRepairUtils.split(FULL_RANGE, 2).iterator(); // Split the full range into two ranges ie (0-100, 100-200 Range tokenRange1 = range.next(); @@ -312,7 +233,8 @@ public void testMergeDifferentTokenRange() { } @Test(expected = IllegalStateException.class) - public void testMergeDifferentKeyspaceName() { + public void testMergeDifferentKeyspaceName() + { // Test merging assignments with different keyspace names List tableNames = Arrays.asList("table1", "table2"); @@ -324,7 +246,8 @@ public void testMergeDifferentKeyspaceName() { } @Test - public void testMergeWithDuplicateTables() { + public void testMergeWithDuplicateTables() + { // Test merging assignments with duplicate table names String keyspaceName = "testKeyspace"; List tableNames1 = Arrays.asList("table1", "table2"); @@ -341,12 +264,14 @@ public void testMergeWithDuplicateTables() { } - private void insertAndFlushSingleTable(String tableName) throws Throwable { + private void insertAndFlushSingleTable(String tableName) throws Throwable + { execute("INSERT INTO %s (k, v) values (?, ?)", 1, 1); flush(); } - private List createAndInsertTables(int count) throws Throwable { + private List createAndInsertTables(int count) throws Throwable + { List tableNames = new ArrayList<>(); for (int i = 0; i < count; i++) { String tableName = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); @@ -356,11 +281,14 @@ private List createAndInsertTables(int count) throws Throwable { return tableNames; } - private void insertAndFlushTable(String tableName) throws Throwable { + private void insertAndFlushTable(String tableName) throws Throwable + { insertAndFlushTable(tableName, 1); ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(KEYSPACE, tableName); } - private void insertAndFlushTable(String tableName, int... vals) throws Throwable { + + private void insertAndFlushTable(String tableName, int... vals) throws Throwable + { for (int i : vals) { executeFormattedQuery("INSERT INTO " + KEYSPACE + '.' + tableName + " (k, v) values (?, ?)", i, i); From 3ab7de60bdf6b6dc1d9dcffa66ccbeca17f8a26b Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sun, 1 Dec 2024 22:42:10 -0600 Subject: [PATCH 089/115] Add more priority and range splitter tests --- .../repair/autorepair/AutoRepair.java | 9 +-- .../autorepair/PrioritizedRepairPlan.java | 2 +- .../autorepair/RepairRangeSplitter.java | 7 +- .../AutoRepairParameterizedTest.java | 27 +++---- .../autorepair/PrioritizedRepairPlanTest.java | 54 +++++++++++-- .../autorepair/RepairRangeSplitterTest.java | 75 +++++++++++++++---- 6 files changed, 129 insertions(+), 45 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 1a6ad2c7db27..6dcbf4e78696 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -84,7 +84,9 @@ public class AutoRepair protected final Map repairStates; @VisibleForTesting - protected static Consumer> shuffleFunc = java.util.Collections::shuffle; + // Auto-repair is likely to be run on multiple nodes independently, we want to avoid running multiple repair + // sessions on overlapping datasets at the same time. Shuffling keyspaces reduces the likelihood of this happening. + protected static Consumer> shuffleFunc = java.util.Collections::shuffle; @VisibleForTesting protected static BiConsumer sleepFunc = Uninterruptibles::sleepUninterruptibly; @@ -210,10 +212,6 @@ public void repair(AutoRepairConfig.RepairType repairType) List keyspaces = new ArrayList<>(); Keyspace.all().forEach(keyspaces::add); - // Auto-repair is likely to be run on multiple nodes independently, we want to avoid running multiple repair - // sessions on overlapping datasets at the same time. Shuffling keyspaces reduces the likelihood of this happening. - shuffleFunc.accept(keyspaces); - // Filter out keyspaces and tables to repair and group into a map by keyspace. Map> keyspacesAndTablesToRepair = new LinkedHashMap<>(); for (Keyspace keyspace : keyspaces) @@ -223,7 +221,6 @@ public void repair(AutoRepairConfig.RepairType repairType) continue; } List tablesToBeRepairedList = retrieveTablesToBeRepaired(keyspace, config, repairType, repairState, collectedRepairStats); - shuffleFunc.accept(tablesToBeRepairedList); keyspacesAndTablesToRepair.put(keyspace.getName(), tablesToBeRepairedList); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java b/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java index 98cdd95421b1..6b10e5fe6b3a 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java +++ b/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java @@ -93,7 +93,7 @@ public String toString() * @param orderFunc A function to order keyspace and tables in the returned plan. * @return Ordered list of plan's by table priorities. */ - public static List build(Map> keyspacesToTableNames, AutoRepairConfig.RepairType repairType, Consumer> orderFunc) + public static List build(Map> keyspacesToTableNames, AutoRepairConfig.RepairType repairType, Consumer> orderFunc) { // Build a map of priority -> (keyspace -> tables) Map>> plans = new HashMap<>(); diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index eb70389ba147..895cde20542d 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -367,10 +367,11 @@ FilteredRepairAssignments filterRepairAssignments(AutoRepairConfig.RepairType re return new FilteredRepairAssignments(assignmentsToReturn, bytesSoFar); } - private static class FilteredRepairAssignments + @VisibleForTesting + static class FilteredRepairAssignments { - private final List repairAssignments; - private final long newBytesSoFar; + final List repairAssignments; + final long newBytesSoFar; private FilteredRepairAssignments(List repairAssignments, long newBytesSoFar) { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index eac3f64fd562..34b2da557fd5 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -71,6 +71,7 @@ import static org.apache.cassandra.repair.autorepair.AutoRepairUtils.RepairTurn.NOT_MY_TURN; 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; import static org.junit.Assert.fail; @@ -590,28 +591,28 @@ public void testRepairShufflesKeyspacesAndTables() { AtomicInteger shuffleKeyspacesCall = new AtomicInteger(); AtomicInteger shuffleTablesCall = new AtomicInteger(); - AutoRepair.shuffleFunc = (List list) -> { - if (!list.isEmpty()) + AtomicInteger keyspaceCount = new AtomicInteger(); + AutoRepair.shuffleFunc = (List list) -> { + // check whether was invoked for keyspaces or tables + if (list.contains(KEYSPACE)) { - assertTrue(list.get(0) instanceof Keyspace || list.get(0) instanceof String); - if (list.get(0) instanceof Keyspace) - { - shuffleKeyspacesCall.getAndIncrement(); - assertFalse(list.isEmpty()); - } - else if (list.get(0) instanceof String) - { - shuffleTablesCall.getAndIncrement(); - } + shuffleKeyspacesCall.getAndIncrement(); + keyspaceCount.set(list.size()); } + else + // presume list not containing a keyspace is for tables. + shuffleTablesCall.getAndIncrement(); }; AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairMinInterval(repairType, "0s"); AutoRepair.instance.repair(repairType); + // Expect a single invocation for keyspaces assertEquals(1, shuffleKeyspacesCall.get()); - assertEquals(5, shuffleTablesCall.get()); + // Expect an invocation for tables for each keyspace + assertNotEquals(0, keyspaceCount.get()); + assertEquals(keyspaceCount.get(), shuffleTablesCall.get()); } @Test diff --git a/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java b/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java index 507d26437e36..011df9d1592e 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java @@ -18,8 +18,11 @@ package org.apache.cassandra.repair.autorepair; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import com.google.common.collect.Lists; import org.junit.Test; import org.apache.cassandra.cql3.CQLTester; @@ -78,25 +81,60 @@ public void testBuildWithSamePriority() @Test public void testBuildWithMixedPriorities() { - String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '3'}"); - String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table4 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); - - // Expect only 3 plans - List prioritizedRepairPlans = PrioritizedRepairPlan.buildSingleKeyspacePlan(AutoRepairConfig.RepairType.FULL, KEYSPACE, table1, table2, table3, table4); - assertEquals(3, prioritizedRepairPlans.size()); + String ks1 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"); + String table1 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table2 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '3'}"); + String table3 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table4 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); + // No priority table should be bucketed at priority 0 + String table5 = createTable(ks1,"CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); + + // Create a new keyspace to ensure its tables get grouped with appropriate priority bucket + String ks2 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"); + String table6 = createTable(ks2,"CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); + String table7 = createTable(ks2,"CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); + + Map> keyspaceToTableMap = new HashMap<>(); + keyspaceToTableMap.put(ks1, Lists.newArrayList(table1, table2, table3, table4, table5)); + keyspaceToTableMap.put(ks2, Lists.newArrayList(table6, table7)); + + // Expect 4 plans + List prioritizedRepairPlans = PrioritizedRepairPlan.build(keyspaceToTableMap, AutoRepairConfig.RepairType.FULL, java.util.Collections::sort); + assertEquals(4, prioritizedRepairPlans.size()); // Verify the order is by descending priority and matches the expected tables assertEquals(3, prioritizedRepairPlans.get(0).getPriority()); + assertEquals(1, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().size()); + assertEquals(ks1, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().get(0).getKeyspaceName()); assertEquals(table2, prioritizedRepairPlans.get(0).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); assertEquals(2, prioritizedRepairPlans.get(1).getPriority()); + assertEquals(1, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().size()); + + assertEquals(ks1, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().get(0).getKeyspaceName()); assertEquals(table1, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); assertEquals(table3, prioritizedRepairPlans.get(1).getKeyspaceRepairPlans().get(0).getTableNames().get(1)); assertEquals(1, prioritizedRepairPlans.get(2).getPriority()); + // 2 keyspaces should be present at priority 1 + assertEquals(2, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().size()); + // ks1.table4 expected in first plan + assertEquals(ks1, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().get(0).getKeyspaceName()); assertEquals(table4, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + // ks2.table7 expected in second plan + assertEquals(ks2, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().get(1).getKeyspaceName()); + assertEquals(table7, prioritizedRepairPlans.get(2).getKeyspaceRepairPlans().get(1).getTableNames().get(0)); + + // Tables without priority should get bucketed at priority 0 + assertEquals(0, prioritizedRepairPlans.get(3).getPriority()); + // 2 keyspaces expected + assertEquals(2, prioritizedRepairPlans.get(3).getKeyspaceRepairPlans().size()); + // ks1.table5 expected in first plan + assertEquals(ks1, prioritizedRepairPlans.get(3).getKeyspaceRepairPlans().get(0).getKeyspaceName()); + assertEquals(table5, prioritizedRepairPlans.get(3).getKeyspaceRepairPlans().get(0).getTableNames().get(0)); + // ks2.table6 expected in second plan + assertEquals(ks2, prioritizedRepairPlans.get(3).getKeyspaceRepairPlans().get(1).getKeyspaceName()); + assertEquals(table6, prioritizedRepairPlans.get(3).getKeyspaceRepairPlans().get(1).getTableNames().get(0)); } @Test diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java index b41da38ed641..9baaecca545e 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java @@ -22,15 +22,18 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.apache.cassandra.config.DataStorageSpec.LongMebibytesBound; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.ColumnFamilyStore; @@ -38,10 +41,15 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; +import org.apache.cassandra.repair.autorepair.RepairRangeSplitter.FilteredRepairAssignments; +import org.apache.cassandra.repair.autorepair.RepairRangeSplitter.SizeEstimate; import org.apache.cassandra.repair.autorepair.RepairRangeSplitter.SizedRepairAssignment; import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.utils.concurrent.Refs; +import static org.apache.cassandra.repair.autorepair.RepairRangeSplitter.MAX_BYTES_PER_SCHEDULE; +import static org.apache.cassandra.repair.autorepair.RepairRangeSplitter.SUBRANGE_SIZE; import static org.apache.cassandra.repair.autorepair.RepairRangeSplitter.TABLE_BATCH_LIMIT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -57,8 +65,7 @@ public static void setUpClass() { CQLTester.setUpClass(); AutoRepairService.setup(); - // TODO: For now, always repair by keyspace to exercise priority tests - AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(AutoRepairConfig.RepairType.FULL, true); + AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(RepairType.FULL, true); FULL_RANGE = new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), DatabaseDescriptor.getPartitioner().getMaximumToken()); } @@ -73,9 +80,9 @@ public void setUp() public void testSizePartitionCount() throws Throwable { insertAndFlushTable(tableName, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); - Refs sstables = RepairRangeSplitter.getSSTableReaderRefs(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, FULL_RANGE); + Refs sstables = RepairRangeSplitter.getSSTableReaderRefs(RepairType.FULL, KEYSPACE, tableName, FULL_RANGE); assertEquals(10, sstables.iterator().next().getEstimatedPartitionSize().count()); - RepairRangeSplitter.SizeEstimate sizes = RepairRangeSplitter.getSizesForRangeOfSSTables(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, FULL_RANGE, sstables); + SizeEstimate sizes = RepairRangeSplitter.getSizesForRangeOfSSTables(RepairType.FULL, KEYSPACE, tableName, FULL_RANGE, sstables); assertEquals(10, sizes.partitions); } @@ -91,10 +98,10 @@ public void testSizePartitionCountSplit() throws Throwable Range tokenRange2 = range.next(); Assert.assertFalse(range.hasNext()); - Refs sstables1 = RepairRangeSplitter.getSSTableReaderRefs(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, tokenRange1); - Refs sstables2 = RepairRangeSplitter.getSSTableReaderRefs(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, tokenRange2); - RepairRangeSplitter.SizeEstimate sizes1 = RepairRangeSplitter.getSizesForRangeOfSSTables(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, tokenRange1, sstables1); - RepairRangeSplitter.SizeEstimate sizes2 = RepairRangeSplitter.getSizesForRangeOfSSTables(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableName, tokenRange2, sstables2); + Refs sstables1 = RepairRangeSplitter.getSSTableReaderRefs(RepairType.FULL, KEYSPACE, tableName, tokenRange1); + Refs sstables2 = RepairRangeSplitter.getSSTableReaderRefs(RepairType.FULL, KEYSPACE, tableName, tokenRange2); + SizeEstimate sizes1 = RepairRangeSplitter.getSizesForRangeOfSSTables(RepairType.FULL, KEYSPACE, tableName, tokenRange1, sstables1); + SizeEstimate sizes2 = RepairRangeSplitter.getSizesForRangeOfSSTables(RepairType.FULL, KEYSPACE, tableName, tokenRange2, sstables2); // +-5% because HLL merge and the applying of range size approx ratio causes estimation errors assertTrue(Math.abs(10000 - (sizes1.partitions + sizes2.partitions)) <= 60); } @@ -103,7 +110,7 @@ public void testSizePartitionCountSplit() throws Throwable public void testGetRepairAssignmentsForTable_NoSSTables() { Collection> ranges = Collections.singleton(new Range<>(Murmur3Partitioner.instance.getMinimumToken(), Murmur3Partitioner.instance.getMaximumToken())); - List assignments = repairRangeSplitter.getRepairAssignmentsForTable(AutoRepairConfig.RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForTable(RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); assertEquals(0, assignments.size()); } @@ -112,7 +119,7 @@ public void testGetRepairAssignmentsForTable_Single() throws Throwable { Collection> ranges = Collections.singleton(new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), DatabaseDescriptor.getPartitioner().getMaximumToken())); insertAndFlushSingleTable(tableName); - List assignments = repairRangeSplitter.getRepairAssignmentsForTable(AutoRepairConfig.RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForTable(RepairType.FULL, CQLTester.KEYSPACE, tableName, ranges); assertEquals(1, assignments.size()); } @@ -123,7 +130,7 @@ public void testGetRepairAssignmentsForTable_BatchingTables() throws Throwable Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(3); - List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, KEYSPACE, tableNames, ranges); // We expect two assignments, one with table1 and table2 batched, and one with table3 assertEquals(2, assignments.size()); @@ -138,7 +145,7 @@ public void testGetRepairAssignmentsForTable_BatchSize() throws Throwable Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(2); - List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, KEYSPACE, tableNames, ranges); // We expect one assignment, with two tables batched assertEquals(1, assignments.size()); @@ -152,7 +159,7 @@ public void testGetRepairAssignmentsForTable_NoBatching() throws Throwable Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(3); - List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, KEYSPACE, tableNames, ranges); assertEquals(3, assignments.size()); } @@ -164,7 +171,7 @@ public void testGetRepairAssignmentsForTable_AllBatched() throws Throwable Collection> ranges = Collections.singleton(FULL_RANGE); List tableNames = createAndInsertTables(5); - List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(AutoRepairConfig.RepairType.FULL, KEYSPACE, tableNames, ranges); + List assignments = repairRangeSplitter.getRepairAssignmentsForKeyspace(RepairType.FULL, KEYSPACE, tableNames, ranges); assertEquals(1, assignments.size()); } @@ -263,6 +270,46 @@ public void testMergeWithDuplicateTables() assertEquals(new HashSet<>(Arrays.asList("table1", "table2", "table3")), new HashSet<>(result.getTableNames())); } + @Test + public void testGetRepairAssignmentsSplitsBySubrangeSizeAndFilterLimitsByMaxBytesPerSchedule() + { + // Ensures that getRepairAssignments splits by SUBRANGE_SIZE and filterRepairAssignments limits by MAX_BYTES_PER_SCHEDULE. + Map parameters = new HashMap<>(); + parameters.put(SUBRANGE_SIZE, "50GiB"); + parameters.put(MAX_BYTES_PER_SCHEDULE, "100GiB"); + repairRangeSplitter = new RepairRangeSplitter(parameters); + + // Given a size estimate of 1024GiB, we should expect 21 splits (50GiB*21 = 1050GiB < 1024GiB) + SizeEstimate sizeEstimate = sizeEstimateByBytes(RepairType.INCREMENTAL, new LongMebibytesBound("1024GiB")); + + List assignments = repairRangeSplitter.getRepairAssignments(Collections.singletonList(sizeEstimate)); + + // Should be 21 assignments, each being ~48.76 GiB + assertEquals(21, assignments.size()); + long expectedBytes = 52357696560L; + for (int i = 0; i < assignments.size(); i++) + { + SizedRepairAssignment assignment = assignments.get(i); + assertEquals("Did not get expected value for assignment " + i, 52357696560L, assignment.getEstimatedBytes()); + } + + // When filtering we should only get 2 assignments back (48.76 * 2 < 100GiB) + FilteredRepairAssignments filteredRepairAssignments = repairRangeSplitter.filterRepairAssignments(RepairType.INCREMENTAL, 0, KEYSPACE, assignments, 0); + List finalRepairAssignments = filteredRepairAssignments.repairAssignments; + assertEquals(2, finalRepairAssignments.size()); + assertEquals(expectedBytes*2, filteredRepairAssignments.newBytesSoFar); + } + + private SizeEstimate sizeEstimateByBytes(RepairType repairType, LongMebibytesBound totalSize) + { + return sizeEstimateByBytes(repairType, totalSize, totalSize); + } + + private SizeEstimate sizeEstimateByBytes(RepairType repairType, LongMebibytesBound sizeInRange, LongMebibytesBound totalSize) + { + return new SizeEstimate(repairType, KEYSPACE, "table1", FULL_RANGE, 1, sizeInRange.toBytes(), totalSize.toBytes()); + } + private void insertAndFlushSingleTable(String tableName) throws Throwable { From 4c6c23fe0cb7b3f9ac9c1a739d07d5e19b08a716 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 10 Dec 2024 11:45:02 -0800 Subject: [PATCH 090/115] Refactor a missed change during the rebase --- .../repair/autorepair/AutoRepairParameterizedTest.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 34b2da557fd5..5a5e66a3db15 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -576,13 +576,9 @@ public void testDefaultAutomatedRepair() for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { Assert.assertTrue(String.format("expected repair type %s to be enabled on table %s", repairType, cfm.name), - cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.FULL).repairEnabled()); - Assert.assertTrue(String.format("expected repair type %s to be enabled on table %s", repairType, cfm.name), - cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).repairEnabled()); - Assert.assertFalse(String.format("expected repair type %s to be disabled on table %s", repairType, cfmDisabledAutoRepair.name), - cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.FULL).repairEnabled()); + cfm.params.automatedRepair.get(repairType).repairEnabled()); Assert.assertFalse(String.format("expected repair type %s to be disabled on table %s", repairType, cfmDisabledAutoRepair.name), - cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).repairEnabled()); + cfmDisabledAutoRepair.params.automatedRepair.get(repairType).repairEnabled()); } } From bf0481ce64af978523062ebc5b517abe3da67674 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Fri, 6 Dec 2024 20:38:23 -0600 Subject: [PATCH 091/115] Set thread name based on RepairType.getConfigName() Changing to upper-case enums caused the executor name to be: AutoRepair-Repair-FULL instead of AutoRepair-Repair-full --- .../org/apache/cassandra/repair/autorepair/AutoRepair.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 6dcbf4e78696..bb64e8f48463 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -104,8 +104,8 @@ protected AutoRepair() repairStates = new EnumMap<>(AutoRepairConfig.RepairType.class); for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { - repairExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-Repair-" + repairType, Thread.NORM_PRIORITY)); - repairRunnableExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-RepairRunnable-" + repairType, Thread.NORM_PRIORITY)); + repairExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-Repair-" + repairType.getConfigName(), Thread.NORM_PRIORITY)); + repairRunnableExecutors.put(repairType, executorFactory().scheduled(false, "AutoRepair-RepairRunnable-" + repairType.getConfigName(), Thread.NORM_PRIORITY)); repairStates.put(repairType, AutoRepairConfig.RepairType.getAutoRepairState(repairType)); tokenRangeSplitters.put(repairType, FBUtilities.newAutoRepairTokenRangeSplitter(config.getTokenRangeSplitter(repairType))); } From 0103a1af5396156cc8f9042fe80b8256623be528 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Sun, 8 Dec 2024 14:01:56 -0800 Subject: [PATCH 092/115] Flatten the auto_repair table property --- pylib/cqlshlib/test/test_cqlsh_output.py | 3 +- .../statements/schema/TableAttributes.java | 10 +- .../repair/autorepair/AutoRepair.java | 4 +- .../autorepair/PrioritizedRepairPlan.java | 2 +- .../cassandra/schema/AutoRepairParams.java | 104 +++++++++++------- .../cassandra/schema/SchemaKeyspace.java | 18 +-- .../apache/cassandra/schema/TableParams.java | 62 ++--------- .../statements/DescribeStatementTest.java | 8 +- .../cassandra/db/SchemaCQLHelperTest.java | 4 +- .../AutoRepairParameterizedTest.java | 18 ++- .../autorepair/PrioritizedRepairPlanTest.java | 24 ++-- 11 files changed, 112 insertions(+), 145 deletions(-) diff --git a/pylib/cqlshlib/test/test_cqlsh_output.py b/pylib/cqlshlib/test/test_cqlsh_output.py index 183df2b0a14a..cbeedb7a4412 100644 --- a/pylib/cqlshlib/test/test_cqlsh_output.py +++ b/pylib/cqlshlib/test/test_cqlsh_output.py @@ -699,8 +699,7 @@ def test_describe_columnfamily_output(self): AND min_index_interval = 128 AND read_repair = 'BLOCKING' AND speculative_retry = '99p' - AND repair_full = {'enabled': 'true'} - AND repair_incremental = {'enabled': 'true'};""" % quote_name(get_keyspace())) + AND auto_repair = {'full_enabled': 'true', 'incremental_enabled': 'true', 'preview_repaired_enabled': 'true', 'priority': '0'};""" % quote_name(get_keyspace())) with cqlsh_testrun(tty=True, env=self.default_env) as c: for cmdword in ('describe table', 'desc columnfamily'): diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index 491a0f6b3da0..7b0e18b304fd 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -153,14 +153,8 @@ private TableParams build(TableParams.Builder builder) if (hasOption(READ_REPAIR)) builder.readRepair(ReadRepairStrategy.fromString(getString(READ_REPAIR))); - if (hasOption(Option.REPAIR_FULL)) - builder.automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.FULL, getMap(Option.REPAIR_FULL))); - - if (hasOption(Option.REPAIR_INCREMENTAL)) - builder.automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.INCREMENTAL, getMap(Option.REPAIR_INCREMENTAL))); - - if (hasOption(Option.REPAIR_PREVIEW_REPAIRED)) - builder.automatedRepairPreviewRepaired(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.PREVIEW_REPAIRED, getMap(Option.REPAIR_PREVIEW_REPAIRED))); + if (hasOption(Option.AUTO_REPAIR)) + builder.automatedRepair(AutoRepairParams.fromMap(getMap(Option.AUTO_REPAIR))); return builder.build(); } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index bb64e8f48463..eafd40eaddf7 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -155,7 +155,7 @@ public void repairAsync(AutoRepairConfig.RepairType repairType) private int getPriority(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName) { ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); - return cfs != null ? cfs.metadata().params.automatedRepair.get(repairType).priority() : 0; + return cfs != null ? cfs.metadata().params.autoRepair.priority() : 0; } // repair runs a repair session of the given type synchronously. @@ -412,7 +412,7 @@ private List retrieveTablesToBeRepaired(Keyspace keyspace, AutoRepairCon String tableName = tableMetadata.name; ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName); - if (!columnFamilyStore.metadata().params.automatedRepair.get(repairType).repairEnabled()) + if (!columnFamilyStore.metadata().params.autoRepair.repairEnabled(repairType)) { logger.info("Repair is disabled for keyspace {} for tables: {}", keyspace.getName(), tableName); repairState.setTotalDisabledTablesRepairCount(repairState.getTotalDisabledTablesRepairCount() + 1); diff --git a/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java b/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java index 6b10e5fe6b3a..857fbe19b406 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java +++ b/src/java/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlan.java @@ -155,6 +155,6 @@ static List buildSingleKeyspacePlan(AutoRepairConfig.Repa private static int getPriority(AutoRepairConfig.RepairType repairType, String keyspaceName, String tableName) { ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); - return cfs != null ? cfs.metadata().params.automatedRepair.get(repairType).priority() : 0; + return cfs != null ? cfs.metadata().params.autoRepair.priority() : 0; } } diff --git a/src/java/org/apache/cassandra/schema/AutoRepairParams.java b/src/java/org/apache/cassandra/schema/AutoRepairParams.java index feadbe4e227e..2b289a8c3b44 100644 --- a/src/java/org/apache/cassandra/schema/AutoRepairParams.java +++ b/src/java/org/apache/cassandra/schema/AutoRepairParams.java @@ -18,10 +18,9 @@ package org.apache.cassandra.schema; import java.util.Arrays; -import java.util.EnumMap; -import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.TreeMap; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableMap; @@ -37,7 +36,9 @@ public final class AutoRepairParams { public enum Option { - ENABLED, + FULL_ENABLED, + INCREMENTAL_ENABLED, + PREVIEW_REPAIRED_ENABLED, PRIORITY; @Override @@ -47,31 +48,29 @@ public String toString() } } - public static final Map DEFAULT_SUB_OPTIONS = ImmutableMap.of( - Option.ENABLED.toString(), Boolean.toString(true), - Option.PRIORITY.toString(), "0" - ); - - public static final Map> DEFAULT_OPTIONS = - ImmutableMap.of(AutoRepairConfig.RepairType.FULL, DEFAULT_SUB_OPTIONS, - AutoRepairConfig.RepairType.INCREMENTAL, DEFAULT_SUB_OPTIONS, - AutoRepairConfig.RepairType.PREVIEW_REPAIRED, DEFAULT_SUB_OPTIONS); - - public final AutoRepairConfig.RepairType type; + private ImmutableMap options; - private Map> options = DEFAULT_OPTIONS; + public static final Map DEFAULT_OPTIONS = ImmutableMap.of( + Option.FULL_ENABLED.name().toLowerCase(), Boolean.toString(true), + Option.INCREMENTAL_ENABLED.name().toLowerCase(), Boolean.toString(true), + Option.PREVIEW_REPAIRED_ENABLED.name().toLowerCase(), Boolean.toString(true), + Option.PRIORITY.toString(), "0" + ); - AutoRepairParams(AutoRepairConfig.RepairType type) + AutoRepairParams(Map options) { - this.type = type; + this.options = ImmutableMap.copyOf(options); } - public static AutoRepairParams create(AutoRepairConfig.RepairType repairType, Map options) + public static final AutoRepairParams DEFAULT = + new AutoRepairParams(DEFAULT_OPTIONS); + + public static AutoRepairParams create(Map options) { - Map> optionsMap = new EnumMap<>(AutoRepairConfig.RepairType.class); - for (Map.Entry> entry : DEFAULT_OPTIONS.entrySet()) + Map optionsMap = new TreeMap<>(); + for (Map.Entry entry : DEFAULT_OPTIONS.entrySet()) { - optionsMap.put(entry.getKey(), new HashMap<>(entry.getValue())); + optionsMap.put(entry.getKey(), entry.getValue()); } if (options != null) { @@ -81,38 +80,61 @@ public static AutoRepairParams create(AutoRepairConfig.RepairType repairType, Ma { throw new ConfigurationException(format("Unknown property '%s'", entry.getKey())); } - optionsMap.get(repairType).put(entry.getKey(), entry.getValue()); + optionsMap.put(entry.getKey(), entry.getValue()); } } - AutoRepairParams repairParams = new AutoRepairParams(repairType); - repairParams.options = optionsMap; - return repairParams; + return new AutoRepairParams(optionsMap); } - public boolean repairEnabled() + public boolean repairEnabled(AutoRepairConfig.RepairType type) { - String enabled = options.get(type).get(Option.ENABLED.toString()); + String option = type.toString().toLowerCase() + "_enabled"; + String enabled = options.get(option); return enabled == null - ? Boolean.parseBoolean(DEFAULT_OPTIONS.get(type).get(Option.ENABLED.toString())) + ? Boolean.parseBoolean(DEFAULT_OPTIONS.get(option)) : Boolean.parseBoolean(enabled); } public int priority() { - String priority = options.get(type).get(Option.PRIORITY.toString()); + String priority = options.get(Option.PRIORITY.toString()); return priority == null - ? Integer.parseInt(DEFAULT_OPTIONS.get(type).get(Option.PRIORITY.toString())) + ? Integer.parseInt(DEFAULT_OPTIONS.get(Option.PRIORITY.toString())) : Integer.parseInt(priority); } public void validate() { - String enabled = options.get(type).get(Option.ENABLED.toString()); - if (enabled != null && !isValidBoolean(enabled)) + for (Option option : Option.values()) + { + if (!options.containsKey(option.toString().toLowerCase())) + { + throw new ConfigurationException(format("Missing repair sub-option '%s'", option)); + } + } + if (options.get(Option.FULL_ENABLED.toString().toLowerCase()) != null && !isValidBoolean(options.get(Option.FULL_ENABLED.toString().toLowerCase()))) { throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be a boolean", - enabled, - Option.ENABLED)); + options.get(Option.FULL_ENABLED.toString().toLowerCase()), + Option.FULL_ENABLED)); + } + if (options.get(Option.INCREMENTAL_ENABLED.toString().toLowerCase()) != null && !isValidBoolean(options.get(Option.INCREMENTAL_ENABLED.toString().toLowerCase()))) + { + throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be a boolean", + options.get(Option.INCREMENTAL_ENABLED.toString().toLowerCase()), + Option.INCREMENTAL_ENABLED)); + } + if (options.get(Option.PREVIEW_REPAIRED_ENABLED.toString().toLowerCase()) != null && !isValidBoolean(options.get(Option.PREVIEW_REPAIRED_ENABLED.toString().toLowerCase()))) + { + throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be a boolean", + options.get(Option.PREVIEW_REPAIRED_ENABLED.toString().toLowerCase()), + Option.PREVIEW_REPAIRED_ENABLED)); + } + if (options.get(Option.PRIORITY.toString().toLowerCase()) != null && !isValidInt(options.get(Option.PRIORITY.toString().toLowerCase()))) + { + throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be an integer", + options.get(Option.PRIORITY.toString().toLowerCase()), + Option.PRIORITY)); } } @@ -121,19 +143,25 @@ public static boolean isValidBoolean(String value) return StringUtils.equalsIgnoreCase(value, "true") || StringUtils.equalsIgnoreCase(value, "false"); } + public static boolean isValidInt(String value) + { + return StringUtils.isNumeric(value); + } + + public Map options() { - return options.get(type); + return options; } - public static AutoRepairParams fromMap(AutoRepairConfig.RepairType repairType, Map map) + public static AutoRepairParams fromMap(Map map) { - return create(repairType, map); + return create(map); } public Map asMap() { - return options.get(type); + return options; } @Override diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index c17f777feac7..d6b5025ced30 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -44,7 +44,6 @@ import org.apache.cassandra.db.partitions.*; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.exceptions.InvalidRequestException; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; import org.apache.cassandra.schema.ColumnMetadata.ClusteringOrder; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; @@ -129,9 +128,7 @@ private SchemaKeyspace() + "additional_write_policy text," + "cdc boolean," + "read_repair text," - + "repair_full frozen>," - + "repair_incremental frozen>," - + "repair_preview_repaired frozen>," + + "auto_repair frozen>," + "PRIMARY KEY ((keyspace_name), table_name))"); private static final TableMetadata Columns = @@ -216,9 +213,7 @@ private SchemaKeyspace() + "additional_write_policy text," + "cdc boolean," + "read_repair text," - + "repair_full frozen>," - + "repair_incremental frozen>," - + "repair_preview_repaired frozen>," + + "auto_repair frozen>," + "PRIMARY KEY ((keyspace_name), view_name))"); private static final TableMetadata Indexes = @@ -567,9 +562,7 @@ private static void addTableParamsToRowBuilder(TableParams params, Row.SimpleBui .add("compression", params.compression.asMap()) .add("read_repair", params.readRepair.toString()) .add("extensions", params.extensions) - .add("repair_full", params.automatedRepair.get(AutoRepairConfig.RepairType.FULL).asMap()) - .add("repair_incremental", params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).asMap()) - .add("repair_preview_repaired", params.automatedRepair.get(AutoRepairConfig.RepairType.PREVIEW_REPAIRED).asMap()); + .add("auto_repair", params.autoRepair.asMap()); // Only add CDC-enabled flag to schema if it's enabled on the node. This is to work around RTE's post-8099 if a 3.8+ @@ -1054,9 +1047,8 @@ static TableParams createTableParamsFromRow(UntypedResultSet.Row row) SpeculativeRetryPolicy.fromString("99PERCENTILE")) .cdc(row.has("cdc") && row.getBoolean("cdc")) .readRepair(getReadRepairStrategy(row)) - .automatedRepairFull(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.FULL, row.getFrozenTextMap("repair_full"))) - .automatedRepairIncremental(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.INCREMENTAL, row.getFrozenTextMap("repair_incremental"))) - .automatedRepairPreviewRepaired(AutoRepairParams.fromMap(AutoRepairConfig.RepairType.PREVIEW_REPAIRED, row.getFrozenTextMap("repair_preview_repaired"))); + .automatedRepair(AutoRepairParams.fromMap(row.getFrozenTextMap("auto_repair"))) +; // allow_auto_snapshot column was introduced in 4.2 if (row.has("allow_auto_snapshot")) diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index fe2681f6c10c..67d90114047d 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.util.EnumMap; import java.util.Map; import java.util.Map.Entry; @@ -35,7 +34,6 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.service.reads.PercentileSpeculativeRetryPolicy; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; @@ -72,9 +70,7 @@ public enum Option CRC_CHECK_CHANCE, CDC, READ_REPAIR, - REPAIR_FULL, - REPAIR_INCREMENTAL, - REPAIR_PREVIEW_REPAIRED, + AUTO_REPAIR, ; @Override @@ -103,8 +99,7 @@ public String toString() public final ImmutableMap extensions; public final boolean cdc; public final ReadRepairStrategy readRepair; - - public final Map automatedRepair; + public final AutoRepairParams autoRepair; private TableParams(Builder builder) { @@ -129,14 +124,7 @@ private TableParams(Builder builder) extensions = builder.extensions; cdc = builder.cdc; readRepair = builder.readRepair; - automatedRepair = new EnumMap(AutoRepairConfig.RepairType.class) - { - { - put(AutoRepairConfig.RepairType.FULL, builder.automatedRepairFull); - put(AutoRepairConfig.RepairType.INCREMENTAL, builder.automatedRepairIncremental); - put(AutoRepairConfig.RepairType.PREVIEW_REPAIRED, builder.automatedRepairPreviewRepaired); - } - }; + autoRepair = builder.autoRepair; } public static Builder builder() @@ -165,9 +153,7 @@ public static Builder builder(TableParams params) .extensions(params.extensions) .cdc(params.cdc) .readRepair(params.readRepair) - .automatedRepairFull(params.automatedRepair.get(AutoRepairConfig.RepairType.FULL)) - .automatedRepairIncremental(params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL)) - .automatedRepairPreviewRepaired(params.automatedRepair.get(AutoRepairConfig.RepairType.PREVIEW_REPAIRED)) + .automatedRepair(params.autoRepair) ; } @@ -224,10 +210,7 @@ public void validate() if (cdc && memtable.factory().writesShouldSkipCommitLog()) fail("CDC cannot work if writes skip the commit log. Check your memtable configuration."); - for (Map.Entry entry : automatedRepair.entrySet()) - { - entry.getValue().validate(); - } + autoRepair.validate(); } private static void fail(String format, Object... args) @@ -265,7 +248,7 @@ public boolean equals(Object o) && extensions.equals(p.extensions) && cdc == p.cdc && readRepair == p.readRepair - && automatedRepair.equals(p.automatedRepair); + && autoRepair.equals(p.autoRepair); } @Override @@ -290,7 +273,7 @@ public int hashCode() extensions, cdc, readRepair, - automatedRepair); + autoRepair); } @Override @@ -316,9 +299,7 @@ public String toString() .add(EXTENSIONS.toString(), extensions) .add(CDC.toString(), cdc) .add(READ_REPAIR.toString(), readRepair) - .add(Option.REPAIR_FULL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.FULL)) - .add(Option.REPAIR_INCREMENTAL.toString(), automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL)) - .add(Option.REPAIR_PREVIEW_REPAIRED.toString(), automatedRepair.get(AutoRepairConfig.RepairType.PREVIEW_REPAIRED)) + .add(Option.AUTO_REPAIR.toString(), autoRepair) .toString(); } @@ -372,11 +353,7 @@ public void appendCqlTo(CqlBuilder builder, boolean isView) .newLine() .append("AND speculative_retry = ").appendWithSingleQuotes(speculativeRetry.toString()) .newLine() - .append("AND repair_full = ").append(automatedRepair.get(AutoRepairConfig.RepairType.FULL).asMap()) - .newLine() - .append("AND repair_incremental = ").append(automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).asMap()) - .newLine() - .append("AND repair_preview_repaired = ").append(automatedRepair.get(AutoRepairConfig.RepairType.PREVIEW_REPAIRED).asMap()); + .append("AND auto_repair = ").append(autoRepair.asMap()); } public static final class Builder @@ -401,10 +378,7 @@ public static final class Builder private boolean cdc; private ReadRepairStrategy readRepair = ReadRepairStrategy.BLOCKING; - private AutoRepairParams automatedRepairFull = new AutoRepairParams(AutoRepairConfig.RepairType.FULL); - private AutoRepairParams automatedRepairIncremental = new AutoRepairParams(AutoRepairConfig.RepairType.INCREMENTAL); - private AutoRepairParams automatedRepairPreviewRepaired = new AutoRepairParams(AutoRepairConfig.RepairType.PREVIEW_REPAIRED); - + private AutoRepairParams autoRepair = AutoRepairParams.DEFAULT; public Builder() { } @@ -528,21 +502,9 @@ public Builder extensions(Map val) return this; } - public Builder automatedRepairFull(AutoRepairParams val) - { - automatedRepairFull = val; - return this; - } - - public Builder automatedRepairIncremental(AutoRepairParams val) - { - automatedRepairIncremental = val; - return this; - } - - public Builder automatedRepairPreviewRepaired(AutoRepairParams val) + public Builder automatedRepair(AutoRepairParams val) { - automatedRepairPreviewRepaired = val; + autoRepair = val; return this; } } diff --git a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java index 569c56947ab8..60ad16a4b815 100644 --- a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java +++ b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java @@ -1075,9 +1075,7 @@ private static String tableParametersCql() " AND min_index_interval = 128\n" + " AND read_repair = 'BLOCKING'\n" + " AND speculative_retry = '99p'\n" + - " AND repair_full = {'enabled': 'true'}\n" + - " AND repair_incremental = {'enabled': 'true'}\n" + - " AND repair_preview_repaired = {'enabled': 'true'};"; + " AND auto_repair = {'full_enabled': 'true', 'incremental_enabled': 'true', 'preview_repaired_enabled': 'true', 'priority': '0'};"; } private static String cqlQuoted(Map map) @@ -1105,9 +1103,7 @@ private static String mvParametersCql() " AND min_index_interval = 128\n" + " AND read_repair = 'BLOCKING'\n" + " AND speculative_retry = '99p'\n" + - " AND repair_full = {'enabled': 'true'}\n" + - " AND repair_incremental = {'enabled': 'true'}\n" + - " AND repair_preview_repaired = {'enabled': 'true'};"; + " AND auto_repair = {'full_enabled': 'true', 'incremental_enabled': 'true', 'preview_repaired_enabled': 'true', 'priority': '0'};"; } private static String keyspaceOutput() diff --git a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java index 422ac8b05f63..52f3d454cb70 100644 --- a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java +++ b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java @@ -307,9 +307,7 @@ public void testCfmOptionsCQL() " AND min_index_interval = 6\n" + " AND read_repair = 'BLOCKING'\n" + " AND speculative_retry = 'ALWAYS'\n" + - " AND repair_full = {'enabled': 'true'}\n" + - " AND repair_incremental = {'enabled': 'true'}\n" + - " AND repair_preview_repaired = {'enabled': 'true'};" + " AND auto_repair = {'full_enabled': 'true', 'incremental_enabled': 'true', 'preview_repaired_enabled': 'true', 'priority': '0'};" )); } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 5a5e66a3db15..a542231538dc 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -128,7 +128,7 @@ public void setup() SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); QueryProcessor.executeInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE)); QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i))", KEYSPACE, TABLE)); - QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i)) WITH repair_full = {'enabled': 'false'} AND repair_incremental = {'enabled': 'false'} AND repair_preview_repaired = {'enabled': 'false'}", KEYSPACE, TABLE_DISABLED_AUTO_REPAIR)); + QueryProcessor.executeInternal(String.format("CREATE TABLE %s.%s (k text, s text static, i int, v text, primary key(k,i)) WITH auto_repair = {'full_enabled': 'false', 'incremental_enabled': 'false', 'preview_repaired_enabled': 'false', 'priority': '0'}", KEYSPACE, TABLE_DISABLED_AUTO_REPAIR)); QueryProcessor.executeInternal(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT i, k from %s.%s " + "WHERE k IS NOT null AND i IS NOT null PRIMARY KEY (i, k)", KEYSPACE, MV, KEYSPACE, TABLE)); @@ -520,10 +520,10 @@ public void testRepairWaitsForRepairToFinishBeforeSchedullingNewSession() throws @Test public void testDisabledAutoRepairForATableThroughTableLevelConfiguration() { - Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.FULL).repairEnabled()); - Assert.assertTrue(cfm.params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).repairEnabled()); - Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.FULL).repairEnabled()); - Assert.assertFalse(cfmDisabledAutoRepair.params.automatedRepair.get(AutoRepairConfig.RepairType.INCREMENTAL).repairEnabled()); + Assert.assertTrue(cfm.params.autoRepair.repairEnabled(AutoRepairConfig.RepairType.FULL)); + Assert.assertTrue(cfm.params.autoRepair.repairEnabled(AutoRepairConfig.RepairType.INCREMENTAL)); + Assert.assertFalse(cfmDisabledAutoRepair.params.autoRepair.repairEnabled(AutoRepairConfig.RepairType.FULL)); + Assert.assertFalse(cfmDisabledAutoRepair.params.autoRepair.repairEnabled(AutoRepairConfig.RepairType.INCREMENTAL)); AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); config.setRepairMinInterval(repairType, "0s"); @@ -565,9 +565,7 @@ public void testTokenRangesNoSplit() @Test public void testTableAttribute() { - assertTrue(TableAttributes.validKeywords().contains("repair_full")); - assertTrue(TableAttributes.validKeywords().contains("repair_incremental")); - assertTrue(TableAttributes.validKeywords().contains("repair_preview_repaired")); + assertTrue(TableAttributes.validKeywords().contains("auto_repair")); } @Test @@ -576,9 +574,9 @@ public void testDefaultAutomatedRepair() for (AutoRepairConfig.RepairType repairType : AutoRepairConfig.RepairType.values()) { Assert.assertTrue(String.format("expected repair type %s to be enabled on table %s", repairType, cfm.name), - cfm.params.automatedRepair.get(repairType).repairEnabled()); + cfm.params.autoRepair.repairEnabled(repairType)); Assert.assertFalse(String.format("expected repair type %s to be disabled on table %s", repairType, cfmDisabledAutoRepair.name), - cfmDisabledAutoRepair.params.automatedRepair.get(repairType).repairEnabled()); + cfmDisabledAutoRepair.params.autoRepair.repairEnabled(repairType)); } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java b/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java index 011df9d1592e..7b95acb879de 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/PrioritizedRepairPlanTest.java @@ -37,9 +37,9 @@ public class PrioritizedRepairPlanTest extends CQLTester public void testBuildWithDifferentPriorities() { // Test reordering assignments with different priorities - String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '3'}"); - String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); + String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '3'}"); + String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '1'}"); List prioritizedRepairPlans = PrioritizedRepairPlan.buildSingleKeyspacePlan(AutoRepairConfig.RepairType.FULL, KEYSPACE, table1, table2, table3); assertEquals(3, prioritizedRepairPlans.size()); @@ -59,9 +59,9 @@ public void testBuildWithDifferentPriorities() public void testBuildWithSamePriority() { // Test reordering assignments with the same priority - String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); + String table2 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); + String table3 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); // Expect only 1 plan since all tables share the same priority List prioritizedRepairPlans = PrioritizedRepairPlan.buildSingleKeyspacePlan(AutoRepairConfig.RepairType.FULL, KEYSPACE, table1, table2, table3); @@ -82,17 +82,17 @@ public void testBuildWithSamePriority() public void testBuildWithMixedPriorities() { String ks1 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"); - String table1 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table2 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '3'}"); - String table3 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '2'}"); - String table4 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); + String table1 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); + String table2 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '3'}"); + String table3 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '2'}"); + String table4 = createTable(ks1, "CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '1'}"); // No priority table should be bucketed at priority 0 String table5 = createTable(ks1,"CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); // Create a new keyspace to ensure its tables get grouped with appropriate priority bucket String ks2 = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"); String table6 = createTable(ks2,"CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); - String table7 = createTable(ks2,"CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '1'}"); + String table7 = createTable(ks2,"CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '1'}"); Map> keyspaceToTableMap = new HashMap<>(); keyspaceToTableMap.put(ks1, Lists.newArrayList(table1, table2, table3, table4, table5)); @@ -149,7 +149,7 @@ public void testBuildWithEmptyTableList() public void testBuildWithOneTable() { // Test with a single element (should remain unchanged) - String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH repair_full = {'enabled': 'true', 'priority': '5'}"); + String table1 = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT) WITH auto_repair = {'full_enabled': 'true', 'priority': '5'}"); // Expect only 1 plans List prioritizedRepairPlans = PrioritizedRepairPlan.buildSingleKeyspacePlan(AutoRepairConfig.RepairType.FULL, KEYSPACE, table1); From 46ca72c308ac33a599c8c56a013e5a08693a1169 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 10 Dec 2024 12:25:59 -0800 Subject: [PATCH 093/115] Fix ant checkstyle errors --- .../statements/schema/TableAttributes.java | 1 - .../repair/autorepair/AutoRepairConfig.java | 3 ++- .../repair/autorepair/RepairAssignment.java | 1 - .../cassandra/schema/AutoRepairParams.java | 27 ++++++++++--------- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index 7b0e18b304fd..cc97a9d525e1 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -25,7 +25,6 @@ import org.apache.cassandra.cql3.statements.PropertyDefinitions; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.repair.autorepair.AutoRepairConfig; import org.apache.cassandra.schema.AutoRepairParams; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.CompactionParams; diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index b84ee5f2f652..8eaef0b6ecc5 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -32,6 +32,7 @@ import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.config.ParameterizedClass; +import org.apache.cassandra.utils.LocalizeString; public class AutoRepairConfig implements Serializable { @@ -65,7 +66,7 @@ public enum RepairType implements Serializable RepairType() { - this.configName = name().toLowerCase(); + this.configName = LocalizeString.toLowerCaseLocalized(name()); } RepairType(String configName) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairAssignment.java b/src/java/org/apache/cassandra/repair/autorepair/RepairAssignment.java index c06d58e71fa2..63f8fbed4426 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairAssignment.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairAssignment.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Objects; -import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; diff --git a/src/java/org/apache/cassandra/schema/AutoRepairParams.java b/src/java/org/apache/cassandra/schema/AutoRepairParams.java index 2b289a8c3b44..105b9f6d0ddd 100644 --- a/src/java/org/apache/cassandra/schema/AutoRepairParams.java +++ b/src/java/org/apache/cassandra/schema/AutoRepairParams.java @@ -28,6 +28,7 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.repair.autorepair.AutoRepairConfig; +import org.apache.cassandra.utils.LocalizeString; import static java.lang.String.format; import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; @@ -51,9 +52,9 @@ public String toString() private ImmutableMap options; public static final Map DEFAULT_OPTIONS = ImmutableMap.of( - Option.FULL_ENABLED.name().toLowerCase(), Boolean.toString(true), - Option.INCREMENTAL_ENABLED.name().toLowerCase(), Boolean.toString(true), - Option.PREVIEW_REPAIRED_ENABLED.name().toLowerCase(), Boolean.toString(true), + LocalizeString.toLowerCaseLocalized(Option.FULL_ENABLED.name()), Boolean.toString(true), + LocalizeString.toLowerCaseLocalized(Option.INCREMENTAL_ENABLED.name()), Boolean.toString(true), + LocalizeString.toLowerCaseLocalized(Option.PREVIEW_REPAIRED_ENABLED.name()), Boolean.toString(true), Option.PRIORITY.toString(), "0" ); @@ -88,7 +89,7 @@ public static AutoRepairParams create(Map options) public boolean repairEnabled(AutoRepairConfig.RepairType type) { - String option = type.toString().toLowerCase() + "_enabled"; + String option = LocalizeString.toLowerCaseLocalized(type.toString()) + "_enabled"; String enabled = options.get(option); return enabled == null ? Boolean.parseBoolean(DEFAULT_OPTIONS.get(option)) @@ -107,33 +108,33 @@ public void validate() { for (Option option : Option.values()) { - if (!options.containsKey(option.toString().toLowerCase())) + if (!options.containsKey(LocalizeString.toLowerCaseLocalized(option.toString()))) { throw new ConfigurationException(format("Missing repair sub-option '%s'", option)); } } - if (options.get(Option.FULL_ENABLED.toString().toLowerCase()) != null && !isValidBoolean(options.get(Option.FULL_ENABLED.toString().toLowerCase()))) + if (options.get(LocalizeString.toLowerCaseLocalized(Option.FULL_ENABLED.toString())) != null && !isValidBoolean(options.get(LocalizeString.toLowerCaseLocalized(Option.FULL_ENABLED.toString())))) { throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be a boolean", - options.get(Option.FULL_ENABLED.toString().toLowerCase()), + options.get(LocalizeString.toLowerCaseLocalized(Option.FULL_ENABLED.toString())), Option.FULL_ENABLED)); } - if (options.get(Option.INCREMENTAL_ENABLED.toString().toLowerCase()) != null && !isValidBoolean(options.get(Option.INCREMENTAL_ENABLED.toString().toLowerCase()))) + if (options.get(LocalizeString.toLowerCaseLocalized(Option.INCREMENTAL_ENABLED.toString())) != null && !isValidBoolean(options.get(LocalizeString.toLowerCaseLocalized(Option.INCREMENTAL_ENABLED.toString())))) { throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be a boolean", - options.get(Option.INCREMENTAL_ENABLED.toString().toLowerCase()), + options.get(LocalizeString.toLowerCaseLocalized(Option.INCREMENTAL_ENABLED.toString())), Option.INCREMENTAL_ENABLED)); } - if (options.get(Option.PREVIEW_REPAIRED_ENABLED.toString().toLowerCase()) != null && !isValidBoolean(options.get(Option.PREVIEW_REPAIRED_ENABLED.toString().toLowerCase()))) + if (options.get(LocalizeString.toLowerCaseLocalized(Option.PREVIEW_REPAIRED_ENABLED.toString())) != null && !isValidBoolean(options.get(LocalizeString.toLowerCaseLocalized(Option.PREVIEW_REPAIRED_ENABLED.toString())))) { throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be a boolean", - options.get(Option.PREVIEW_REPAIRED_ENABLED.toString().toLowerCase()), + options.get(LocalizeString.toLowerCaseLocalized(Option.PREVIEW_REPAIRED_ENABLED.toString())), Option.PREVIEW_REPAIRED_ENABLED)); } - if (options.get(Option.PRIORITY.toString().toLowerCase()) != null && !isValidInt(options.get(Option.PRIORITY.toString().toLowerCase()))) + if (options.get(LocalizeString.toLowerCaseLocalized(Option.PRIORITY.toString())) != null && !isValidInt(options.get(LocalizeString.toLowerCaseLocalized(Option.PRIORITY.toString())))) { throw new ConfigurationException(format("Invalid value %s for '%s' repair sub-option - must be an integer", - options.get(Option.PRIORITY.toString().toLowerCase()), + options.get(LocalizeString.toLowerCaseLocalized(Option.PRIORITY.toString())), Option.PRIORITY)); } } From efb979388b1563f6af7650746aa06e82bfca5c03 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 10 Dec 2024 14:17:51 -0800 Subject: [PATCH 094/115] Ignore keyspaces with MetaStrategy --- .../apache/cassandra/repair/autorepair/AutoRepairUtils.java | 3 ++- .../repair/autorepair/AutoRepairParameterizedTest.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index 77bdfbfb2e5b..0fd021690873 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -55,6 +55,7 @@ import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.MetaStrategy; import org.apache.cassandra.locator.NetworkTopologyStrategy; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; @@ -802,7 +803,7 @@ public static boolean checkNodeContainsKeyspaceReplica(Keyspace ks) ksReplicaOnNode = false; } } - if (replicationStrategy instanceof LocalStrategy) + if (replicationStrategy instanceof LocalStrategy || replicationStrategy instanceof MetaStrategy) { ksReplicaOnNode = false; } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index a542231538dc..fd7403c0e391 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -648,7 +648,7 @@ public void testRepairMaxRetries() // system_auth.resource_role_permissons_index,system_traces.sessions,system_traces.events,ks.tbl, // system_distributed.auto_repair_priority,system_distributed.repair_history,system_distributed.auto_repair_history, // system_distributed.view_build_status,system_distributed.parent_repair_history,system_distributed.partition_denylist - int exptedTablesGoingThroughRepair = 18; + int exptedTablesGoingThroughRepair = 17; assertEquals(config.getRepairMaxRetries()*exptedTablesGoingThroughRepair, sleepCalls.get()); verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(0); verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); From 51e86d5053876a057dccb53d06ab60ccc6ad4890 Mon Sep 17 00:00:00 2001 From: Jaydeepkumar Chovatia Date: Tue, 10 Dec 2024 14:44:18 -0800 Subject: [PATCH 095/115] Adjust BigTableScanner.getScanner as per 5.1 API set --- .../apache/cassandra/repair/autorepair/RepairRangeSplitter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index 895cde20542d..ecae4dc2c2a7 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -577,7 +577,7 @@ static SizeEstimate getSizesForRangeOfSSTables(AutoRepairConfig.RepairType repai // get the bounds of the sstable for this range using the index file but do not actually read it. List> bounds = BigTableScanner.makeBounds(reader, Collections.singleton(tokenRange)); - ISSTableScanner rangeScanner = BigTableScanner.getScanner((BigTableReader) reader, Collections.singleton(tokenRange)); + ISSTableScanner rangeScanner = reader.getScanner(Collections.singleton(tokenRange)); // Type check scanner returned as it may be an EmptySSTableScanner if the range is not covered in the // SSTable, in this case we will avoid incrementing approxBytesInRange. if (rangeScanner instanceof BigTableScanner) From d784844144aaa044e03937a6884cd37743d1cc15 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 10 Dec 2024 15:56:06 -0800 Subject: [PATCH 096/115] Fix antcheckstyle --- .../apache/cassandra/repair/autorepair/RepairRangeSplitter.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java index ecae4dc2c2a7..fab0f07fb195 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java +++ b/src/java/org/apache/cassandra/repair/autorepair/RepairRangeSplitter.java @@ -50,7 +50,6 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.ISSTableScanner; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.io.sstable.format.big.BigTableReader; import org.apache.cassandra.io.sstable.format.big.BigTableScanner; import org.apache.cassandra.io.sstable.metadata.CompactionMetadata; import org.apache.cassandra.io.sstable.metadata.MetadataType; From da3e0235df709b195d12b415eeef290c275bc364 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Wed, 11 Dec 2024 20:30:24 -0800 Subject: [PATCH 097/115] Fix ant test-latest failures for RepairRangeSplitterTest-latest_jdk17 Tests run: 15, Failures: 7 --- .../cassandra/repair/autorepair/RepairRangeSplitterTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java index 9baaecca545e..d93a40b689f9 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/RepairRangeSplitterTest.java @@ -41,6 +41,7 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.big.BigFormat; import org.apache.cassandra.repair.autorepair.AutoRepairConfig.RepairType; import org.apache.cassandra.repair.autorepair.RepairRangeSplitter.FilteredRepairAssignments; import org.apache.cassandra.repair.autorepair.RepairRangeSplitter.SizeEstimate; @@ -67,6 +68,7 @@ public static void setUpClass() AutoRepairService.setup(); AutoRepairService.instance.getAutoRepairConfig().setRepairByKeyspace(RepairType.FULL, true); FULL_RANGE = new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), DatabaseDescriptor.getPartitioner().getMaximumToken()); + DatabaseDescriptor.setSelectedSSTableFormat(DatabaseDescriptor.getSSTableFormats().get(BigFormat.NAME)); } @Before @@ -74,6 +76,7 @@ public void setUp() { repairRangeSplitter = new RepairRangeSplitter(Collections.emptyMap()); tableName = createTable("CREATE TABLE %s (k INT PRIMARY KEY, v INT)"); + assertTrue(BigFormat.isSelected()); } @Test From 607b1a15c68130b37ab134214eb9022b90b92772 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Sun, 15 Dec 2024 10:54:20 -0800 Subject: [PATCH 098/115] utest: calculate expected tables dynamically instead of a fixed value --- .../AutoRepairParameterizedTest.java | 31 ++++++++++++++----- 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index fd7403c0e391..74615905ab24 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -37,6 +37,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.repair.RepairCoordinator; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.locator.LocalStrategy; import org.apache.cassandra.schema.SystemDistributedKeyspace; import org.apache.cassandra.service.StorageService; @@ -100,6 +101,9 @@ public class AutoRepairParameterizedTest extends CQLTester RepairCoordinator repairRunnable; private static AutoRepairConfig defaultConfig; + // Expected number of tables that should be repaired. + private static int expectedTablesGoingThroughRepair; + @Parameterized.Parameter() public AutoRepairConfig.RepairType repairType; @@ -120,6 +124,22 @@ public static void setupClass() throws Exception AutoRepairUtils.setup(); StorageService.instance.doAutoRepairSetup(); DatabaseDescriptor.setCDCEnabled(false); + + // Calculate the expected number of tables to be repaired, this should be all system keyspaces that are + // distributed, plus 1 for the table we created (ks.tbl), excluding the 'mv' materialized view and + // 'tbl_disabled_auto_repair' we created. + expectedTablesGoingThroughRepair = 0; + for (Keyspace keyspace : Keyspace.all()) + { + // skip LocalStrategy keyspaces as these aren't repaired. + if (keyspace.getReplicationStrategy() instanceof LocalStrategy) + { + continue; + } + + int expectedTables = keyspace.getName().equals("ks") ? 1 : keyspace.getColumnFamilyStores().size(); + expectedTablesGoingThroughRepair += expectedTables; + } } @Before @@ -644,15 +664,10 @@ public void testRepairMaxRetries() AutoRepair.instance.repair(repairType); - //system_auth.role_permissions,system_auth.network_permissions,system_auth.role_members,system_auth.roles, - // system_auth.resource_role_permissons_index,system_traces.sessions,system_traces.events,ks.tbl, - // system_distributed.auto_repair_priority,system_distributed.repair_history,system_distributed.auto_repair_history, - // system_distributed.view_build_status,system_distributed.parent_repair_history,system_distributed.partition_denylist - int exptedTablesGoingThroughRepair = 17; - assertEquals(config.getRepairMaxRetries()*exptedTablesGoingThroughRepair, sleepCalls.get()); + assertEquals(config.getRepairMaxRetries()*expectedTablesGoingThroughRepair, sleepCalls.get()); verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(0); verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); - verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(exptedTablesGoingThroughRepair); + verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(expectedTablesGoingThroughRepair); } @Test @@ -678,7 +693,7 @@ public void testRepairSuccessAfterRetry() AutoRepair.instance.repair(repairType); assertEquals(1, sleepCalls.get()); - verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(18); + verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(expectedTablesGoingThroughRepair); verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(0); } From ab71a155c27f61edfde46da2d4a0bb055772db78 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Tue, 17 Dec 2024 20:16:54 -0800 Subject: [PATCH 099/115] Ignore repair for system_traces keyspace --- .../cassandra/repair/autorepair/AutoRepair.java | 2 +- .../repair/autorepair/AutoRepairUtils.java | 16 +++++++++++----- .../autorepair/AutoRepairParameterizedTest.java | 6 ++++++ .../repair/autorepair/AutoRepairTest.java | 4 ++-- .../repair/autorepair/AutoRepairUtilsTest.java | 12 +++++++++--- 5 files changed, 29 insertions(+), 11 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index eafd40eaddf7..d6d58746f45f 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -216,7 +216,7 @@ public void repair(AutoRepairConfig.RepairType repairType) Map> keyspacesAndTablesToRepair = new LinkedHashMap<>(); for (Keyspace keyspace : keyspaces) { - if (!AutoRepairUtils.checkNodeContainsKeyspaceReplica(keyspace)) + if (!AutoRepairUtils.shouldConsiderKeyspace(keyspace)) { continue; } diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java index 0fd021690873..56cbf208b801 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairUtils.java @@ -790,24 +790,30 @@ public static Set getPriorityHosts(RepairType repairType) return hosts; } - public static boolean checkNodeContainsKeyspaceReplica(Keyspace ks) + public static boolean shouldConsiderKeyspace(Keyspace ks) { AbstractReplicationStrategy replicationStrategy = ks.getReplicationStrategy(); - boolean ksReplicaOnNode = true; + boolean repair = true; if (replicationStrategy instanceof NetworkTopologyStrategy) { Set datacenters = ((NetworkTopologyStrategy) replicationStrategy).getDatacenters(); String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddressAndPort()); if (!datacenters.contains(localDC)) { - ksReplicaOnNode = false; + repair = false; } } if (replicationStrategy instanceof LocalStrategy || replicationStrategy instanceof MetaStrategy) { - ksReplicaOnNode = false; + repair = false; } - return ksReplicaOnNode; + if (ks.getName().equalsIgnoreCase(SchemaConstants.TRACE_KEYSPACE_NAME)) + { + // by default, ignore the tables under system_traces as they do not have + // that much important data + repair = false; + } + return repair; } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 74615905ab24..672076e261ee 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -62,6 +62,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.LocalizeString; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; @@ -136,6 +137,11 @@ public static void setupClass() throws Exception { continue; } + // skip system_traces keyspaces + if (keyspace.getName().equalsIgnoreCase(SchemaConstants.TRACE_KEYSPACE_NAME)) + { + continue; + } int expectedTables = keyspace.getName().equals("ks") ? 1 : keyspace.getColumnFamilyStores().size(); expectedTablesGoingThroughRepair += expectedTables; diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java index 425dc9b31f37..283527f61117 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairTest.java @@ -141,14 +141,14 @@ public void testCheckNTSreplicationNodeInsideOutsideDC() // case 1 : // node reside in "datacenter1" // keyspace has replica in "datacenter1" - Assert.assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + Assert.assertTrue(AutoRepairUtils.shouldConsiderKeyspace(ks)); } else if (ks.getName().equals(ksname2)) { // case 2 : // node reside in "datacenter1" // keyspace has replica in "datacenter2" - Assert.assertFalse(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + Assert.assertFalse(AutoRepairUtils.shouldConsiderKeyspace(ks)); } } } diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java index c831bc625586..766507aaa750 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -424,7 +424,7 @@ public void testCheckNodeContainsKeyspaceReplica() { Keyspace ks = Keyspace.open("ks"); - assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(ks)); + assertTrue(AutoRepairUtils.shouldConsiderKeyspace(ks)); } @Test @@ -473,8 +473,8 @@ public void testMyTurnToRunRepairShouldReturnMyTurnWhenRepairOngoing() @Test public void testLocalStrategyAndNetworkKeyspace() { - assertFalse(AutoRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open("system"))); - assertTrue(AutoRepairUtils.checkNodeContainsKeyspaceReplica(Keyspace.open(KEYSPACE))); + assertFalse(AutoRepairUtils.shouldConsiderKeyspace(Keyspace.open("system"))); + assertTrue(AutoRepairUtils.shouldConsiderKeyspace(Keyspace.open(KEYSPACE))); } @Test @@ -496,4 +496,10 @@ public void testGetLastRepairTimeForNodeWhenHistoryIsEmpty() assertEquals(0, AutoRepairUtils.getLastRepairTimeForNode(repairType, myID)); } + + @Test + public void tesSkipSystemTraces() + { + assertFalse(AutoRepairUtils.shouldConsiderKeyspace(Keyspace.open(SchemaConstants.TRACE_KEYSPACE_NAME))); + } } From 851f2c68e0e5cdbc0c7516b59e7ddd2928949d14 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Thu, 19 Dec 2024 09:59:36 -0800 Subject: [PATCH 100/115] CR from Andy T --- .../repair/autorepair/AutoRepairParameterizedTest.java | 1 - .../apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 672076e261ee..d2561af74c42 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -62,7 +62,6 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.LocalizeString; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java index 766507aaa750..c74325e46dd0 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairUtilsTest.java @@ -498,7 +498,7 @@ public void testGetLastRepairTimeForNodeWhenHistoryIsEmpty() } @Test - public void tesSkipSystemTraces() + public void testSkipSystemTraces() { assertFalse(AutoRepairUtils.shouldConsiderKeyspace(Keyspace.open(SchemaConstants.TRACE_KEYSPACE_NAME))); } From ff67c3a222df114eb9187f8d7bf64d9f2b184faa Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Wed, 18 Dec 2024 20:35:17 -0800 Subject: [PATCH 101/115] Fix diff40 --- test/data/jmxdump/cassandra-4.0-jmx.yaml | 768 ++++++++++++++++++++++- 1 file changed, 740 insertions(+), 28 deletions(-) diff --git a/test/data/jmxdump/cassandra-4.0-jmx.yaml b/test/data/jmxdump/cassandra-4.0-jmx.yaml index e0d01272c83a..43fe3a03fbed 100644 --- a/test/data/jmxdump/cassandra-4.0-jmx.yaml +++ b/test/data/jmxdump/cassandra-4.0-jmx.yaml @@ -10190,6 +10190,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,n org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -10204,6 +10209,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,n org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11067,6 +11077,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11081,6 +11096,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11944,6 +11964,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11958,6 +11983,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -12821,6 +12851,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -12835,6 +12870,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -13698,6 +13738,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -13712,6 +13757,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -14575,6 +14625,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -14589,6 +14644,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -15452,6 +15512,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -15466,6 +15531,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -16329,6 +16399,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -16343,6 +16418,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -17206,6 +17286,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -17220,6 +17305,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18083,6 +18173,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18097,6 +18192,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18960,6 +19060,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18974,6 +19079,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19837,6 +19947,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,na org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19851,6 +19966,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,na org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -20714,6 +20834,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_st org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_statements,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -20728,6 +20853,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_st org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_statements,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -21591,6 +21721,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -21605,6 +21740,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -22468,6 +22608,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estima org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -22482,6 +22627,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estima org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -23345,6 +23495,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_act org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_activity,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -23359,6 +23514,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_act org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_activity,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -24222,6 +24382,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estim org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -24236,6 +24401,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estim org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25099,6 +25269,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25113,6 +25288,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25976,6 +26156,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25990,6 +26175,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -26853,6 +27043,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=view_builds org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=view_builds_in_progress,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -27730,6 +27925,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=networ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=network_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -29484,6 +29684,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_m org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_members,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -29498,6 +29703,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_m org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_members,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -30361,6 +30571,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_p org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -30375,6 +30590,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_p org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -31238,6 +31458,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles, org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -31252,6 +31477,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles, org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -32992,6 +33222,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope=repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -33869,6 +34104,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope=view_build_status,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -34746,6 +34986,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggr org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggregates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -34760,6 +35005,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggr org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggregates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -35623,6 +35873,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=colu org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -35637,6 +35892,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=colu org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -36500,6 +36760,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=drop org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=dropped_columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -37377,6 +37642,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=func org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=functions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -37391,6 +37661,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=func org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=functions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -38254,6 +38529,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=inde org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=indexes,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -38268,6 +38548,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=inde org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=indexes,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -39131,6 +39416,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keys org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keyspaces,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -39145,6 +39435,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keys org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keyspaces,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40008,6 +40303,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tabl org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tables,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40022,6 +40322,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tabl org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tables,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40885,6 +41190,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=trig org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=triggers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40899,6 +41209,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=trig org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=triggers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -41762,6 +42077,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=type org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=types,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -41776,6 +42096,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=type org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=types,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42639,6 +42964,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=view org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42653,6 +42983,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=view org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -43516,6 +43851,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=even org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -43530,6 +43870,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=even org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -44393,6 +44738,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sess org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sessions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -44407,6 +44757,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sess org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sessions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -45267,13 +45622,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BloomFilterOffHeapMemoryUsed - name: objectName parameters: [] returnType: javax.management.ObjectName -org.apache.cassandra.metrics:type=ColumnFamily,name=BytesAnticompacted: - attributes: - - {access: read-only, name: Value, type: java.lang.Object} - operations: - - name: objectName - parameters: [] - returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesFlushed: attributes: - {access: read-only, name: Value, type: java.lang.Object} @@ -45281,13 +45629,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BytesFlushed: - name: objectName parameters: [] returnType: javax.management.ObjectName -org.apache.cassandra.metrics:type=ColumnFamily,name=BytesMutatedAnticompaction: - attributes: - - {access: read-only, name: Value, type: java.lang.Object} - operations: - - name: objectName - parameters: [] - returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesPendingRepair: attributes: - {access: read-only, name: Value, type: java.lang.Object} @@ -57152,6 +57493,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=Blo org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -57166,6 +57512,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=Byt org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -58233,6 +58584,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -58247,6 +58603,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -59314,6 +59675,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -59328,6 +59694,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -60395,6 +60766,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=Bloom org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -60409,6 +60785,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=Bytes org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -61476,6 +61857,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -61490,6 +61876,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -62557,6 +62948,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -62571,6 +62967,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -63638,6 +64039,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -63652,6 +64058,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -64719,6 +65130,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -64733,6 +65149,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -65800,6 +66221,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -65814,6 +66240,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -66881,6 +67312,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -66895,6 +67331,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -67962,6 +68403,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -67976,6 +68422,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -69043,6 +69494,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=Bloo org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -69057,6 +69513,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=Byte org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -70124,6 +70585,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statement org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statements,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -70138,6 +70604,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statement org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statements,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -71205,6 +71676,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=Bloom org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -71219,6 +71695,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=Bytes org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -72286,6 +72767,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -72300,6 +72786,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -73367,6 +73858,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -73381,6 +73877,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -74448,6 +74949,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,na org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -74462,6 +74968,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,na org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -75529,6 +76040,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -75543,6 +76059,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -76610,6 +77131,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -76624,6 +77150,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -77691,6 +78222,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_pro org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_progress,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -77705,6 +78241,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_pro org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_progress,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -78772,6 +79313,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -78786,6 +79332,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -79853,6 +80404,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=resource_role org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=resource_role_permissons_index,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -80934,6 +81490,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members, org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -80948,6 +81509,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members, org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -82015,6 +82581,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -82029,6 +82600,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -83096,6 +83672,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=Bl org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -83110,6 +83691,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=By org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -84177,6 +84763,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=parent org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=parent_repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -85258,6 +85849,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -85272,6 +85868,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -86339,6 +86940,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=view_b org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=view_build_status,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -87420,6 +88026,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates, org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -87434,6 +88045,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates, org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -88501,6 +89117,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -88515,6 +89136,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -89582,6 +90208,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_col org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -89596,6 +90227,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_col org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -90663,6 +91299,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -90677,6 +91318,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -91744,6 +92390,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -91758,6 +92409,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -92825,6 +93481,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -92839,6 +93500,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -93906,6 +94572,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -93920,6 +94591,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -94987,6 +95663,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,na org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -95001,6 +95682,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,na org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -96068,6 +96754,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -96082,6 +96773,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -97149,6 +97845,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -97163,6 +97864,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -98230,6 +98936,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -98244,6 +98955,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -99311,6 +100027,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,na org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -99325,6 +100046,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,na org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -100389,13 +101115,6 @@ org.apache.cassandra.metrics:type=Table,name=BloomFilterOffHeapMemoryUsed: - name: objectName parameters: [] returnType: javax.management.ObjectName -org.apache.cassandra.metrics:type=Table,name=BytesAnticompacted: - attributes: - - {access: read-only, name: Value, type: java.lang.Object} - operations: - - name: objectName - parameters: [] - returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesFlushed: attributes: - {access: read-only, name: Value, type: java.lang.Object} @@ -100403,13 +101122,6 @@ org.apache.cassandra.metrics:type=Table,name=BytesFlushed: - name: objectName parameters: [] returnType: javax.management.ObjectName -org.apache.cassandra.metrics:type=Table,name=BytesMutatedAnticompaction: - attributes: - - {access: read-only, name: Value, type: java.lang.Object} - operations: - - name: objectName - parameters: [] - returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesPendingRepair: attributes: - {access: read-only, name: Value, type: java.lang.Object} From 8243151b6261910a1ed394edc43917d0384e9556 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Wed, 18 Dec 2024 20:44:57 -0800 Subject: [PATCH 102/115] Fix diff41 --- test/data/jmxdump/cassandra-4.1-jmx.yaml | 768 ++++++++++++++++++++++- 1 file changed, 740 insertions(+), 28 deletions(-) diff --git a/test/data/jmxdump/cassandra-4.1-jmx.yaml b/test/data/jmxdump/cassandra-4.1-jmx.yaml index a5ea2a74a16b..1e85c6d48fcd 100644 --- a/test/data/jmxdump/cassandra-4.1-jmx.yaml +++ b/test/data/jmxdump/cassandra-4.1-jmx.yaml @@ -10190,6 +10190,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,n org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -10204,6 +10209,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,n org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11067,6 +11077,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11081,6 +11096,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11944,6 +11964,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11958,6 +11983,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -12821,6 +12851,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -12835,6 +12870,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -13698,6 +13738,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -13712,6 +13757,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -14575,6 +14625,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -14589,6 +14644,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -15452,6 +15512,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -15466,6 +15531,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -16329,6 +16399,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -16343,6 +16418,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -17206,6 +17286,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -17220,6 +17305,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18083,6 +18173,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18097,6 +18192,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18960,6 +19060,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18974,6 +19079,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19837,6 +19947,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,na org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19851,6 +19966,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,na org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -20714,6 +20834,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_st org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_statements,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -20728,6 +20853,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_st org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_statements,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -21591,6 +21721,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -21605,6 +21740,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -22468,6 +22608,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estima org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -22482,6 +22627,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estima org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -23345,6 +23495,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_act org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_activity_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -23359,6 +23514,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_act org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_activity_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -24222,6 +24382,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estim org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -24236,6 +24401,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estim org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25099,6 +25269,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25113,6 +25288,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25976,6 +26156,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25990,6 +26175,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -26853,6 +27043,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=view_builds org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=view_builds_in_progress,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -27730,6 +27925,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=networ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=network_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -29484,6 +29684,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_m org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_members,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -29498,6 +29703,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_m org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_members,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -30361,6 +30571,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_p org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -30375,6 +30590,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_p org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -31238,6 +31458,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles, org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -31252,6 +31477,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles, org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -32992,6 +33222,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope=repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -33869,6 +34104,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope=view_build_status,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -34746,6 +34986,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggr org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggregates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -34760,6 +35005,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggr org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggregates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -35623,6 +35873,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=colu org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -35637,6 +35892,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=colu org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -36500,6 +36760,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=drop org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=dropped_columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -37377,6 +37642,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=func org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=functions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -37391,6 +37661,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=func org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=functions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -38254,6 +38529,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=inde org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=indexes,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -38268,6 +38548,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=inde org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=indexes,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -39131,6 +39416,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keys org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keyspaces,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -39145,6 +39435,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keys org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keyspaces,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40008,6 +40303,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tabl org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tables,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40022,6 +40322,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tabl org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tables,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40885,6 +41190,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=trig org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=triggers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40899,6 +41209,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=trig org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=triggers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -41762,6 +42077,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=type org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=types,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -41776,6 +42096,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=type org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=types,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42639,6 +42964,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=view org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42653,6 +42983,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=view org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -43516,6 +43851,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=even org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -43530,6 +43870,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=even org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -44393,6 +44738,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sess org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sessions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -44407,6 +44757,11 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sess org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sessions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -45267,13 +45622,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BloomFilterOffHeapMemoryUsed - name: objectName parameters: [] returnType: javax.management.ObjectName -org.apache.cassandra.metrics:type=ColumnFamily,name=BytesAnticompacted: - attributes: - - {access: read-only, name: Value, type: java.lang.Object} - operations: - - name: objectName - parameters: [] - returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesFlushed: attributes: - {access: read-only, name: Value, type: java.lang.Object} @@ -45281,13 +45629,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BytesFlushed: - name: objectName parameters: [] returnType: javax.management.ObjectName -org.apache.cassandra.metrics:type=ColumnFamily,name=BytesMutatedAnticompaction: - attributes: - - {access: read-only, name: Value, type: java.lang.Object} - operations: - - name: objectName - parameters: [] - returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesPendingRepair: attributes: - {access: read-only, name: Value, type: java.lang.Object} @@ -57152,6 +57493,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=Blo org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -57166,6 +57512,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=Byt org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -58233,6 +58584,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -58247,6 +58603,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -59314,6 +59675,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -59328,6 +59694,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -60395,6 +60766,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=Bloom org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -60409,6 +60785,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=Bytes org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -61476,6 +61857,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -61490,6 +61876,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -62557,6 +62948,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -62571,6 +62967,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -63638,6 +64039,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -63652,6 +64058,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -64719,6 +65130,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -64733,6 +65149,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -65800,6 +66221,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -65814,6 +66240,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -66881,6 +67312,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -66895,6 +67331,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -67962,6 +68403,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -67976,6 +68422,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -69043,6 +69494,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=Bloo org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -69057,6 +69513,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=Byte org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -70124,6 +70585,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statement org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statements,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -70138,6 +70604,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statement org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statements,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -71205,6 +71676,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=Bloom org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -71219,6 +71695,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=Bytes org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -72286,6 +72767,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -72300,6 +72786,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -73367,6 +73858,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -73381,6 +73877,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -74448,6 +74949,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,na org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -74462,6 +74968,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,na org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -75529,6 +76040,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -75543,6 +76059,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -76610,6 +77131,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -76624,6 +77150,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -77691,6 +78222,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_pro org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_progress,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -77705,6 +78241,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_pro org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_progress,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -78772,6 +79313,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -78786,6 +79332,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -79853,6 +80404,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=resource_role org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=resource_role_permissons_index,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -80934,6 +81490,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members, org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -80948,6 +81509,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members, org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -82015,6 +82581,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -82029,6 +82600,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -83096,6 +83672,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=Bl org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -83110,6 +83691,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=By org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -84177,6 +84763,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=parent org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=parent_repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -85258,6 +85849,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -85272,6 +85868,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -86339,6 +86940,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=view_b org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=view_build_status,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -87420,6 +88026,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates, org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -87434,6 +88045,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates, org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -88501,6 +89117,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -88515,6 +89136,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -89582,6 +90208,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_col org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -89596,6 +90227,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_col org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -90663,6 +91299,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -90677,6 +91318,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -91744,6 +92390,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -91758,6 +92409,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -92825,6 +93481,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -92839,6 +93500,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -93906,6 +94572,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -93920,6 +94591,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -94987,6 +95663,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,na org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -95001,6 +95682,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,na org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -96068,6 +96754,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -96082,6 +96773,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -97149,6 +97845,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -97163,6 +97864,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -98230,6 +98936,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -98244,6 +98955,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -99311,6 +100027,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,na org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -99325,6 +100046,11 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,na org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -100389,13 +101115,6 @@ org.apache.cassandra.metrics:type=Table,name=BloomFilterOffHeapMemoryUsed: - name: objectName parameters: [] returnType: javax.management.ObjectName -org.apache.cassandra.metrics:type=Table,name=BytesAnticompacted: - attributes: - - {access: read-only, name: Value, type: java.lang.Object} - operations: - - name: objectName - parameters: [] - returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesFlushed: attributes: - {access: read-only, name: Value, type: java.lang.Object} @@ -100403,13 +101122,6 @@ org.apache.cassandra.metrics:type=Table,name=BytesFlushed: - name: objectName parameters: [] returnType: javax.management.ObjectName -org.apache.cassandra.metrics:type=Table,name=BytesMutatedAnticompaction: - attributes: - - {access: read-only, name: Value, type: java.lang.Object} - operations: - - name: objectName - parameters: [] - returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesPendingRepair: attributes: - {access: read-only, name: Value, type: java.lang.Object} From 6c45efde23468cb9356c7dbe10f08ca9b9e7ed79 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Fri, 20 Dec 2024 08:05:01 -0800 Subject: [PATCH 103/115] Implement minimum repair task duration setting for auto-repair scheduler --- .../repair/autorepair/AutoRepair.java | 18 +++++-- .../repair/autorepair/AutoRepairConfig.java | 14 ++++++ .../cassandra/service/AutoRepairService.java | 7 +++ .../service/AutoRepairServiceMBean.java | 2 + .../org/apache/cassandra/tools/NodeProbe.java | 5 ++ .../tools/nodetool/GetAutoRepairConfig.java | 1 + .../tools/nodetool/SetAutoRepairConfig.java | 6 ++- .../autorepair/AutoRepairConfigTest.java | 10 +++- .../AutoRepairParameterizedTest.java | 47 +++++++++++++++++++ .../nodetool/SetAutoRepairConfigTest.java | 11 ++++- 10 files changed, 115 insertions(+), 6 deletions(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index d6d58746f45f..50d8e9e5dee8 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -319,8 +319,8 @@ private void repairKeyspace(AutoRepairConfig.RepairType repairType, boolean prim while (retryCount <= config.getRepairMaxRetries()) { RepairCoordinator task = repairState.getRepairRunnable(keyspaceName, - Lists.newArrayList(curRepairAssignment.getTableNames()), - ranges, primaryRangeOnly); + Lists.newArrayList(curRepairAssignment.getTableNames()), + ranges, primaryRangeOnly); repairState.resetWaitCondition(); f = repairRunnableExecutors.get(repairType).submit(task); try @@ -489,6 +489,18 @@ public AutoRepairState getRepairState(AutoRepairConfig.RepairType repairType) return repairStates.get(repairType); } + private void soakAfterRepair(long startTimeMilis, long minDurationMilis) + { + long currentTime = timeFunc.get(); + long timeElapsed = currentTime - startTimeMilis; + if (timeElapsed < minDurationMilis) + { + long timeToSoak = minDurationMilis - timeElapsed; + logger.info("Soaking for {} ms after repair", timeToSoak); + sleepFunc.accept(timeToSoak, TimeUnit.MILLISECONDS); + } + } + static class CollectedRepairStats { int failedTokenRanges = 0; @@ -496,4 +508,4 @@ static class CollectedRepairStats int skippedTokenRanges = 0; int skippedTables = 0; } -} +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java index 8eaef0b6ecc5..c3f39c1878bc 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepairConfig.java @@ -52,6 +52,10 @@ public class AutoRepairConfig implements Serializable public volatile Integer repair_max_retries = 3; // the backoff time in seconds for retrying a repair session. public volatile DurationSpec.LongSecondsBound repair_retry_backoff = new DurationSpec.LongSecondsBound("30s"); + // the minimum duration for the execution of a single repair task (i.e.: RepairRunnable). + // This helps prevent the auto-repair scheduler from overwhelming the node by scheduling a large number of + // repair tasks in a short period of time. + public volatile DurationSpec.LongSecondsBound repair_task_min_duration = new DurationSpec.LongSecondsBound("5s"); // global_settings overides Options.defaultOptions for all repair types public volatile Options global_settings; @@ -160,6 +164,16 @@ public void setRepairRetryBackoff(String interval) repair_retry_backoff = new DurationSpec.LongSecondsBound(interval); } + public DurationSpec.LongSecondsBound getRepairTaskMinDuration() + { + return repair_task_min_duration; + } + + public void setRepairTaskMinDuration(String duration) + { + repair_task_min_duration = new DurationSpec.LongSecondsBound(duration); + } + public boolean isAutoRepairEnabled(RepairType repairType) { return enabled && applyOverrides(repairType, opt -> opt.enabled); diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index d8fd68253596..5e8bbee6eb90 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -18,6 +18,7 @@ package org.apache.cassandra.service; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.autorepair.AutoRepairConfig; @@ -142,6 +143,12 @@ public void setAutoRepairRetryBackoff(String interval) config.setRepairRetryBackoff(interval); } + @Override + public void setAutoRepairMinRepairTaskDuration(String duration) + { + config.setRepairTaskMinDuration(duration); + } + @Override public void setRepairSSTableCountHigherThreshold(RepairType repairType, int sstableHigherThreshold) { diff --git a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java index 121c9a480303..cac2680c3f01 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java +++ b/src/java/org/apache/cassandra/service/AutoRepairServiceMBean.java @@ -50,6 +50,8 @@ public interface AutoRepairServiceMBean public void setAutoRepairRetryBackoff(String interval); + public void setAutoRepairMinRepairTaskDuration(String duration); + public void setRepairSSTableCountHigherThreshold(RepairType repairType, int ssTableHigherThreshold); public void setAutoRepairTableMaxRepairTime(RepairType repairType, String autoRepairTableMaxRepairTime); diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 987616ddb338..b615e3ffd729 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -2474,6 +2474,11 @@ public void setAutoRepairRetryBackoff(String interval) autoRepairProxy.setAutoRepairRetryBackoff(interval); } + public void setAutoRepairMinRepairTaskDuration(String duration) + { + autoRepairProxy.setAutoRepairMinRepairTaskDuration(duration); + } + public void setRepairSSTableCountHigherThreshold(AutoRepairConfig.RepairType repairType, int ssTableHigherThreshold) { autoRepairProxy.setRepairSSTableCountHigherThreshold(repairType, ssTableHigherThreshold); diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java index 48f8d54de3f6..681b8533824b 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/GetAutoRepairConfig.java @@ -49,6 +49,7 @@ public void execute(NodeProbe probe) sb.append("\n\tTTL for repair history for dead nodes: " + config.getAutoRepairHistoryClearDeleteHostsBufferInterval()); sb.append("\n\tmax retries for repair: " + config.getRepairMaxRetries()); sb.append("\n\tretry backoff: " + config.getRepairRetryBackoff()); + sb.append("\n\tmin repair job duration: " + config.getRepairTaskMinDuration().toSeconds() + " seconds"); for (RepairType repairType : RepairType.values()) { sb.append(formatRepairTypeConfig(probe, repairType, config)); diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java index 2929c944442a..75cdea6cd1c0 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java +++ b/src/java/org/apache/cassandra/tools/nodetool/SetAutoRepairConfig.java @@ -46,7 +46,8 @@ public class SetAutoRepairConfig extends NodeToolCmd "[start_scheduler|number_of_repair_threads|number_of_subranges|min_repair_interval|sstable_upper_threshold" + "|enabled|table_max_repair_time|priority_hosts|forcerepair_hosts|ignore_dcs" + "|history_clear_delete_hosts_buffer_interval|repair_primary_token_range_only" + - "|parallel_repair_count|parallel_repair_percentage|mv_repair_enabled|repair_max_retries|repair_retry_backoff|repair_session_timeout]", + "|parallel_repair_count|parallel_repair_percentage|mv_repair_enabled|repair_max_retries" + + "|repair_retry_backoff|repair_session_timeout|min_repair_task_duration]", required = true) protected List args = new ArrayList<>(); @@ -88,6 +89,9 @@ public void execute(NodeProbe probe) case "repair_retry_backoff": probe.setAutoRepairRetryBackoff(paramVal); return; + case "min_repair_task_duration": + probe.setAutoRepairMinRepairTaskDuration(paramVal); + return; default: // proceed to options that require --repair-type option break; diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java index 08ae9631a192..79b371fca347 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairConfigTest.java @@ -90,6 +90,15 @@ public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsEnabled() assertTrue(config.isAutoRepairEnabled(repairType)); } + @Test + public void testRepairMinDuration() + { + config = new AutoRepairConfig(false); + + config.setRepairTaskMinDuration("3s"); + assertEquals(3L, config.getRepairTaskMinDuration().toSeconds()); + } + @Test public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledGlobally() { @@ -98,7 +107,6 @@ public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledGlobally() assertFalse(config.isAutoRepairEnabled(repairType)); } - @Test public void testIsAutoRepairEnabledReturnsTrueWhenRepairIsDisabledForRepairType() { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index d2561af74c42..48b5f7904717 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -177,6 +177,7 @@ public void setup() timeFuncCalls = 0; AutoRepair.timeFunc = System::currentTimeMillis; + AutoRepair.sleepFunc = (Long startTime, TimeUnit unit) -> {}; resetCounters(); resetConfig(); @@ -219,6 +220,7 @@ private void resetConfig() config.global_settings = defaultConfig.global_settings; config.history_clear_delete_hosts_buffer_interval = defaultConfig.history_clear_delete_hosts_buffer_interval; config.setRepairSubRangeNum(repairType, 1); + config.repair_task_min_duration = new DurationSpec.LongSecondsBound("0s"); } private void executeCQL() @@ -749,4 +751,49 @@ public void testRepairThrowsForIRWithCDCReplay() AutoRepair.instance.repair(repairType); } } + + + @Test + public void testSoakAfterImmediateRepair() + { + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + when(autoRepairState.isSuccess()).thenReturn(true); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.repair_task_min_duration = new DurationSpec.LongSecondsBound("1s"); + AtomicInteger sleepCalls = new AtomicInteger(); + AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + sleepCalls.getAndIncrement(); + assertEquals(TimeUnit.MILLISECONDS, unit); + assertTrue(config.getRepairTaskMinDuration().toMilliseconds() >= duration); + }; + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + + AutoRepair.instance.repair(repairType); + + assertEquals(expectedTablesGoingThroughRepair, sleepCalls.get()); + verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(expectedTablesGoingThroughRepair); + verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); + verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(0); + } + + @Test + public void testNoSoakAfterRepair() + { + when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); + when(autoRepairState.isSuccess()).thenReturn(true); + AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); + config.repair_task_min_duration = new DurationSpec.LongSecondsBound("0s"); + AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { + fail("Should not sleep after repair"); + }; + config.setRepairMinInterval(repairType, "0s"); + AutoRepair.instance.repairStates.put(repairType, autoRepairState); + + AutoRepair.instance.repair(repairType); + + verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(expectedTablesGoingThroughRepair); + verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); + verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(0); + } } diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java index 6e2d79def026..e12af34af8d8 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/SetAutoRepairConfigTest.java @@ -105,7 +105,6 @@ public void testRepairMaxRetries() verify(probe, times(1)).setAutoRepairMaxRetriesCount(2); } - @Test public void testRetryBackoffInSec() { @@ -131,6 +130,16 @@ public void testStartScheduler() verify(probe, times(1)).startScheduler(); } + + @Test + public void testMinRepairDuration() + { + cmd.args = ImmutableList.of("min_repair_task_duration", "4s"); + + cmd.execute(probe); + + verify(probe, times(1)).setAutoRepairMinRepairTaskDuration("4s"); + } } @RunWith(Parameterized.class) From 0caa56329ebff3ee8e194a5ac14d8bdc6abddaf7 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Fri, 20 Dec 2024 08:09:45 -0800 Subject: [PATCH 104/115] Cleanup --- .../cassandra/repair/autorepair/AutoRepairParameterizedTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 48b5f7904717..809de187c38e 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -104,7 +104,6 @@ public class AutoRepairParameterizedTest extends CQLTester // Expected number of tables that should be repaired. private static int expectedTablesGoingThroughRepair; - @Parameterized.Parameter() public AutoRepairConfig.RepairType repairType; From 9f555fce0d95eba0e4d20d758a98d1313411166c Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Fri, 20 Dec 2024 08:17:22 -0800 Subject: [PATCH 105/115] Cleanup --- src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java | 2 ++ .../repair/autorepair/AutoRepairParameterizedTest.java | 1 - 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 50d8e9e5dee8..df0db2854355 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -325,7 +325,9 @@ private void repairKeyspace(AutoRepairConfig.RepairType repairType, boolean prim f = repairRunnableExecutors.get(repairType).submit(task); try { + long jobStartTime = timeFunc.get(); repairState.waitForRepairToComplete(config.getRepairSessionTimeout(repairType)); + soakAfterRepair(jobStartTime, config.getRepairTaskMinDuration().toMilliseconds()); } catch (InterruptedException e) { diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 809de187c38e..7a44a05a498d 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -751,7 +751,6 @@ public void testRepairThrowsForIRWithCDCReplay() } } - @Test public void testSoakAfterImmediateRepair() { From 3c6bc2546429fabefd3497f1a9e75f555d5c2e45 Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys Date: Sat, 21 Dec 2024 08:29:55 -0800 Subject: [PATCH 106/115] Address comment --- .../repair/autorepair/AutoRepairParameterizedTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java index 7a44a05a498d..4f60d8c95af5 100644 --- a/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java +++ b/test/unit/org/apache/cassandra/repair/autorepair/AutoRepairParameterizedTest.java @@ -757,19 +757,20 @@ public void testSoakAfterImmediateRepair() when(autoRepairState.getRepairRunnable(any(), any(), any(), anyBoolean())).thenReturn(repairRunnable); when(autoRepairState.isSuccess()).thenReturn(true); AutoRepairConfig config = AutoRepairService.instance.getAutoRepairConfig(); - config.repair_task_min_duration = new DurationSpec.LongSecondsBound("1s"); + config.repair_task_min_duration = new DurationSpec.LongSecondsBound("10s"); AtomicInteger sleepCalls = new AtomicInteger(); AutoRepair.sleepFunc = (Long duration, TimeUnit unit) -> { sleepCalls.getAndIncrement(); assertEquals(TimeUnit.MILLISECONDS, unit); assertTrue(config.getRepairTaskMinDuration().toMilliseconds() >= duration); + config.repair_task_min_duration = new DurationSpec.LongSecondsBound("0s"); }; config.setRepairMinInterval(repairType, "0s"); AutoRepair.instance.repairStates.put(repairType, autoRepairState); AutoRepair.instance.repair(repairType); - assertEquals(expectedTablesGoingThroughRepair, sleepCalls.get()); + assertEquals(1, sleepCalls.get()); verify(autoRepairState, Mockito.times(1)).setSucceededTokenRangesCount(expectedTablesGoingThroughRepair); verify(autoRepairState, Mockito.times(1)).setSkippedTokenRangesCount(0); verify(autoRepairState, Mockito.times(1)).setFailedTokenRangesCount(0); From c06acafa0da3fb7d38beb6b595f0d0c272960444 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Sat, 21 Dec 2024 10:22:33 -0800 Subject: [PATCH 107/115] Remove unused imports --- src/java/org/apache/cassandra/service/AutoRepairService.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/java/org/apache/cassandra/service/AutoRepairService.java b/src/java/org/apache/cassandra/service/AutoRepairService.java index 5e8bbee6eb90..167e4507a7c1 100644 --- a/src/java/org/apache/cassandra/service/AutoRepairService.java +++ b/src/java/org/apache/cassandra/service/AutoRepairService.java @@ -18,7 +18,6 @@ package org.apache.cassandra.service; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.autorepair.AutoRepairConfig; From c08b8d9d95c0e0576cad654927193d553763bbd2 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Sat, 21 Dec 2024 10:37:33 -0800 Subject: [PATCH 108/115] formatting --- src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index df0db2854355..1bd49e1b214a 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -510,4 +510,4 @@ static class CollectedRepairStats int skippedTokenRanges = 0; int skippedTables = 0; } -} \ No newline at end of file +} From 9b8a582f3db9bcc4e3d6a04db090b56f54b66188 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Fri, 20 Dec 2024 11:10:38 -0800 Subject: [PATCH 109/115] Convert the metrics to TableMeter --- .../cassandra/metrics/KeyspaceMetrics.java | 10 ++++++ .../cassandra/metrics/TableMetrics.java | 31 +++++++++++-------- test/data/jmxdump/cassandra-4.0-jmx.yaml | 24 ++++++++++++++ test/data/jmxdump/cassandra-4.1-jmx.yaml | 24 ++++++++++++++ 4 files changed, 76 insertions(+), 13 deletions(-) diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java index 237fd03e2d7d..148a7495f9cf 100644 --- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java +++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java @@ -180,6 +180,11 @@ public class KeyspaceMetrics public final Meter tooManySSTableIndexesReadWarnings; public final Meter tooManySSTableIndexesReadAborts; + public final Meter bytesAnticompacted; + public final Meter bytesMutatedAnticompaction; + public final Meter bytesPreviewed; + public final Meter tokenRangesPreviewedDesynchronized; + public final Meter bytesPreviewedDesynchronized; public final ImmutableMap, ImmutableMap>> formatSpecificGauges; @@ -288,6 +293,11 @@ public KeyspaceMetrics(final Keyspace ks) outOfRangeTokenReads = createKeyspaceCounter("ReadOutOfRangeToken"); outOfRangeTokenWrites = createKeyspaceCounter("WriteOutOfRangeToken"); outOfRangeTokenPaxosRequests = createKeyspaceCounter("PaxosOutOfRangeToken"); + bytesAnticompacted = createKeyspaceMeter("BytesAnticompacted"); + bytesMutatedAnticompaction = createKeyspaceMeter("BytesMutatedAnticompaction"); + bytesPreviewed = createKeyspaceMeter("BytesPreviewed"); + tokenRangesPreviewedDesynchronized = createKeyspaceMeter("TokenRangesPreviewedDesynchronized"); + bytesPreviewedDesynchronized = createKeyspaceMeter("BytesPreviewedDesynchronized"); } /** diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index 5e5890311025..67b5d0c1e3f5 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -207,15 +207,15 @@ public class TableMetrics /** number of partitions read creating merkle trees */ public final TableHistogram partitionsValidated; /** number of bytes read while doing anticompaction */ - public final Meter bytesAnticompacted; + public final TableMeter bytesAnticompacted; /** number of bytes where the whole sstable was contained in a repairing range so that we only mutated the repair status */ - public final Meter bytesMutatedAnticompaction; + public final TableMeter bytesMutatedAnticompaction; /** number of bytes that were scanned during preview repair */ - public final Meter bytesPreviewed; + public final TableMeter bytesPreviewed; /** number of desynchronized token ranges that were detected during preview repair */ - public final Meter tokenRangesPreviewedDesynchronized; + public final TableMeter tokenRangesPreviewedDesynchronized; /** number of desynchronized bytes that were detected during preview repair */ - public final Meter bytesPreviewedDesynchronized; + public final TableMeter bytesPreviewedDesynchronized; /** ratio of how much we anticompact vs how much we could mutate the repair status*/ public final Gauge mutatedAnticompactionGauge; @@ -814,15 +814,15 @@ public Long getValue() bytesValidated = createTableHistogram("BytesValidated", cfs.keyspace.metric.bytesValidated, false); partitionsValidated = createTableHistogram("PartitionsValidated", cfs.keyspace.metric.partitionsValidated, false); - bytesAnticompacted = createTableMeter("BytesAnticompacted"); - bytesMutatedAnticompaction = createTableMeter("BytesMutatedAnticompaction"); - bytesPreviewed = createTableMeter("BytesPreviewed"); - tokenRangesPreviewedDesynchronized = createTableMeter("TokenRangesPreviewedDesynchronized"); - bytesPreviewedDesynchronized = createTableMeter("BytesPreviewedDesynchronized"); + bytesAnticompacted = createTableMeter("BytesAnticompacted", cfs.keyspace.metric.bytesAnticompacted); + bytesMutatedAnticompaction = createTableMeter("BytesMutatedAnticompaction", cfs.keyspace.metric.bytesMutatedAnticompaction); + bytesPreviewed = createTableMeter("BytesPreviewed", cfs.keyspace.metric.bytesPreviewed); + tokenRangesPreviewedDesynchronized = createTableMeter("TokenRangesPreviewedDesynchronized", cfs.keyspace.metric.tokenRangesPreviewedDesynchronized); + bytesPreviewedDesynchronized = createTableMeter("BytesPreviewedDesynchronized", cfs.keyspace.metric.bytesPreviewedDesynchronized); mutatedAnticompactionGauge = createTableGauge("MutatedAnticompactionGauge", () -> { - double bytesMutated = bytesMutatedAnticompaction.getCount(); - double bytesAnticomp = bytesAnticompacted.getCount(); + double bytesMutated = bytesMutatedAnticompaction.table.getCount(); + double bytesAnticomp = bytesAnticompacted.table.getCount(); if (bytesAnticomp + bytesMutated > 0) return bytesMutated / (bytesAnticomp + bytesMutated); return 0.0; @@ -1154,10 +1154,15 @@ private TableMeter(Meter table, Meter keyspace, Meter global) } public void mark() + { + mark(1L); + } + + public void mark(long val) { for (Meter meter : all) { - meter.mark(); + meter.mark(val); } } } diff --git a/test/data/jmxdump/cassandra-4.0-jmx.yaml b/test/data/jmxdump/cassandra-4.0-jmx.yaml index 43fe3a03fbed..5490ffcd4170 100644 --- a/test/data/jmxdump/cassandra-4.0-jmx.yaml +++ b/test/data/jmxdump/cassandra-4.0-jmx.yaml @@ -101115,6 +101115,18 @@ org.apache.cassandra.metrics:type=Table,name=BloomFilterOffHeapMemoryUsed: - name: objectName parameters: [] returnType: javax.management.ObjectName +org.apache.cassandra.metrics:type=Table,name=BytesAnticompacted: + attributes: + - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} + operations: + - name: objectName + parameters: [] + returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesFlushed: attributes: - {access: read-only, name: Value, type: java.lang.Object} @@ -101122,6 +101134,18 @@ org.apache.cassandra.metrics:type=Table,name=BytesFlushed: - name: objectName parameters: [] returnType: javax.management.ObjectName +org.apache.cassandra.metrics:type=Table,name=BytesMutatedAnticompaction: + attributes: + - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} + operations: + - name: objectName + parameters: [] + returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesPendingRepair: attributes: - {access: read-only, name: Value, type: java.lang.Object} diff --git a/test/data/jmxdump/cassandra-4.1-jmx.yaml b/test/data/jmxdump/cassandra-4.1-jmx.yaml index 1e85c6d48fcd..8623da7bf414 100644 --- a/test/data/jmxdump/cassandra-4.1-jmx.yaml +++ b/test/data/jmxdump/cassandra-4.1-jmx.yaml @@ -101115,6 +101115,18 @@ org.apache.cassandra.metrics:type=Table,name=BloomFilterOffHeapMemoryUsed: - name: objectName parameters: [] returnType: javax.management.ObjectName +org.apache.cassandra.metrics:type=Table,name=BytesAnticompacted: + attributes: + - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} + operations: + - name: objectName + parameters: [] + returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesFlushed: attributes: - {access: read-only, name: Value, type: java.lang.Object} @@ -101122,6 +101134,18 @@ org.apache.cassandra.metrics:type=Table,name=BytesFlushed: - name: objectName parameters: [] returnType: javax.management.ObjectName +org.apache.cassandra.metrics:type=Table,name=BytesMutatedAnticompaction: + attributes: + - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} + operations: + - name: objectName + parameters: [] + returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesPendingRepair: attributes: - {access: read-only, name: Value, type: java.lang.Object} From c265030b621151180acbf28470ad35214f5607b2 Mon Sep 17 00:00:00 2001 From: Jaydeepkumar Chovatia Date: Fri, 20 Dec 2024 11:15:40 -0800 Subject: [PATCH 110/115] Add a global type=ColumnFamily --- test/data/jmxdump/cassandra-4.0-jmx.yaml | 24 ++++++++++++++++++++++++ test/data/jmxdump/cassandra-4.1-jmx.yaml | 24 ++++++++++++++++++++++++ 2 files changed, 48 insertions(+) diff --git a/test/data/jmxdump/cassandra-4.0-jmx.yaml b/test/data/jmxdump/cassandra-4.0-jmx.yaml index 5490ffcd4170..e3c86eb84365 100644 --- a/test/data/jmxdump/cassandra-4.0-jmx.yaml +++ b/test/data/jmxdump/cassandra-4.0-jmx.yaml @@ -45622,6 +45622,18 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BloomFilterOffHeapMemoryUsed - name: objectName parameters: [] returnType: javax.management.ObjectName +org.apache.cassandra.metrics:type=ColumnFamily,name=BytesAnticompacted: + attributes: + - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} + operations: + - name: objectName + parameters: [] + returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesFlushed: attributes: - {access: read-only, name: Value, type: java.lang.Object} @@ -45629,6 +45641,18 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BytesFlushed: - name: objectName parameters: [] returnType: javax.management.ObjectName +org.apache.cassandra.metrics:type=ColumnFamily,name=BytesMutatedAnticompaction: + attributes: + - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} + operations: + - name: objectName + parameters: [] + returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesPendingRepair: attributes: - {access: read-only, name: Value, type: java.lang.Object} diff --git a/test/data/jmxdump/cassandra-4.1-jmx.yaml b/test/data/jmxdump/cassandra-4.1-jmx.yaml index 8623da7bf414..e1c141989ec2 100644 --- a/test/data/jmxdump/cassandra-4.1-jmx.yaml +++ b/test/data/jmxdump/cassandra-4.1-jmx.yaml @@ -45622,6 +45622,18 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BloomFilterOffHeapMemoryUsed - name: objectName parameters: [] returnType: javax.management.ObjectName +org.apache.cassandra.metrics:type=ColumnFamily,name=BytesAnticompacted: + attributes: + - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} + operations: + - name: objectName + parameters: [] + returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesFlushed: attributes: - {access: read-only, name: Value, type: java.lang.Object} @@ -45629,6 +45641,18 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BytesFlushed: - name: objectName parameters: [] returnType: javax.management.ObjectName +org.apache.cassandra.metrics:type=ColumnFamily,name=BytesMutatedAnticompaction: + attributes: + - {access: read-only, name: Count, type: long} + - {access: read-only, name: FifteenMinuteRate, type: double} + - {access: read-only, name: FiveMinuteRate, type: double} + - {access: read-only, name: MeanRate, type: double} + - {access: read-only, name: OneMinuteRate, type: double} + - {access: read-only, name: RateUnit, type: java.lang.String} + operations: + - name: objectName + parameters: [] + returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesPendingRepair: attributes: - {access: read-only, name: Value, type: java.lang.Object} From d19138a41a34fdd65204384a92167d7df95bc4be Mon Sep 17 00:00:00 2001 From: Jaydeepkumar Chovatia Date: Fri, 20 Dec 2024 11:18:21 -0800 Subject: [PATCH 111/115] Remove an extra empty line From cfc67053cb3c861875c44303cd105eae4dd773c0 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 Date: Fri, 20 Dec 2024 14:22:29 -0800 Subject: [PATCH 112/115] Revert the cassandra-4.x.xml changes --- test/data/jmxdump/cassandra-4.0-jmx.yaml | 768 +---------------------- test/data/jmxdump/cassandra-4.1-jmx.yaml | 768 +---------------------- 2 files changed, 8 insertions(+), 1528 deletions(-) diff --git a/test/data/jmxdump/cassandra-4.0-jmx.yaml b/test/data/jmxdump/cassandra-4.0-jmx.yaml index e3c86eb84365..e0d01272c83a 100644 --- a/test/data/jmxdump/cassandra-4.0-jmx.yaml +++ b/test/data/jmxdump/cassandra-4.0-jmx.yaml @@ -10190,11 +10190,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,n org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -10209,11 +10204,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,n org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11077,11 +11067,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11096,11 +11081,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11964,11 +11944,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11983,11 +11958,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -12851,11 +12821,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -12870,11 +12835,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -13738,11 +13698,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -13757,11 +13712,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -14625,11 +14575,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -14644,11 +14589,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -15512,11 +15452,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -15531,11 +15466,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -16399,11 +16329,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -16418,11 +16343,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -17286,11 +17206,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -17305,11 +17220,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18173,11 +18083,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18192,11 +18097,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19060,11 +18960,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19079,11 +18974,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19947,11 +19837,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,na org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19966,11 +19851,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,na org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -20834,11 +20714,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_st org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_statements,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -20853,11 +20728,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_st org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_statements,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -21721,11 +21591,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -21740,11 +21605,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -22608,11 +22468,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estima org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -22627,11 +22482,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estima org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -23495,11 +23345,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_act org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_activity,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -23514,11 +23359,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_act org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_activity,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -24382,11 +24222,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estim org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -24401,11 +24236,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estim org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25269,11 +25099,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25288,11 +25113,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -26156,11 +25976,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -26175,11 +25990,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -27043,11 +26853,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=view_builds org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=view_builds_in_progress,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -27925,11 +27730,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=networ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=network_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -29684,11 +29484,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_m org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_members,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -29703,11 +29498,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_m org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_members,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -30571,11 +30361,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_p org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -30590,11 +30375,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_p org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -31458,11 +31238,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles, org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -31477,11 +31252,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles, org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -33222,11 +32992,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope=repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -34104,11 +33869,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope=view_build_status,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -34986,11 +34746,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggr org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggregates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -35005,11 +34760,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggr org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggregates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -35873,11 +35623,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=colu org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -35892,11 +35637,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=colu org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -36760,11 +36500,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=drop org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=dropped_columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -37642,11 +37377,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=func org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=functions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -37661,11 +37391,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=func org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=functions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -38529,11 +38254,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=inde org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=indexes,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -38548,11 +38268,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=inde org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=indexes,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -39416,11 +39131,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keys org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keyspaces,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -39435,11 +39145,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keys org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keyspaces,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40303,11 +40008,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tabl org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tables,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40322,11 +40022,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tabl org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tables,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -41190,11 +40885,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=trig org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=triggers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -41209,11 +40899,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=trig org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=triggers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42077,11 +41762,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=type org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=types,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42096,11 +41776,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=type org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=types,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42964,11 +42639,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=view org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42983,11 +42653,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=view org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -43851,11 +43516,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=even org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -43870,11 +43530,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=even org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -44738,11 +44393,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sess org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sessions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -44757,11 +44407,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sess org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sessions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -45624,12 +45269,7 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BloomFilterOffHeapMemoryUsed returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesAnticompacted: attributes: - - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} + - {access: read-only, name: Value, type: java.lang.Object} operations: - name: objectName parameters: [] @@ -45643,12 +45283,7 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BytesFlushed: returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesMutatedAnticompaction: attributes: - - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} + - {access: read-only, name: Value, type: java.lang.Object} operations: - name: objectName parameters: [] @@ -57517,11 +57152,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=Blo org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -57536,11 +57166,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=Byt org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -58608,11 +58233,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -58627,11 +58247,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -59699,11 +59314,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -59718,11 +59328,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -60790,11 +60395,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=Bloom org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -60809,11 +60409,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=Bytes org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -61881,11 +61476,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -61900,11 +61490,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -62972,11 +62557,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -62991,11 +62571,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -64063,11 +63638,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -64082,11 +63652,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -65154,11 +64719,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -65173,11 +64733,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -66245,11 +65800,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -66264,11 +65814,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -67336,11 +66881,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -67355,11 +66895,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -68427,11 +67962,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -68446,11 +67976,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -69518,11 +69043,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=Bloo org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -69537,11 +69057,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=Byte org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -70609,11 +70124,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statement org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statements,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -70628,11 +70138,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statement org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statements,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -71700,11 +71205,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=Bloom org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -71719,11 +71219,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=Bytes org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -72791,11 +72286,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -72810,11 +72300,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -73882,11 +73367,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -73901,11 +73381,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -74973,11 +74448,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,na org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -74992,11 +74462,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,na org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -76064,11 +75529,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -76083,11 +75543,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -77155,11 +76610,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -77174,11 +76624,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -78246,11 +77691,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_pro org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_progress,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -78265,11 +77705,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_pro org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_progress,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -79337,11 +78772,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -79356,11 +78786,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -80428,11 +79853,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=resource_role org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=resource_role_permissons_index,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -81514,11 +80934,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members, org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -81533,11 +80948,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members, org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -82605,11 +82015,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -82624,11 +82029,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -83696,11 +83096,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=Bl org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -83715,11 +83110,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=By org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -84787,11 +84177,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=parent org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=parent_repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -85873,11 +85258,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -85892,11 +85272,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -86964,11 +86339,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=view_b org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=view_build_status,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -88050,11 +87420,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates, org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -88069,11 +87434,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates, org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -89141,11 +88501,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -89160,11 +88515,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -90232,11 +89582,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_col org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -90251,11 +89596,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_col org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -91323,11 +90663,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -91342,11 +90677,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -92414,11 +91744,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -92433,11 +91758,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -93505,11 +92825,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -93524,11 +92839,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -94596,11 +93906,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -94615,11 +93920,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -95687,11 +94987,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,na org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -95706,11 +95001,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,na org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -96778,11 +96068,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -96797,11 +96082,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -97869,11 +97149,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -97888,11 +97163,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -98960,11 +98230,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -98979,11 +98244,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -100051,11 +99311,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,na org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -100070,11 +99325,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,na org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -101141,12 +100391,7 @@ org.apache.cassandra.metrics:type=Table,name=BloomFilterOffHeapMemoryUsed: returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesAnticompacted: attributes: - - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} + - {access: read-only, name: Value, type: java.lang.Object} operations: - name: objectName parameters: [] @@ -101160,12 +100405,7 @@ org.apache.cassandra.metrics:type=Table,name=BytesFlushed: returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesMutatedAnticompaction: attributes: - - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} + - {access: read-only, name: Value, type: java.lang.Object} operations: - name: objectName parameters: [] diff --git a/test/data/jmxdump/cassandra-4.1-jmx.yaml b/test/data/jmxdump/cassandra-4.1-jmx.yaml index e1c141989ec2..a5ea2a74a16b 100644 --- a/test/data/jmxdump/cassandra-4.1-jmx.yaml +++ b/test/data/jmxdump/cassandra-4.1-jmx.yaml @@ -10190,11 +10190,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,n org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -10209,11 +10204,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,n org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11077,11 +11067,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11096,11 +11081,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11964,11 +11944,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -11983,11 +11958,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_r org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -12851,11 +12821,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -12870,11 +12835,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -13738,11 +13698,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -13757,11 +13712,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -14625,11 +14575,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -14644,11 +14589,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -15512,11 +15452,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -15531,11 +15466,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -16399,11 +16329,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -16418,11 +16343,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -17286,11 +17206,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -17305,11 +17220,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18173,11 +18083,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -18192,11 +18097,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peer_events_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19060,11 +18960,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19079,11 +18974,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name= org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19947,11 +19837,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,na org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -19966,11 +19851,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,na org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=peers_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -20834,11 +20714,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_st org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_statements,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -20853,11 +20728,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_st org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=prepared_statements,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -21721,11 +21591,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -21740,11 +21605,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,nam org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=repairs,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -22608,11 +22468,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estima org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -22627,11 +22482,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estima org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=size_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -23495,11 +23345,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_act org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_activity_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -23514,11 +23359,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_act org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=sstable_activity_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -24382,11 +24222,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estim org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -24401,11 +24236,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estim org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=table_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25269,11 +25099,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -25288,11 +25113,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -26156,11 +25976,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -26175,11 +25990,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=transferred_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -27043,11 +26853,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=view_builds org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=view_builds_in_progress,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -27925,11 +27730,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=networ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=network_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -29684,11 +29484,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_m org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_members,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -29703,11 +29498,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_m org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_members,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -30571,11 +30361,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_p org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -30590,11 +30375,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_p org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=role_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -31458,11 +31238,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles, org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -31477,11 +31252,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles, org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_auth,scope=roles,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -33222,11 +32992,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope=repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -34104,11 +33869,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_distributed,scope=view_build_status,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -34986,11 +34746,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggr org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggregates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -35005,11 +34760,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggr org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=aggregates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -35873,11 +35623,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=colu org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -35892,11 +35637,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=colu org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -36760,11 +36500,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=drop org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=dropped_columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -37642,11 +37377,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=func org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=functions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -37661,11 +37391,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=func org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=functions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -38529,11 +38254,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=inde org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=indexes,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -38548,11 +38268,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=inde org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=indexes,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -39416,11 +39131,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keys org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keyspaces,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -39435,11 +39145,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keys org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=keyspaces,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40303,11 +40008,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tabl org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tables,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -40322,11 +40022,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tabl org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=tables,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -41190,11 +40885,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=trig org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=triggers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -41209,11 +40899,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=trig org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=triggers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42077,11 +41762,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=type org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=types,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42096,11 +41776,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=type org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=types,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42964,11 +42639,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=view org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -42983,11 +42653,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=view org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_schema,scope=views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -43851,11 +43516,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=even org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -43870,11 +43530,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=even org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -44738,11 +44393,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sess org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sessions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -44757,11 +44407,6 @@ org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sess org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system_traces,scope=sessions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -45624,12 +45269,7 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BloomFilterOffHeapMemoryUsed returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesAnticompacted: attributes: - - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} + - {access: read-only, name: Value, type: java.lang.Object} operations: - name: objectName parameters: [] @@ -45643,12 +45283,7 @@ org.apache.cassandra.metrics:type=ColumnFamily,name=BytesFlushed: returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=ColumnFamily,name=BytesMutatedAnticompaction: attributes: - - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} + - {access: read-only, name: Value, type: java.lang.Object} operations: - name: objectName parameters: [] @@ -57517,11 +57152,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=Blo org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -57536,11 +57166,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=Byt org.apache.cassandra.metrics:type=Table,keyspace=system,scope=IndexInfo,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -58608,11 +58233,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -58627,11 +58247,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,n org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -59699,11 +59314,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -59718,11 +59328,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=available_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -60790,11 +60395,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=Bloom org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -60809,11 +60409,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=Bytes org.apache.cassandra.metrics:type=Table,keyspace=system,scope=batches,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -61881,11 +61476,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -61900,11 +61490,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=built_views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -62972,11 +62557,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -62991,11 +62571,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history org.apache.cassandra.metrics:type=Table,keyspace=system,scope=compaction_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -64063,11 +63638,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -64082,11 +63652,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -65154,11 +64719,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -65173,11 +64733,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=paxos,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -66245,11 +65800,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -66264,11 +65814,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=B org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -67336,11 +66881,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -67355,11 +66895,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peer_events_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -68427,11 +67962,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BloomFi org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -68446,11 +67976,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesFl org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -69518,11 +69043,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=Bloo org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -69537,11 +69057,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=Byte org.apache.cassandra.metrics:type=Table,keyspace=system,scope=peers_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -70609,11 +70124,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statement org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statements,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -70628,11 +70138,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statement org.apache.cassandra.metrics:type=Table,keyspace=system,scope=prepared_statements,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -71700,11 +71205,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=Bloom org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -71719,11 +71219,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=Bytes org.apache.cassandra.metrics:type=Table,keyspace=system,scope=repairs,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -72791,11 +72286,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -72810,11 +72300,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,nam org.apache.cassandra.metrics:type=Table,keyspace=system,scope=size_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -73882,11 +73367,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -73901,11 +73381,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity_v org.apache.cassandra.metrics:type=Table,keyspace=system,scope=sstable_activity_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -74973,11 +74448,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,na org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -74992,11 +74462,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,na org.apache.cassandra.metrics:type=Table,keyspace=system,scope=table_estimates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -76064,11 +75529,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -76083,11 +75543,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -77155,11 +76610,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges_v2,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -77174,11 +76624,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges org.apache.cassandra.metrics:type=Table,keyspace=system,scope=transferred_ranges_v2,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -78246,11 +77691,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_pro org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_progress,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -78265,11 +77705,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_pro org.apache.cassandra.metrics:type=Table,keyspace=system,scope=view_builds_in_progress,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -79337,11 +78772,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -79356,11 +78786,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=network_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -80428,11 +79853,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=resource_role org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=resource_role_permissons_index,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -81514,11 +80934,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members, org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -81533,11 +80948,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members, org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_members,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -82605,11 +82015,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -82624,11 +82029,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissi org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=role_permissions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -83696,11 +83096,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=Bl org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -83715,11 +83110,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=By org.apache.cassandra.metrics:type=Table,keyspace=system_auth,scope=roles,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -84787,11 +84177,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=parent org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=parent_repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -85873,11 +85258,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair_history,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -85892,11 +85272,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=repair_history,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -86964,11 +86339,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=view_b org.apache.cassandra.metrics:type=Table,keyspace=system_distributed,scope=view_build_status,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -88050,11 +87420,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates, org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -88069,11 +87434,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates, org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=aggregates,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -89141,11 +88501,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -89160,11 +88515,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -90232,11 +89582,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_col org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_columns,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -90251,11 +89596,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_col org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=dropped_columns,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -91323,11 +90663,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -91342,11 +90677,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=functions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -92414,11 +91744,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -92433,11 +91758,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,nam org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=indexes,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -93505,11 +92825,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -93524,11 +92839,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,n org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=keyspaces,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -94596,11 +93906,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -94615,11 +93920,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=tables,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -95687,11 +94987,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,na org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -95706,11 +95001,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,na org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=triggers,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -96778,11 +96068,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -96797,11 +96082,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=types,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -97869,11 +97149,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -97888,11 +97163,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name= org.apache.cassandra.metrics:type=Table,keyspace=system_schema,scope=views,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -98960,11 +98230,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -98979,11 +98244,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=events,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -100051,11 +99311,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,na org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,name=BytesAnticompacted: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -100070,11 +99325,6 @@ org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,na org.apache.cassandra.metrics:type=Table,keyspace=system_traces,scope=sessions,name=BytesMutatedAnticompaction: attributes: - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} operations: - name: objectName parameters: [] @@ -101141,12 +100391,7 @@ org.apache.cassandra.metrics:type=Table,name=BloomFilterOffHeapMemoryUsed: returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesAnticompacted: attributes: - - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} + - {access: read-only, name: Value, type: java.lang.Object} operations: - name: objectName parameters: [] @@ -101160,12 +100405,7 @@ org.apache.cassandra.metrics:type=Table,name=BytesFlushed: returnType: javax.management.ObjectName org.apache.cassandra.metrics:type=Table,name=BytesMutatedAnticompaction: attributes: - - {access: read-only, name: Count, type: long} - - {access: read-only, name: FifteenMinuteRate, type: double} - - {access: read-only, name: FiveMinuteRate, type: double} - - {access: read-only, name: MeanRate, type: double} - - {access: read-only, name: OneMinuteRate, type: double} - - {access: read-only, name: RateUnit, type: java.lang.String} + - {access: read-only, name: Value, type: java.lang.Object} operations: - name: objectName parameters: [] From de582fda1112a6f8a815241d5720d76e97d71d51 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Fri, 20 Dec 2024 21:54:44 -0800 Subject: [PATCH 113/115] gauge compatible meter --- .../metrics/CassandraMetricsRegistry.java | 52 +++++++++++++++++-- .../cassandra/metrics/TableMetrics.java | 15 ++++-- 2 files changed, 58 insertions(+), 9 deletions(-) diff --git a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java index 919d3163e317..2fe4f1b23f89 100644 --- a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java +++ b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java @@ -301,9 +301,14 @@ public Counter counter(MetricName... name) } public Meter meter(MetricName... name) + { + return meter(false, name); + } + + public Meter meter(boolean gaugeCompatible, MetricName... name) { Meter meter = super.meter(name[0].getMetricName()); - Stream.of(name).forEach(n -> register(n, meter)); + Stream.of(name).forEach(n -> register(gaugeCompatible, n, meter)); return meter; } @@ -370,6 +375,11 @@ public static SnapshottingReservoir createReservoir(TimeUnit durationUnit) } public T register(MetricName name, T metric) + { + return register(false, name, metric); + } + + public T register(boolean gaugeCompatible, MetricName name, T metric) { if (metric instanceof MetricSet) throw new IllegalArgumentException("MetricSet registration using MetricName is not supported"); @@ -377,7 +387,7 @@ public T register(MetricName name, T metric) try { verifyUnknownMetric(name); - registerMBean(metric, name.getMBeanName(), MBeanWrapper.instance); + registerMBean(metric, name.getMBeanName(), MBeanWrapper.instance, gaugeCompatible); return super.register(name.getMetricName(), metric); } catch (IllegalArgumentException e) @@ -491,7 +501,7 @@ public interface MetricNameResolver @Nullable String resolve(String fullName); } - private void registerMBean(Metric metric, ObjectName name, MBeanWrapper mBeanServer) + public void registerMBean(Metric metric, ObjectName name, MBeanWrapper mBeanServer, boolean gaugeCompatible) { AbstractBean mbean; @@ -504,7 +514,18 @@ else if (metric instanceof Histogram) else if (metric instanceof Timer) mbean = new JmxTimer((Timer) metric, name, TimeUnit.SECONDS, DEFAULT_TIMER_UNIT); else if (metric instanceof Metered) - mbean = new JmxMeter((Metered) metric, name, TimeUnit.SECONDS); + { + // If a gauge compatible meter is requested, create a special implementation which + // also yields a 'Value' attribute for backwards compatibility. + if (gaugeCompatible) + { + mbean = new JmxMeterGaugeCompatible((Metered) metric, name, TimeUnit.SECONDS); + } + else + { + mbean = new JmxMeter((Metered) metric, name, TimeUnit.SECONDS); + } + } else throw new IllegalArgumentException("Unknown metric type: " + metric.getClass()); @@ -816,6 +837,29 @@ private String calculateRateUnit(TimeUnit unit) } } + public interface JmxMeterGaugeCompatibleMBean extends JmxMeterMBean, JmxGaugeMBean {} + + /** + * An implementation of {@link JmxMeter} that is compatible with {@link JmxGaugeMBean} in that it also + * implements {@link JmxGaugeMBean}. This is useful for metrics that were migrated from {@link JmxGauge} + * to {@link JmxMeter} like {@link TableMetrics#bytesAnticompacted} and + * {@link TableMetrics#bytesMutatedAnticompaction}. + */ + private static class JmxMeterGaugeCompatible extends JmxMeter implements JmxMeterGaugeCompatibleMBean + { + + private JmxMeterGaugeCompatible(Metered metric, ObjectName objectName, TimeUnit rateUnit) + { + super(metric, objectName, rateUnit); + } + + @Override + public Object getValue() + { + return getCount(); + } + } + /** * Exports a timer as a JMX MBean, check corresponding {@link org.apache.cassandra.db.virtual.model.TimerMetricRow} * for the same functionality for virtual tables. diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index 67b5d0c1e3f5..5714aea46480 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -814,8 +814,8 @@ public Long getValue() bytesValidated = createTableHistogram("BytesValidated", cfs.keyspace.metric.bytesValidated, false); partitionsValidated = createTableHistogram("PartitionsValidated", cfs.keyspace.metric.partitionsValidated, false); - bytesAnticompacted = createTableMeter("BytesAnticompacted", cfs.keyspace.metric.bytesAnticompacted); - bytesMutatedAnticompaction = createTableMeter("BytesMutatedAnticompaction", cfs.keyspace.metric.bytesMutatedAnticompaction); + bytesAnticompacted = createTableMeter("BytesAnticompacted", cfs.keyspace.metric.bytesAnticompacted, true); + bytesMutatedAnticompaction = createTableMeter("BytesMutatedAnticompaction", cfs.keyspace.metric.bytesMutatedAnticompaction, true); bytesPreviewed = createTableMeter("BytesPreviewed", cfs.keyspace.metric.bytesPreviewed); tokenRangesPreviewedDesynchronized = createTableMeter("TokenRangesPreviewedDesynchronized", cfs.keyspace.metric.tokenRangesPreviewedDesynchronized); bytesPreviewedDesynchronized = createTableMeter("BytesPreviewedDesynchronized", cfs.keyspace.metric.bytesPreviewedDesynchronized); @@ -1087,16 +1087,21 @@ protected SnapshottingTimer createTableTimer(String name) protected TableMeter createTableMeter(String name, Meter keyspaceMeter) { - return createTableMeter(name, name, keyspaceMeter); + return createTableMeter(name, keyspaceMeter, false); } - protected TableMeter createTableMeter(String name, String alias, Meter keyspaceMeter) + protected TableMeter createTableMeter(String name, Meter keyspaceMeter, boolean globalMeterGaugeCompatible) + { + return createTableMeter(name, name, keyspaceMeter, globalMeterGaugeCompatible); + } + + protected TableMeter createTableMeter(String name, String alias, Meter keyspaceMeter, boolean globalMeterGaugeCompatible) { Meter meter = Metrics.meter(factory.createMetricName(name), aliasFactory.createMetricName(alias)); register(name, alias, meter); return new TableMeter(meter, keyspaceMeter, - Metrics.meter(GLOBAL_FACTORY.createMetricName(name), + Metrics.meter(globalMeterGaugeCompatible, GLOBAL_FACTORY.createMetricName(name), GLOBAL_ALIAS_FACTORY.createMetricName(alias))); } From c210d72efba2862801709a39983bcd9c6c81622e Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Sun, 17 Nov 2024 20:50:21 -0600 Subject: [PATCH 114/115] DO NOT COMMIT circleci --- .circleci/config.yml | 214 +++++++++++++++++++++---------------------- 1 file changed, 107 insertions(+), 107 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 864919b8f418..02e3aed428a7 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -24,7 +24,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -111,10 +111,10 @@ jobs: j17_dtests: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -220,7 +220,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 10 steps: - attach_workspace: at: /home/cassandra @@ -402,7 +402,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -489,10 +489,10 @@ jobs: j11_dtests_latest_repeat: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -690,7 +690,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -804,10 +804,10 @@ jobs: j17_cqlsh_dtests_py38_vnode: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -912,10 +912,10 @@ jobs: j17_dtests_vnode_repeat: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -1066,10 +1066,10 @@ jobs: j11_dtests_vnode_repeat: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -1199,10 +1199,10 @@ jobs: j17_cqlsh_dtests_py311_latest: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -1310,7 +1310,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -1428,7 +1428,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -1518,7 +1518,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -1632,10 +1632,10 @@ jobs: j11_cqlsh_dtests_py38_latest: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -1744,7 +1744,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -1830,10 +1830,10 @@ jobs: j11_cqlsh_dtests_py311: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -1939,10 +1939,10 @@ jobs: j17_dtests_large_vnode_repeat: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -2071,10 +2071,10 @@ jobs: j17_dtests_repeat: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -2228,7 +2228,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -2346,7 +2346,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -2432,10 +2432,10 @@ jobs: j17_cqlsh_dtests_py311: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -2540,10 +2540,10 @@ jobs: j11_cqlsh_dtests_py38: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -2652,7 +2652,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -2741,7 +2741,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -2831,7 +2831,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -2948,7 +2948,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -3141,7 +3141,7 @@ jobs: j11_dtests_large_vnode: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: xlarge working_directory: ~/ shell: /bin/bash -eo pipefail -l parallelism: 4 @@ -3226,10 +3226,10 @@ jobs: j11_dtests_large_vnode_repeat: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -3359,7 +3359,7 @@ jobs: j11_dtests_large: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: xlarge working_directory: ~/ shell: /bin/bash -eo pipefail -l parallelism: 4 @@ -3447,7 +3447,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -3537,7 +3537,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -3652,10 +3652,10 @@ jobs: j11_upgrade_dtests_repeat: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: xlarge working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -3788,7 +3788,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 10 steps: - attach_workspace: at: /home/cassandra @@ -3905,7 +3905,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -4206,7 +4206,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -4324,7 +4324,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -4413,7 +4413,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 10 steps: - attach_workspace: at: /home/cassandra @@ -4602,7 +4602,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -4692,7 +4692,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -4957,7 +4957,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -5160,7 +5160,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 4 steps: - attach_workspace: at: /home/cassandra @@ -5350,7 +5350,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -5437,10 +5437,10 @@ jobs: j17_dtests_vnode: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -5543,10 +5543,10 @@ jobs: j11_dtests_repeat: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -5679,7 +5679,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -5797,7 +5797,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -5912,10 +5912,10 @@ jobs: j17_dtests_latest_repeat: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -6047,7 +6047,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -6134,10 +6134,10 @@ jobs: j11_upgrade_dtests: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 100 steps: - attach_workspace: at: /home/cassandra @@ -6219,10 +6219,10 @@ jobs: j11_dtests_large_repeat: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -6355,7 +6355,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -6512,7 +6512,7 @@ jobs: j17_dtests_large_vnode: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: xlarge working_directory: ~/ shell: /bin/bash -eo pipefail -l parallelism: 4 @@ -6596,10 +6596,10 @@ jobs: j11_cqlsh_dtests_py38_vnode: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -6708,7 +6708,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -6798,7 +6798,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -6885,10 +6885,10 @@ jobs: j11_cqlsh_dtests_py311_latest: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -7057,10 +7057,10 @@ jobs: j11_cqlsh_dtests_py311_vnode: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -7169,7 +7169,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -7283,10 +7283,10 @@ jobs: j17_cqlsh_dtests_py38_latest: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -7391,10 +7391,10 @@ jobs: j17_cqlsh_dtests_py311_vnode: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -7502,7 +7502,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 1 + parallelism: 10 steps: - attach_workspace: at: /home/cassandra @@ -7617,10 +7617,10 @@ jobs: j17_dtests_latest: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -7726,7 +7726,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -8021,7 +8021,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -8108,10 +8108,10 @@ jobs: j11_dtests_latest: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -8257,10 +8257,10 @@ jobs: j11_dtests: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -8342,10 +8342,10 @@ jobs: j17_cqlsh_dtests_py38: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -8453,7 +8453,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -8570,7 +8570,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -8656,7 +8656,7 @@ jobs: j17_dtests_large: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: xlarge working_directory: ~/ shell: /bin/bash -eo pipefail -l parallelism: 4 @@ -8743,7 +8743,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -8904,7 +8904,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -8990,10 +8990,10 @@ jobs: j11_dtests_vnode: docker: - image: apache/cassandra-testing-ubuntu2004-java11-w-dependencies:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 50 steps: - attach_workspace: at: /home/cassandra @@ -9078,7 +9078,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -9164,10 +9164,10 @@ jobs: j17_dtests_large_repeat: docker: - image: apache/cassandra-testing-ubuntu2004-java11:latest - resource_class: medium + resource_class: large working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -9299,7 +9299,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra @@ -9388,7 +9388,7 @@ jobs: resource_class: medium working_directory: ~/ shell: /bin/bash -eo pipefail -l - parallelism: 4 + parallelism: 25 steps: - attach_workspace: at: /home/cassandra From 25802d116b623cb2176a71cb067b10e3e7c6d7d8 Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Tue, 7 Jan 2025 00:53:10 -0600 Subject: [PATCH 115/115] add guardrail for multiple version repairs being blocked --- .../org/apache/cassandra/config/Config.java | 1 + .../cassandra/config/GuardrailsOptions.java | 16 +++++++++ .../cassandra/db/guardrails/Guardrails.java | 34 +++++++++++++++++++ .../db/guardrails/GuardrailsConfig.java | 6 ++++ .../db/guardrails/GuardrailsMBean.java | 10 ++++++ .../repair/autorepair/AutoRepair.java | 6 ++++ .../cassandra/service/StorageService.java | 5 ++- .../distributed/test/GossipTest.java | 29 ++++++++++++++++ 8 files changed, 106 insertions(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index cbe1b64bcac9..0a571d5bf5cd 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -423,6 +423,7 @@ public static class SSTableConfig // When false, it behaves the same as normal streaming. public volatile boolean cdc_on_repair_enabled = true; public String cdc_raw_directory; + public boolean mixed_version_repairs_enabled = true; @Replaces(oldName = "cdc_total_space_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE_INT, deprecated = true) public DataStorageSpec.IntMebibytesBound cdc_total_space = new DataStorageSpec.IntMebibytesBound("0MiB"); @Replaces(oldName = "cdc_free_space_check_interval_ms", converter = Converters.MILLIS_DURATION_INT, deprecated = true) diff --git a/src/java/org/apache/cassandra/config/GuardrailsOptions.java b/src/java/org/apache/cassandra/config/GuardrailsOptions.java index e1c2fe892037..13e7f1646b25 100644 --- a/src/java/org/apache/cassandra/config/GuardrailsOptions.java +++ b/src/java/org/apache/cassandra/config/GuardrailsOptions.java @@ -205,6 +205,22 @@ public int getMaterializedViewsPerTableWarnThreshold() return config.materialized_views_per_table_warn_threshold; } + @Override + public boolean getMixedVersionRepairsEnabled() + { + return config.mixed_version_repairs_enabled; + } + + @Override + public boolean setMixedVersionRepairsEnabled(boolean enabled) + { + updatePropertyWithLogging("mixed_version_repairs_enabled", + enabled, + () -> config.mixed_version_repairs_enabled, + x -> config.mixed_version_repairs_enabled = x); + return enabled; + } + @Override public int getPartitionKeysInSelectWarnThreshold() { diff --git a/src/java/org/apache/cassandra/db/guardrails/Guardrails.java b/src/java/org/apache/cassandra/db/guardrails/Guardrails.java index df463d8b0f9f..153b6b5d07a5 100644 --- a/src/java/org/apache/cassandra/db/guardrails/Guardrails.java +++ b/src/java/org/apache/cassandra/db/guardrails/Guardrails.java @@ -39,7 +39,12 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.disk.usage.DiskUsageBroadcaster; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.Directory; +import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.tcm.membership.NodeVersion; import org.apache.cassandra.utils.MBeanWrapper; +import org.apache.cassandra.utils.btree.BTreeMap; import static java.lang.String.format; @@ -289,6 +294,23 @@ public final class Guardrails implements GuardrailsMBean state -> CONFIG_PROVIDER.getOrCreate(state).getAllowFilteringEnabled(), "Querying with ALLOW FILTERING"); + /** + * Guardrail disabling repairs when there are mixed versions + */ + public static final EnableFlag mixedRepairsEnabled = + new EnableFlag("mixed_version_repairs", + "Mixed mode repairs and streaming adds many unknowns and additional performance impacts during upgrades.", + state -> { + if (CONFIG_PROVIDER.getOrCreate(state).getMixedVersionRepairsEnabled()) + return true; + Directory directory = ClusterMetadata.current().directory; + Set nodes = directory.states.keySet(); + BTreeMap versions = directory.versions; + NodeVersion version = versions.get(nodes.iterator().next()); + return nodes.stream().allMatch(node -> versions.get(node).equals(version)); + }, + "Running repairs during mixed Cassandra versions"); + /** * Guardrail disabling setting SimpleStrategy via keyspace creation or alteration */ @@ -594,6 +616,18 @@ public void setKeyspacesThreshold(int warn, int fail) DEFAULT_CONFIG.setKeyspacesThreshold(warn, fail); } + @Override + public boolean getMixedVersionRepairsEnabled() + { + return DEFAULT_CONFIG.getMixedVersionRepairsEnabled(); + } + + @Override + public void setMixedVersionRepairsEnabled(boolean enabled) + { + DEFAULT_CONFIG.setMixedVersionRepairsEnabled(enabled); + } + @Override public int getTablesWarnThreshold() { diff --git a/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java b/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java index d2814a5d7865..995f7178e0e8 100644 --- a/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java +++ b/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java @@ -106,6 +106,12 @@ public interface GuardrailsConfig */ int getPartitionKeysInSelectFailThreshold(); + /** + * @return if repairs during mixed versions are allowed + */ + boolean getMixedVersionRepairsEnabled(); + boolean setMixedVersionRepairsEnabled(boolean enabled); + /** * @return The threshold to fail when creating more materialized views per table 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..3116f7443f0a 100644 --- a/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java +++ b/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java @@ -54,6 +54,16 @@ public interface GuardrailsMBean */ void setKeyspacesThreshold(int warn, int fail); + /** + * @return If mixed cassandra version repairs are enabled. + */ + boolean getMixedVersionRepairsEnabled(); + + /** + * @param enabled If mixed cassandra version repairs are enabled. + */ + void setMixedVersionRepairsEnabled(boolean enabled); + /** * @return The threshold to warn when creating more tables than threshold. * -1 means disabled. diff --git a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java index 1bd49e1b214a..cfc598dea679 100644 --- a/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java +++ b/src/java/org/apache/cassandra/repair/autorepair/AutoRepair.java @@ -36,6 +36,7 @@ import com.google.common.collect.Lists; import com.google.common.util.concurrent.Uninterruptibles; +import org.apache.cassandra.db.guardrails.Guardrails; import org.apache.cassandra.repair.RepairCoordinator; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.config.DurationSpec; @@ -167,6 +168,11 @@ public void repair(AutoRepairConfig.RepairType repairType) logger.debug("Auto-repair is disabled for repair type {}", repairType); return; } + if (!Guardrails.mixedRepairsEnabled.isEnabled(null)) + { + logger.debug("Cannot run repair when nodes in the cluster have different versions."); + return; + } AutoRepairService.instance.checkCanRun(repairType); AutoRepairState repairState = repairStates.get(repairType); try diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 7eb10e0542b0..c5d7a02c825d 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -3298,7 +3298,10 @@ private FutureTask createRepairTask(final int cmd, final String keyspace datacenters.removeAll(existingDatacenters); throw new IllegalArgumentException("data center(s) " + datacenters.toString() + " not found"); } - + if (!Guardrails.mixedRepairsEnabled.isEnabled(null)) + { + throw new IllegalStateException("Repairs while in mixed mode is disabled."); + } RepairCoordinator task = new RepairCoordinator(this, cmd, options, keyspace); task.addProgressListener(progressSupport); for (ProgressListener listener : listeners) diff --git a/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java index e66a85a3f2f7..78e55a0c02dc 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java @@ -22,6 +22,7 @@ import java.net.InetSocketAddress; import java.util.Collection; import java.util.Collections; +import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; @@ -36,6 +37,7 @@ 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.distributed.api.NodeToolResult; import org.apache.cassandra.distributed.api.TokenSupplier; import org.apache.cassandra.distributed.shared.ClusterUtils; import org.apache.cassandra.gms.Gossiper; @@ -46,9 +48,15 @@ import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.Transformation; +import org.apache.cassandra.tcm.membership.Location; +import org.apache.cassandra.tcm.membership.MembershipUtils; +import org.apache.cassandra.tcm.membership.NodeAddresses; 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.PrepareMove; +import org.apache.cassandra.tcm.transformations.Register; +import org.apache.cassandra.utils.CassandraVersion; import org.assertj.core.api.Assertions; import static org.apache.cassandra.config.CassandraRelevantProperties.JOIN_RING; @@ -61,6 +69,7 @@ import static org.apache.cassandra.distributed.shared.ClusterUtils.replaceHostAndStart; import static org.apache.cassandra.distributed.shared.ClusterUtils.stopUnchecked; import static org.apache.cassandra.distributed.shared.NetworkTopology.singleDcNetworkTopology; +import static org.apache.cassandra.tcm.membership.NodeVersion.CURRENT_METADATA_VERSION; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -174,6 +183,26 @@ public void gossipShutdownUpdatesTokenMetadata() throws Exception } } + @Test + public void repairsDisabledOnMultiVersionGuardrail() throws Exception + { + try (Cluster cluster = builder() + .withNodes(2) + .withConfig(config -> config.with(GOSSIP) + .set("mixed_version_repairs_enabled", false)) + .start()) + { + cluster.get(1).runOnInstance(() -> { + NodeAddresses addresses = MembershipUtils.nodeAddresses(new Random()); + ClusterMetadataService.instance().commit( + new Register(addresses, new Location("dc1", "rack1"), new NodeVersion(new CassandraVersion("4.0"), CURRENT_METADATA_VERSION)) + ); + }); + NodeToolResult result = cluster.get(1).nodetoolResult("repair", "system_distributed"); + assertTrue(result.getStderr().contains("Repairs while in mixed mode is disabled.")); + } + } + @Test public void restartGossipOnGossippingOnlyMember() throws Throwable {