You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2020/02/14 02:08:20 UTC

[GitHub] [cassandra] dcapwell opened a new pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

dcapwell opened a new pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390170372
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+
+public class RepairCoordinatorBase extends DistributedTestBase
+{
+    protected static Cluster CLUSTER;
+
+    protected final RepairType repairType;
+    protected final RepairParallelism parallelism;
+    protected final boolean withNotifications;
 
 Review comment:
   I am thinking whether we should make `withNotifications` always true... this can reduce half of tests and simplify some code in `DTestNodeTool/InternalNodeProbe`.. wdyt?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390150093
 
 

 ##########
 File path: test/unit/org/apache/cassandra/utils/Retry.java
 ##########
 @@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.IntToLongFunction;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+
+/**
+ * Class for retryable actions.
+ *
+ * @see {@link #retryWithBackoff(int, Supplier, Predicate)}
+ */
+public final class Retry
+{
+    private static final ScheduledExecutorService SCHEDULED = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("RetryScheduler"));
+
+    private Retry()
+    {
+
+    }
+
+    /**
+     * Schedule code to run after the defined duration.
+     *
+     * Since a executor was not defined, the global {@link ForkJoinPool#commonPool()} executor will be used, if this
+     * is not desirable then should use {@link #schedule(Duration, Executor, Runnable)}.
+     *
+     * @param duration how long to delay
+     * @param fn code to run
+     * @return future representing result
+     */
+    public static CompletableFuture<Void> schedule(final Duration duration, final Runnable fn)
+    {
+        return schedule(duration, ForkJoinPool.commonPool(), fn);
+    }
+
+    /**
+     * Schedule code to run after the defined duration on the provided executor.
+     *
+     * @param duration how long to delay
+     * @param executor to run on
+     * @param fn code to run
+     * @return future representing result
+     */
+    public static CompletableFuture<Void> schedule(final Duration duration, final Executor executor, final Runnable fn)
+    {
+        long nanos = duration.toNanos();
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        SCHEDULED.schedule(() -> run0(executor, future, fn), nanos, TimeUnit.NANOSECONDS);
 
 Review comment:
   can we simplify this as `Future future = SCHEDULED.schedule(fn, nanos, TimeUnit.NANOSECONDS);` and get rid of `run0` ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384007569
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/NodeToolResult.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.management.Notification;
+
+import org.junit.Assert;
+
+public class NodeToolResult
+{
+    public final String[] commandAndArgs;
 
 Review comment:
   Should all these fields be public?..

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390101382
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,167 +145,239 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
 
 Review comment:
   When there is no valid CFs to be repaired, a skip event with 100/100 progress is fired, and then another `success` event with `progressCounter/totalProgress` is fired in `success()`.. 
   
   Should we use `progressCounter/totalProgress` for skip progress event?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390139488
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedRepairUtils.java
 ##########
 @@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.QueryResult;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.impl.AbstractCluster;
+import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.metrics.StorageMetrics;
+
+import static org.apache.cassandra.utils.Retry.retryWithBackoffBlocking;
+
+public final class DistributedRepairUtils
+{
+    public static final int DEFAULT_COORDINATOR = 1;
+
+    private DistributedRepairUtils()
+    {
+
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, RepairType repairType, boolean withNotifications, String... args) {
+        return repair(cluster, DEFAULT_COORDINATOR, repairType, withNotifications, args);
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, int node, RepairType repairType, boolean withNotifications, String... args) {
+        args = repairType.append(args);
+        args = ArrayUtils.addAll(new String[] { "repair" }, args);
+        return cluster.get(node).nodetoolResult(withNotifications, args);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster)
 
 Review comment:
   ```suggestion
       public static long getRepairExceptions(AbstractCluster<IInvokableInstance> cluster)
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390679857
 
 

 ##########
 File path: test/unit/org/apache/cassandra/utils/Retry.java
 ##########
 @@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.IntToLongFunction;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+
+/**
+ * Class for retryable actions.
+ *
+ * @see {@link #retryWithBackoff(int, Supplier, Predicate)}
+ */
+public final class Retry
+{
+    private static final ScheduledExecutorService SCHEDULED = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("RetryScheduler"));
+
+    private Retry()
+    {
+
+    }
+
+    /**
+     * Schedule code to run after the defined duration.
+     *
+     * Since a executor was not defined, the global {@link ForkJoinPool#commonPool()} executor will be used, if this
+     * is not desirable then should use {@link #schedule(Duration, Executor, Runnable)}.
+     *
+     * @param duration how long to delay
+     * @param fn code to run
+     * @return future representing result
+     */
+    public static CompletableFuture<Void> schedule(final Duration duration, final Runnable fn)
+    {
+        return schedule(duration, ForkJoinPool.commonPool(), fn);
+    }
+
+    /**
+     * Schedule code to run after the defined duration on the provided executor.
+     *
+     * @param duration how long to delay
+     * @param executor to run on
+     * @param fn code to run
+     * @return future representing result
+     */
+    public static CompletableFuture<Void> schedule(final Duration duration, final Executor executor, final Runnable fn)
+    {
+        long nanos = duration.toNanos();
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        SCHEDULED.schedule(() -> run0(executor, future, fn), nanos, TimeUnit.NANOSECONDS);
+        return future;
+    }
+
+    private static void run0(final Executor executor, final CompletableFuture<Void> future, final Runnable fn)
+    {
+        try
+        {
+            executor.execute(() -> {
+                try
+                {
+                    fn.run();
+                    future.complete(null);
+                }
+                catch (Exception e)
+                {
+                    future.completeExceptionally(e);
+                }
+            });
+        }
+        catch (Exception e)
+        {
+            future.completeExceptionally(e);
+        }
+    }
+
+    /**
+     * Continously attempting to call the provided future supplier until successful or until no longer able to retry.
+     *
+     * @param maxRetries to allow
+     * @param fn asyncronous operation to retry
+     * @param retryableException used to say if retry is allowed
+     * @return future representing the result.  If retries were not able to get a successful result, the exception is the last exception seen.
+     */
+    public static <A> CompletableFuture<A> retryWithBackoff(final int maxRetries,
+                                                            final Supplier<CompletableFuture<A>> fn,
+                                                            final Predicate<Throwable> retryableException)
+    {
+        CompletableFuture<A> future = new CompletableFuture<>();
+        retryWithBackoff0(future, 0, maxRetries, fn, retryableException, retryCount -> computeSleepTimeMillis(retryCount, 50, 1000));
+        return future;
+    }
+
+    /**
+     * This is the same as {@link #retryWithBackoff(int, Supplier, Predicate)}, but takes a blocking retryable action
+     * and blocks the caller until done.
+     */
+    public static <A> A retryWithBackoffBlocking(final int maxRetries, final Supplier<A> fn)
+    {
+        return retryWithBackoffBlocking(maxRetries, fn, (ignore) -> true);
+    }
+
+    /**
+     * This is the same as {@link #retryWithBackoff(int, Supplier, Predicate)}, but takes a blocking retryable action
+     * and blocks the caller until done.
+     */
+    public static <A> A retryWithBackoffBlocking(final int maxRetries,
+                                                 final Supplier<A> fn,
+                                                 final Predicate<Throwable> retryableException)
+    {
+        return retryWithBackoff(maxRetries, () -> CompletableFuture.completedFuture(fn.get()), retryableException).join();
+    }
+
+    private static <A> void retryWithBackoff0(final CompletableFuture<A> result,
+                                              final int retryCount,
+                                              final int maxRetry,
+                                              final Supplier<CompletableFuture<A>> body,
 
 Review comment:
   this is the private implementation, so can't.  The `retryWithBackoffBlocking` calls use `Supplier<A>`  where as `retryWithBackoff` doesn't and uses a `Supplier<CompletableFuture<A>>`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r383812137
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorFast.java
 ##########
 @@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Set;
+
+import com.google.common.collect.Iterables;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.distributed.api.IMessageFilters;
+import org.apache.cassandra.distributed.api.LongTokenRange;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.NodeToolResult.ProgressEventType;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.service.StorageService;
+
+import static java.lang.String.format;
+import static org.apache.cassandra.distributed.api.IMessageFilters.Matcher.of;
+import static org.apache.cassandra.distributed.test.DistributedRepairUtils.assertParentRepairFailedWithMessageContains;
+import static org.apache.cassandra.distributed.test.DistributedRepairUtils.assertParentRepairNotExist;
+import static org.apache.cassandra.distributed.test.DistributedRepairUtils.assertParentRepairSuccess;
+import static org.apache.cassandra.distributed.test.DistributedRepairUtils.getRepairExceptions;
+
+public abstract class RepairCoordinatorFast extends RepairCoordinatorBase
+{
+    public RepairCoordinatorFast(RepairType repairType, RepairParallelism parallelism, boolean withNotifications)
+    {
+        super(repairType, parallelism, withNotifications);
+    }
+
+    @Test(timeout = 1 * 60 * 1000)
+    public void simple() {
+        String table = tableName("simple");
+        CLUSTER.schemaChange(format("CREATE TABLE %s.%s (key text, PRIMARY KEY (key))", KEYSPACE, table));
+        CLUSTER.coordinator(1).execute(format("INSERT INTO %s.%s (key) VALUES (?)", KEYSPACE, table), ConsistencyLevel.ANY, "some text");
+
+        long repairExceptions = getRepairExceptions(CLUSTER, 2);
+        NodeToolResult result = repair(2, KEYSPACE, table);
+        result.asserts().ok();
+        if (withNotifications)
+        {
+            result.asserts()
+                  .notificationContains(ProgressEventType.START, "Starting repair command")
+                  .notificationContains(ProgressEventType.START, "repairing keyspace " + KEYSPACE + " with repair options")
+                  .notificationContains(ProgressEventType.SUCCESS, repairType != RepairType.PREVIEW ? "Repair completed successfully": "Repair preview completed successfully")
+                  .notificationContains(ProgressEventType.COMPLETE, "finished");
+        }
+
+        if (repairType != RepairType.PREVIEW)
+        {
+            assertParentRepairSuccess(CLUSTER, KEYSPACE, table);
+        }
+        else
+        {
+            assertParentRepairNotExist(CLUSTER, KEYSPACE, table);
+        }
+
+        Assert.assertEquals(repairExceptions, getRepairExceptions(CLUSTER, 2));
+    }
+
+    @Test(timeout = 1 * 60 * 1000)
+    public void missingKeyspace()
+    {
+        // as of this moment the check is done in nodetool so the JMX notifications are not imporant
+        // nor is the history stored
+        long repairExceptions = getRepairExceptions(CLUSTER, 2);
+        NodeToolResult result = repair(2, "doesnotexist");
+        result.asserts()
+              .notOk()
+              .errorContains("Keyspace [doesnotexist] does not exist.");
+
+        Assert.assertEquals(repairExceptions, getRepairExceptions(CLUSTER, 2));
+
+        assertParentRepairNotExist(CLUSTER, "doesnotexist");
+    }
+
+    @Test(timeout = 1 * 60 * 1000)
+    public void missingTable()
+    {
+        long repairExceptions = getRepairExceptions(CLUSTER, 2);
+        NodeToolResult result = repair(2, KEYSPACE, "doesnotexist");
+        result.asserts()
+              .notOk();
+        if (withNotifications)
+        {
+            result.asserts()
+                  .errorContains("failed with error Unknown keyspace/cf pair (distributed_test_keyspace.doesnotexist)")
+                  // Start notification is ignored since this is checked during setup (aka before start)
+                  .notificationContains(ProgressEventType.ERROR, "failed with error Unknown keyspace/cf pair (distributed_test_keyspace.doesnotexist)")
+                  .notificationContains(ProgressEventType.COMPLETE, "finished with error");
+        }
+
+        assertParentRepairNotExist(CLUSTER, KEYSPACE, "doesnotexist");
+
+        Assert.assertEquals(repairExceptions + 1, getRepairExceptions(CLUSTER, 2));
+    }
+
+    @Test(timeout = 1 * 60 * 1000)
+    public void noTablesToRepair()
+    {
+        // index CF currently don't support repair, so they get dropped when listed
+        // this is done in this test to cause the keyspace to have 0 tables to repair, which causes repair to no-op
+        // early and skip.
+        String table = tableName("withindex");
+        CLUSTER.schemaChange(format("CREATE TABLE %s.%s (key text, value text, PRIMARY KEY (key))", KEYSPACE, table));
+        CLUSTER.schemaChange(format("CREATE INDEX value_%s ON %s.%s (value)", postfix(), KEYSPACE, table));
+
+        long repairExceptions = getRepairExceptions(CLUSTER, 2);
+        // if CF has a . in it, it is assumed to be a 2i which rejects repairs
+        NodeToolResult result = repair(2, KEYSPACE, table + ".value");
+        result.asserts().ok();
+        if (withNotifications)
+        {
+            result.asserts()
+                  .notificationContains("Empty keyspace")
+                  .notificationContains("skipping repair: " + KEYSPACE)
+                  // Start notification is ignored since this is checked during setup (aka before start)
+                  .notificationContains(ProgressEventType.SUCCESS, "Empty keyspace") // will fail since success isn't returned; only complete
+                  .notificationContains(ProgressEventType.COMPLETE, "finished"); // will fail since it doesn't do this
+        }
+
+        assertParentRepairNotExist(CLUSTER, KEYSPACE, table + ".value");
+
+        // this is actually a SKIP and not a FAILURE, so shouldn't increment
+        Assert.assertEquals(repairExceptions, getRepairExceptions(CLUSTER, 2));
+    }
+
+    @Test(timeout = 1 * 60 * 1000)
+    public void intersectingRange()
+    {
+        // this test exists to show that this case will cause repair to finish; success or failure isn't imporant
+        // if repair is enhanced to allow intersecting ranges w/ local then this test will fail saying that we expected
+        // repair to fail but it didn't, this would be fine and this test should be updated to reflect the new
+        // semantic
+        String table = tableName("intersectingrange");
+        CLUSTER.schemaChange(format("CREATE TABLE %s.%s (key text, value text, PRIMARY KEY (key))", KEYSPACE, table));
+
+        //TODO dtest api for this?
 
 Review comment:
   Looks like this TODO has been done.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384161591
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -199,63 +277,48 @@ protected void runMayThrow() throws Exception
         }
         else
         {
-            fireProgressEvent(new ProgressEvent(ProgressEventType.START, 0, 100, message));
             traceState = null;
         }
 
-        Set<InetAddressAndPort> allNeighbors = new HashSet<>();
-        List<CommonRange> commonRanges = new ArrayList<>();
-
-        try
-        {
-            //pre-calculate output of getLocalReplicas and pass it to getNeighbors to increase performance and prevent
-            //calculation multiple times
-            Iterable<Range<Token>> keyspaceLocalRanges = storageService.getLocalReplicas(keyspace).ranges();
+        // Why is this before start when its publishing the start event?  For backwards compatability
+        // Before we finish validating we actually trigger this, so kept publishing early even though its not
+        // correct...
+        String message = String.format("Starting repair command #%d (%s), repairing keyspace %s with %s", cmd, parentSession, keyspace,
 
 Review comment:
   Nit: `final`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384629539
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedRepairUtils.java
 ##########
 @@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.ResultSet;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.impl.AbstractCluster;
+import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.metrics.StorageMetrics;
+
+public final class DistributedRepairUtils
+{
+    public static final int DEFAULT_COORDINATOR = 1;
+
+    private DistributedRepairUtils()
+    {
+
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, RepairType repairType, boolean withNotifications, String... args) {
+        return repair(cluster, DEFAULT_COORDINATOR, repairType, withNotifications, args);
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, int node, RepairType repairType, boolean withNotifications, String... args) {
+        args = repairType.append(args);
+        args = ArrayUtils.addAll(new String[] { "repair" }, args);
+        return cluster.get(node).nodetoolResult(withNotifications, args);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster)
+    {
+        return getRepairExceptions(cluster, DEFAULT_COORDINATOR);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster, int node)
+    {
+        return cluster.get(node).callOnInstance(() -> StorageMetrics.repairExceptions.getCount());
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, String ks, String table)
+    {
+        return queryParentRepairHistory(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        // This is kinda brittle since the caller never gets the ID and can't ask for the ID; it needs to infer the id
+        // this logic makes the assumption the ks/table pairs are unique (should be or else create should fail) so any
+        // repair for that pair will be the repair id
+        Set<String> tableNames = table == null? Collections.emptySet() : ImmutableSet.of(table);
+        ResultSet rs = null;
+        Exception latestException = null;
+        for (int i = 0; i < 10; i++)
+        {
+            try
+            {
+                rs = cluster.coordinator(coordinator)
+                            .executeWithResult("SELECT * FROM system_distributed.parent_repair_history", ConsistencyLevel.QUORUM)
+                            .filter(row -> ks.equals(row.getString("keyspace_name")))
+                            .filter(row -> tableNames.equals(row.getSet("columnfamily_names")));
+                break;
+            }
+            catch (Exception e)
+            {
+                latestException = e;
+                rs = null;
+                //TODO do we have a backoff stategy I can leverage?  I would prefer expotential but don't want to add for somethinr minor
+                Uninterruptibles.sleepUninterruptibly( (i + 1) * 300, TimeUnit.MILLISECONDS);
+            }
+        }
+        if (rs == null)
+        {
+            // exception should exist
+            if (latestException == null)
+            {
+                Assert.fail("Unable to query system_distributed.parent_repair_history, got back neither result set or exception ");
+            }
+            if (latestException instanceof RuntimeException)
 
 Review comment:
   That being said, with the change to call a retry function, this is no longer a issue =D

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390131976
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java
 ##########
 @@ -61,7 +79,26 @@ protected void connect()
         mbeanServerConn = null;
         jmxc = null;
 
-        ssProxy = StorageService.instance;
+        if (withNotifications)
+        {
+            ssProxy = StorageService.instance;
+        }
+        else
+        {
+            // replace the notification apis with a no-op method
+            StorageServiceMBean mock = Mockito.spy(StorageService.instance);
+            Mockito.doNothing().when(mock).addNotificationListener(Mockito.any(), Mockito.any(), Mockito.any());
+            try
+            {
+                Mockito.doNothing().when(mock).removeNotificationListener(Mockito.any(), Mockito.any(), Mockito.any());
+                Mockito.doNothing().when(mock).removeNotificationListener(Mockito.any());
+            }
+            catch (ListenerNotFoundException e)
+            {
+                throw new AssertionError(e);
+            }
+            ssProxy = mock;
 
 Review comment:
   I don't understand what are the cases that a no-op storage-service is needed..

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390125644
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -552,9 +568,66 @@ public int liveMemberCount()
         }).call();
     }
 
-    public int nodetool(String... commandAndArgs)
+    public NodeToolResult nodetoolResult(boolean withNotifications, String... commandAndArgs)
     {
-        return sync(() -> new NodeTool(new InternalNodeProbeFactory()).execute(commandAndArgs)).call();
+        return sync(() -> {
+            DTestNodeTool nodetool = new DTestNodeTool(withNotifications);
+            int rc =  nodetool.execute(commandAndArgs);
+            return new NodeToolResult(commandAndArgs, rc, new ArrayList<>(nodetool.notifications.notifications), nodetool.latestError);
+        }).call();
+    }
+
+    private static class DTestNodeTool extends NodeTool {
+        private final StorageServiceMBean storageProxy;
+        private final CollectingNotificationListener notifications = new CollectingNotificationListener();
+
+        private Throwable latestError;
+
+        DTestNodeTool(boolean withNotifications) {
+            super(new InternalNodeProbeFactory(withNotifications));
+            storageProxy = InternalNodeProbe.create(withNotifications).getStorageService();
 
 Review comment:
   Will `InternalNodeProbe.create(withNotifications)` cause any jmx connection leak?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384016748
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -360,9 +361,9 @@ public void startup(ICluster cluster)
                     throw new RuntimeException(e);
                 }
 
+                registerFilter(cluster);
 
 Review comment:
   I like how now we use inboundSink in both cases: "real" networking and "fake" networking, too.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384014298
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -374,6 +375,8 @@ public void startup(ICluster cluster)
                 }
                 JVMStabilityInspector.replaceKiller(new InstanceKiller());
 
+                ActiveRepairService.instance.start();
 
 Review comment:
   Do we want to hide this behind a feature flag? I'd say yes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384226130
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+
+public class RepairCoordinatorBase extends DistributedTestBase implements Serializable
 
 Review comment:
   tests pass without it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384218296
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/ResultSet.java
 ##########
 @@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Date;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+public class ResultSet implements Iterator<Row>
 
 Review comment:
   any recommendations?  Only one that comes to mind is `DResultSet` >_>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384218449
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -360,9 +361,9 @@ public void startup(ICluster cluster)
                     throw new RuntimeException(e);
                 }
 
+                registerFilter(cluster);
 
 Review comment:
   ^_^

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r383967815
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+
+public class RepairCoordinatorBase extends DistributedTestBase implements Serializable
 
 Review comment:
   I'm not a very big fan of marking test classes as `Serializable`. Can we find a way around it? This usually means we haven't serialized and passed the required state while passing it to the lambda, and instead have passed the instance of the test case. This might be also caused by the fact we're using parametrized tests (since that's already the case in one other test somewhere in the repo), but I think we shuoldn't use pass test class instance into the in-jvm cluster instances.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384173826
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/ICoordinator.java
 ##########
 @@ -27,7 +27,11 @@
 {
     // a bit hacky, but ConsistencyLevel draws in too many dependent classes, so we cannot have a cross-version
     // method signature that accepts ConsistencyLevel directly.  So we just accept an Enum<?> and cast.
-    Object[][] execute(String query, Enum<?> consistencyLevel, Object... boundValues);
+    default Object[][] execute(String query, Enum<?> consistencyLevel, Object... boundValues)
+    {
+        return executeWithResult(query, consistencyLevel, boundValues).toObjectArrays();
+    }
+    ResultSet executeWithResult(String query, Enum<?> consistencyLevel, Object... boundValues);
 
 Review comment:
   +1 Please get rid of `Object[][]`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r383984227
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+
+public class RepairCoordinatorBase extends DistributedTestBase implements Serializable
+{
+    protected static Cluster CLUSTER;
+
+    protected final RepairType repairType;
+    protected final RepairParallelism parallelism;
+    protected final boolean withNotifications;
+
+    public RepairCoordinatorBase(RepairType repairType, RepairParallelism parallelism, boolean withNotifications)
+    {
+        this.repairType = repairType;
+        this.parallelism = parallelism;
+        this.withNotifications = withNotifications;
+    }
+
+    @Parameterized.Parameters(name = "{0}/{1}")
+    public static Collection<Object[]> testsWithoutType()
+    {
+        List<Object[]> tests = new ArrayList<>();
+        for (RepairParallelism p : RepairParallelism.values())
+        {
+            tests.add(new Object[] { p, true });
+            tests.add(new Object[] { p, false });
+        }
+        return tests;
+    }
+
+    @BeforeClass
+    public static void before()
+    {
+        // This only works because the way CI works
+        // In CI a new JVM is spun up for each test file, so this doesn't have to worry about another test file
+        // getting this set first
+        System.setProperty("cassandra.nodetool.jmx_notification_poll_interval_seconds", "1");
+        DatabaseDescriptor.clientInitialization();
 
 Review comment:
   At least a preliminary test hasn't confirmed that we need to call `clientInitialization` here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390470787
 
 

 ##########
 File path: src/java/org/apache/cassandra/service/ActiveRepairService.java
 ##########
 @@ -525,6 +525,20 @@ public synchronized ParentRepairSession removeParentRepairSession(UUID parentSes
         return parentRepairSessions.remove(parentSessionId);
     }
 
+    public synchronized ParentRepairSession removeParentRepairSessionIfPresent(UUID parentSessionId)
 
 Review comment:
   they are the same minus the "don't fail if not present thing". The code that called `removeParentRepairSession` assumes the session exists so would be an error it it doesn't, the new code which calls `removeParentRepairSessionIfPresent` calls it during cases were the session may not be setup yet, so may see a null.
   
   I could refactor so the `removeParentRepairSession` calls this one and still fails, but I didn't want to change the semantics of that method.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384006508
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
 ##########
 @@ -151,9 +152,16 @@ public boolean isShutdown()
         @Override
         public synchronized void startup()
         {
+            startup(AbstractCluster.this);
+        }
+
+        public synchronized void startup(ICluster cluster)
+        {
+            if (cluster != AbstractCluster.this)
+                throw new IllegalArgumentException("Only the owning cluster can be used for startup"); //TODO why have this in the API?
             if (!isShutdown)
                 throw new IllegalStateException();
-            delegate().startup(AbstractCluster.this);
+            delegate().startup(cluster);
 
 Review comment:
   I think you can just do `delegate().startup()`. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384217171
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedRepairUtils.java
 ##########
 @@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.ResultSet;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.impl.AbstractCluster;
+import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.metrics.StorageMetrics;
+
+public final class DistributedRepairUtils
+{
+    public static final int DEFAULT_COORDINATOR = 1;
+
+    private DistributedRepairUtils()
+    {
+
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, RepairType repairType, boolean withNotifications, String... args) {
+        return repair(cluster, DEFAULT_COORDINATOR, repairType, withNotifications, args);
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, int node, RepairType repairType, boolean withNotifications, String... args) {
+        args = repairType.append(args);
+        args = ArrayUtils.addAll(new String[] { "repair" }, args);
+        return cluster.get(node).nodetoolResult(withNotifications, args);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster)
+    {
+        return getRepairExceptions(cluster, DEFAULT_COORDINATOR);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster, int node)
+    {
+        return cluster.get(node).callOnInstance(() -> StorageMetrics.repairExceptions.getCount());
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, String ks, String table)
+    {
+        return queryParentRepairHistory(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        // This is kinda brittle since the caller never gets the ID and can't ask for the ID; it needs to infer the id
+        // this logic makes the assumption the ks/table pairs are unique (should be or else create should fail) so any
+        // repair for that pair will be the repair id
+        Set<String> tableNames = table == null? Collections.emptySet() : ImmutableSet.of(table);
+        ResultSet rs = null;
+        Exception latestException = null;
+        for (int i = 0; i < 10; i++)
+        {
+            try
+            {
+                rs = cluster.coordinator(coordinator)
+                            .executeWithResult("SELECT * FROM system_distributed.parent_repair_history", ConsistencyLevel.QUORUM)
+                            .filter(row -> ks.equals(row.getString("keyspace_name")))
+                            .filter(row -> tableNames.equals(row.getSet("columnfamily_names")));
+                break;
+            }
+            catch (Exception e)
+            {
+                latestException = e;
+                rs = null;
+                //TODO do we have a backoff stategy I can leverage?  I would prefer expotential but don't want to add for somethinr minor
+                Uninterruptibles.sleepUninterruptibly( (i + 1) * 300, TimeUnit.MILLISECONDS);
+            }
+        }
+        if (rs == null)
+        {
+            // exception should exist
+            if (latestException == null)
+            {
+                Assert.fail("Unable to query system_distributed.parent_repair_history, got back neither result set or exception ");
+            }
+            if (latestException instanceof RuntimeException)
+                throw (RuntimeException) latestException;
+            throw new RuntimeException(latestException);
+        }
+        return rs;
+    }
+
+    public static void assertParentRepairNotExist(AbstractCluster<?> cluster, String ks, String table)
+    {
+        assertParentRepairNotExist(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static void assertParentRepairNotExist(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        ResultSet rs = queryParentRepairHistory(cluster, coordinator, ks, table);
+        Assert.assertFalse("No repairs should be found but at least one found", rs.hasNext());
+    }
+
+    public static void assertParentRepairNotExist(AbstractCluster<?> cluster, String ks)
+    {
+        assertParentRepairNotExist(cluster, DEFAULT_COORDINATOR, ks);
+    }
+
+    public static void assertParentRepairNotExist(AbstractCluster<?> cluster, int coordinator, String ks)
+    {
+        ResultSet rs = queryParentRepairHistory(cluster, coordinator, ks, null);
+        Assert.assertFalse("No repairs should be found but at least one found", rs.hasNext());
+    }
+
+    public static void assertParentRepairSuccess(AbstractCluster<?> cluster, String ks, String table)
+    {
+        assertParentRepairSuccess(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static void assertParentRepairSuccess(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        ResultSet rs = queryParentRepairHistory(cluster, coordinator, ks, table);
+        validateExistingParentRepair(rs, row -> {
+            // check completed
+            Assert.assertNotNull("finished_at not found, alears the repair is not complete?", rs.getTimestamp("finished_at"));
+
+            // check not failed (aka success)
+            Assert.assertNull("Exception found", rs.getString("exception_stacktrace"));
+            Assert.assertNull("Exception found", rs.getString("exception_message"));
+        });
+    }
+
+    public static void assertParentRepairFailedWithMessageContains(AbstractCluster<?> cluster, String ks, String table, String message)
+    {
+        assertParentRepairFailedWithMessageContains(cluster, DEFAULT_COORDINATOR, ks, table, message);
+    }
+
+    public static void assertParentRepairFailedWithMessageContains(AbstractCluster<?> cluster, int coordinator, String ks, String table, String message)
+    {
+        ResultSet rs = queryParentRepairHistory(cluster, coordinator, ks, table);
+        validateExistingParentRepair(rs, row -> {
+            // check completed
+            Assert.assertNotNull("finished_at not found, alears the repair is not complete?", rs.getTimestamp("finished_at"));
 
 Review comment:
   copy/paste for the win!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384239848
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/NodeToolResult.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.management.Notification;
+
+import org.junit.Assert;
+
+public class NodeToolResult
+{
+    public final String[] commandAndArgs;
 
 Review comment:
   pushed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384234884
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/LongTokenRange.java
 ##########
 @@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.io.Serializable;
+
+public final class LongTokenRange implements Serializable
 
 Review comment:
   > I think we might want to limit partitioner to Murmur3Partitioner and allow using long tokens only
   
   for now sure.  If we want to move to a actual total order partitioner we could always relook at the testing implications of that.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390130474
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -552,9 +568,66 @@ public int liveMemberCount()
         }).call();
     }
 
-    public int nodetool(String... commandAndArgs)
+    public NodeToolResult nodetoolResult(boolean withNotifications, String... commandAndArgs)
     {
-        return sync(() -> new NodeTool(new InternalNodeProbeFactory()).execute(commandAndArgs)).call();
+        return sync(() -> {
+            DTestNodeTool nodetool = new DTestNodeTool(withNotifications);
+            int rc =  nodetool.execute(commandAndArgs);
+            return new NodeToolResult(commandAndArgs, rc, new ArrayList<>(nodetool.notifications.notifications), nodetool.latestError);
+        }).call();
+    }
+
+    private static class DTestNodeTool extends NodeTool {
+        private final StorageServiceMBean storageProxy;
+        private final CollectingNotificationListener notifications = new CollectingNotificationListener();
+
+        private Throwable latestError;
+
+        DTestNodeTool(boolean withNotifications) {
+            super(new InternalNodeProbeFactory(withNotifications));
 
 Review comment:
   since we are creating `InternalNodeProbe` instance to get `storageProxy ` in the next line. It looks like we don't need `withNotifications` for `InternalNodeProbeFactory`..

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384238304
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
 ##########
 @@ -151,9 +152,16 @@ public boolean isShutdown()
         @Override
         public synchronized void startup()
         {
+            startup(AbstractCluster.this);
+        }
+
+        public synchronized void startup(ICluster cluster)
+        {
+            if (cluster != AbstractCluster.this)
+                throw new IllegalArgumentException("Only the owning cluster can be used for startup"); //TODO why have this in the API?
 
 Review comment:
   Really, the only way to solve this is to split the user API from the implementation API.  Ill file a JIRA for this

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on issue #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on issue #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#issuecomment-591176680
 
 
   @ifesdjeen thoughts on killing off `Object[][]`?  I got two for it and I do not want to maintain this if I add it here...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r383808940
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/Row.java
 ##########
 @@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Arrays;
+import java.util.Date;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import javax.annotation.Nullable;
+
+public class Row
+{
+    private final String[] names;
+    @Nullable private Object[] results; // mutable to avoid allocations in loops
+
+    public Row(String[] names)
+    {
+        this.names = names;
+    }
+
+    void setResults(@Nullable Object[] results)
+    {
+        this.results = results;
+    }
+
+    public <T> T get(String name)
+    {
+        checkAccess();
+        int idx = findIndex(name);
 
 Review comment:
   Shouold we use hashmap? Main reason is when we're using it with something like Harry, where we're doing quite a few queries, and big-Oh can start to matter.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] clohfink commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
clohfink commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384161021
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/ResultSet.java
 ##########
 @@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Date;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+public class ResultSet implements Iterator<Row>
 
 Review comment:
   Can we rename this slightly? Theres already a driver ResultSet and a ResultSet in cql3 package. Its a rather overloaded class name where each is representing same thing but slightly different.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384211054
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/Row.java
 ##########
 @@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Arrays;
+import java.util.Date;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import javax.annotation.Nullable;
+
+public class Row
+{
+    private final String[] names;
+    @Nullable private Object[] results; // mutable to avoid allocations in loops
+
+    public Row(String[] names)
+    {
+        this.names = names;
+    }
+
+    void setResults(@Nullable Object[] results)
+    {
+        this.results = results;
+    }
+
+    public <T> T get(String name)
+    {
+        checkAccess();
+        int idx = findIndex(name);
 
 Review comment:
   I was being lazy, will switch

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r383984344
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+
+public class RepairCoordinatorBase extends DistributedTestBase implements Serializable
 
 Review comment:
   At least a preliminary test hasn't confirmed that we need `Serializable` here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384216555
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/NodeToolResult.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.management.Notification;
+
+import org.junit.Assert;
+
+public class NodeToolResult
+{
+    public final String[] commandAndArgs;
+    public final int rc;
+    public final List<Notification> notifications;
+    public final Throwable error;
+
+    public NodeToolResult(String[] commandAndArgs, int rc, List<Notification> notifications, Throwable error)
+    {
+        this.commandAndArgs = commandAndArgs;
+        this.rc = rc;
+        this.notifications = notifications;
+        this.error = error;
+    }
+
+    public Asserts asserts()
+    {
+        return new Asserts();
+    }
+
+    public final class Asserts {
+        public Asserts ok() {
+            Assert.assertEquals("nodetool command " + commandAndArgs[0] + " was not successful", 0, rc);
+            return this;
+        }
+
+        public Asserts notOk() {
 
 Review comment:
   https://en.wikipedia.org/wiki/Exit_status
   
   > EXIT_SUCCESS and EXIT_FAILURE. On Unix-like systems these are equal to 0 and 1 respectively.
   
   sure, its consistent with other docs about RC

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell closed pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell closed pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384620030
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedRepairUtils.java
 ##########
 @@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.ResultSet;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.impl.AbstractCluster;
+import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.metrics.StorageMetrics;
+
+public final class DistributedRepairUtils
+{
+    public static final int DEFAULT_COORDINATOR = 1;
+
+    private DistributedRepairUtils()
+    {
+
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, RepairType repairType, boolean withNotifications, String... args) {
+        return repair(cluster, DEFAULT_COORDINATOR, repairType, withNotifications, args);
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, int node, RepairType repairType, boolean withNotifications, String... args) {
+        args = repairType.append(args);
+        args = ArrayUtils.addAll(new String[] { "repair" }, args);
+        return cluster.get(node).nodetoolResult(withNotifications, args);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster)
+    {
+        return getRepairExceptions(cluster, DEFAULT_COORDINATOR);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster, int node)
+    {
+        return cluster.get(node).callOnInstance(() -> StorageMetrics.repairExceptions.getCount());
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, String ks, String table)
+    {
+        return queryParentRepairHistory(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        // This is kinda brittle since the caller never gets the ID and can't ask for the ID; it needs to infer the id
+        // this logic makes the assumption the ks/table pairs are unique (should be or else create should fail) so any
+        // repair for that pair will be the repair id
+        Set<String> tableNames = table == null? Collections.emptySet() : ImmutableSet.of(table);
+        ResultSet rs = null;
+        Exception latestException = null;
+        for (int i = 0; i < 10; i++)
+        {
+            try
+            {
+                rs = cluster.coordinator(coordinator)
+                            .executeWithResult("SELECT * FROM system_distributed.parent_repair_history", ConsistencyLevel.QUORUM)
+                            .filter(row -> ks.equals(row.getString("keyspace_name")))
+                            .filter(row -> tableNames.equals(row.getSet("columnfamily_names")));
+                break;
+            }
+            catch (Exception e)
+            {
+                latestException = e;
+                rs = null;
+                //TODO do we have a backoff stategy I can leverage?  I would prefer expotential but don't want to add for somethinr minor
 
 Review comment:
   ok, copied the same retry logic I always use... patch is larger now but Retry offers more reuse.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382176472
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,53 +146,128 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
+
+        success(msg);
+    }
+
+    private void success(String msg)
     {
+        fireProgressEvent(new ProgressEvent(ProgressEventType.SUCCESS, progressCounter.get(), totalProgress, msg));
+        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.COMPLETED,
+                                                        ImmutableList.of(msg));
+        complete(null);
+    }
+
+    public void notifyError(Throwable error)
+    {
+        // exception should be ignored
+        if (error instanceof SomeRepairFailedException)
+            return;
+        logger.error("Repair {} failed:", parentSession, error);
+
         StorageMetrics.repairExceptions.inc();
-        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, message);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCount, totalProgress, errorMessage));
-        String completionMessage = String.format("Repair command #%d finished with error", cmd);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCount, totalProgress, completionMessage));
-        recordFailure(errorMessage, completionMessage);
+        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, error.getMessage());
+        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCounter.get(), totalProgress, errorMessage));
+
+        // since this can fail, update table only after updating in-memory and notification state
+        if (!options.isPreview())
+        {
+            SystemDistributedKeyspace.failParentRepair(parentSession, error);
+        }
     }
 
+    private void fail(String reason)
+    {
+        if (reason == null)
+            reason = "Some repair failed";
+        String completionMessage = String.format("Repair command #%d finished with error", cmd);
+
+        // Note we rely on the first message being the reason for the failure
+        // when inspecting this state from RepairRunner.queryForCompletedRepair
+        ActiveRepairService.instance.recordRepairStatus(cmd, ParentRepairStatus.FAILED,
+                                                        ImmutableList.of(reason, completionMessage));
+
+        complete(completionMessage);
+    }
 
-    protected void runMayThrow() throws Exception
+    private void complete(String msg)
     {
-        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.IN_PROGRESS, ImmutableList.of());
-        final TraceState traceState;
-        final UUID parentSession = UUIDGen.getTimeUUID();
-        final String tag = "repair:" + cmd;
+        long durationMillis = System.currentTimeMillis() - creationTimeMillis;
+        if (msg == null) {
+            String duration = DurationFormatUtils.formatDurationWords(durationMillis,true, true);
+            msg = String.format("Repair command #%d finished in %s", cmd, duration);
+        }
 
-        final AtomicInteger progress = new AtomicInteger();
-        final int totalProgress = 4 + options.getRanges().size(); // get valid column families, calculate neighbors, validation, prepare for repair + number of ranges to repair
+        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCounter.get(), totalProgress, msg));
 
-        String[] columnFamilies = options.getColumnFamilies().toArray(new String[options.getColumnFamilies().size()]);
-        Iterable<ColumnFamilyStore> validColumnFamilies;
-        try
+        if (ActiveRepairService.instance.hasParentRepairSession(parentSession))
 
 Review comment:
   This code is inherently race prone as you're actually checking and removing from a `ConcurrentMap`. It would be better to simply have `removeIfPresent`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384214589
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedRepairUtils.java
 ##########
 @@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.ResultSet;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.impl.AbstractCluster;
+import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.metrics.StorageMetrics;
+
+public final class DistributedRepairUtils
+{
+    public static final int DEFAULT_COORDINATOR = 1;
+
+    private DistributedRepairUtils()
+    {
+
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, RepairType repairType, boolean withNotifications, String... args) {
+        return repair(cluster, DEFAULT_COORDINATOR, repairType, withNotifications, args);
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, int node, RepairType repairType, boolean withNotifications, String... args) {
+        args = repairType.append(args);
+        args = ArrayUtils.addAll(new String[] { "repair" }, args);
+        return cluster.get(node).nodetoolResult(withNotifications, args);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster)
+    {
+        return getRepairExceptions(cluster, DEFAULT_COORDINATOR);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster, int node)
+    {
+        return cluster.get(node).callOnInstance(() -> StorageMetrics.repairExceptions.getCount());
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, String ks, String table)
+    {
+        return queryParentRepairHistory(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        // This is kinda brittle since the caller never gets the ID and can't ask for the ID; it needs to infer the id
+        // this logic makes the assumption the ks/table pairs are unique (should be or else create should fail) so any
+        // repair for that pair will be the repair id
+        Set<String> tableNames = table == null? Collections.emptySet() : ImmutableSet.of(table);
+        ResultSet rs = null;
+        Exception latestException = null;
+        for (int i = 0; i < 10; i++)
+        {
+            try
+            {
+                rs = cluster.coordinator(coordinator)
+                            .executeWithResult("SELECT * FROM system_distributed.parent_repair_history", ConsistencyLevel.QUORUM)
+                            .filter(row -> ks.equals(row.getString("keyspace_name")))
+                            .filter(row -> tableNames.equals(row.getSet("columnfamily_names")));
+                break;
+            }
+            catch (Exception e)
+            {
+                latestException = e;
+                rs = null;
+                //TODO do we have a backoff stategy I can leverage?  I would prefer expotential but don't want to add for somethinr minor
 
 Review comment:
   I keep copy/pasting a similar one, and @jrwest asked me to create a util since this behavior is needed.
   
   I guess I should create that util now...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384235549
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/LongTokenRange.java
 ##########
 @@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.io.Serializable;
+
+public final class LongTokenRange implements Serializable
 
 Review comment:
   > We already have API for bootstrap that works with long tokens, and having this in addition sort of hints at a need to do something like that.
   
   you mean ```
   public static interface TokenSupplier
       {
           public long token(int nodeId);
       }
   ```
   
   Yeah, that looks to add startup integration, this is a hack to get runtime accessor.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384016231
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -199,23 +206,16 @@ public void schemaChangeInternal(String query)
 
     private void registerMockMessaging(ICluster cluster)
     {
-        BiConsumer<InetAddressAndPort, IMessage> deliverToInstance = (to, message) -> cluster.get(to).receiveMessage(message);
-        BiConsumer<InetAddressAndPort, IMessage> deliverToInstanceIfNotFiltered = (to, message) -> {
-            if (permitMessage(cluster, to, message))
-                deliverToInstance.accept(to, message);
-        };
-
         MessagingService.instance().outboundSink.add((message, to) -> {
-            deliverToInstanceIfNotFiltered.accept(to, serializeMessage(message.from(), to, message));
+            cluster.get(to).receiveMessage(serializeMessage(message.from(), to, message));
             return false;
         });
     }
 
-    // unnecessary if registerMockMessaging used
     private void registerFilter(ICluster cluster)
     {
-        MessagingService.instance().outboundSink.add((message, to) -> {
-            return permitMessage(cluster, to, serializeMessage(message.from(), to, message));
+        MessagingService.instance().inboundSink.add(message -> {
+            return permitMessage(cluster, FBUtilities.getBroadcastAddressAndPort(), serializeMessage(message.from(), FBUtilities.getBroadcastAddressAndPort(), message));
 
 Review comment:
   I think we can even inline `permitMessage` now. WDYT?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r391520032
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/LongTokenRange.java
 ##########
 @@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.io.Serializable;
+
+public final class LongTokenRange implements Serializable
 
 Review comment:
   Yeah; this is a "known limitation" :)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] clohfink commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
clohfink commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384159342
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/ICoordinator.java
 ##########
 @@ -27,7 +27,11 @@
 {
     // a bit hacky, but ConsistencyLevel draws in too many dependent classes, so we cannot have a cross-version
     // method signature that accepts ConsistencyLevel directly.  So we just accept an Enum<?> and cast.
-    Object[][] execute(String query, Enum<?> consistencyLevel, Object... boundValues);
+    default Object[][] execute(String query, Enum<?> consistencyLevel, Object... boundValues)
+    {
+        return executeWithResult(query, consistencyLevel, boundValues).toObjectArrays();
+    }
+    ResultSet executeWithResult(String query, Enum<?> consistencyLevel, Object... boundValues);
 
 Review comment:
   I am a fan of getting rid of it, but Object[][] used in places other than this so might be another large refactor to change everything it touches and the methods like it

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384032682
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedRepairUtils.java
 ##########
 @@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.ResultSet;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.impl.AbstractCluster;
+import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.metrics.StorageMetrics;
+
+public final class DistributedRepairUtils
+{
+    public static final int DEFAULT_COORDINATOR = 1;
+
+    private DistributedRepairUtils()
+    {
+
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, RepairType repairType, boolean withNotifications, String... args) {
+        return repair(cluster, DEFAULT_COORDINATOR, repairType, withNotifications, args);
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, int node, RepairType repairType, boolean withNotifications, String... args) {
+        args = repairType.append(args);
+        args = ArrayUtils.addAll(new String[] { "repair" }, args);
+        return cluster.get(node).nodetoolResult(withNotifications, args);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster)
+    {
+        return getRepairExceptions(cluster, DEFAULT_COORDINATOR);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster, int node)
+    {
+        return cluster.get(node).callOnInstance(() -> StorageMetrics.repairExceptions.getCount());
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, String ks, String table)
+    {
+        return queryParentRepairHistory(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        // This is kinda brittle since the caller never gets the ID and can't ask for the ID; it needs to infer the id
+        // this logic makes the assumption the ks/table pairs are unique (should be or else create should fail) so any
+        // repair for that pair will be the repair id
+        Set<String> tableNames = table == null? Collections.emptySet() : ImmutableSet.of(table);
+        ResultSet rs = null;
+        Exception latestException = null;
+        for (int i = 0; i < 10; i++)
+        {
+            try
+            {
+                rs = cluster.coordinator(coordinator)
+                            .executeWithResult("SELECT * FROM system_distributed.parent_repair_history", ConsistencyLevel.QUORUM)
+                            .filter(row -> ks.equals(row.getString("keyspace_name")))
+                            .filter(row -> tableNames.equals(row.getSet("columnfamily_names")));
+                break;
+            }
+            catch (Exception e)
+            {
+                latestException = e;
+                rs = null;
+                //TODO do we have a backoff stategy I can leverage?  I would prefer expotential but don't want to add for somethinr minor
+                Uninterruptibles.sleepUninterruptibly( (i + 1) * 300, TimeUnit.MILLISECONDS);
+            }
+        }
+        if (rs == null)
+        {
+            // exception should exist
+            if (latestException == null)
+            {
+                Assert.fail("Unable to query system_distributed.parent_repair_history, got back neither result set or exception ");
+            }
+            if (latestException instanceof RuntimeException)
+                throw (RuntimeException) latestException;
+            throw new RuntimeException(latestException);
+        }
+        return rs;
+    }
+
+    public static void assertParentRepairNotExist(AbstractCluster<?> cluster, String ks, String table)
+    {
+        assertParentRepairNotExist(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static void assertParentRepairNotExist(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        ResultSet rs = queryParentRepairHistory(cluster, coordinator, ks, table);
+        Assert.assertFalse("No repairs should be found but at least one found", rs.hasNext());
+    }
+
+    public static void assertParentRepairNotExist(AbstractCluster<?> cluster, String ks)
+    {
+        assertParentRepairNotExist(cluster, DEFAULT_COORDINATOR, ks);
+    }
+
+    public static void assertParentRepairNotExist(AbstractCluster<?> cluster, int coordinator, String ks)
+    {
+        ResultSet rs = queryParentRepairHistory(cluster, coordinator, ks, null);
+        Assert.assertFalse("No repairs should be found but at least one found", rs.hasNext());
+    }
+
+    public static void assertParentRepairSuccess(AbstractCluster<?> cluster, String ks, String table)
+    {
+        assertParentRepairSuccess(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static void assertParentRepairSuccess(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        ResultSet rs = queryParentRepairHistory(cluster, coordinator, ks, table);
+        validateExistingParentRepair(rs, row -> {
+            // check completed
+            Assert.assertNotNull("finished_at not found, alears the repair is not complete?", rs.getTimestamp("finished_at"));
+
+            // check not failed (aka success)
+            Assert.assertNull("Exception found", rs.getString("exception_stacktrace"));
+            Assert.assertNull("Exception found", rs.getString("exception_message"));
+        });
+    }
+
+    public static void assertParentRepairFailedWithMessageContains(AbstractCluster<?> cluster, String ks, String table, String message)
+    {
+        assertParentRepairFailedWithMessageContains(cluster, DEFAULT_COORDINATOR, ks, table, message);
+    }
+
+    public static void assertParentRepairFailedWithMessageContains(AbstractCluster<?> cluster, int coordinator, String ks, String table, String message)
+    {
+        ResultSet rs = queryParentRepairHistory(cluster, coordinator, ks, table);
+        validateExistingParentRepair(rs, row -> {
+            // check completed
+            Assert.assertNotNull("finished_at not found, alears the repair is not complete?", rs.getTimestamp("finished_at"));
 
 Review comment:
   Typo in "alears" (in several places in this file) ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390150093
 
 

 ##########
 File path: test/unit/org/apache/cassandra/utils/Retry.java
 ##########
 @@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.IntToLongFunction;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+
+/**
+ * Class for retryable actions.
+ *
+ * @see {@link #retryWithBackoff(int, Supplier, Predicate)}
+ */
+public final class Retry
+{
+    private static final ScheduledExecutorService SCHEDULED = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("RetryScheduler"));
+
+    private Retry()
+    {
+
+    }
+
+    /**
+     * Schedule code to run after the defined duration.
+     *
+     * Since a executor was not defined, the global {@link ForkJoinPool#commonPool()} executor will be used, if this
+     * is not desirable then should use {@link #schedule(Duration, Executor, Runnable)}.
+     *
+     * @param duration how long to delay
+     * @param fn code to run
+     * @return future representing result
+     */
+    public static CompletableFuture<Void> schedule(final Duration duration, final Runnable fn)
+    {
+        return schedule(duration, ForkJoinPool.commonPool(), fn);
+    }
+
+    /**
+     * Schedule code to run after the defined duration on the provided executor.
+     *
+     * @param duration how long to delay
+     * @param executor to run on
+     * @param fn code to run
+     * @return future representing result
+     */
+    public static CompletableFuture<Void> schedule(final Duration duration, final Executor executor, final Runnable fn)
+    {
+        long nanos = duration.toNanos();
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        SCHEDULED.schedule(() -> run0(executor, future, fn), nanos, TimeUnit.NANOSECONDS);
 
 Review comment:
   can we simplify this as `Future future = SCHEDULED.schedule(fn, nanos, TimeUnit.NANOSECONDS);` ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384627403
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
 ##########
 @@ -151,9 +152,16 @@ public boolean isShutdown()
         @Override
         public synchronized void startup()
         {
+            startup(AbstractCluster.this);
+        }
+
+        public synchronized void startup(ICluster cluster)
+        {
+            if (cluster != AbstractCluster.this)
+                throw new IllegalArgumentException("Only the owning cluster can be used for startup"); //TODO why have this in the API?
 
 Review comment:
   Created https://issues.apache.org/jira/browse/CASSANDRA-15602. This would also help with binary compatibility, if we make changes to the user API it wouldn't affect the plugin API so each branch doesn't need to be changed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382170649
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -199,63 +286,39 @@ protected void runMayThrow() throws Exception
         }
         else
         {
-            fireProgressEvent(new ProgressEvent(ProgressEventType.START, 0, 100, message));
             traceState = null;
         }
 
         Set<InetAddressAndPort> allNeighbors = new HashSet<>();
         List<CommonRange> commonRanges = new ArrayList<>();
 
-        try
-        {
-            //pre-calculate output of getLocalReplicas and pass it to getNeighbors to increase performance and prevent
-            //calculation multiple times
-            Iterable<Range<Token>> keyspaceLocalRanges = storageService.getLocalReplicas(keyspace).ranges();
+        //pre-calculate output of getLocalReplicas and pass it to getNeighbors to increase performance and prevent
+        //calculation multiple times
+        Iterable<Range<Token>> keyspaceLocalRanges = storageService.getLocalReplicas(keyspace).ranges();
 
-            for (Range<Token> range : options.getRanges())
-            {
-                EndpointsForRange neighbors = ActiveRepairService.getNeighbors(keyspace, keyspaceLocalRanges, range,
-                                                                               options.getDataCenters(),
-                                                                               options.getHosts());
-
-                addRangeToNeighbors(commonRanges, range, neighbors);
-                allNeighbors.addAll(neighbors.endpoints());
-            }
-
-            progress.incrementAndGet();
-        }
-        catch (IllegalArgumentException e)
+        for (Range<Token> range : options.getRanges())
         {
-            logger.error("Repair {} failed:", parentSession, e);
-            fireErrorAndComplete(progress.get(), totalProgress, e.getMessage());
-            return;
+            EndpointsForRange neighbors = ActiveRepairService.getNeighbors(keyspace, keyspaceLocalRanges, range,
 
 Review comment:
   Can be `final`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390676885
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -552,9 +568,66 @@ public int liveMemberCount()
         }).call();
     }
 
-    public int nodetool(String... commandAndArgs)
+    public NodeToolResult nodetoolResult(boolean withNotifications, String... commandAndArgs)
     {
-        return sync(() -> new NodeTool(new InternalNodeProbeFactory()).execute(commandAndArgs)).call();
+        return sync(() -> {
+            DTestNodeTool nodetool = new DTestNodeTool(withNotifications);
+            int rc =  nodetool.execute(commandAndArgs);
+            return new NodeToolResult(commandAndArgs, rc, new ArrayList<>(nodetool.notifications.notifications), nodetool.latestError);
+        }).call();
+    }
+
+    private static class DTestNodeTool extends NodeTool {
+        private final StorageServiceMBean storageProxy;
+        private final CollectingNotificationListener notifications = new CollectingNotificationListener();
+
+        private Throwable latestError;
+
+        DTestNodeTool(boolean withNotifications) {
+            super(new InternalNodeProbeFactory(withNotifications));
 
 Review comment:
   do you mean 
   
   ```
   storageProxy = new InternalNodeProbe(withNotifications).getStorageService();
   ```
   
   Need it there as well.  `storageProxy` is what allows notifications so need to disable here as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390677892
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java
 ##########
 @@ -61,7 +79,26 @@ protected void connect()
         mbeanServerConn = null;
         jmxc = null;
 
-        ssProxy = StorageService.instance;
+        if (withNotifications)
+        {
+            ssProxy = StorageService.instance;
+        }
+        else
+        {
+            // replace the notification apis with a no-op method
+            StorageServiceMBean mock = Mockito.spy(StorageService.instance);
+            Mockito.doNothing().when(mock).addNotificationListener(Mockito.any(), Mockito.any(), Mockito.any());
+            try
+            {
+                Mockito.doNothing().when(mock).removeNotificationListener(Mockito.any(), Mockito.any(), Mockito.any());
+                Mockito.doNothing().when(mock).removeNotificationListener(Mockito.any());
+            }
+            catch (ListenerNotFoundException e)
+            {
+                throw new AssertionError(e);
+            }
+            ssProxy = mock;
 
 Review comment:
   Very very good question!
   
   This only happens when `withNotifications == false`.  The thing about JMX notifications are they are lossy.  Given the lossy nature we need the ability to test that we do the right thing when notifications are not seen on the caller.  So we disable notifications by creating a "spy" on StorageService and making `removeNotificationListener` do nothing.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390472529
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/NodeToolResult.java
 ##########
 @@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.management.Notification;
+
+import org.junit.Assert;
+
+public class NodeToolResult
+{
+    private final String[] commandAndArgs;
+    private final int rc;
+    private final List<Notification> notifications;
+    private final Throwable error;
+
+    public NodeToolResult(String[] commandAndArgs, int rc, List<Notification> notifications, Throwable error)
+    {
+        this.commandAndArgs = commandAndArgs;
+        this.rc = rc;
+        this.notifications = notifications;
+        this.error = error;
+    }
+
+    public String[] getCommandAndArgs()
+    {
+        return commandAndArgs;
+    }
+
+    public int getRc()
+    {
+        return rc;
+    }
+
+    public List<Notification> getNotifications()
+    {
+        return notifications;
+    }
+
+    public Throwable getError()
+    {
+        return error;
+    }
+
+    public Asserts asserts()
+    {
+        return new Asserts();
+    }
+
+    public final class Asserts {
+        public Asserts success() {
+            Assert.assertEquals("nodetool command " + commandAndArgs[0] + " was not successful", 0, rc);
+            return this;
+        }
+
+        public Asserts failure() {
+            Assert.assertNotEquals("nodetool command " + commandAndArgs[0] + " was successful but not expected to be", 0, rc);
+            return this;
+        }
+
+        public Asserts errorContains(String msg) {
+            Assert.assertNotNull("No exception was found but expected one", error);
+            Assert.assertTrue("Error message '" + error.getMessage() + "' does not contain '" + msg + "'", error.getMessage().contains(msg));
+            return this;
+        }
+
+        public Asserts notificationContains(String msg) {
+            Assert.assertNotNull("notifications not defined", notifications);
+            Assert.assertFalse("notifications not defined", notifications.isEmpty());
+            for (Notification n : notifications) {
+                if (n.getMessage().contains(msg)) {
+                    return this;
+                }
+            }
+            Assert.fail("Unable to locate message " + msg + " in notifications: " + notifications);
+            return this; // unreachable
+        }
+
+        public Asserts notificationContains(ProgressEventType type, String msg) {
+            int userType = type.ordinal();
+            Assert.assertNotNull("notifications not defined", notifications);
+            Assert.assertFalse("notifications not defined", notifications.isEmpty());
+            for (Notification n : notifications) {
+                if (notificationType(n) == userType) {
+                    if (n.getMessage().contains(msg)) {
+                        return this;
+                    }
+                }
+            }
+            Assert.fail("Unable to locate message '" + msg + "' in notifications: " + notifications);
+            return this; // unreachable
+        }
+    }
+
+    private static int notificationType(Notification n)
+    {
+        return ((Map<String, Integer>) n.getUserData()).get("type").intValue();
+    }
+
+    public String toString()
+    {
+        return "NodeToolResult{" +
+               "commandAndArgs=" + Arrays.toString(commandAndArgs) +
+               ", rc=" + rc +
+               ", notifications=[" + notifications.stream().map(n -> ProgressEventType.values()[notificationType(n)].name()).collect(Collectors.joining(", ")) + "]" +
+               ", error=" + error +
+               '}';
+    }
+
+    /**
+     * Progress event type.
+     *
+     * <p>
+     * Progress starts by emitting {@link #START}, followed by emitting zero or more {@link #PROGRESS} events,
+     * then it emits either one of {@link #ERROR}/{@link #ABORT}/{@link #SUCCESS}.
+     * Progress indicates its completion by emitting {@link #COMPLETE} at the end of process.
+     * </p>
+     * <p>
+     * {@link #NOTIFICATION} event type is used to just notify message without progress.
+     * </p>
+     */
+    public enum ProgressEventType
 
 Review comment:
   Yes, but its based off another jira!
   
   @ifesdjeen is moving dtest into its own repo and making it so the public api does not depend on Cassandra; the current dtest logic depends on Cassandra which is causing a ton of issues (maintenance, compatibility for upgrade, etc.).  Given the fact I am aware of this, I didn't want to depend on it and require @ifesdjeen to remove it...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384228225
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/ResultSet.java
 ##########
 @@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Date;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+public class ResultSet implements Iterator<Row>
+{
+    public static final ResultSet EMPTY = new ResultSet(new String[0], null);
+
+    private final String[] names;
+    private final Object[][] results;
+    private final Predicate<Row> filter;
+    private final Row row;
+    private int offset = -1;
+
+    public ResultSet(String[] names, Object[][] results)
+    {
+        this.names = names;
+        this.results = results;
+        this.row = new Row(names);
+        this.filter = ignore -> true;
+    }
+
+    private ResultSet(String[] names, Object[][] results, Predicate<Row> filter, int offset)
+    {
+        this.names = names;
+        this.results = results;
+        this.filter = filter;
+        this.offset = offset;
+        this.row = new Row(names);
+    }
+
+    public String[] getNames()
+    {
+        return names;
+    }
+
+    public boolean isEmpty()
+    {
+        return results.length == 0;
+    }
+
+    public int size()
+    {
+        return results.length;
+    }
+
+    public ResultSet filter(Predicate<Row> fn)
+    {
+        return new ResultSet(names, results, filter.and(fn), offset);
+    }
+
+    public Object[][] toObjectArrays()
+    {
+        return results;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+        if (results == null)
+            return false;
+        while ((offset += 1) < results.length)
+        {
+            row.setResults(results[offset]);
+            if (filter.test(row))
+            {
+                return true;
+            }
+        }
+        row.setResults(null);
+        return false;
+    }
+
+    @Override
+    public Row next()
+    {
+        if (offset < 0 || offset >= results.length)
+            throw new NoSuchElementException();
+        return row;
 
 Review comment:
   Im not sure how I feel about this.  
   
   ```
   while (rs.hasNext()) {
     list.add(rs.next());
   }
   ```
   
   vs
   
   ```
   rs.forEach(list::add)
   ```
   
   Both are wrong.
   
   How about this, I added a `public Row copy()` method so you can copy if needed.  I don't like the idea of blocking Iterable since you still need to iterate.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384229586
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/Row.java
 ##########
 @@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Arrays;
+import java.util.Date;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import javax.annotation.Nullable;
+
+public class Row
+{
+    private final String[] names;
+    @Nullable private Object[] results; // mutable to avoid allocations in loops
+
+    public Row(String[] names)
+    {
+        this.names = names;
+    }
+
+    void setResults(@Nullable Object[] results)
+    {
+        this.results = results;
+    }
+
+    public <T> T get(String name)
+    {
+        checkAccess();
+        int idx = findIndex(name);
 
 Review comment:
   pushed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390769868
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/QueryResult.java
 ##########
 @@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.function.Predicate;
+
+/**
+ * A table of data representing a complete query result.
+ *
+ * A <code>QueryResult</code> is different from {@link java.sql.ResultSet} in several key ways:
+ *
+ * <ul>
+ *     <li>represents a complete result rather than a cursor</li>
+ *     <li>returns a {@link Row} to access the current row of data</li>
+ *     <li>relies on object pooling; {@link #hasNext()} may return the same object just with different data, accessing a
+ *     {@link Row} from a previous {@link #hasNext()} call has undefined behavior.</li>
+ *     <li>includes {@link #filter(Predicate)}, this will do client side filtering since Apache Cassandra is more
+ *     restrictive on server side filtering</li>
+ * </ul>
+ *
+ * <h2>Unsafe patterns</h2>
+ *
+ * Below are a few unsafe patterns which may lead to unexpected results
+ *
+ * <code>{@code
+ * while (rs.hasNext()) {
+ *   list.add(rs.next());
 
 Review comment:
   got it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390672358
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,167 +145,239 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
 
 Review comment:
   switched to that and refactored all usage to use a new `notification` method.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390106009
 
 

 ##########
 File path: src/java/org/apache/cassandra/service/ActiveRepairService.java
 ##########
 @@ -525,6 +525,20 @@ public synchronized ParentRepairSession removeParentRepairSession(UUID parentSes
         return parentRepairSessions.remove(parentSessionId);
     }
 
+    public synchronized ParentRepairSession removeParentRepairSessionIfPresent(UUID parentSessionId)
 
 Review comment:
   this method looks very similar to `removeParentRepairSession`.. should we refactor it?
   
   alternatively, we can change `removeParentRepairSession` (also used in multiple places) to gracefully handle non-existing parent session.. there is no point throw NPE if parent session is already gone..

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390117550
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/NodeToolResult.java
 ##########
 @@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.management.Notification;
+
+import org.junit.Assert;
+
+public class NodeToolResult
+{
+    private final String[] commandAndArgs;
+    private final int rc;
+    private final List<Notification> notifications;
+    private final Throwable error;
+
+    public NodeToolResult(String[] commandAndArgs, int rc, List<Notification> notifications, Throwable error)
+    {
+        this.commandAndArgs = commandAndArgs;
+        this.rc = rc;
+        this.notifications = notifications;
+        this.error = error;
+    }
+
+    public String[] getCommandAndArgs()
+    {
+        return commandAndArgs;
+    }
+
+    public int getRc()
+    {
+        return rc;
+    }
+
+    public List<Notification> getNotifications()
+    {
+        return notifications;
+    }
+
+    public Throwable getError()
+    {
+        return error;
+    }
+
+    public Asserts asserts()
+    {
+        return new Asserts();
+    }
+
+    public final class Asserts {
+        public Asserts success() {
+            Assert.assertEquals("nodetool command " + commandAndArgs[0] + " was not successful", 0, rc);
+            return this;
+        }
+
+        public Asserts failure() {
+            Assert.assertNotEquals("nodetool command " + commandAndArgs[0] + " was successful but not expected to be", 0, rc);
+            return this;
+        }
+
+        public Asserts errorContains(String msg) {
+            Assert.assertNotNull("No exception was found but expected one", error);
+            Assert.assertTrue("Error message '" + error.getMessage() + "' does not contain '" + msg + "'", error.getMessage().contains(msg));
+            return this;
+        }
+
+        public Asserts notificationContains(String msg) {
+            Assert.assertNotNull("notifications not defined", notifications);
+            Assert.assertFalse("notifications not defined", notifications.isEmpty());
+            for (Notification n : notifications) {
+                if (n.getMessage().contains(msg)) {
+                    return this;
+                }
+            }
+            Assert.fail("Unable to locate message " + msg + " in notifications: " + notifications);
+            return this; // unreachable
+        }
+
+        public Asserts notificationContains(ProgressEventType type, String msg) {
+            int userType = type.ordinal();
+            Assert.assertNotNull("notifications not defined", notifications);
+            Assert.assertFalse("notifications not defined", notifications.isEmpty());
+            for (Notification n : notifications) {
+                if (notificationType(n) == userType) {
+                    if (n.getMessage().contains(msg)) {
+                        return this;
+                    }
+                }
+            }
+            Assert.fail("Unable to locate message '" + msg + "' in notifications: " + notifications);
+            return this; // unreachable
+        }
+    }
+
+    private static int notificationType(Notification n)
+    {
+        return ((Map<String, Integer>) n.getUserData()).get("type").intValue();
+    }
+
+    public String toString()
+    {
+        return "NodeToolResult{" +
+               "commandAndArgs=" + Arrays.toString(commandAndArgs) +
+               ", rc=" + rc +
+               ", notifications=[" + notifications.stream().map(n -> ProgressEventType.values()[notificationType(n)].name()).collect(Collectors.joining(", ")) + "]" +
+               ", error=" + error +
+               '}';
+    }
+
+    /**
+     * Progress event type.
+     *
+     * <p>
+     * Progress starts by emitting {@link #START}, followed by emitting zero or more {@link #PROGRESS} events,
+     * then it emits either one of {@link #ERROR}/{@link #ABORT}/{@link #SUCCESS}.
+     * Progress indicates its completion by emitting {@link #COMPLETE} at the end of process.
+     * </p>
+     * <p>
+     * {@link #NOTIFICATION} event type is used to just notify message without progress.
+     * </p>
+     */
+    public enum ProgressEventType
 
 Review comment:
   is there a reason not to use `org.apache.cassandra.utils.progress.ProgressEventType` directly?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r383986316
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/ICoordinator.java
 ##########
 @@ -27,7 +27,11 @@
 {
     // a bit hacky, but ConsistencyLevel draws in too many dependent classes, so we cannot have a cross-version
     // method signature that accepts ConsistencyLevel directly.  So we just accept an Enum<?> and cast.
-    Object[][] execute(String query, Enum<?> consistencyLevel, Object... boundValues);
+    default Object[][] execute(String query, Enum<?> consistencyLevel, Object... boundValues)
+    {
+        return executeWithResult(query, consistencyLevel, boundValues).toObjectArrays();
+    }
+    ResultSet executeWithResult(String query, Enum<?> consistencyLevel, Object... boundValues);
 
 Review comment:
   In a long run, do we even want to keep `object[][]`? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384216109
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedRepairUtils.java
 ##########
 @@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.ResultSet;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.impl.AbstractCluster;
+import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.metrics.StorageMetrics;
+
+public final class DistributedRepairUtils
+{
+    public static final int DEFAULT_COORDINATOR = 1;
+
+    private DistributedRepairUtils()
+    {
+
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, RepairType repairType, boolean withNotifications, String... args) {
+        return repair(cluster, DEFAULT_COORDINATOR, repairType, withNotifications, args);
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, int node, RepairType repairType, boolean withNotifications, String... args) {
+        args = repairType.append(args);
+        args = ArrayUtils.addAll(new String[] { "repair" }, args);
+        return cluster.get(node).nodetoolResult(withNotifications, args);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster)
+    {
+        return getRepairExceptions(cluster, DEFAULT_COORDINATOR);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster, int node)
+    {
+        return cluster.get(node).callOnInstance(() -> StorageMetrics.repairExceptions.getCount());
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, String ks, String table)
+    {
+        return queryParentRepairHistory(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        // This is kinda brittle since the caller never gets the ID and can't ask for the ID; it needs to infer the id
+        // this logic makes the assumption the ks/table pairs are unique (should be or else create should fail) so any
+        // repair for that pair will be the repair id
+        Set<String> tableNames = table == null? Collections.emptySet() : ImmutableSet.of(table);
+        ResultSet rs = null;
+        Exception latestException = null;
+        for (int i = 0; i < 10; i++)
+        {
+            try
+            {
+                rs = cluster.coordinator(coordinator)
+                            .executeWithResult("SELECT * FROM system_distributed.parent_repair_history", ConsistencyLevel.QUORUM)
+                            .filter(row -> ks.equals(row.getString("keyspace_name")))
+                            .filter(row -> tableNames.equals(row.getSet("columnfamily_names")));
+                break;
+            }
+            catch (Exception e)
+            {
+                latestException = e;
+                rs = null;
+                //TODO do we have a backoff stategy I can leverage?  I would prefer expotential but don't want to add for somethinr minor
+                Uninterruptibles.sleepUninterruptibly( (i + 1) * 300, TimeUnit.MILLISECONDS);
+            }
+        }
+        if (rs == null)
+        {
+            // exception should exist
+            if (latestException == null)
+            {
+                Assert.fail("Unable to query system_distributed.parent_repair_history, got back neither result set or exception ");
+            }
+            if (latestException instanceof RuntimeException)
 
 Review comment:
   I looked closer and its true with java, but could also be not true...
   
   ```
   Exception latestException = null;
   ...
   try
               {
                   rs = cluster.coordinator(coordinator)
                               .executeWithResult("SELECT * FROM system_distributed.parent_repair_history", ConsistencyLevel.QUORUM)
                               .filter(row -> ks.equals(row.getString("keyspace_name")))
                               .filter(row -> tableNames.equals(row.getSet("columnfamily_names")));
                   break;
               }
               catch (Exception e)
               {
                   latestException = e;
                   rs = null;
                   //TODO do we have a backoff stategy I can leverage?  I would prefer expotential but don't want to add for somethinr minor
                   Uninterruptibles.sleepUninterruptibly( (i + 1) * 300, TimeUnit.MILLISECONDS);
               }
   ```
   
   its only set in that catch block, and the APIs do not explicitly throw, so for `javac` this only happens if its a RuntimeException or a Error.  Now, lets say for some reason we implement cluster in Scala (god no... please no...), Exception isn't checked there so something like IOException could now be thrown!!!!!!
   
   This is just me being defensive.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390131976
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java
 ##########
 @@ -61,7 +79,26 @@ protected void connect()
         mbeanServerConn = null;
         jmxc = null;
 
-        ssProxy = StorageService.instance;
+        if (withNotifications)
+        {
+            ssProxy = StorageService.instance;
+        }
+        else
+        {
+            // replace the notification apis with a no-op method
+            StorageServiceMBean mock = Mockito.spy(StorageService.instance);
+            Mockito.doNothing().when(mock).addNotificationListener(Mockito.any(), Mockito.any(), Mockito.any());
+            try
+            {
+                Mockito.doNothing().when(mock).removeNotificationListener(Mockito.any(), Mockito.any(), Mockito.any());
+                Mockito.doNothing().when(mock).removeNotificationListener(Mockito.any());
+            }
+            catch (ListenerNotFoundException e)
+            {
+                throw new AssertionError(e);
+            }
+            ssProxy = mock;
 
 Review comment:
   I don't understand what are the cases that a no-op storage-service is needed.. what happened if we just use `StorageService.instance`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384233479
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,53 +146,128 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
+
+        success(msg);
+    }
+
+    private void success(String msg)
     {
+        fireProgressEvent(new ProgressEvent(ProgressEventType.SUCCESS, progressCounter.get(), totalProgress, msg));
+        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.COMPLETED,
+                                                        ImmutableList.of(msg));
+        complete(null);
+    }
+
+    public void notifyError(Throwable error)
+    {
+        // exception should be ignored
+        if (error instanceof SomeRepairFailedException)
+            return;
+        logger.error("Repair {} failed:", parentSession, error);
+
         StorageMetrics.repairExceptions.inc();
-        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, message);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCount, totalProgress, errorMessage));
-        String completionMessage = String.format("Repair command #%d finished with error", cmd);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCount, totalProgress, completionMessage));
-        recordFailure(errorMessage, completionMessage);
+        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, error.getMessage());
+        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCounter.get(), totalProgress, errorMessage));
+
+        // since this can fail, update table only after updating in-memory and notification state
+        if (!options.isPreview())
+        {
+            SystemDistributedKeyspace.failParentRepair(parentSession, error);
+        }
     }
 
+    private void fail(String reason)
+    {
+        if (reason == null)
+            reason = "Some repair failed";
+        String completionMessage = String.format("Repair command #%d finished with error", cmd);
+
+        // Note we rely on the first message being the reason for the failure
+        // when inspecting this state from RepairRunner.queryForCompletedRepair
+        ActiveRepairService.instance.recordRepairStatus(cmd, ParentRepairStatus.FAILED,
+                                                        ImmutableList.of(reason, completionMessage));
+
+        complete(completionMessage);
+    }
 
-    protected void runMayThrow() throws Exception
+    private void complete(String msg)
     {
-        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.IN_PROGRESS, ImmutableList.of());
-        final TraceState traceState;
-        final UUID parentSession = UUIDGen.getTimeUUID();
-        final String tag = "repair:" + cmd;
+        long durationMillis = System.currentTimeMillis() - creationTimeMillis;
+        if (msg == null) {
+            String duration = DurationFormatUtils.formatDurationWords(durationMillis,true, true);
+            msg = String.format("Repair command #%d finished in %s", cmd, duration);
+        }
 
-        final AtomicInteger progress = new AtomicInteger();
-        final int totalProgress = 4 + options.getRanges().size(); // get valid column families, calculate neighbors, validation, prepare for repair + number of ranges to repair
+        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCounter.get(), totalProgress, msg));
 
-        String[] columnFamilies = options.getColumnFamilies().toArray(new String[options.getColumnFamilies().size()]);
-        Iterable<ColumnFamilyStore> validColumnFamilies;
-        try
+        if (ActiveRepairService.instance.hasParentRepairSession(parentSession))
+            ActiveRepairService.instance.removeParentRepairSession(parentSession);
+        Context ctx = context;
+        if (options.isTraced() && ctx != null && ctx.traceState != null)
         {
-            validColumnFamilies = storageService.getValidColumnFamilies(false, false, keyspace, columnFamilies);
-            progress.incrementAndGet();
+            for (ProgressListener listener : listeners)
+                ctx.traceState.removeProgressListener(listener);
+            // Because DebuggableThreadPoolExecutor#afterExecute and this callback
+            // run in a nondeterministic order (within the same thread), the
+            // TraceState may have been nulled out at this point. The TraceState
+            // should be traceState, so just set it without bothering to check if it
+            // actually was nulled out.
+            Tracing.instance.set(ctx.traceState);
+            Tracing.traceRepair(msg);
+            Tracing.instance.stopSession();
+        }
+
+        Keyspace.open(keyspace).metric.repairTime.update(durationMillis, TimeUnit.MILLISECONDS);
+    }
+
+    public void run()
+    {
+        try {
+            // Why is this before setup/start when its publishing the start event?  For backwards compatability
+            // One of the first things we did before was publish this before validating, so publish early to keep
+            // that the same.
+            String message = String.format("Starting repair command #%d (%s), repairing keyspace %s with %s", cmd, parentSession, keyspace,
+                                           options);
+            logger.info(message);
+            Tracing.traceRepair(message);
+            fireProgressEvent(new ProgressEvent(ProgressEventType.START, 0, 100, message));
+
+            Pair<Context, String> setup = setup();
 
 Review comment:
   I added Either...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390468434
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -578,86 +648,26 @@ public RepairCompleteCallback(UUID parentSession,
 
         public void onSuccess(Object result)
         {
-            if (!options.isPreview())
-            {
-                SystemDistributedKeyspace.successfulParentRepair(parentSession, successfulRanges);
-            }
-            final String message;
+            maybeStoreParentRepairSuccess(successfulRanges);
             if (hasFailure.get())
             {
-                StorageMetrics.repairExceptions.inc();
-                message = "Some repair failed";
-                fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progress.get(), totalProgress,
-                                                    message));
+                fail(null);
 
 Review comment:
   Nope!  
   
   The issue is that errors don't get aggregated so the call to fail the job may not know the root cause(s).  If we add an error when `fail` is called then the client will get this last; producing a somewhat meaningless error message.
   
   Right now error messages are sent to client at the source of the error (lets say 100 RepairJobs submitted and 2 fail; client gets 2 error events).  The current nodetool logic doesn't log the errors but saves the last seen error as the "root cause" and prints that on complete.  With the current example one of the repair jobs will randomly be chosen as the root cause (could be two distinct issues...) and reported to the operator.  If we switch to fail producing an error (which happens on trunk in some cases) the operator gets a meaningless error message saying "something broke".
   
   This code assumes the root cause triggered an error event; it was an even larger refactor to propagate errors up, so chose not to do that.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382205353
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -418,6 +481,12 @@ public void onSuccess(List<RepairSessionResult> results)
             {
                 try
                 {
+                    if (results == null || results.stream().anyMatch(s -> s == null))
+                    {
+                        // something failed
+                        fail(null);
 
 Review comment:
   In this case no without a larger rewrite. All paths of repair rely on list of futures to future as list where null means “error”. To propagate error is more invasive and would prefer in the JIRA about repair hanging.
   
   I did change things a little bit. The errors are sent via notification (lossy) so may see a more detailed error; this is a fall back if notification was lost. 
   
   I test with and without notification for reasons like this

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r379650296
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -374,6 +375,8 @@ public void startup(ICluster cluster)
                 }
                 JVMStabilityInspector.replaceKiller(new InstanceKiller());
 
+                ActiveRepairService.instance.start();
 
 Review comment:
   this enables IR

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390769595
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/NodeToolResult.java
 ##########
 @@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.management.Notification;
+
+import org.junit.Assert;
+
+public class NodeToolResult
+{
+    private final String[] commandAndArgs;
+    private final int rc;
+    private final List<Notification> notifications;
+    private final Throwable error;
+
+    public NodeToolResult(String[] commandAndArgs, int rc, List<Notification> notifications, Throwable error)
+    {
+        this.commandAndArgs = commandAndArgs;
+        this.rc = rc;
+        this.notifications = notifications;
+        this.error = error;
+    }
+
+    public String[] getCommandAndArgs()
+    {
+        return commandAndArgs;
+    }
+
+    public int getRc()
+    {
+        return rc;
+    }
+
+    public List<Notification> getNotifications()
+    {
+        return notifications;
+    }
+
+    public Throwable getError()
+    {
+        return error;
+    }
+
+    public Asserts asserts()
+    {
+        return new Asserts();
+    }
+
+    public final class Asserts {
+        public Asserts success() {
+            Assert.assertEquals("nodetool command " + commandAndArgs[0] + " was not successful", 0, rc);
+            return this;
+        }
+
+        public Asserts failure() {
+            Assert.assertNotEquals("nodetool command " + commandAndArgs[0] + " was successful but not expected to be", 0, rc);
+            return this;
+        }
+
+        public Asserts errorContains(String msg) {
+            Assert.assertNotNull("No exception was found but expected one", error);
+            Assert.assertTrue("Error message '" + error.getMessage() + "' does not contain '" + msg + "'", error.getMessage().contains(msg));
+            return this;
+        }
+
+        public Asserts notificationContains(String msg) {
+            Assert.assertNotNull("notifications not defined", notifications);
+            Assert.assertFalse("notifications not defined", notifications.isEmpty());
+            for (Notification n : notifications) {
+                if (n.getMessage().contains(msg)) {
+                    return this;
+                }
+            }
+            Assert.fail("Unable to locate message " + msg + " in notifications: " + notifications);
+            return this; // unreachable
+        }
+
+        public Asserts notificationContains(ProgressEventType type, String msg) {
+            int userType = type.ordinal();
+            Assert.assertNotNull("notifications not defined", notifications);
+            Assert.assertFalse("notifications not defined", notifications.isEmpty());
+            for (Notification n : notifications) {
+                if (notificationType(n) == userType) {
+                    if (n.getMessage().contains(msg)) {
+                        return this;
+                    }
+                }
+            }
+            Assert.fail("Unable to locate message '" + msg + "' in notifications: " + notifications);
+            return this; // unreachable
+        }
+    }
+
+    private static int notificationType(Notification n)
+    {
+        return ((Map<String, Integer>) n.getUserData()).get("type").intValue();
+    }
+
+    public String toString()
+    {
+        return "NodeToolResult{" +
+               "commandAndArgs=" + Arrays.toString(commandAndArgs) +
+               ", rc=" + rc +
+               ", notifications=[" + notifications.stream().map(n -> ProgressEventType.values()[notificationType(n)].name()).collect(Collectors.joining(", ")) + "]" +
+               ", error=" + error +
+               '}';
+    }
+
+    /**
+     * Progress event type.
+     *
+     * <p>
+     * Progress starts by emitting {@link #START}, followed by emitting zero or more {@link #PROGRESS} events,
+     * then it emits either one of {@link #ERROR}/{@link #ABORT}/{@link #SUCCESS}.
+     * Progress indicates its completion by emitting {@link #COMPLETE} at the end of process.
+     * </p>
+     * <p>
+     * {@link #NOTIFICATION} event type is used to just notify message without progress.
+     * </p>
+     */
+    public enum ProgressEventType
 
 Review comment:
   thanks for the explanation.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384210809
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -374,6 +375,8 @@ public void startup(ICluster cluster)
                 }
                 JVMStabilityInspector.replaceKiller(new InstanceKiller());
 
+                ActiveRepairService.instance.start();
 
 Review comment:
   we don't block repair behind a feature flag; this is a boolean saying 'yep, IR was started'

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382172794
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -418,6 +481,12 @@ public void onSuccess(List<RepairSessionResult> results)
             {
                 try
                 {
+                    if (results == null || results.stream().anyMatch(s -> s == null))
+                    {
+                        // something failed
+                        fail(null);
 
 Review comment:
   "Something failed" is not very useful to the operator. Can we instead have a descriptive message so the operator can tell the difference between failing at this location vs failing at line [634](https://github.com/apache/cassandra/pull/446/files#diff-4bc513a60150419e20a9449c70a64a66R634)?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390117908
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/QueryResult.java
 ##########
 @@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.function.Predicate;
+
+/**
+ * A table of data representing a complete query result.
+ *
+ * A <code>QueryResult</code> is different from {@link java.sql.ResultSet} in several key ways:
+ *
+ * <ul>
+ *     <li>represents a complete result rather than a cursor</li>
+ *     <li>returns a {@link Row} to access the current row of data</li>
+ *     <li>relies on object pooling; {@link #hasNext()} may return the same object just with different data, accessing a
+ *     {@link Row} from a previous {@link #hasNext()} call has undefined behavior.</li>
+ *     <li>includes {@link #filter(Predicate)}, this will do client side filtering since Apache Cassandra is more
+ *     restrictive on server side filtering</li>
+ * </ul>
+ *
+ * <h2>Unsafe patterns</h2>
+ *
+ * Below are a few unsafe patterns which may lead to unexpected results
+ *
+ * <code>{@code
+ * while (rs.hasNext()) {
+ *   list.add(rs.next());
 
 Review comment:
   just curious, is there any benefit of reusing the same `Row` object?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382163012
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -91,10 +101,15 @@
     private final String keyspace;
 
     private final String tag;
-    private final AtomicInteger progress = new AtomicInteger();
+    private final AtomicInteger progressCounter = new AtomicInteger();
     private final int totalProgress;
 
+    private final long creationTimeMillis = System.currentTimeMillis();
 
 Review comment:
   `System.currentTimeMillis();` prevents us from Injecting and mocking the clock. Could you evaluate whether we can replace these calls with `MonotonicClock`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390481562
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -552,9 +568,66 @@ public int liveMemberCount()
         }).call();
     }
 
-    public int nodetool(String... commandAndArgs)
+    public NodeToolResult nodetoolResult(boolean withNotifications, String... commandAndArgs)
     {
-        return sync(() -> new NodeTool(new InternalNodeProbeFactory()).execute(commandAndArgs)).call();
+        return sync(() -> {
+            DTestNodeTool nodetool = new DTestNodeTool(withNotifications);
+            int rc =  nodetool.execute(commandAndArgs);
+            return new NodeToolResult(commandAndArgs, rc, new ArrayList<>(nodetool.notifications.notifications), nodetool.latestError);
+        }).call();
+    }
+
+    private static class DTestNodeTool extends NodeTool {
+        private final StorageServiceMBean storageProxy;
+        private final CollectingNotificationListener notifications = new CollectingNotificationListener();
+
+        private Throwable latestError;
+
+        DTestNodeTool(boolean withNotifications) {
+            super(new InternalNodeProbeFactory(withNotifications));
+            storageProxy = InternalNodeProbe.create(withNotifications).getStorageService();
 
 Review comment:
   connection on, since JMX shouldn't be on in dtest.  it is possible this object (DTestNodeTool) leaks since `javax.management.NotificationBroadcasterSupport#listenerList` will still hold reference...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384014506
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -552,9 +555,66 @@ public int liveMemberCount()
         }).call();
     }
 
-    public int nodetool(String... commandAndArgs)
+    public NodeToolResult nodetoolResult(boolean withNotifications, String... commandAndArgs)
     {
-        return sync(() -> new NodeTool(new InternalNodeProbeFactory()).execute(commandAndArgs)).call();
+        return sync(() -> {
+            DtestNodeTool nodetool = new DtestNodeTool(withNotifications);
+            int rc =  nodetool.execute(commandAndArgs);
+            return new NodeToolResult(commandAndArgs, rc, new ArrayList<>(nodetool.notifications.notifications), nodetool.latestError);
+        }).call();
+    }
+
+    private static class DtestNodeTool extends NodeTool {
 
 Review comment:
   What do you think about `DTestNodeTool` (different casing)?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390676346
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java
 ##########
 @@ -45,14 +47,30 @@
 import org.apache.cassandra.service.GCInspector;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.StorageServiceMBean;
 import org.apache.cassandra.streaming.StreamManager;
 import org.apache.cassandra.tools.NodeProbe;
+import org.mockito.Mockito;
 
 public class InternalNodeProbe extends NodeProbe
 {
-    public InternalNodeProbe() throws IOException
+    private final boolean withNotifications;
+
+    public InternalNodeProbe(boolean withNotifications) throws IOException
     {
         super("", 0);
+        this.withNotifications = withNotifications;
+        connect(); // need to double "connect" since constructor calls connect; withNotifications isn't visable the first time
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390475580
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/QueryResult.java
 ##########
 @@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.function.Predicate;
+
+/**
+ * A table of data representing a complete query result.
+ *
+ * A <code>QueryResult</code> is different from {@link java.sql.ResultSet} in several key ways:
+ *
+ * <ul>
+ *     <li>represents a complete result rather than a cursor</li>
+ *     <li>returns a {@link Row} to access the current row of data</li>
+ *     <li>relies on object pooling; {@link #hasNext()} may return the same object just with different data, accessing a
+ *     {@link Row} from a previous {@link #hasNext()} call has undefined behavior.</li>
+ *     <li>includes {@link #filter(Predicate)}, this will do client side filtering since Apache Cassandra is more
+ *     restrictive on server side filtering</li>
+ * </ul>
+ *
+ * <h2>Unsafe patterns</h2>
+ *
+ * Below are a few unsafe patterns which may lead to unexpected results
+ *
+ * <code>{@code
+ * while (rs.hasNext()) {
+ *   list.add(rs.next());
 
 Review comment:
   avoid allocations; thats about it.
   
   Given the fact the common access pattern doesn't actually need to hold reference to `Row` outside the local loop scope, there isn't much of a need to keep creating objects.
   
   This is a me thing though.  I prefer to avoid allocations in loops if possible since we don't know how large loops are.  

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382164930
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,53 +146,128 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
+
+        success(msg);
+    }
+
+    private void success(String msg)
     {
+        fireProgressEvent(new ProgressEvent(ProgressEventType.SUCCESS, progressCounter.get(), totalProgress, msg));
+        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.COMPLETED,
+                                                        ImmutableList.of(msg));
+        complete(null);
+    }
+
+    public void notifyError(Throwable error)
+    {
+        // exception should be ignored
+        if (error instanceof SomeRepairFailedException)
+            return;
+        logger.error("Repair {} failed:", parentSession, error);
+
         StorageMetrics.repairExceptions.inc();
-        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, message);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCount, totalProgress, errorMessage));
-        String completionMessage = String.format("Repair command #%d finished with error", cmd);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCount, totalProgress, completionMessage));
-        recordFailure(errorMessage, completionMessage);
+        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, error.getMessage());
+        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCounter.get(), totalProgress, errorMessage));
+
+        // since this can fail, update table only after updating in-memory and notification state
+        if (!options.isPreview())
+        {
+            SystemDistributedKeyspace.failParentRepair(parentSession, error);
+        }
     }
 
+    private void fail(String reason)
+    {
+        if (reason == null)
+            reason = "Some repair failed";
+        String completionMessage = String.format("Repair command #%d finished with error", cmd);
+
+        // Note we rely on the first message being the reason for the failure
+        // when inspecting this state from RepairRunner.queryForCompletedRepair
+        ActiveRepairService.instance.recordRepairStatus(cmd, ParentRepairStatus.FAILED,
+                                                        ImmutableList.of(reason, completionMessage));
+
+        complete(completionMessage);
+    }
 
-    protected void runMayThrow() throws Exception
+    private void complete(String msg)
     {
-        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.IN_PROGRESS, ImmutableList.of());
-        final TraceState traceState;
-        final UUID parentSession = UUIDGen.getTimeUUID();
-        final String tag = "repair:" + cmd;
+        long durationMillis = System.currentTimeMillis() - creationTimeMillis;
+        if (msg == null) {
+            String duration = DurationFormatUtils.formatDurationWords(durationMillis,true, true);
 
 Review comment:
   Can be `final`.
   
   Nit: Need space between `,` & `true`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390678592
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedRepairUtils.java
 ##########
 @@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.QueryResult;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.impl.AbstractCluster;
+import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.metrics.StorageMetrics;
+
+import static org.apache.cassandra.utils.Retry.retryWithBackoffBlocking;
+
+public final class DistributedRepairUtils
+{
+    public static final int DEFAULT_COORDINATOR = 1;
+
+    private DistributedRepairUtils()
+    {
+
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, RepairType repairType, boolean withNotifications, String... args) {
+        return repair(cluster, DEFAULT_COORDINATOR, repairType, withNotifications, args);
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, int node, RepairType repairType, boolean withNotifications, String... args) {
+        args = repairType.append(args);
+        args = ArrayUtils.addAll(new String[] { "repair" }, args);
+        return cluster.get(node).nodetoolResult(withNotifications, args);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster)
 
 Review comment:
   I don't remember why I did this, both yours and mine will work with Cluster and UpgradeCluster.  Is there a reason you want me to drop the `C` part?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384239181
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
 ##########
 @@ -151,9 +152,16 @@ public boolean isShutdown()
         @Override
         public synchronized void startup()
         {
+            startup(AbstractCluster.this);
+        }
+
+        public synchronized void startup(ICluster cluster)
+        {
+            if (cluster != AbstractCluster.this)
+                throw new IllegalArgumentException("Only the owning cluster can be used for startup"); //TODO why have this in the API?
             if (!isShutdown)
                 throw new IllegalStateException();
-            delegate().startup(AbstractCluster.this);
+            delegate().startup(cluster);
 
 Review comment:
   pushed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390679400
 
 

 ##########
 File path: test/unit/org/apache/cassandra/utils/Retry.java
 ##########
 @@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.IntToLongFunction;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+
+/**
+ * Class for retryable actions.
+ *
+ * @see {@link #retryWithBackoff(int, Supplier, Predicate)}
+ */
+public final class Retry
+{
+    private static final ScheduledExecutorService SCHEDULED = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("RetryScheduler"));
+
+    private Retry()
+    {
+
+    }
+
+    /**
+     * Schedule code to run after the defined duration.
+     *
+     * Since a executor was not defined, the global {@link ForkJoinPool#commonPool()} executor will be used, if this
+     * is not desirable then should use {@link #schedule(Duration, Executor, Runnable)}.
+     *
+     * @param duration how long to delay
+     * @param fn code to run
+     * @return future representing result
+     */
+    public static CompletableFuture<Void> schedule(final Duration duration, final Runnable fn)
+    {
+        return schedule(duration, ForkJoinPool.commonPool(), fn);
+    }
+
+    /**
+     * Schedule code to run after the defined duration on the provided executor.
+     *
+     * @param duration how long to delay
+     * @param executor to run on
+     * @param fn code to run
+     * @return future representing result
+     */
+    public static CompletableFuture<Void> schedule(final Duration duration, final Executor executor, final Runnable fn)
+    {
+        long nanos = duration.toNanos();
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        SCHEDULED.schedule(() -> run0(executor, future, fn), nanos, TimeUnit.NANOSECONDS);
 
 Review comment:
   Sadly no.
   
   * We can't return `Future` as its a useless interface =(
   * `java.util.concurrent.Executor#execute` can fail, so if it fails any listener for the future will hang. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390106009
 
 

 ##########
 File path: src/java/org/apache/cassandra/service/ActiveRepairService.java
 ##########
 @@ -525,6 +525,20 @@ public synchronized ParentRepairSession removeParentRepairSession(UUID parentSes
         return parentRepairSessions.remove(parentSessionId);
     }
 
+    public synchronized ParentRepairSession removeParentRepairSessionIfPresent(UUID parentSessionId)
 
 Review comment:
   this method looks very similar to `removeParentRepairSession`.. should we refactor it?
   
   alternatively, we can change `removeParentRepairSession` to gracefully handle non-existing parent session.. there is no point throw NPE if parent session is already gone..

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384006452
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
 ##########
 @@ -151,9 +152,16 @@ public boolean isShutdown()
         @Override
         public synchronized void startup()
         {
+            startup(AbstractCluster.this);
+        }
+
+        public synchronized void startup(ICluster cluster)
+        {
+            if (cluster != AbstractCluster.this)
+                throw new IllegalArgumentException("Only the owning cluster can be used for startup"); //TODO why have this in the API?
 
 Review comment:
   To my best memory, this is because it's called from `DelegatingInvokableInstance`. But we can make it package-private.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384234611
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/ICoordinator.java
 ##########
 @@ -27,7 +27,11 @@
 {
     // a bit hacky, but ConsistencyLevel draws in too many dependent classes, so we cannot have a cross-version
     // method signature that accepts ConsistencyLevel directly.  So we just accept an Enum<?> and cast.
-    Object[][] execute(String query, Enum<?> consistencyLevel, Object... boundValues);
+    default Object[][] execute(String query, Enum<?> consistencyLevel, Object... boundValues)
+    {
+        return executeWithResult(query, consistencyLevel, boundValues).toObjectArrays();
+    }
+    ResultSet executeWithResult(String query, Enum<?> consistencyLevel, Object... boundValues);
 
 Review comment:
   I hate Object[][], but I rather not do in this JIRA.  I don't mind pushing to another JIRA and doing that work after this merges, but I don't want to block this on that.
   
   > In a long run, do we even want to keep object[][]?
   
   as a user Object[][] is unusable since the column order != table definition order, so I have no way of knowing what the 3rd column is!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384031064
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/NodeToolResult.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.management.Notification;
+
+import org.junit.Assert;
+
+public class NodeToolResult
+{
+    public final String[] commandAndArgs;
+    public final int rc;
+    public final List<Notification> notifications;
+    public final Throwable error;
+
+    public NodeToolResult(String[] commandAndArgs, int rc, List<Notification> notifications, Throwable error)
+    {
+        this.commandAndArgs = commandAndArgs;
+        this.rc = rc;
+        this.notifications = notifications;
+        this.error = error;
+    }
+
+    public Asserts asserts()
+    {
+        return new Asserts();
+    }
+
+    public final class Asserts {
+        public Asserts ok() {
+            Assert.assertEquals("nodetool command " + commandAndArgs[0] + " was not successful", 0, rc);
+            return this;
+        }
+
+        public Asserts notOk() {
 
 Review comment:
   Should we call it `succeeded/failed` instead?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390680062
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+
+public class RepairCoordinatorBase extends DistributedTestBase
+{
+    protected static Cluster CLUSTER;
+
+    protected final RepairType repairType;
+    protected final RepairParallelism parallelism;
+    protected final boolean withNotifications;
 
 Review comment:
   can't, its there to make sure we work properly when notifications don't work.  This has been a problem area for nodetool so its important to be able to test that nodetool does the right thing when notifications are lost.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384214346
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+
+public class RepairCoordinatorBase extends DistributedTestBase implements Serializable
+{
+    protected static Cluster CLUSTER;
+
+    protected final RepairType repairType;
+    protected final RepairParallelism parallelism;
+    protected final boolean withNotifications;
+
+    public RepairCoordinatorBase(RepairType repairType, RepairParallelism parallelism, boolean withNotifications)
+    {
+        this.repairType = repairType;
+        this.parallelism = parallelism;
+        this.withNotifications = withNotifications;
+    }
+
+    @Parameterized.Parameters(name = "{0}/{1}")
+    public static Collection<Object[]> testsWithoutType()
+    {
+        List<Object[]> tests = new ArrayList<>();
+        for (RepairParallelism p : RepairParallelism.values())
+        {
+            tests.add(new Object[] { p, true });
+            tests.add(new Object[] { p, false });
+        }
+        return tests;
+    }
+
+    @BeforeClass
+    public static void before()
+    {
+        // This only works because the way CI works
+        // In CI a new JVM is spun up for each test file, so this doesn't have to worry about another test file
+        // getting this set first
+        System.setProperty("cassandra.nodetool.jmx_notification_poll_interval_seconds", "1");
+        DatabaseDescriptor.clientInitialization();
 
 Review comment:
   removed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382170132
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,53 +146,128 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
+
+        success(msg);
+    }
+
+    private void success(String msg)
     {
+        fireProgressEvent(new ProgressEvent(ProgressEventType.SUCCESS, progressCounter.get(), totalProgress, msg));
+        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.COMPLETED,
+                                                        ImmutableList.of(msg));
+        complete(null);
+    }
+
+    public void notifyError(Throwable error)
+    {
+        // exception should be ignored
+        if (error instanceof SomeRepairFailedException)
+            return;
+        logger.error("Repair {} failed:", parentSession, error);
+
         StorageMetrics.repairExceptions.inc();
-        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, message);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCount, totalProgress, errorMessage));
-        String completionMessage = String.format("Repair command #%d finished with error", cmd);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCount, totalProgress, completionMessage));
-        recordFailure(errorMessage, completionMessage);
+        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, error.getMessage());
+        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCounter.get(), totalProgress, errorMessage));
+
+        // since this can fail, update table only after updating in-memory and notification state
+        if (!options.isPreview())
+        {
+            SystemDistributedKeyspace.failParentRepair(parentSession, error);
+        }
     }
 
+    private void fail(String reason)
+    {
+        if (reason == null)
+            reason = "Some repair failed";
+        String completionMessage = String.format("Repair command #%d finished with error", cmd);
+
+        // Note we rely on the first message being the reason for the failure
+        // when inspecting this state from RepairRunner.queryForCompletedRepair
+        ActiveRepairService.instance.recordRepairStatus(cmd, ParentRepairStatus.FAILED,
+                                                        ImmutableList.of(reason, completionMessage));
+
+        complete(completionMessage);
+    }
 
-    protected void runMayThrow() throws Exception
+    private void complete(String msg)
     {
-        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.IN_PROGRESS, ImmutableList.of());
-        final TraceState traceState;
-        final UUID parentSession = UUIDGen.getTimeUUID();
-        final String tag = "repair:" + cmd;
+        long durationMillis = System.currentTimeMillis() - creationTimeMillis;
+        if (msg == null) {
+            String duration = DurationFormatUtils.formatDurationWords(durationMillis,true, true);
+            msg = String.format("Repair command #%d finished in %s", cmd, duration);
+        }
 
-        final AtomicInteger progress = new AtomicInteger();
-        final int totalProgress = 4 + options.getRanges().size(); // get valid column families, calculate neighbors, validation, prepare for repair + number of ranges to repair
+        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCounter.get(), totalProgress, msg));
 
-        String[] columnFamilies = options.getColumnFamilies().toArray(new String[options.getColumnFamilies().size()]);
-        Iterable<ColumnFamilyStore> validColumnFamilies;
-        try
+        if (ActiveRepairService.instance.hasParentRepairSession(parentSession))
+            ActiveRepairService.instance.removeParentRepairSession(parentSession);
+        Context ctx = context;
+        if (options.isTraced() && ctx != null && ctx.traceState != null)
         {
-            validColumnFamilies = storageService.getValidColumnFamilies(false, false, keyspace, columnFamilies);
-            progress.incrementAndGet();
+            for (ProgressListener listener : listeners)
+                ctx.traceState.removeProgressListener(listener);
+            // Because DebuggableThreadPoolExecutor#afterExecute and this callback
+            // run in a nondeterministic order (within the same thread), the
+            // TraceState may have been nulled out at this point. The TraceState
+            // should be traceState, so just set it without bothering to check if it
+            // actually was nulled out.
+            Tracing.instance.set(ctx.traceState);
+            Tracing.traceRepair(msg);
+            Tracing.instance.stopSession();
+        }
+
+        Keyspace.open(keyspace).metric.repairTime.update(durationMillis, TimeUnit.MILLISECONDS);
+    }
+
+    public void run()
+    {
+        try {
+            // Why is this before setup/start when its publishing the start event?  For backwards compatability
+            // One of the first things we did before was publish this before validating, so publish early to keep
+            // that the same.
+            String message = String.format("Starting repair command #%d (%s), repairing keyspace %s with %s", cmd, parentSession, keyspace,
+                                           options);
+            logger.info(message);
+            Tracing.traceRepair(message);
+            fireProgressEvent(new ProgressEvent(ProgressEventType.START, 0, 100, message));
+
+            Pair<Context, String> setup = setup();
+            if (setup.right != null)
+            {
+                skip(setup.right);
+                return;
+            }
+            Context ctx = setup.left;
+            assert ctx != null : "Context is required but was not found";
+            this.context = ctx;
+            start(ctx);
         }
-        catch (IllegalArgumentException | IOException e)
+        catch (Exception | Error e)
         {
-            logger.error("Repair {} failed:", parentSession, e);
-            fireErrorAndComplete(progress.get(), totalProgress, e.getMessage());
-            return;
+            notifyError(e);
+            fail(e.getMessage());
         }
+    }
+
+    private Pair<Context, String> setup() throws Exception
+    {
+        ActiveRepairService.instance.recordRepairStatus(cmd, ParentRepairStatus.IN_PROGRESS, ImmutableList.of());
+
+        String[] columnFamilies = options.getColumnFamilies().toArray(new String[options.getColumnFamilies().size()]);
+        Iterable<ColumnFamilyStore> validColumnFamilies = storageService.getValidColumnFamilies(false, false, keyspace, columnFamilies);
 
 Review comment:
   Can be `final`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382164141
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,53 +146,128 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
+
+        success(msg);
+    }
+
+    private void success(String msg)
     {
+        fireProgressEvent(new ProgressEvent(ProgressEventType.SUCCESS, progressCounter.get(), totalProgress, msg));
+        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.COMPLETED,
+                                                        ImmutableList.of(msg));
+        complete(null);
+    }
+
+    public void notifyError(Throwable error)
+    {
+        // exception should be ignored
+        if (error instanceof SomeRepairFailedException)
+            return;
+        logger.error("Repair {} failed:", parentSession, error);
+
         StorageMetrics.repairExceptions.inc();
-        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, message);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCount, totalProgress, errorMessage));
-        String completionMessage = String.format("Repair command #%d finished with error", cmd);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCount, totalProgress, completionMessage));
-        recordFailure(errorMessage, completionMessage);
+        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, error.getMessage());
+        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCounter.get(), totalProgress, errorMessage));
+
+        // since this can fail, update table only after updating in-memory and notification state
+        if (!options.isPreview())
+        {
+            SystemDistributedKeyspace.failParentRepair(parentSession, error);
+        }
     }
 
+    private void fail(String reason)
+    {
+        if (reason == null)
+            reason = "Some repair failed";
+        String completionMessage = String.format("Repair command #%d finished with error", cmd);
 
 Review comment:
   Can be `final`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384211610
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+
+public class RepairCoordinatorBase extends DistributedTestBase implements Serializable
 
 Review comment:
   oh, I always do that since dtest keeps getting mad at me... this was copy/paste <_<

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382166208
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,53 +146,128 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
+
+        success(msg);
+    }
+
+    private void success(String msg)
     {
+        fireProgressEvent(new ProgressEvent(ProgressEventType.SUCCESS, progressCounter.get(), totalProgress, msg));
+        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.COMPLETED,
+                                                        ImmutableList.of(msg));
+        complete(null);
+    }
+
+    public void notifyError(Throwable error)
+    {
+        // exception should be ignored
+        if (error instanceof SomeRepairFailedException)
+            return;
+        logger.error("Repair {} failed:", parentSession, error);
+
         StorageMetrics.repairExceptions.inc();
-        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, message);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCount, totalProgress, errorMessage));
-        String completionMessage = String.format("Repair command #%d finished with error", cmd);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCount, totalProgress, completionMessage));
-        recordFailure(errorMessage, completionMessage);
+        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, error.getMessage());
+        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCounter.get(), totalProgress, errorMessage));
+
+        // since this can fail, update table only after updating in-memory and notification state
+        if (!options.isPreview())
+        {
+            SystemDistributedKeyspace.failParentRepair(parentSession, error);
+        }
     }
 
+    private void fail(String reason)
+    {
+        if (reason == null)
+            reason = "Some repair failed";
+        String completionMessage = String.format("Repair command #%d finished with error", cmd);
+
+        // Note we rely on the first message being the reason for the failure
+        // when inspecting this state from RepairRunner.queryForCompletedRepair
+        ActiveRepairService.instance.recordRepairStatus(cmd, ParentRepairStatus.FAILED,
+                                                        ImmutableList.of(reason, completionMessage));
+
+        complete(completionMessage);
+    }
 
-    protected void runMayThrow() throws Exception
+    private void complete(String msg)
     {
-        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.IN_PROGRESS, ImmutableList.of());
-        final TraceState traceState;
-        final UUID parentSession = UUIDGen.getTimeUUID();
-        final String tag = "repair:" + cmd;
+        long durationMillis = System.currentTimeMillis() - creationTimeMillis;
+        if (msg == null) {
+            String duration = DurationFormatUtils.formatDurationWords(durationMillis,true, true);
+            msg = String.format("Repair command #%d finished in %s", cmd, duration);
+        }
 
-        final AtomicInteger progress = new AtomicInteger();
-        final int totalProgress = 4 + options.getRanges().size(); // get valid column families, calculate neighbors, validation, prepare for repair + number of ranges to repair
+        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCounter.get(), totalProgress, msg));
 
-        String[] columnFamilies = options.getColumnFamilies().toArray(new String[options.getColumnFamilies().size()]);
-        Iterable<ColumnFamilyStore> validColumnFamilies;
-        try
+        if (ActiveRepairService.instance.hasParentRepairSession(parentSession))
+            ActiveRepairService.instance.removeParentRepairSession(parentSession);
+        Context ctx = context;
+        if (options.isTraced() && ctx != null && ctx.traceState != null)
         {
-            validColumnFamilies = storageService.getValidColumnFamilies(false, false, keyspace, columnFamilies);
-            progress.incrementAndGet();
+            for (ProgressListener listener : listeners)
+                ctx.traceState.removeProgressListener(listener);
+            // Because DebuggableThreadPoolExecutor#afterExecute and this callback
+            // run in a nondeterministic order (within the same thread), the
+            // TraceState may have been nulled out at this point. The TraceState
+            // should be traceState, so just set it without bothering to check if it
+            // actually was nulled out.
+            Tracing.instance.set(ctx.traceState);
+            Tracing.traceRepair(msg);
+            Tracing.instance.stopSession();
+        }
+
+        Keyspace.open(keyspace).metric.repairTime.update(durationMillis, TimeUnit.MILLISECONDS);
+    }
+
+    public void run()
+    {
+        try {
+            // Why is this before setup/start when its publishing the start event?  For backwards compatability
+            // One of the first things we did before was publish this before validating, so publish early to keep
+            // that the same.
+            String message = String.format("Starting repair command #%d (%s), repairing keyspace %s with %s", cmd, parentSession, keyspace,
 
 Review comment:
   Can be `final`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384007712
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/NodeToolResult.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.management.Notification;
+
+import org.junit.Assert;
+
+public class NodeToolResult
+{
+    public final String[] commandAndArgs;
 
 Review comment:
   Should all these fields be public?..

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390674270
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -552,9 +568,66 @@ public int liveMemberCount()
         }).call();
     }
 
-    public int nodetool(String... commandAndArgs)
+    public NodeToolResult nodetoolResult(boolean withNotifications, String... commandAndArgs)
     {
-        return sync(() -> new NodeTool(new InternalNodeProbeFactory()).execute(commandAndArgs)).call();
+        return sync(() -> {
+            DTestNodeTool nodetool = new DTestNodeTool(withNotifications);
+            int rc =  nodetool.execute(commandAndArgs);
+            return new NodeToolResult(commandAndArgs, rc, new ArrayList<>(nodetool.notifications.notifications), nodetool.latestError);
+        }).call();
+    }
+
+    private static class DTestNodeTool extends NodeTool {
+        private final StorageServiceMBean storageProxy;
+        private final CollectingNotificationListener notifications = new CollectingNotificationListener();
+
+        private Throwable latestError;
+
+        DTestNodeTool(boolean withNotifications) {
+            super(new InternalNodeProbeFactory(withNotifications));
+            storageProxy = InternalNodeProbe.create(withNotifications).getStorageService();
 
 Review comment:
   added close method to remove listener.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r386047795
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/ResultSet.java
 ##########
 @@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Date;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+public class ResultSet implements Iterator<Row>
+{
+    public static final ResultSet EMPTY = new ResultSet(new String[0], null);
+
+    private final String[] names;
+    private final Object[][] results;
+    private final Predicate<Row> filter;
+    private final Row row;
+    private int offset = -1;
+
+    public ResultSet(String[] names, Object[][] results)
+    {
+        this.names = names;
+        this.results = results;
+        this.row = new Row(names);
+        this.filter = ignore -> true;
+    }
+
+    private ResultSet(String[] names, Object[][] results, Predicate<Row> filter, int offset)
+    {
+        this.names = names;
+        this.results = results;
+        this.filter = filter;
+        this.offset = offset;
+        this.row = new Row(names);
+    }
+
+    public String[] getNames()
+    {
+        return names;
+    }
+
+    public boolean isEmpty()
+    {
+        return results.length == 0;
+    }
+
+    public int size()
+    {
+        return results.length;
+    }
+
+    public ResultSet filter(Predicate<Row> fn)
+    {
+        return new ResultSet(names, results, filter.and(fn), offset);
+    }
+
+    public Object[][] toObjectArrays()
+    {
+        return results;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+        if (results == null)
+            return false;
+        while ((offset += 1) < results.length)
+        {
+            row.setResults(results[offset]);
+            if (filter.test(row))
+            {
+                return true;
+            }
+        }
+        row.setResults(null);
+        return false;
+    }
+
+    @Override
+    public Row next()
+    {
+        if (offset < 0 || offset >= results.length)
+            throw new NoSuchElementException();
+        return row;
 
 Review comment:
   I documented both classes about this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390104182
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -578,86 +648,26 @@ public RepairCompleteCallback(UUID parentSession,
 
         public void onSuccess(Object result)
         {
-            if (!options.isPreview())
-            {
-                SystemDistributedKeyspace.successfulParentRepair(parentSession, successfulRanges);
-            }
-            final String message;
+            maybeStoreParentRepairSuccess(successfulRanges);
             if (hasFailure.get())
             {
-                StorageMetrics.repairExceptions.inc();
-                message = "Some repair failed";
-                fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progress.get(), totalProgress,
-                                                    message));
+                fail(null);
 
 Review comment:
   When there is failure, should we fire `ERROR` event instead of `COMPLETE` event inside `fail() -> complete()`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384983565
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/ResultSet.java
 ##########
 @@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Date;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+public class ResultSet implements Iterator<Row>
+{
+    public static final ResultSet EMPTY = new ResultSet(new String[0], null);
+
+    private final String[] names;
+    private final Object[][] results;
+    private final Predicate<Row> filter;
+    private final Row row;
+    private int offset = -1;
+
+    public ResultSet(String[] names, Object[][] results)
+    {
+        this.names = names;
+        this.results = results;
+        this.row = new Row(names);
+        this.filter = ignore -> true;
+    }
+
+    private ResultSet(String[] names, Object[][] results, Predicate<Row> filter, int offset)
+    {
+        this.names = names;
+        this.results = results;
+        this.filter = filter;
+        this.offset = offset;
+        this.row = new Row(names);
+    }
+
+    public String[] getNames()
+    {
+        return names;
+    }
+
+    public boolean isEmpty()
+    {
+        return results.length == 0;
+    }
+
+    public int size()
+    {
+        return results.length;
+    }
+
+    public ResultSet filter(Predicate<Row> fn)
+    {
+        return new ResultSet(names, results, filter.and(fn), offset);
+    }
+
+    public Object[][] toObjectArrays()
+    {
+        return results;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+        if (results == null)
+            return false;
+        while ((offset += 1) < results.length)
+        {
+            row.setResults(results[offset]);
+            if (filter.test(row))
+            {
+                return true;
+            }
+        }
+        row.setResults(null);
+        return false;
+    }
+
+    @Override
+    public Row next()
+    {
+        if (offset < 0 || offset >= results.length)
+            throw new NoSuchElementException();
+        return row;
 
 Review comment:
   All right; let's also add a comment about that.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384226935
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorFast.java
 ##########
 @@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Set;
+
+import com.google.common.collect.Iterables;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.distributed.api.IMessageFilters;
+import org.apache.cassandra.distributed.api.LongTokenRange;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.NodeToolResult.ProgressEventType;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+import org.apache.cassandra.net.Verb;
+import org.apache.cassandra.service.StorageService;
+
+import static java.lang.String.format;
+import static org.apache.cassandra.distributed.api.IMessageFilters.Matcher.of;
+import static org.apache.cassandra.distributed.test.DistributedRepairUtils.assertParentRepairFailedWithMessageContains;
+import static org.apache.cassandra.distributed.test.DistributedRepairUtils.assertParentRepairNotExist;
+import static org.apache.cassandra.distributed.test.DistributedRepairUtils.assertParentRepairSuccess;
+import static org.apache.cassandra.distributed.test.DistributedRepairUtils.getRepairExceptions;
+
+public abstract class RepairCoordinatorFast extends RepairCoordinatorBase
+{
+    public RepairCoordinatorFast(RepairType repairType, RepairParallelism parallelism, boolean withNotifications)
+    {
+        super(repairType, parallelism, withNotifications);
+    }
+
+    @Test(timeout = 1 * 60 * 1000)
+    public void simple() {
+        String table = tableName("simple");
+        CLUSTER.schemaChange(format("CREATE TABLE %s.%s (key text, PRIMARY KEY (key))", KEYSPACE, table));
+        CLUSTER.coordinator(1).execute(format("INSERT INTO %s.%s (key) VALUES (?)", KEYSPACE, table), ConsistencyLevel.ANY, "some text");
+
+        long repairExceptions = getRepairExceptions(CLUSTER, 2);
+        NodeToolResult result = repair(2, KEYSPACE, table);
+        result.asserts().ok();
+        if (withNotifications)
+        {
+            result.asserts()
+                  .notificationContains(ProgressEventType.START, "Starting repair command")
+                  .notificationContains(ProgressEventType.START, "repairing keyspace " + KEYSPACE + " with repair options")
+                  .notificationContains(ProgressEventType.SUCCESS, repairType != RepairType.PREVIEW ? "Repair completed successfully": "Repair preview completed successfully")
+                  .notificationContains(ProgressEventType.COMPLETE, "finished");
+        }
+
+        if (repairType != RepairType.PREVIEW)
+        {
+            assertParentRepairSuccess(CLUSTER, KEYSPACE, table);
+        }
+        else
+        {
+            assertParentRepairNotExist(CLUSTER, KEYSPACE, table);
+        }
+
+        Assert.assertEquals(repairExceptions, getRepairExceptions(CLUSTER, 2));
+    }
+
+    @Test(timeout = 1 * 60 * 1000)
+    public void missingKeyspace()
+    {
+        // as of this moment the check is done in nodetool so the JMX notifications are not imporant
+        // nor is the history stored
+        long repairExceptions = getRepairExceptions(CLUSTER, 2);
+        NodeToolResult result = repair(2, "doesnotexist");
+        result.asserts()
+              .notOk()
+              .errorContains("Keyspace [doesnotexist] does not exist.");
+
+        Assert.assertEquals(repairExceptions, getRepairExceptions(CLUSTER, 2));
+
+        assertParentRepairNotExist(CLUSTER, "doesnotexist");
+    }
+
+    @Test(timeout = 1 * 60 * 1000)
+    public void missingTable()
+    {
+        long repairExceptions = getRepairExceptions(CLUSTER, 2);
+        NodeToolResult result = repair(2, KEYSPACE, "doesnotexist");
+        result.asserts()
+              .notOk();
+        if (withNotifications)
+        {
+            result.asserts()
+                  .errorContains("failed with error Unknown keyspace/cf pair (distributed_test_keyspace.doesnotexist)")
+                  // Start notification is ignored since this is checked during setup (aka before start)
+                  .notificationContains(ProgressEventType.ERROR, "failed with error Unknown keyspace/cf pair (distributed_test_keyspace.doesnotexist)")
+                  .notificationContains(ProgressEventType.COMPLETE, "finished with error");
+        }
+
+        assertParentRepairNotExist(CLUSTER, KEYSPACE, "doesnotexist");
+
+        Assert.assertEquals(repairExceptions + 1, getRepairExceptions(CLUSTER, 2));
+    }
+
+    @Test(timeout = 1 * 60 * 1000)
+    public void noTablesToRepair()
+    {
+        // index CF currently don't support repair, so they get dropped when listed
+        // this is done in this test to cause the keyspace to have 0 tables to repair, which causes repair to no-op
+        // early and skip.
+        String table = tableName("withindex");
+        CLUSTER.schemaChange(format("CREATE TABLE %s.%s (key text, value text, PRIMARY KEY (key))", KEYSPACE, table));
+        CLUSTER.schemaChange(format("CREATE INDEX value_%s ON %s.%s (value)", postfix(), KEYSPACE, table));
+
+        long repairExceptions = getRepairExceptions(CLUSTER, 2);
+        // if CF has a . in it, it is assumed to be a 2i which rejects repairs
+        NodeToolResult result = repair(2, KEYSPACE, table + ".value");
+        result.asserts().ok();
+        if (withNotifications)
+        {
+            result.asserts()
+                  .notificationContains("Empty keyspace")
+                  .notificationContains("skipping repair: " + KEYSPACE)
+                  // Start notification is ignored since this is checked during setup (aka before start)
+                  .notificationContains(ProgressEventType.SUCCESS, "Empty keyspace") // will fail since success isn't returned; only complete
+                  .notificationContains(ProgressEventType.COMPLETE, "finished"); // will fail since it doesn't do this
+        }
+
+        assertParentRepairNotExist(CLUSTER, KEYSPACE, table + ".value");
+
+        // this is actually a SKIP and not a FAILURE, so shouldn't increment
+        Assert.assertEquals(repairExceptions, getRepairExceptions(CLUSTER, 2));
+    }
+
+    @Test(timeout = 1 * 60 * 1000)
+    public void intersectingRange()
+    {
+        // this test exists to show that this case will cause repair to finish; success or failure isn't imporant
+        // if repair is enhanced to allow intersecting ranges w/ local then this test will fail saying that we expected
+        // repair to fail but it didn't, this would be fine and this test should be updated to reflect the new
+        // semantic
+        String table = tableName("intersectingrange");
+        CLUSTER.schemaChange(format("CREATE TABLE %s.%s (key text, value text, PRIMARY KEY (key))", KEYSPACE, table));
+
+        //TODO dtest api for this?
 
 Review comment:
   nah.  This is a total hack. 
   
   This is saying "for a instance, should be able to ask what the tokens are", but I have to get into the cluster and serialize out to get access (which is why that class is under .api... if under .tests class path doesn't match)
   
   TL;DR
   
   ```
   LongTokenRange tokenRange = CLUSTER.get(2).callOnInstance(() -> {
               Set<Range<Token>> ranges = StorageService.instance.getLocalReplicas(KEYSPACE).ranges();
               Range<Token> range = Iterables.getFirst(ranges, null);
               long left = (long) range.left.getTokenValue();
               long right = (long) range.right.getTokenValue();
               return new LongTokenRange(left, right);
           });
   ```
   
   should be
   
   
   ```
   LongTokenRange tokenRange = failIfVnodes(CLUSTER.get(2).getTokens())
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390160965
 
 

 ##########
 File path: test/unit/org/apache/cassandra/utils/Retry.java
 ##########
 @@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.IntToLongFunction;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+
+/**
+ * Class for retryable actions.
+ *
+ * @see {@link #retryWithBackoff(int, Supplier, Predicate)}
+ */
+public final class Retry
+{
+    private static final ScheduledExecutorService SCHEDULED = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("RetryScheduler"));
+
+    private Retry()
+    {
+
+    }
+
+    /**
+     * Schedule code to run after the defined duration.
+     *
+     * Since a executor was not defined, the global {@link ForkJoinPool#commonPool()} executor will be used, if this
+     * is not desirable then should use {@link #schedule(Duration, Executor, Runnable)}.
+     *
+     * @param duration how long to delay
+     * @param fn code to run
+     * @return future representing result
+     */
+    public static CompletableFuture<Void> schedule(final Duration duration, final Runnable fn)
+    {
+        return schedule(duration, ForkJoinPool.commonPool(), fn);
+    }
+
+    /**
+     * Schedule code to run after the defined duration on the provided executor.
+     *
+     * @param duration how long to delay
+     * @param executor to run on
+     * @param fn code to run
+     * @return future representing result
+     */
+    public static CompletableFuture<Void> schedule(final Duration duration, final Executor executor, final Runnable fn)
+    {
+        long nanos = duration.toNanos();
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        SCHEDULED.schedule(() -> run0(executor, future, fn), nanos, TimeUnit.NANOSECONDS);
+        return future;
+    }
+
+    private static void run0(final Executor executor, final CompletableFuture<Void> future, final Runnable fn)
+    {
+        try
+        {
+            executor.execute(() -> {
+                try
+                {
+                    fn.run();
+                    future.complete(null);
+                }
+                catch (Exception e)
+                {
+                    future.completeExceptionally(e);
+                }
+            });
+        }
+        catch (Exception e)
+        {
+            future.completeExceptionally(e);
+        }
+    }
+
+    /**
+     * Continously attempting to call the provided future supplier until successful or until no longer able to retry.
+     *
+     * @param maxRetries to allow
+     * @param fn asyncronous operation to retry
+     * @param retryableException used to say if retry is allowed
+     * @return future representing the result.  If retries were not able to get a successful result, the exception is the last exception seen.
+     */
+    public static <A> CompletableFuture<A> retryWithBackoff(final int maxRetries,
+                                                            final Supplier<CompletableFuture<A>> fn,
+                                                            final Predicate<Throwable> retryableException)
+    {
+        CompletableFuture<A> future = new CompletableFuture<>();
+        retryWithBackoff0(future, 0, maxRetries, fn, retryableException, retryCount -> computeSleepTimeMillis(retryCount, 50, 1000));
+        return future;
+    }
+
+    /**
+     * This is the same as {@link #retryWithBackoff(int, Supplier, Predicate)}, but takes a blocking retryable action
+     * and blocks the caller until done.
+     */
+    public static <A> A retryWithBackoffBlocking(final int maxRetries, final Supplier<A> fn)
+    {
+        return retryWithBackoffBlocking(maxRetries, fn, (ignore) -> true);
+    }
+
+    /**
+     * This is the same as {@link #retryWithBackoff(int, Supplier, Predicate)}, but takes a blocking retryable action
+     * and blocks the caller until done.
+     */
+    public static <A> A retryWithBackoffBlocking(final int maxRetries,
+                                                 final Supplier<A> fn,
+                                                 final Predicate<Throwable> retryableException)
+    {
+        return retryWithBackoff(maxRetries, () -> CompletableFuture.completedFuture(fn.get()), retryableException).join();
+    }
+
+    private static <A> void retryWithBackoff0(final CompletableFuture<A> result,
+                                              final int retryCount,
+                                              final int maxRetry,
+                                              final Supplier<CompletableFuture<A>> body,
 
 Review comment:
   can we pass `Supplier<A> fn` directly? it might simplify a bit..

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382165519
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,53 +146,128 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
+
+        success(msg);
+    }
+
+    private void success(String msg)
     {
+        fireProgressEvent(new ProgressEvent(ProgressEventType.SUCCESS, progressCounter.get(), totalProgress, msg));
+        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.COMPLETED,
+                                                        ImmutableList.of(msg));
+        complete(null);
+    }
+
+    public void notifyError(Throwable error)
+    {
+        // exception should be ignored
+        if (error instanceof SomeRepairFailedException)
+            return;
+        logger.error("Repair {} failed:", parentSession, error);
+
         StorageMetrics.repairExceptions.inc();
-        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, message);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCount, totalProgress, errorMessage));
-        String completionMessage = String.format("Repair command #%d finished with error", cmd);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCount, totalProgress, completionMessage));
-        recordFailure(errorMessage, completionMessage);
+        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, error.getMessage());
+        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCounter.get(), totalProgress, errorMessage));
+
+        // since this can fail, update table only after updating in-memory and notification state
+        if (!options.isPreview())
+        {
+            SystemDistributedKeyspace.failParentRepair(parentSession, error);
+        }
     }
 
+    private void fail(String reason)
+    {
+        if (reason == null)
+            reason = "Some repair failed";
+        String completionMessage = String.format("Repair command #%d finished with error", cmd);
+
+        // Note we rely on the first message being the reason for the failure
+        // when inspecting this state from RepairRunner.queryForCompletedRepair
+        ActiveRepairService.instance.recordRepairStatus(cmd, ParentRepairStatus.FAILED,
+                                                        ImmutableList.of(reason, completionMessage));
+
+        complete(completionMessage);
+    }
 
-    protected void runMayThrow() throws Exception
+    private void complete(String msg)
     {
-        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.IN_PROGRESS, ImmutableList.of());
-        final TraceState traceState;
-        final UUID parentSession = UUIDGen.getTimeUUID();
-        final String tag = "repair:" + cmd;
+        long durationMillis = System.currentTimeMillis() - creationTimeMillis;
+        if (msg == null) {
+            String duration = DurationFormatUtils.formatDurationWords(durationMillis,true, true);
+            msg = String.format("Repair command #%d finished in %s", cmd, duration);
+        }
 
-        final AtomicInteger progress = new AtomicInteger();
-        final int totalProgress = 4 + options.getRanges().size(); // get valid column families, calculate neighbors, validation, prepare for repair + number of ranges to repair
+        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCounter.get(), totalProgress, msg));
 
-        String[] columnFamilies = options.getColumnFamilies().toArray(new String[options.getColumnFamilies().size()]);
-        Iterable<ColumnFamilyStore> validColumnFamilies;
-        try
+        if (ActiveRepairService.instance.hasParentRepairSession(parentSession))
+            ActiveRepairService.instance.removeParentRepairSession(parentSession);
+        Context ctx = context;
 
 Review comment:
   Can be `final`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r383989437
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/LongTokenRange.java
 ##########
 @@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.io.Serializable;
+
+public final class LongTokenRange implements Serializable
 
 Review comment:
   I think we might want to limit partitioner to `Murmur3Partitioner` and allow using long tokens only. We already have API for bootstrap that works with long tokens, and having this in addition sort of hints at a need to do something like that.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390131976
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java
 ##########
 @@ -61,7 +79,26 @@ protected void connect()
         mbeanServerConn = null;
         jmxc = null;
 
-        ssProxy = StorageService.instance;
+        if (withNotifications)
+        {
+            ssProxy = StorageService.instance;
+        }
+        else
+        {
+            // replace the notification apis with a no-op method
+            StorageServiceMBean mock = Mockito.spy(StorageService.instance);
+            Mockito.doNothing().when(mock).addNotificationListener(Mockito.any(), Mockito.any(), Mockito.any());
+            try
+            {
+                Mockito.doNothing().when(mock).removeNotificationListener(Mockito.any(), Mockito.any(), Mockito.any());
+                Mockito.doNothing().when(mock).removeNotificationListener(Mockito.any());
+            }
+            catch (ListenerNotFoundException e)
+            {
+                throw new AssertionError(e);
+            }
+            ssProxy = mock;
 
 Review comment:
   I don't understand what are the cases that a no-op storage-service is needed.. what happened if we just use `StorageService.instance`..

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390129323
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java
 ##########
 @@ -45,14 +47,30 @@
 import org.apache.cassandra.service.GCInspector;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.StorageServiceMBean;
 import org.apache.cassandra.streaming.StreamManager;
 import org.apache.cassandra.tools.NodeProbe;
+import org.mockito.Mockito;
 
 public class InternalNodeProbe extends NodeProbe
 {
-    public InternalNodeProbe() throws IOException
+    private final boolean withNotifications;
+
+    public InternalNodeProbe(boolean withNotifications) throws IOException
     {
         super("", 0);
+        this.withNotifications = withNotifications;
+        connect(); // need to double "connect" since constructor calls connect; withNotifications isn't visable the first time
 
 Review comment:
   how about adding a new `NodeProble` constructor with param `boolean connect`, so we don't have to connect twice... 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384632037
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/ResultSet.java
 ##########
 @@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Date;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+public class ResultSet implements Iterator<Row>
 
 Review comment:
   switched to `QueryResult`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390673481
 
 

 ##########
 File path: src/java/org/apache/cassandra/service/ActiveRepairService.java
 ##########
 @@ -525,6 +525,20 @@ public synchronized ParentRepairSession removeParentRepairSession(UUID parentSes
         return parentRepairSessions.remove(parentSessionId);
     }
 
+    public synchronized ParentRepairSession removeParentRepairSessionIfPresent(UUID parentSessionId)
 
 Review comment:
   got rid of this method and made `removeParentRepairSession` handle the case the map doesn't know of the session.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384213652
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+
+public class RepairCoordinatorBase extends DistributedTestBase implements Serializable
+{
+    protected static Cluster CLUSTER;
+
+    protected final RepairType repairType;
+    protected final RepairParallelism parallelism;
+    protected final boolean withNotifications;
+
+    public RepairCoordinatorBase(RepairType repairType, RepairParallelism parallelism, boolean withNotifications)
+    {
+        this.repairType = repairType;
+        this.parallelism = parallelism;
+        this.withNotifications = withNotifications;
+    }
+
+    @Parameterized.Parameters(name = "{0}/{1}")
+    public static Collection<Object[]> testsWithoutType()
+    {
+        List<Object[]> tests = new ArrayList<>();
+        for (RepairParallelism p : RepairParallelism.values())
+        {
+            tests.add(new Object[] { p, true });
+            tests.add(new Object[] { p, false });
+        }
+        return tests;
+    }
+
+    @BeforeClass
+    public static void before()
+    {
+        // This only works because the way CI works
+        // In CI a new JVM is spun up for each test file, so this doesn't have to worry about another test file
+        // getting this set first
+        System.setProperty("cassandra.nodetool.jmx_notification_poll_interval_seconds", "1");
+        DatabaseDescriptor.clientInitialization();
 
 Review comment:
   I don't remember, it could be copy/paste from nodetool!
   
   I am running the tests without it to see if they fail.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r390770793
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedRepairUtils.java
 ##########
 @@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.QueryResult;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.impl.AbstractCluster;
+import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.metrics.StorageMetrics;
+
+import static org.apache.cassandra.utils.Retry.retryWithBackoffBlocking;
+
+public final class DistributedRepairUtils
+{
+    public static final int DEFAULT_COORDINATOR = 1;
+
+    private DistributedRepairUtils()
+    {
+
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, RepairType repairType, boolean withNotifications, String... args) {
+        return repair(cluster, DEFAULT_COORDINATOR, repairType, withNotifications, args);
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, int node, RepairType repairType, boolean withNotifications, String... args) {
+        args = repairType.append(args);
+        args = ArrayUtils.addAll(new String[] { "repair" }, args);
+        return cluster.get(node).nodetoolResult(withNotifications, args);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster)
 
 Review comment:
   just nits..no strong opinon.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r383968758
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairParallelism;
+import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType;
+
+public class RepairCoordinatorBase extends DistributedTestBase implements Serializable
+{
+    protected static Cluster CLUSTER;
+
+    protected final RepairType repairType;
+    protected final RepairParallelism parallelism;
+    protected final boolean withNotifications;
+
+    public RepairCoordinatorBase(RepairType repairType, RepairParallelism parallelism, boolean withNotifications)
+    {
+        this.repairType = repairType;
+        this.parallelism = parallelism;
+        this.withNotifications = withNotifications;
+    }
+
+    @Parameterized.Parameters(name = "{0}/{1}")
+    public static Collection<Object[]> testsWithoutType()
+    {
+        List<Object[]> tests = new ArrayList<>();
+        for (RepairParallelism p : RepairParallelism.values())
+        {
+            tests.add(new Object[] { p, true });
+            tests.add(new Object[] { p, false });
+        }
+        return tests;
+    }
+
+    @BeforeClass
+    public static void before()
+    {
+        // This only works because the way CI works
+        // In CI a new JVM is spun up for each test file, so this doesn't have to worry about another test file
+        // getting this set first
+        System.setProperty("cassandra.nodetool.jmx_notification_poll_interval_seconds", "1");
+        DatabaseDescriptor.clientInitialization();
 
 Review comment:
   I'm curious why we need to initialize client here. Usually we need to do this in Cassandra instance itself, but not on the test class. Do we need it for JMX to work?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384236061
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
 ##########
 @@ -151,9 +152,16 @@ public boolean isShutdown()
         @Override
         public synchronized void startup()
         {
+            startup(AbstractCluster.this);
+        }
+
+        public synchronized void startup(ICluster cluster)
+        {
+            if (cluster != AbstractCluster.this)
+                throw new IllegalArgumentException("Only the owning cluster can be used for startup"); //TODO why have this in the API?
 
 Review comment:
   I was bitten by this when doing restart since I kept passing in the cluster; I don't mind removing the API as its unsafe anyways.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382169286
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,53 +146,128 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
+
+        success(msg);
+    }
+
+    private void success(String msg)
     {
+        fireProgressEvent(new ProgressEvent(ProgressEventType.SUCCESS, progressCounter.get(), totalProgress, msg));
+        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.COMPLETED,
+                                                        ImmutableList.of(msg));
+        complete(null);
+    }
+
+    public void notifyError(Throwable error)
+    {
+        // exception should be ignored
+        if (error instanceof SomeRepairFailedException)
+            return;
+        logger.error("Repair {} failed:", parentSession, error);
+
         StorageMetrics.repairExceptions.inc();
-        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, message);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCount, totalProgress, errorMessage));
-        String completionMessage = String.format("Repair command #%d finished with error", cmd);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCount, totalProgress, completionMessage));
-        recordFailure(errorMessage, completionMessage);
+        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, error.getMessage());
+        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCounter.get(), totalProgress, errorMessage));
+
+        // since this can fail, update table only after updating in-memory and notification state
+        if (!options.isPreview())
+        {
+            SystemDistributedKeyspace.failParentRepair(parentSession, error);
+        }
     }
 
+    private void fail(String reason)
+    {
+        if (reason == null)
+            reason = "Some repair failed";
+        String completionMessage = String.format("Repair command #%d finished with error", cmd);
+
+        // Note we rely on the first message being the reason for the failure
+        // when inspecting this state from RepairRunner.queryForCompletedRepair
+        ActiveRepairService.instance.recordRepairStatus(cmd, ParentRepairStatus.FAILED,
+                                                        ImmutableList.of(reason, completionMessage));
+
+        complete(completionMessage);
+    }
 
-    protected void runMayThrow() throws Exception
+    private void complete(String msg)
     {
-        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.IN_PROGRESS, ImmutableList.of());
-        final TraceState traceState;
-        final UUID parentSession = UUIDGen.getTimeUUID();
-        final String tag = "repair:" + cmd;
+        long durationMillis = System.currentTimeMillis() - creationTimeMillis;
+        if (msg == null) {
+            String duration = DurationFormatUtils.formatDurationWords(durationMillis,true, true);
+            msg = String.format("Repair command #%d finished in %s", cmd, duration);
+        }
 
-        final AtomicInteger progress = new AtomicInteger();
-        final int totalProgress = 4 + options.getRanges().size(); // get valid column families, calculate neighbors, validation, prepare for repair + number of ranges to repair
+        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCounter.get(), totalProgress, msg));
 
-        String[] columnFamilies = options.getColumnFamilies().toArray(new String[options.getColumnFamilies().size()]);
-        Iterable<ColumnFamilyStore> validColumnFamilies;
-        try
+        if (ActiveRepairService.instance.hasParentRepairSession(parentSession))
+            ActiveRepairService.instance.removeParentRepairSession(parentSession);
+        Context ctx = context;
+        if (options.isTraced() && ctx != null && ctx.traceState != null)
         {
-            validColumnFamilies = storageService.getValidColumnFamilies(false, false, keyspace, columnFamilies);
-            progress.incrementAndGet();
+            for (ProgressListener listener : listeners)
+                ctx.traceState.removeProgressListener(listener);
+            // Because DebuggableThreadPoolExecutor#afterExecute and this callback
+            // run in a nondeterministic order (within the same thread), the
+            // TraceState may have been nulled out at this point. The TraceState
+            // should be traceState, so just set it without bothering to check if it
+            // actually was nulled out.
+            Tracing.instance.set(ctx.traceState);
+            Tracing.traceRepair(msg);
+            Tracing.instance.stopSession();
+        }
+
+        Keyspace.open(keyspace).metric.repairTime.update(durationMillis, TimeUnit.MILLISECONDS);
+    }
+
+    public void run()
+    {
+        try {
+            // Why is this before setup/start when its publishing the start event?  For backwards compatability
+            // One of the first things we did before was publish this before validating, so publish early to keep
+            // that the same.
+            String message = String.format("Starting repair command #%d (%s), repairing keyspace %s with %s", cmd, parentSession, keyspace,
+                                           options);
+            logger.info(message);
+            Tracing.traceRepair(message);
+            fireProgressEvent(new ProgressEvent(ProgressEventType.START, 0, 100, message));
+
+            Pair<Context, String> setup = setup();
 
 Review comment:
   Its not immediately obvious to the reader what the `Pair` represents. I would prefer if you had a nested static class with descriptive fields like you've done with the `Context`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382164733
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,53 +146,128 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
+
+        success(msg);
+    }
+
+    private void success(String msg)
     {
+        fireProgressEvent(new ProgressEvent(ProgressEventType.SUCCESS, progressCounter.get(), totalProgress, msg));
+        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.COMPLETED,
+                                                        ImmutableList.of(msg));
+        complete(null);
+    }
+
+    public void notifyError(Throwable error)
+    {
+        // exception should be ignored
+        if (error instanceof SomeRepairFailedException)
+            return;
+        logger.error("Repair {} failed:", parentSession, error);
+
         StorageMetrics.repairExceptions.inc();
-        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, message);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCount, totalProgress, errorMessage));
-        String completionMessage = String.format("Repair command #%d finished with error", cmd);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCount, totalProgress, completionMessage));
-        recordFailure(errorMessage, completionMessage);
+        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, error.getMessage());
+        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCounter.get(), totalProgress, errorMessage));
+
+        // since this can fail, update table only after updating in-memory and notification state
+        if (!options.isPreview())
+        {
+            SystemDistributedKeyspace.failParentRepair(parentSession, error);
+        }
     }
 
+    private void fail(String reason)
+    {
+        if (reason == null)
+            reason = "Some repair failed";
+        String completionMessage = String.format("Repair command #%d finished with error", cmd);
+
+        // Note we rely on the first message being the reason for the failure
+        // when inspecting this state from RepairRunner.queryForCompletedRepair
+        ActiveRepairService.instance.recordRepairStatus(cmd, ParentRepairStatus.FAILED,
+                                                        ImmutableList.of(reason, completionMessage));
+
+        complete(completionMessage);
+    }
 
-    protected void runMayThrow() throws Exception
+    private void complete(String msg)
     {
-        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.IN_PROGRESS, ImmutableList.of());
-        final TraceState traceState;
-        final UUID parentSession = UUIDGen.getTimeUUID();
-        final String tag = "repair:" + cmd;
+        long durationMillis = System.currentTimeMillis() - creationTimeMillis;
 
 Review comment:
   Can be `final`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382206051
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -91,10 +101,15 @@
     private final String keyspace;
 
     private final String tag;
-    private final AtomicInteger progress = new AtomicInteger();
+    private final AtomicInteger progressCounter = new AtomicInteger();
     private final int totalProgress;
 
+    private final long creationTimeMillis = System.currentTimeMillis();
 
 Review comment:
   Rather not in this patch since I don’t test time. 
   
   Better argument for the parent jira since it tracks state change time 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384985756
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -374,6 +375,8 @@ public void startup(ICluster cluster)
                 }
                 JVMStabilityInspector.replaceKiller(new InstanceKiller());
 
+                ActiveRepairService.instance.start();
 
 Review comment:
   From what I can see, we're starting a scheduled thread here:
   
   ```
   consistent.local.start();
           ScheduledExecutors.optionalTasks.scheduleAtFixedRate(consistent.local::cleanup, 0,
                                                                LocalSessions.CLEANUP_INTERVAL,
                                                                TimeUnit.SECONDS);
   ```
   
   It looks like it will get shut down together with scheduled executors, but I still think we should hide it behind a flag just like netwroking and gossip.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r391659785
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -374,6 +375,8 @@ public void startup(ICluster cluster)
                 }
                 JVMStabilityInspector.replaceKiller(new InstanceKiller());
 
+                ActiveRepairService.instance.start();
 
 Review comment:
   btw a good idea would be to start it already _after_ native protocol is started; however when you rebase this will already be the case

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r386045512
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
 ##########
 @@ -374,6 +375,8 @@ public void startup(ICluster cluster)
                 }
                 JVMStabilityInspector.replaceKiller(new InstanceKiller());
 
+                ActiveRepairService.instance.start();
 
 Review comment:
   @ifesdjeen can you explain why feature flags exist?  Maybe there is a mismatch between what I thought and your thinking; I honestly thought it was for performance to start a cluster.
   
   What feels weird to me is that full, validation, and preview repair will work without the new flag, but incremental repair will require it.  So to me it feels off.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r382170024
 
 

 ##########
 File path: src/java/org/apache/cassandra/repair/RepairRunnable.java
 ##########
 @@ -131,53 +146,128 @@ protected void fireProgressEvent(ProgressEvent event)
         }
     }
 
-    protected void fireErrorAndComplete(int progressCount, int totalProgress, String message)
+    private void skip(String msg)
+    {
+        logger.info("Repair {} skipped: {}", parentSession, msg);
+        fireProgressEvent(new ProgressEvent(ProgressEventType.NOTIFICATION, 100, 100, "Repair " + parentSession + " skipped: " + msg));
+
+        success(msg);
+    }
+
+    private void success(String msg)
     {
+        fireProgressEvent(new ProgressEvent(ProgressEventType.SUCCESS, progressCounter.get(), totalProgress, msg));
+        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.COMPLETED,
+                                                        ImmutableList.of(msg));
+        complete(null);
+    }
+
+    public void notifyError(Throwable error)
+    {
+        // exception should be ignored
+        if (error instanceof SomeRepairFailedException)
+            return;
+        logger.error("Repair {} failed:", parentSession, error);
+
         StorageMetrics.repairExceptions.inc();
-        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, message);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCount, totalProgress, errorMessage));
-        String completionMessage = String.format("Repair command #%d finished with error", cmd);
-        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCount, totalProgress, completionMessage));
-        recordFailure(errorMessage, completionMessage);
+        String errorMessage = String.format("Repair command #%d failed with error %s", cmd, error.getMessage());
+        fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCounter.get(), totalProgress, errorMessage));
+
+        // since this can fail, update table only after updating in-memory and notification state
+        if (!options.isPreview())
+        {
+            SystemDistributedKeyspace.failParentRepair(parentSession, error);
+        }
     }
 
+    private void fail(String reason)
+    {
+        if (reason == null)
+            reason = "Some repair failed";
+        String completionMessage = String.format("Repair command #%d finished with error", cmd);
+
+        // Note we rely on the first message being the reason for the failure
+        // when inspecting this state from RepairRunner.queryForCompletedRepair
+        ActiveRepairService.instance.recordRepairStatus(cmd, ParentRepairStatus.FAILED,
+                                                        ImmutableList.of(reason, completionMessage));
+
+        complete(completionMessage);
+    }
 
-    protected void runMayThrow() throws Exception
+    private void complete(String msg)
     {
-        ActiveRepairService.instance.recordRepairStatus(cmd, ActiveRepairService.ParentRepairStatus.IN_PROGRESS, ImmutableList.of());
-        final TraceState traceState;
-        final UUID parentSession = UUIDGen.getTimeUUID();
-        final String tag = "repair:" + cmd;
+        long durationMillis = System.currentTimeMillis() - creationTimeMillis;
+        if (msg == null) {
+            String duration = DurationFormatUtils.formatDurationWords(durationMillis,true, true);
+            msg = String.format("Repair command #%d finished in %s", cmd, duration);
+        }
 
-        final AtomicInteger progress = new AtomicInteger();
-        final int totalProgress = 4 + options.getRanges().size(); // get valid column families, calculate neighbors, validation, prepare for repair + number of ranges to repair
+        fireProgressEvent(new ProgressEvent(ProgressEventType.COMPLETE, progressCounter.get(), totalProgress, msg));
 
-        String[] columnFamilies = options.getColumnFamilies().toArray(new String[options.getColumnFamilies().size()]);
-        Iterable<ColumnFamilyStore> validColumnFamilies;
-        try
+        if (ActiveRepairService.instance.hasParentRepairSession(parentSession))
+            ActiveRepairService.instance.removeParentRepairSession(parentSession);
+        Context ctx = context;
+        if (options.isTraced() && ctx != null && ctx.traceState != null)
         {
-            validColumnFamilies = storageService.getValidColumnFamilies(false, false, keyspace, columnFamilies);
-            progress.incrementAndGet();
+            for (ProgressListener listener : listeners)
+                ctx.traceState.removeProgressListener(listener);
+            // Because DebuggableThreadPoolExecutor#afterExecute and this callback
+            // run in a nondeterministic order (within the same thread), the
+            // TraceState may have been nulled out at this point. The TraceState
+            // should be traceState, so just set it without bothering to check if it
+            // actually was nulled out.
+            Tracing.instance.set(ctx.traceState);
+            Tracing.traceRepair(msg);
+            Tracing.instance.stopSession();
+        }
+
+        Keyspace.open(keyspace).metric.repairTime.update(durationMillis, TimeUnit.MILLISECONDS);
+    }
+
+    public void run()
+    {
+        try {
+            // Why is this before setup/start when its publishing the start event?  For backwards compatability
+            // One of the first things we did before was publish this before validating, so publish early to keep
+            // that the same.
+            String message = String.format("Starting repair command #%d (%s), repairing keyspace %s with %s", cmd, parentSession, keyspace,
+                                           options);
+            logger.info(message);
+            Tracing.traceRepair(message);
+            fireProgressEvent(new ProgressEvent(ProgressEventType.START, 0, 100, message));
+
+            Pair<Context, String> setup = setup();
+            if (setup.right != null)
+            {
+                skip(setup.right);
+                return;
+            }
+            Context ctx = setup.left;
+            assert ctx != null : "Context is required but was not found";
+            this.context = ctx;
+            start(ctx);
         }
-        catch (IllegalArgumentException | IOException e)
+        catch (Exception | Error e)
         {
-            logger.error("Repair {} failed:", parentSession, e);
-            fireErrorAndComplete(progress.get(), totalProgress, e.getMessage());
-            return;
+            notifyError(e);
+            fail(e.getMessage());
         }
+    }
+
+    private Pair<Context, String> setup() throws Exception
+    {
+        ActiveRepairService.instance.recordRepairStatus(cmd, ParentRepairStatus.IN_PROGRESS, ImmutableList.of());
+
+        String[] columnFamilies = options.getColumnFamilies().toArray(new String[options.getColumnFamilies().size()]);
 
 Review comment:
   Can be `final`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384235549
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/LongTokenRange.java
 ##########
 @@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.io.Serializable;
+
+public final class LongTokenRange implements Serializable
 
 Review comment:
   > We already have API for bootstrap that works with long tokens, and having this in addition sort of hints at a need to do something like that.
   
   you mean 
   
   ```
   public static interface TokenSupplier
       {
           public long token(int nodeId);
       }
   ```
   
   Yeah, that looks to add startup integration, this is a hack to get runtime accessor.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384027782
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedRepairUtils.java
 ##########
 @@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.ResultSet;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.impl.AbstractCluster;
+import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.metrics.StorageMetrics;
+
+public final class DistributedRepairUtils
+{
+    public static final int DEFAULT_COORDINATOR = 1;
+
+    private DistributedRepairUtils()
+    {
+
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, RepairType repairType, boolean withNotifications, String... args) {
+        return repair(cluster, DEFAULT_COORDINATOR, repairType, withNotifications, args);
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, int node, RepairType repairType, boolean withNotifications, String... args) {
+        args = repairType.append(args);
+        args = ArrayUtils.addAll(new String[] { "repair" }, args);
+        return cluster.get(node).nodetoolResult(withNotifications, args);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster)
+    {
+        return getRepairExceptions(cluster, DEFAULT_COORDINATOR);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster, int node)
+    {
+        return cluster.get(node).callOnInstance(() -> StorageMetrics.repairExceptions.getCount());
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, String ks, String table)
+    {
+        return queryParentRepairHistory(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        // This is kinda brittle since the caller never gets the ID and can't ask for the ID; it needs to infer the id
+        // this logic makes the assumption the ks/table pairs are unique (should be or else create should fail) so any
+        // repair for that pair will be the repair id
+        Set<String> tableNames = table == null? Collections.emptySet() : ImmutableSet.of(table);
+        ResultSet rs = null;
+        Exception latestException = null;
+        for (int i = 0; i < 10; i++)
+        {
+            try
+            {
+                rs = cluster.coordinator(coordinator)
+                            .executeWithResult("SELECT * FROM system_distributed.parent_repair_history", ConsistencyLevel.QUORUM)
+                            .filter(row -> ks.equals(row.getString("keyspace_name")))
+                            .filter(row -> tableNames.equals(row.getSet("columnfamily_names")));
+                break;
+            }
+            catch (Exception e)
+            {
+                latestException = e;
+                rs = null;
+                //TODO do we have a backoff stategy I can leverage?  I would prefer expotential but don't want to add for somethinr minor
+                Uninterruptibles.sleepUninterruptibly( (i + 1) * 300, TimeUnit.MILLISECONDS);
+            }
+        }
+        if (rs == null)
+        {
+            // exception should exist
+            if (latestException == null)
+            {
+                Assert.fail("Unable to query system_distributed.parent_repair_history, got back neither result set or exception ");
+            }
+            if (latestException instanceof RuntimeException)
 
 Review comment:
   Intellij says this one is always `true`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384236353
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java
 ##########
 @@ -151,9 +152,16 @@ public boolean isShutdown()
         @Override
         public synchronized void startup()
         {
+            startup(AbstractCluster.this);
+        }
+
+        public synchronized void startup(ICluster cluster)
+        {
+            if (cluster != AbstractCluster.this)
+                throw new IllegalArgumentException("Only the owning cluster can be used for startup"); //TODO why have this in the API?
 
 Review comment:
   actually its too big a change.  The user API is the same as the plugin api (instance), so this leaks to user but is needed by plugin.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384235708
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/LongTokenRange.java
 ##########
 @@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.io.Serializable;
+
+public final class LongTokenRange implements Serializable
 
 Review comment:
   also just noticed `TokenSupplier` doesn't support vnodes...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r383810654
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/ResultSet.java
 ##########
 @@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.util.Date;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+public class ResultSet implements Iterator<Row>
+{
+    public static final ResultSet EMPTY = new ResultSet(new String[0], null);
+
+    private final String[] names;
+    private final Object[][] results;
+    private final Predicate<Row> filter;
+    private final Row row;
+    private int offset = -1;
+
+    public ResultSet(String[] names, Object[][] results)
+    {
+        this.names = names;
+        this.results = results;
+        this.row = new Row(names);
+        this.filter = ignore -> true;
+    }
+
+    private ResultSet(String[] names, Object[][] results, Predicate<Row> filter, int offset)
+    {
+        this.names = names;
+        this.results = results;
+        this.filter = filter;
+        this.offset = offset;
+        this.row = new Row(names);
+    }
+
+    public String[] getNames()
+    {
+        return names;
+    }
+
+    public boolean isEmpty()
+    {
+        return results.length == 0;
+    }
+
+    public int size()
+    {
+        return results.length;
+    }
+
+    public ResultSet filter(Predicate<Row> fn)
+    {
+        return new ResultSet(names, results, filter.and(fn), offset);
+    }
+
+    public Object[][] toObjectArrays()
+    {
+        return results;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+        if (results == null)
+            return false;
+        while ((offset += 1) < results.length)
+        {
+            row.setResults(results[offset]);
+            if (filter.test(row))
+            {
+                return true;
+            }
+        }
+        row.setResults(null);
+        return false;
+    }
+
+    @Override
+    public Row next()
+    {
+        if (offset < 0 || offset >= results.length)
+            throw new NoSuchElementException();
+        return row;
 
 Review comment:
   I don't mind pooling / reusing objects in general. Should we change API from `Iterator` to something that prevents leaking instances unless person is deliberately trying to do something harmful for themselves?
   
   Something like `forEach(Row)` that is exposed by `Iterable` and maybe `map` to produce a derivative collection, but without exposing `Iterator` itself, since if someone tries to hold on for `Row` instance, its contents will change, somewhat unexpectedly for them, since they've received a legit row instance.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
ifesdjeen commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384027021
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/test/DistributedRepairUtils.java
 ##########
 @@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Assert;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.NodeToolResult;
+import org.apache.cassandra.distributed.api.ResultSet;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.impl.AbstractCluster;
+import org.apache.cassandra.distributed.impl.IInvokableInstance;
+import org.apache.cassandra.metrics.StorageMetrics;
+
+public final class DistributedRepairUtils
+{
+    public static final int DEFAULT_COORDINATOR = 1;
+
+    private DistributedRepairUtils()
+    {
+
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, RepairType repairType, boolean withNotifications, String... args) {
+        return repair(cluster, DEFAULT_COORDINATOR, repairType, withNotifications, args);
+    }
+
+    public static NodeToolResult repair(AbstractCluster<?> cluster, int node, RepairType repairType, boolean withNotifications, String... args) {
+        args = repairType.append(args);
+        args = ArrayUtils.addAll(new String[] { "repair" }, args);
+        return cluster.get(node).nodetoolResult(withNotifications, args);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster)
+    {
+        return getRepairExceptions(cluster, DEFAULT_COORDINATOR);
+    }
+
+    public static <I extends IInvokableInstance, C extends AbstractCluster<I>> long getRepairExceptions(C cluster, int node)
+    {
+        return cluster.get(node).callOnInstance(() -> StorageMetrics.repairExceptions.getCount());
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, String ks, String table)
+    {
+        return queryParentRepairHistory(cluster, DEFAULT_COORDINATOR, ks, table);
+    }
+
+    public static ResultSet queryParentRepairHistory(AbstractCluster<?> cluster, int coordinator, String ks, String table)
+    {
+        // This is kinda brittle since the caller never gets the ID and can't ask for the ID; it needs to infer the id
+        // this logic makes the assumption the ks/table pairs are unique (should be or else create should fail) so any
+        // repair for that pair will be the repair id
+        Set<String> tableNames = table == null? Collections.emptySet() : ImmutableSet.of(table);
+        ResultSet rs = null;
+        Exception latestException = null;
+        for (int i = 0; i < 10; i++)
+        {
+            try
+            {
+                rs = cluster.coordinator(coordinator)
+                            .executeWithResult("SELECT * FROM system_distributed.parent_repair_history", ConsistencyLevel.QUORUM)
+                            .filter(row -> ks.equals(row.getString("keyspace_name")))
+                            .filter(row -> tableNames.equals(row.getSet("columnfamily_names")));
+                break;
+            }
+            catch (Exception e)
+            {
+                latestException = e;
+                rs = null;
+                //TODO do we have a backoff stategy I can leverage?  I would prefer expotential but don't want to add for somethinr minor
 
 Review comment:
   You can just take just take it from `ExponentialBackoffRetry` from driver, it's quite tiny:
   
   ```
        protected long calculateSleepTime(int retries) {
               return this.sleepTime * (long)this.r.nextInt(1 << retries + 1);
           }
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #446: CASSANDRA-15564 Refactor repair coordinator to centralize stage change logic and improved the public facing errors
URL: https://github.com/apache/cassandra/pull/446#discussion_r384234884
 
 

 ##########
 File path: test/distributed/org/apache/cassandra/distributed/api/LongTokenRange.java
 ##########
 @@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.api;
+
+import java.io.Serializable;
+
+public final class LongTokenRange implements Serializable
 
 Review comment:
   > I think we might want to limit partitioner to Murmur3Partitioner and allow using long tokens only
   
   for now sure.  If we want to move to a actual total order partitioner we could always relook at the testing implications of that then.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org