You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yu...@apache.org on 2014/11/24 22:26:18 UTC

[1/8] cassandra git commit: Add DC-aware sequential repair

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 17de36f24 -> 41469ecf6
  refs/heads/cassandra-2.1 eac7781e7 -> 326a9ff2f
  refs/heads/trunk 584113103 -> c023d4922


Add DC-aware sequential repair

patch by Jimmy Mårdell; reviewed by yukim for CASSANDRA-8193


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/41469ecf
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/41469ecf
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/41469ecf

Branch: refs/heads/cassandra-2.0
Commit: 41469ecf6a27e94441f96ef905ed3b5354c23987
Parents: 17de36f
Author: Jimmy Mårdell <ya...@spotify.com>
Authored: Mon Nov 24 15:07:33 2014 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Mon Nov 24 15:09:41 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../DatacenterAwareRequestCoordinator.java      |  73 +++++++++++
 .../cassandra/repair/IRequestCoordinator.java   |  28 ++++
 .../cassandra/repair/IRequestProcessor.java     |  23 ++++
 .../repair/ParallelRequestCoordinator.java      |  49 +++++++
 .../org/apache/cassandra/repair/RepairJob.java  |  32 ++++-
 .../cassandra/repair/RepairParallelism.java     |  22 ++++
 .../apache/cassandra/repair/RepairSession.java  |  14 +-
 .../cassandra/repair/RequestCoordinator.java    | 128 -------------------
 .../repair/SequentialRequestCoordinator.java    |  58 +++++++++
 .../cassandra/service/ActiveRepairService.java  |   6 +-
 .../cassandra/service/StorageService.java       |  64 ++++++----
 .../cassandra/service/StorageServiceMBean.java  |  19 ++-
 .../org/apache/cassandra/tools/NodeCmd.java     |  21 ++-
 .../org/apache/cassandra/tools/NodeProbe.java   |  30 +++--
 .../apache/cassandra/tools/NodeToolHelp.yaml    |   1 +
 .../repair/RequestCoordinatorTest.java          | 124 ++++++++++++++++++
 17 files changed, 506 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index fe23248..7519653 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -24,6 +24,7 @@
  * Allow concurrent writing of the same table in the same JVM using
    CQLSSTableWriter (CASSANDRA-7463)
  * Fix totalDiskSpaceUsed calculation (CASSANDRA-8205)
+ * Add DC-aware sequential repair (CASSANDRA-8193)
 
 
 2.0.11:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java b/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java
new file mode 100644
index 0000000..ab3e03e
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+import java.net.InetAddress;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Queue;
+
+public class DatacenterAwareRequestCoordinator implements IRequestCoordinator<InetAddress>
+{
+    private Map<String, Queue<InetAddress>> requestsByDatacenter = new HashMap<>();
+    private int remaining = 0;
+    private final IRequestProcessor<InetAddress> processor;
+
+    protected DatacenterAwareRequestCoordinator(IRequestProcessor<InetAddress> processor)
+    {
+        this.processor = processor;
+    }
+
+    public void add(InetAddress request)
+    {
+        String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(request);
+        Queue<InetAddress> queue = requestsByDatacenter.get(dc);
+        if (queue == null)
+        {
+            queue = new LinkedList<>();
+            requestsByDatacenter.put(dc, queue);
+        }
+        queue.add(request);
+        remaining++;
+    }
+
+    public void start()
+    {
+        for (Queue<InetAddress> requests : requestsByDatacenter.values())
+        {
+            if (!requests.isEmpty())
+              processor.process(requests.peek());
+        }
+    }
+
+    // Returns how many request remains
+    public int completed(InetAddress request)
+    {
+        String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(request);
+        Queue<InetAddress> requests = requestsByDatacenter.get(dc);
+        assert requests != null;
+        assert request.equals(requests.peek());
+        requests.poll();
+        if (!requests.isEmpty())
+            processor.process(requests.peek());
+        return --remaining;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/IRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/IRequestCoordinator.java b/src/java/org/apache/cassandra/repair/IRequestCoordinator.java
new file mode 100644
index 0000000..db2ffe3
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/IRequestCoordinator.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+public interface IRequestCoordinator<R>
+{
+    void add(R request);
+
+    void start();
+
+    // Returns how many request remains
+    int completed(R request);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/IRequestProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/IRequestProcessor.java b/src/java/org/apache/cassandra/repair/IRequestProcessor.java
new file mode 100644
index 0000000..c7b49d7
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/IRequestProcessor.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+public interface IRequestProcessor<R>
+{
+    void process(R request);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java b/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java
new file mode 100644
index 0000000..839bb43
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class ParallelRequestCoordinator<R> implements IRequestCoordinator<R>
+{
+    private final Set<R> requests = new HashSet<>();
+    private final IRequestProcessor<R> processor;
+
+    public ParallelRequestCoordinator(IRequestProcessor<R> processor)
+    {
+        this.processor = processor;
+    }
+
+    @Override
+    public void add(R request) { requests.add(request); }
+
+    @Override
+    public void start()
+    {
+        for (R request : requests)
+            processor.process(request);
+    }
+
+    @Override
+    public int completed(R request)
+    {
+        requests.remove(request);
+        return requests.size();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RepairJob.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java
index 931f95a..7c791aa 100644
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@ -43,9 +43,9 @@ public class RepairJob
     private static Logger logger = LoggerFactory.getLogger(RepairJob.class);
 
     public final RepairJobDesc desc;
-    private final boolean isSequential;
+    private final RepairParallelism parallelismDegree;
     // first we send tree requests. this tracks the endpoints remaining to hear from
-    private final RequestCoordinator<InetAddress> treeRequests;
+    private final IRequestCoordinator<InetAddress> treeRequests;
     // tree responses are then tracked here
     private final List<TreeResponse> trees = new ArrayList<>();
     // once all responses are received, each tree is compared with each other, and differencer tasks
@@ -68,21 +68,38 @@ public class RepairJob
                      String keyspace,
                      String columnFamily,
                      Range<Token> range,
-                     boolean isSequential,
+                     RepairParallelism parallelismDegree,
                      ListeningExecutorService taskExecutor)
     {
         this.listener = listener;
         this.desc = new RepairJobDesc(sessionId, keyspace, columnFamily, range);
-        this.isSequential = isSequential;
+        this.parallelismDegree = parallelismDegree;
         this.taskExecutor = taskExecutor;
-        this.treeRequests = new RequestCoordinator<InetAddress>(isSequential)
+
+        IRequestProcessor<InetAddress> processor = new IRequestProcessor<InetAddress>()
         {
-            public void send(InetAddress endpoint)
+            @Override
+            public void process(InetAddress endpoint)
             {
                 ValidationRequest request = new ValidationRequest(desc, gcBefore);
                 MessagingService.instance().sendOneWay(request.createMessage(), endpoint);
             }
         };
+
+        switch (parallelismDegree)
+        {
+            case SEQUENTIAL:
+                this.treeRequests = new SequentialRequestCoordinator<>(processor);
+                break;
+            case PARALLEL:
+                this.treeRequests = new ParallelRequestCoordinator<>(processor);
+                break;
+            case DATACENTER_AWARE:
+                this.treeRequests = new DatacenterAwareRequestCoordinator(processor);
+                break;
+            default:
+                throw new AssertionError("Unknown degree of parallelism specified");
+        }
     }
 
     /**
@@ -102,7 +119,8 @@ public class RepairJob
         List<InetAddress> allEndpoints = new ArrayList<>(endpoints);
         allEndpoints.add(FBUtilities.getBroadcastAddress());
 
-        if (isSequential)
+        // Create a snapshot at all nodes unless we're using pure parallel repairs
+        if (parallelismDegree != RepairParallelism.PARALLEL)
         {
             List<ListenableFuture<InetAddress>> snapshotTasks = new ArrayList<>(allEndpoints.size());
             for (InetAddress endpoint : allEndpoints)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RepairParallelism.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairParallelism.java b/src/java/org/apache/cassandra/repair/RepairParallelism.java
new file mode 100644
index 0000000..12c22ca
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/RepairParallelism.java
@@ -0,0 +1,22 @@
+package org.apache.cassandra.repair;
+
+/**
+ * Specify the degree of parallelism when calculating the merkle trees in a repair job.
+ */
+public enum RepairParallelism
+{
+    /**
+     * One node at a time
+     */
+    SEQUENTIAL,
+
+    /**
+     * All nodes at the same time
+     */
+    PARALLEL,
+
+    /**
+     * One node per data center at a time
+     */
+    DATACENTER_AWARE
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RepairSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairSession.java b/src/java/org/apache/cassandra/repair/RepairSession.java
index c9a9671..f2b95eb 100644
--- a/src/java/org/apache/cassandra/repair/RepairSession.java
+++ b/src/java/org/apache/cassandra/repair/RepairSession.java
@@ -83,7 +83,7 @@ public class RepairSession extends WrappedRunnable implements IEndpointStateChan
     private final UUID id;
     public final String keyspace;
     private final String[] cfnames;
-    public final boolean isSequential;
+    public final RepairParallelism parallelismDegree;
     /** Range to repair */
     public final Range<Token> range;
     public final Set<InetAddress> endpoints;
@@ -110,19 +110,19 @@ public class RepairSession extends WrappedRunnable implements IEndpointStateChan
      *
      * @param range range to repair
      * @param keyspace name of keyspace
-     * @param isSequential true if performing repair on snapshots sequentially
+     * @param parallelismDegree specifies the degree of parallelism when calculating the merkle trees
      * @param dataCenters the data centers that should be part of the repair; null for all DCs
      * @param cfnames names of columnfamilies
      */
-    public RepairSession(Range<Token> range, String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
+    public RepairSession(Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
     {
-        this(UUIDGen.getTimeUUID(), range, keyspace, isSequential, dataCenters, hosts, cfnames);
+        this(UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, dataCenters, hosts, cfnames);
     }
 
-    public RepairSession(UUID id, Range<Token> range, String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, String[] cfnames)
+    public RepairSession(UUID id, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String[] cfnames)
     {
         this.id = id;
-        this.isSequential = isSequential;
+        this.parallelismDegree = parallelismDegree;
         this.keyspace = keyspace;
         this.cfnames = cfnames;
         assert cfnames.length > 0 : "Repairing no column families seems pointless, doesn't it";
@@ -270,7 +270,7 @@ public class RepairSession extends WrappedRunnable implements IEndpointStateChan
             // Create and queue a RepairJob for each column family
             for (String cfname : cfnames)
             {
-                RepairJob job = new RepairJob(this, id, keyspace, cfname, range, isSequential, taskExecutor);
+                RepairJob job = new RepairJob(this, id, keyspace, cfname, range, parallelismDegree, taskExecutor);
                 jobs.offer(job);
             }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RequestCoordinator.java b/src/java/org/apache/cassandra/repair/RequestCoordinator.java
deleted file mode 100644
index ed089ef..0000000
--- a/src/java/org/apache/cassandra/repair/RequestCoordinator.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.repair;
-
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Queue;
-import java.util.Set;
-
-/**
-*/
-public abstract class RequestCoordinator<R>
-{
-    private final Order<R> orderer;
-
-    public RequestCoordinator(boolean isSequential)
-    {
-        this.orderer = isSequential ? new SequentialOrder(this) : new ParallelOrder(this);
-    }
-
-    public abstract void send(R request);
-
-    public void add(R request)
-    {
-        orderer.add(request);
-    }
-
-    public void start()
-    {
-        orderer.start();
-    }
-
-    // Returns how many request remains
-    public int completed(R request)
-    {
-        return orderer.completed(request);
-    }
-
-    private static abstract class Order<R>
-    {
-        protected final RequestCoordinator<R> coordinator;
-
-        Order(RequestCoordinator<R> coordinator)
-        {
-            this.coordinator = coordinator;
-        }
-
-        public abstract void add(R request);
-        public abstract void start();
-        public abstract int completed(R request);
-    }
-
-    private static class SequentialOrder<R> extends Order<R>
-    {
-        private final Queue<R> requests = new LinkedList<>();
-
-        SequentialOrder(RequestCoordinator<R> coordinator)
-        {
-            super(coordinator);
-        }
-
-        public void add(R request)
-        {
-            requests.add(request);
-        }
-
-        public void start()
-        {
-            if (requests.isEmpty())
-                return;
-
-            coordinator.send(requests.peek());
-        }
-
-        public int completed(R request)
-        {
-            assert request.equals(requests.peek());
-            requests.poll();
-            int remaining = requests.size();
-            if (remaining != 0)
-                coordinator.send(requests.peek());
-            return remaining;
-        }
-    }
-
-    private static class ParallelOrder<R> extends Order<R>
-    {
-        private final Set<R> requests = new HashSet<>();
-
-        ParallelOrder(RequestCoordinator<R> coordinator)
-        {
-            super(coordinator);
-        }
-
-        public void add(R request)
-        {
-            requests.add(request);
-        }
-
-        public void start()
-        {
-            for (R request : requests)
-                coordinator.send(request);
-        }
-
-        public int completed(R request)
-        {
-            requests.remove(request);
-            return requests.size();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java b/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java
new file mode 100644
index 0000000..6bb5de9
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import java.util.LinkedList;
+import java.util.Queue;
+
+public class SequentialRequestCoordinator<R> implements IRequestCoordinator<R>
+{
+    private final Queue<R> requests = new LinkedList<>();
+    private final IRequestProcessor<R> processor;
+
+    public SequentialRequestCoordinator(IRequestProcessor<R> processor)
+    {
+        this.processor = processor;
+    }
+
+    @Override
+    public void add(R request)
+    {
+        requests.add(request);
+    }
+
+    @Override
+    public void start()
+    {
+        if (requests.isEmpty())
+            return;
+
+        processor.process(requests.peek());
+    }
+
+    @Override
+    public int completed(R request)
+    {
+        assert request.equals(requests.peek());
+        requests.poll();
+        int remaining = requests.size();
+        if (remaining != 0)
+            processor.process(requests.peek());
+        return remaining;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index aac9f9a..da81e8f 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -92,9 +92,9 @@ public class ActiveRepairService
      *
      * @return Future for asynchronous call or null if there is no need to repair
      */
-    public RepairFuture submitRepairSession(Range<Token> range, String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
+    public RepairFuture submitRepairSession(Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
     {
-        RepairSession session = new RepairSession(range, keyspace, isSequential, dataCenters, hosts, cfnames);
+        RepairSession session = new RepairSession(range, keyspace, parallelismDegree, dataCenters, hosts, cfnames);
         if (session.endpoints.isEmpty())
             return null;
         RepairFuture futureTask = new RepairFuture(session);
@@ -128,7 +128,7 @@ public class ActiveRepairService
     // add it to the sessions (avoid NPE in tests)
     RepairFuture submitArtificialRepairSession(RepairJobDesc desc)
     {
-        RepairSession session = new RepairSession(desc.sessionId, desc.range, desc.keyspace, false, null, null, new String[]{desc.columnFamily});
+        RepairSession session = new RepairSession(desc.sessionId, desc.range, desc.keyspace, RepairParallelism.PARALLEL, null, null, new String[]{desc.columnFamily});
         sessions.put(session.getId(), session);
         RepairFuture futureTask = new RepairFuture(session);
         executor.execute(futureTask);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 4bc1eee..3d42d1c 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -77,6 +77,7 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.ResponseVerbHandler;
 import org.apache.cassandra.repair.RepairFuture;
 import org.apache.cassandra.repair.RepairMessageVerbHandler;
+import org.apache.cassandra.repair.RepairParallelism;
 import org.apache.cassandra.service.paxos.CommitVerbHandler;
 import org.apache.cassandra.service.paxos.PrepareVerbHandler;
 import org.apache.cassandra.service.paxos.ProposeVerbHandler;
@@ -334,7 +335,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         {
             throw new IllegalStateException("No configured daemon");
         }
-        
+
         try
         {
             daemon.nativeServer.start();
@@ -432,10 +433,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             if (Gossiper.instance.getEndpointStateForEndpoint(DatabaseDescriptor.getReplaceAddress()).getApplicationState(ApplicationState.TOKENS) == null)
                 throw new RuntimeException("Could not find tokens for " + DatabaseDescriptor.getReplaceAddress() + " to replace");
             Collection<Token> tokens = TokenSerializer.deserialize(getPartitioner(), new DataInputStream(new ByteArrayInputStream(getApplicationStateValue(DatabaseDescriptor.getReplaceAddress(), ApplicationState.TOKENS))));
-            
+
             SystemKeyspace.setLocalHostId(hostId); // use the replacee's host Id as our own so we receive hints, etc
             Gossiper.instance.resetEndpointStateMap(); // clean up since we have what we need
-            return tokens;        
+            return tokens;
         }
         catch (IOException e)
         {
@@ -2408,16 +2409,21 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int forceRepairAsync(final String keyspace, final boolean isSequential, final Collection<String> dataCenters, final Collection<String> hosts, final boolean primaryRange, final String... columnFamilies)
     {
+        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, columnFamilies);
+    }
+
+    public int forceRepairAsync(final String keyspace, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts, final boolean primaryRange, final String... columnFamilies)
+    {
         // when repairing only primary range, dataCenter nor hosts can be set
         if (primaryRange && (dataCenters != null || hosts != null))
         {
             throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
         }
         final Collection<Range<Token>> ranges = primaryRange ? getLocalPrimaryRanges(keyspace) : getLocalRanges(keyspace);
-        return forceRepairAsync(keyspace, isSequential, dataCenters, hosts, ranges, columnFamilies);
+        return forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, ranges, columnFamilies);
     }
 
-    public int forceRepairAsync(final String keyspace, final boolean isSequential, final Collection<String> dataCenters, final Collection<String> hosts,  final Collection<Range<Token>> ranges, final String... columnFamilies)
+    public int forceRepairAsync(final String keyspace, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts,  final Collection<Range<Token>> ranges, final String... columnFamilies)
     {
         if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
             return 0;
@@ -2425,7 +2431,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         final int cmd = nextRepairCommand.incrementAndGet();
         if (ranges.size() > 0)
         {
-            new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, hosts, columnFamilies)).start();
+            new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, hosts, columnFamilies)).start();
         }
         return cmd;
     }
@@ -2438,37 +2444,42 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
         }
         final Collection<Range<Token>> ranges = primaryRange ? getLocalPrimaryRanges(keyspace) : getLocalRanges(keyspace);
-        return forceRepairAsync(keyspace, isSequential, isLocal, ranges, columnFamilies);
+        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, ranges, columnFamilies);
     }
 
-    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, Collection<Range<Token>> ranges, String... columnFamilies)
+    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, boolean isLocal, Collection<Range<Token>> ranges, String... columnFamilies)
     {
         if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
             return 0;
 
         final int cmd = nextRepairCommand.incrementAndGet();
-        if (!FBUtilities.isUnix() && isSequential)
+        if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
         {
             logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-            isSequential = false;
+            parallelismDegree = RepairParallelism.PARALLEL;
         }
-        new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, isLocal, columnFamilies)).start();
+        new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, isLocal, columnFamilies)).start();
         return cmd;
     }
 
     public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
     {
+        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, columnFamilies);
+    }
+
+    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
+    {
         Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
 
         logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
                     repairingRange, keyspaceName, columnFamilies);
 
-        if (!FBUtilities.isUnix() && isSequential)
+        if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
         {
             logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-            isSequential = false;
+            parallelismDegree = RepairParallelism.PARALLEL;
         }
-        return forceRepairAsync(keyspaceName, isSequential, dataCenters, hosts, repairingRange, columnFamilies);
+        return forceRepairAsync(keyspaceName, parallelismDegree, dataCenters, hosts, repairingRange, columnFamilies);
     }
 
     public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies)
@@ -2486,7 +2497,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public void forceKeyspaceRepair(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
     {
-        forceKeyspaceRepairRange(keyspaceName, getLocalRanges(keyspaceName), isSequential, isLocal, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, getLocalRanges(keyspaceName), isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, columnFamilies);
     }
 
     public void forceKeyspaceRepairPrimaryRange(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
@@ -2498,7 +2509,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
         }
 
-        forceKeyspaceRepairRange(keyspaceName, getLocalPrimaryRanges(keyspaceName), isSequential, false, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, getLocalPrimaryRanges(keyspaceName), isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, false, columnFamilies);
     }
 
     public void forceKeyspaceRepairRange(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
@@ -2507,14 +2518,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
                            repairingRange, keyspaceName, columnFamilies);
-        forceKeyspaceRepairRange(keyspaceName, repairingRange, isSequential, isLocal, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, repairingRange, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, columnFamilies);
     }
 
-    public void forceKeyspaceRepairRange(final String keyspaceName, final Collection<Range<Token>> ranges, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
+    public void forceKeyspaceRepairRange(final String keyspaceName, final Collection<Range<Token>> ranges, RepairParallelism parallelismDegree, boolean isLocal, final String... columnFamilies) throws IOException
     {
         if (ranges.isEmpty() || Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor() < 2)
             return;
-        createRepairTask(nextRepairCommand.incrementAndGet(), keyspaceName, ranges, isSequential, isLocal, columnFamilies).run();
+        createRepairTask(nextRepairCommand.incrementAndGet(), keyspaceName, ranges, parallelismDegree, isLocal, columnFamilies).run();
     }
 
     /**
@@ -2556,17 +2567,17 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return repairingRange;
     }
 
-    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final boolean isSequential, final boolean isLocal, final String... columnFamilies)
+    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final RepairParallelism parallelismDegree, final boolean isLocal, final String... columnFamilies)
     {
         Set<String> dataCenters = null;
         if (isLocal)
         {
             dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
         }
-        return createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, null, columnFamilies);
+        return createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, null, columnFamilies);
     }
 
-    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final boolean isSequential, final Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
+    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
     {
         if (dataCenters != null && !dataCenters.contains(DatabaseDescriptor.getLocalDataCenter()))
         {
@@ -2587,7 +2598,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                     RepairFuture future;
                     try
                     {
-                        future = forceKeyspaceRepair(range, keyspace, isSequential, dataCenters, hosts, columnFamilies);
+                        future = forceKeyspaceRepair(range, keyspace, parallelismDegree, dataCenters, hosts, columnFamilies);
                     }
                     catch (IllegalArgumentException e)
                     {
@@ -2639,6 +2650,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public RepairFuture forceKeyspaceRepair(final Range<Token> range, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, final String... columnFamilies) throws IOException
     {
+        return forceKeyspaceRepair(range, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, columnFamilies);
+    }
+
+    public RepairFuture forceKeyspaceRepair(final Range<Token> range, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, final String... columnFamilies) throws IOException
+    {
         ArrayList<String> names = new ArrayList<String>();
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
@@ -2651,7 +2667,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             return null;
         }
 
-        return ActiveRepairService.instance.submitRepairSession(range, keyspaceName, isSequential, dataCenters, hosts, names.toArray(new String[names.size()]));
+        return ActiveRepairService.instance.submitRepairSession(range, keyspaceName, parallelismDegree, dataCenters, hosts, names.toArray(new String[names.size()]));
     }
 
     public void forceTerminateAllRepairSessions() {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 0e2f55d..2386fc8 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -17,6 +17,8 @@
  */
 package org.apache.cassandra.service;
 
+import org.apache.cassandra.repair.RepairParallelism;
+
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -260,10 +262,25 @@ public interface StorageServiceMBean extends NotificationEmitter
     public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, String... columnFamilies);
 
     /**
+     * Invoke repair asynchronously.
+     * You can track repair progress by subscribing JMX notification sent from this StorageServiceMBean.
+     * Notification format is:
+     *   type: "repair"
+     *   userObject: int array of length 2, [0]=command number, [1]=ordinal of AntiEntropyService.Status
+     *
+     * @return Repair command number, or 0 if nothing to repair
+     */
+    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, String... columnFamilies);
+
+    /**
      * Same as forceRepairAsync, but handles a specified range
      */
     public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts,  final String... columnFamilies);
 
+    /**
+     * Same as forceRepairAsync, but handles a specified range
+     */
+    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts,  final String... columnFamilies);
 
     /**
      * Invoke repair asynchronously.
@@ -477,7 +494,7 @@ public interface StorageServiceMBean extends NotificationEmitter
 
     /**
      * Enables/Disables tracing for the whole system. Only thrift requests can start tracing currently.
-     * 
+     *
      * @param probability
      *            ]0,1[ will enable tracing on a partial number of requests with the provided probability. 0 will
      *            disable tracing and 1 will enable tracing for all requests (which mich severely cripple the system)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/tools/NodeCmd.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeCmd.java b/src/java/org/apache/cassandra/tools/NodeCmd.java
index a397244..2d7809a 100644
--- a/src/java/org/apache/cassandra/tools/NodeCmd.java
+++ b/src/java/org/apache/cassandra/tools/NodeCmd.java
@@ -47,6 +47,7 @@ import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.net.MessagingServiceMBean;
+import org.apache.cassandra.repair.RepairParallelism;
 import org.apache.cassandra.service.CacheServiceMBean;
 import org.apache.cassandra.service.StorageProxyMBean;
 import org.apache.cassandra.streaming.StreamState;
@@ -70,6 +71,7 @@ public class NodeCmd
     private static final Pair<String, String> TOKENS_OPT = Pair.create("T", "tokens");
     private static final Pair<String, String> PRIMARY_RANGE_OPT = Pair.create("pr", "partitioner-range");
     private static final Pair<String, String> PARALLEL_REPAIR_OPT = Pair.create("par", "parallel");
+    private static final Pair<String, String> DCPARALLEL_REPAIR_OPT = Pair.create("dcpar", "dcparallel");
     private static final Pair<String, String> LOCAL_DC_REPAIR_OPT = Pair.create("local", "in-local-dc");
     private static final Pair<String, String> HOST_REPAIR_OPT = Pair.create("hosts", "in-host");
     private static final Pair<String, String> DC_REPAIR_OPT = Pair.create("dc", "in-dc");
@@ -100,6 +102,7 @@ public class NodeCmd
         options.addOption(TOKENS_OPT,   false, "display all tokens");
         options.addOption(PRIMARY_RANGE_OPT, false, "only repair the first range returned by the partitioner for the node");
         options.addOption(PARALLEL_REPAIR_OPT, false, "repair nodes in parallel.");
+        options.addOption(DCPARALLEL_REPAIR_OPT, false, "repair data centers in parallel.");
         options.addOption(LOCAL_DC_REPAIR_OPT, false, "only repair against nodes in the same datacenter");
         options.addOption(DC_REPAIR_OPT, true, "only repair against nodes in the specified datacenters (comma separated)");
         options.addOption(HOST_REPAIR_OPT, true, "only repair against specified nodes (comma separated)");
@@ -203,10 +206,10 @@ public class NodeCmd
         StringBuilder header = new StringBuilder(512);
         header.append("\nAvailable commands\n");
         final NodeToolHelp ntHelp = loadHelp();
-        Collections.sort(ntHelp.commands, new Comparator<NodeToolHelp.NodeToolCommand>() 
+        Collections.sort(ntHelp.commands, new Comparator<NodeToolHelp.NodeToolCommand>()
         {
             @Override
-            public int compare(NodeToolHelp.NodeToolCommand o1, NodeToolHelp.NodeToolCommand o2) 
+            public int compare(NodeToolHelp.NodeToolCommand o1, NodeToolHelp.NodeToolCommand o2)
             {
                 return o1.name.compareTo(o2.name);
             }
@@ -525,7 +528,7 @@ public class NodeCmd
         }
     }
 
-    private Map<String, SetHostStat> getOwnershipByDc(boolean resolveIp, Map<String, String> tokenToEndpoint, 
+    private Map<String, SetHostStat> getOwnershipByDc(boolean resolveIp, Map<String, String> tokenToEndpoint,
                                                       Map<InetAddress, Float> ownerships) throws UnknownHostException
     {
         Map<String, SetHostStat> ownershipByDc = Maps.newLinkedHashMap();
@@ -574,7 +577,7 @@ public class NodeCmd
         public final Float owns;
         public final String token;
 
-        public HostStat(String token, InetAddress endpoint, boolean resolveIp, Float owns) 
+        public HostStat(String token, InetAddress endpoint, boolean resolveIp, Float owns)
         {
             this.token = token;
             this.endpoint = endpoint;
@@ -1668,7 +1671,11 @@ public class NodeCmd
             switch (nc)
             {
                 case REPAIR  :
-                    boolean sequential = !cmd.hasOption(PARALLEL_REPAIR_OPT.left);
+                    RepairParallelism parallelismDegree = RepairParallelism.SEQUENTIAL;
+                    if (cmd.hasOption(PARALLEL_REPAIR_OPT.left))
+                        parallelismDegree = RepairParallelism.PARALLEL;
+                    else if (cmd.hasOption(DCPARALLEL_REPAIR_OPT.left))
+                        parallelismDegree = RepairParallelism.DATACENTER_AWARE;
                     boolean localDC = cmd.hasOption(LOCAL_DC_REPAIR_OPT.left);
                     boolean specificDC = cmd.hasOption(DC_REPAIR_OPT.left);
                     boolean specificHosts = cmd.hasOption(HOST_REPAIR_OPT.left);
@@ -1686,9 +1693,9 @@ public class NodeCmd
                     else if(specificHosts)
                         hosts  = Arrays.asList(cmd.getOptionValue(HOST_REPAIR_OPT.left).split(","));
                     if (cmd.hasOption(START_TOKEN_OPT.left) || cmd.hasOption(END_TOKEN_OPT.left))
-                        probe.forceRepairRangeAsync(System.out, keyspace, sequential, dataCenters, hosts, cmd.getOptionValue(START_TOKEN_OPT.left), cmd.getOptionValue(END_TOKEN_OPT.left), columnFamilies);
+                        probe.forceRepairRangeAsync(System.out, keyspace, parallelismDegree, dataCenters, hosts, cmd.getOptionValue(START_TOKEN_OPT.left), cmd.getOptionValue(END_TOKEN_OPT.left), columnFamilies);
                     else
-                        probe.forceRepairAsync(System.out, keyspace, sequential, dataCenters, hosts, primaryRange, columnFamilies);
+                        probe.forceRepairAsync(System.out, keyspace, parallelismDegree, dataCenters, hosts, primaryRange, columnFamilies);
                     break;
                 case FLUSH   :
                     try { probe.forceKeyspaceFlush(keyspace, columnFamilies); }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 849e368..261d416 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -54,6 +54,7 @@ import org.apache.cassandra.gms.FailureDetectorMBean;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.MessagingServiceMBean;
+import org.apache.cassandra.repair.RepairParallelism;
 import org.apache.cassandra.service.*;
 import org.apache.cassandra.streaming.StreamState;
 import org.apache.cassandra.streaming.StreamManagerMBean;
@@ -217,12 +218,17 @@ public class NodeProbe
 
     public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts,  boolean primaryRange, String... columnFamilies) throws IOException
     {
+        forceRepairAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, columnFamilies);
+    }
+
+    public void forceRepairAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts,  boolean primaryRange, String... columnFamilies) throws IOException
+    {
         RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
         try
         {
             jmxc.addConnectionNotificationListener(runner, null, null);
             ssProxy.addNotificationListener(runner, null, null);
-            if (!runner.repairAndWait(ssProxy, isSequential, dataCenters, hosts, primaryRange))
+            if (!runner.repairAndWait(ssProxy, parallelismDegree, dataCenters, hosts, primaryRange))
                 failed = true;
         }
         catch (Exception e)
@@ -239,15 +245,19 @@ public class NodeProbe
             catch (Throwable ignored) {}
         }
     }
-
     public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, String... columnFamilies) throws IOException
     {
+        forceRepairRangeAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, startToken, endToken, columnFamilies);
+    }
+
+    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, String... columnFamilies) throws IOException
+    {
         RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
         try
         {
             jmxc.addConnectionNotificationListener(runner, null, null);
             ssProxy.addNotificationListener(runner, null, null);
-            if (!runner.repairRangeAndWait(ssProxy,  isSequential, dataCenters, hosts, startToken, endToken))
+            if (!runner.repairRangeAndWait(ssProxy, parallelismDegree, dataCenters, hosts, startToken, endToken))
                 failed = true;
         }
         catch (Exception e)
@@ -896,17 +906,17 @@ public class NodeProbe
     {
         return failed;
     }
-    
+
     public long getReadRepairAttempted()
     {
         return spProxy.getReadRepairAttempted();
     }
-    
+
     public long getReadRepairRepairedBlocking()
     {
         return spProxy.getReadRepairRepairedBlocking();
     }
-    
+
     public long getReadRepairRepairedBackground()
     {
         return spProxy.getReadRepairRepairedBackground();
@@ -1060,16 +1070,16 @@ class RepairRunner implements NotificationListener
         this.columnFamilies = columnFamilies;
     }
 
-    public boolean repairAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRangeOnly) throws Exception
+    public boolean repairAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRangeOnly) throws Exception
     {
-        cmd = ssProxy.forceRepairAsync(keyspace, isSequential, dataCenters, hosts, primaryRangeOnly, columnFamilies);
+        cmd = ssProxy.forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, primaryRangeOnly, columnFamilies);
         waitForRepair();
         return success;
     }
 
-    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, String startToken, String endToken) throws Exception
+    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, String startToken, String endToken) throws Exception
     {
-        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, isSequential, dataCenters, hosts, columnFamilies);
+        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, parallelismDegree, dataCenters, hosts, columnFamilies);
         waitForRepair();
         return success;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml b/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
index 523335e..b254bac 100644
--- a/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
+++ b/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
@@ -155,6 +155,7 @@ commands:
     help: |
       Repair one or more column families
          Use -dc to repair specific datacenters (csv list).
+         Use -dcpar to repair datacenters in parallel.
          Use -et to specify a token at which repair range ends.
          Use -local to only repair against nodes in the same datacenter.
          Use -pr to repair only the first range returned by the partitioner.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java b/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java
new file mode 100644
index 0000000..ad6eabe
--- /dev/null
+++ b/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+
+public class RequestCoordinatorTest implements IRequestProcessor<InetAddress>
+{
+    private InetAddress[] endpoints;
+    private List<InetAddress> activeRequests;
+    private static Random random = new Random(0);
+
+    @Before
+    public void setup() throws UnknownHostException
+    {
+        endpoints = new InetAddress[12];
+        for (int i = 0; i < 12; i++)
+            endpoints[i] = InetAddress.getByName("127.0.0." + (i + 1));
+        activeRequests = new ArrayList<>();
+        DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
+        {
+            @Override
+            public String getRack(InetAddress endpoint)
+            {
+                return "rack1";
+            }
+
+            @Override
+            public String getDatacenter(InetAddress endpoint)
+            {
+                // 127.0.0.1, 127.0.0.2, 127.0.0.3 -> datacenter1
+                // 127.0.0.4, 127.0.0.5, 127.0.0.6 -> datacenter2 etc
+                int no = endpoint.getAddress()[3] - 1;
+                return "datacenter" + (no / 3 + 1);
+            }
+
+            @Override
+            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+            {
+                return 0;
+            }
+        });
+    }
+
+    @Override
+    public void process(InetAddress request)
+    {
+        activeRequests.add(request);
+    }
+
+    @Test
+    public void testSequentialRequestCoordinator()
+    {
+        SequentialRequestCoordinator<InetAddress> coordinator = new SequentialRequestCoordinator<>(this);
+        for (InetAddress endpoint : endpoints)
+            coordinator.add(endpoint);
+        coordinator.start();
+        int max = finishRequests(coordinator);
+        assertEquals(1, max);
+    }
+
+    @Test
+    public void testParallelRequestCoordinator()
+    {
+        ParallelRequestCoordinator<InetAddress> coordinator = new ParallelRequestCoordinator<>(this);
+        for (InetAddress endpoint : endpoints)
+            coordinator.add(endpoint);
+        coordinator.start();
+        int max = finishRequests(coordinator);
+        assertEquals(endpoints.length, max);
+    }
+
+    @Test
+    public void testDatacenterAwareRequestCoordinator()
+    {
+        DatacenterAwareRequestCoordinator coordinator = new DatacenterAwareRequestCoordinator(this);
+        for (InetAddress endpoint : endpoints)
+            coordinator.add(endpoint);
+        coordinator.start();
+        int max = finishRequests(coordinator);
+        assertEquals(4, max);
+    }
+
+    private int finishRequests(IRequestCoordinator<InetAddress> requestCoordinator)
+    {
+        int max = 0;
+        while (activeRequests.size() > 0)
+        {
+            max = Math.max(max, activeRequests.size());
+            // Finish a request
+            int ix = random.nextInt(activeRequests.size());
+            InetAddress finished = activeRequests.get(ix);
+            activeRequests.remove(ix);
+            requestCoordinator.completed(finished);
+        }
+        return max;
+    }
+}


[8/8] cassandra git commit: Merge branch 'cassandra-2.1' into trunk

Posted by yu...@apache.org.
Merge branch 'cassandra-2.1' into trunk

Conflicts:
	src/java/org/apache/cassandra/repair/RepairJob.java
	src/java/org/apache/cassandra/repair/RepairSession.java
	src/java/org/apache/cassandra/service/ActiveRepairService.java
	src/java/org/apache/cassandra/service/StorageService.java
	src/java/org/apache/cassandra/service/StorageServiceMBean.java
	src/java/org/apache/cassandra/tools/NodeProbe.java
	src/java/org/apache/cassandra/tools/NodeTool.java


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c023d492
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c023d492
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c023d492

Branch: refs/heads/trunk
Commit: c023d4922863bd4e7d3c959035a8634cd370a829
Parents: 5841131 326a9ff
Author: Yuki Morishita <yu...@apache.org>
Authored: Mon Nov 24 15:25:55 2014 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Mon Nov 24 15:25:55 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../org/apache/cassandra/repair/RepairJob.java  | 123 ++++++++++++++++---
 .../cassandra/repair/RepairParallelism.java     |  57 +++++++++
 .../apache/cassandra/repair/RepairSession.java  |  10 +-
 .../cassandra/repair/messages/RepairOption.java |  27 ++--
 .../cassandra/service/ActiveRepairService.java  |   5 +-
 .../cassandra/service/StorageService.java       |  42 +++++--
 .../cassandra/service/StorageServiceMBean.java  |  10 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   7 +-
 .../org/apache/cassandra/tools/NodeTool.java    |  11 +-
 .../cassandra/repair/RepairSessionTest.java     |   2 +-
 .../repair/messages/RepairOptionTest.java       |   8 +-
 12 files changed, 249 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/src/java/org/apache/cassandra/repair/RepairJob.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/repair/RepairJob.java
index 0b40d4a,20d5d97..34b4217
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@@ -18,18 -18,22 +18,17 @@@
  package org.apache.cassandra.repair;
  
  import java.net.InetAddress;
- import java.util.ArrayList;
- import java.util.Collection;
- import java.util.List;
+ import java.util.*;
 -import java.util.concurrent.atomic.AtomicInteger;
 -import java.util.concurrent.locks.Condition;
  
  import com.google.common.util.concurrent.*;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
++import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.Keyspace;
 -import org.apache.cassandra.dht.Range;
 -import org.apache.cassandra.dht.Token;
 -import org.apache.cassandra.net.MessagingService;
 -import org.apache.cassandra.repair.messages.ValidationRequest;
 +import org.apache.cassandra.gms.FailureDetector;
  import org.apache.cassandra.utils.FBUtilities;
 -import org.apache.cassandra.utils.MerkleTree;
 -import org.apache.cassandra.utils.concurrent.SimpleCondition;
 +import org.apache.cassandra.utils.Pair;
  
  /**
   * RepairJob runs repair on given ColumnFamily.
@@@ -38,49 -42,87 +37,50 @@@ public class RepairJob extends Abstract
  {
      private static Logger logger = LoggerFactory.getLogger(RepairJob.class);
  
 -    public final RepairJobDesc desc;
 +    private final RepairSession session;
 +    private final RepairJobDesc desc;
-     private final boolean isSequential;
+     private final RepairParallelism parallelismDegree;
 -    // first we send tree requests. this tracks the endpoints remaining to hear from
 -    private final IRequestCoordinator<InetAddress> treeRequests;
 -    // tree responses are then tracked here
 -    private final List<TreeResponse> trees = new ArrayList<>();
 -    // once all responses are received, each tree is compared with each other, and differencer tasks
 -    // are submitted. the job is done when all differencers are complete.
 +    private final long repairedAt;
      private final ListeningExecutorService taskExecutor;
  
      /**
       * Create repair job to run on specific columnfamily
 +     *
 +     * @param session RepairSession that this RepairJob belongs
 +     * @param columnFamily name of the ColumnFamily to repair
-      * @param isSequential when true, validation runs sequentially among replica
++     * @param parallelismDegree how to run repair job in parallel
 +     * @param repairedAt when the repair occurred (millis)
 +     * @param taskExecutor Executor to run various repair tasks
       */
 -    public RepairJob(IRepairJobEventListener listener,
 -                     UUID parentSessionId,
 -                     UUID sessionId,
 -                     String keyspace,
 +    public RepairJob(RepairSession session,
                       String columnFamily,
-                      boolean isSequential,
 -                     Range<Token> range,
+                      RepairParallelism parallelismDegree,
 +                     long repairedAt,
                       ListeningExecutorService taskExecutor)
      {
 -        this.listener = listener;
 -        this.desc = new RepairJobDesc(parentSessionId, sessionId, keyspace, columnFamily, range);
 -        this.parallelismDegree = parallelismDegree;
 +        this.session = session;
 +        this.desc = new RepairJobDesc(session.parentRepairSession, session.getId(), session.keyspace, columnFamily, session.getRange());
-         this.isSequential = isSequential;
 +        this.repairedAt = repairedAt;
          this.taskExecutor = taskExecutor;
 -
 -        IRequestProcessor<InetAddress> processor = new IRequestProcessor<InetAddress>()
 -        {
 -            @Override
 -            public void process(InetAddress endpoint)
 -            {
 -                ValidationRequest request = new ValidationRequest(desc, gcBefore);
 -                MessagingService.instance().sendOneWay(request.createMessage(), endpoint);
 -            }
 -        };
 -
 -        switch (parallelismDegree)
 -        {
 -            case SEQUENTIAL:
 -                this.treeRequests = new SequentialRequestCoordinator<>(processor);
 -                break;
 -            case PARALLEL:
 -                this.treeRequests = new ParallelRequestCoordinator<>(processor);
 -                break;
 -            case DATACENTER_AWARE:
 -                this.treeRequests = new DatacenterAwareRequestCoordinator(processor);
 -                break;
 -            default:
 -                throw new AssertionError("Unknown degree of parallelism specified");
 -        }
 -    }
 -
 -    /**
 -     * @return true if this job failed
 -     */
 -    public boolean isFailed()
 -    {
 -        return failed;
++        this.parallelismDegree = parallelismDegree;
      }
  
      /**
 -     * Send merkle tree request to every involved neighbor.
 +     * Runs repair job.
 +     *
 +     * This sets up necessary task and runs them on given {@code taskExecutor}.
 +     * After submitting all tasks, waits until validation with replica completes.
       */
 -    public void sendTreeRequests(Collection<InetAddress> endpoints)
 +    public void run()
      {
 -        // send requests to all nodes
 -        List<InetAddress> allEndpoints = new ArrayList<>(endpoints);
 +        List<InetAddress> allEndpoints = new ArrayList<>(session.endpoints);
          allEndpoints.add(FBUtilities.getBroadcastAddress());
  
 +        ListenableFuture<List<TreeResponse>> validations;
-         if (isSequential)
+         // Create a snapshot at all nodes unless we're using pure parallel repairs
+         if (parallelismDegree != RepairParallelism.PARALLEL)
          {
 +            // Request snapshot to all replica
              List<ListenableFuture<InetAddress>> snapshotTasks = new ArrayList<>(allEndpoints.size());
              for (InetAddress endpoint : allEndpoints)
              {
@@@ -88,110 -130,102 +88,201 @@@
                  snapshotTasks.add(snapshotTask);
                  taskExecutor.execute(snapshotTask);
              }
 +            // When all snapshot complete, send validation requests
              ListenableFuture<List<InetAddress>> allSnapshotTasks = Futures.allAsList(snapshotTasks);
 -            // Execute send tree request after all snapshot complete
 -            Futures.addCallback(allSnapshotTasks, new FutureCallback<List<InetAddress>>()
 +            validations = Futures.transform(allSnapshotTasks, new AsyncFunction<List<InetAddress>, List<TreeResponse>>()
              {
 -                public void onSuccess(List<InetAddress> endpoints)
 -                {
 -                    sendTreeRequestsInternal(endpoints);
 -                }
 -
 -                public void onFailure(Throwable throwable)
 +                public ListenableFuture<List<TreeResponse>> apply(List<InetAddress> endpoints) throws Exception
                  {
-                     return sendValidationRequest(endpoints);
 -                    // TODO need to propagate error to RepairSession
 -                    logger.error("Error occurred during snapshot phase", throwable);
 -                    listener.failedSnapshot();
 -                    failed = true;
++                    logger.info(String.format("[repair #%s] requesting merkle trees for %s (to %s)", desc.sessionId, desc.columnFamily, endpoints));
++                    if (parallelismDegree == RepairParallelism.SEQUENTIAL)
++                        return sendSequentialValidationRequest(endpoints);
++                    else
++                        return sendDCAwareValidationRequest(endpoints);
                  }
              }, taskExecutor);
          }
          else
          {
 -            sendTreeRequestsInternal(allEndpoints);
++            logger.info(String.format("[repair #%s] requesting merkle trees for %s (to %s)", desc.sessionId, desc.columnFamily, allEndpoints));
 +            // If not sequential, just send validation request to all replica
 +            validations = sendValidationRequest(allEndpoints);
          }
 -    }
  
 -    private void sendTreeRequestsInternal(Collection<InetAddress> endpoints)
 -    {
 -        this.gcBefore = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).gcBefore(System.currentTimeMillis());
 -        for (InetAddress endpoint : endpoints)
 -            treeRequests.add(endpoint);
 +        // When all validations complete, submit sync tasks
 +        ListenableFuture<List<SyncStat>> syncResults = Futures.transform(validations, new AsyncFunction<List<TreeResponse>, List<SyncStat>>()
 +        {
 +            public ListenableFuture<List<SyncStat>> apply(List<TreeResponse> trees) throws Exception
 +            {
 +                // Unregister from FailureDetector once we've completed synchronizing Merkle trees.
 +                // After this point, we rely on tcp_keepalive for individual sockets to notify us when a connection is down.
 +                // See CASSANDRA-3569
 +                FailureDetector.instance.unregisterFailureDetectionEventListener(session);
 +
 +                InetAddress local = FBUtilities.getLocalAddress();
 +
 +                List<SyncTask> syncTasks = new ArrayList<>();
 +                // We need to difference all trees one against another
 +                for (int i = 0; i < trees.size() - 1; ++i)
 +                {
 +                    TreeResponse r1 = trees.get(i);
 +                    for (int j = i + 1; j < trees.size(); ++j)
 +                    {
 +                        TreeResponse r2 = trees.get(j);
 +                        SyncTask task;
 +                        if (r1.endpoint.equals(local) || r2.endpoint.equals(local))
 +                        {
 +                            task = new LocalSyncTask(desc, r1, r2, repairedAt);
 +                        }
 +                        else
 +                        {
 +                            task = new RemoteSyncTask(desc, r1, r2);
 +                            // RemoteSyncTask expects SyncComplete message sent back.
 +                            // Register task to RepairSession to receive response.
 +                            session.waitForSync(Pair.create(desc, new NodePair(r1.endpoint, r2.endpoint)), (RemoteSyncTask) task);
 +                        }
 +                        syncTasks.add(task);
 +                        taskExecutor.submit(task);
 +                    }
 +                }
 +                return Futures.allAsList(syncTasks);
 +            }
 +        }, taskExecutor);
  
 -        logger.info(String.format("[repair #%s] requesting merkle trees for %s (to %s)", desc.sessionId, desc.columnFamily, endpoints));
 -        treeRequests.start();
 -        requestsSent.signalAll();
 +        // When all sync complete, set the final result
 +        Futures.addCallback(syncResults, new FutureCallback<List<SyncStat>>()
 +        {
 +            public void onSuccess(List<SyncStat> stats)
 +            {
 +                logger.info(String.format("[repair #%s] %s is fully synced", session.getId(), desc.columnFamily));
 +                set(new RepairResult(desc, stats));
 +            }
 +
 +            /**
 +             * Snapshot, validation and sync failures are all handled here
 +             */
 +            public void onFailure(Throwable t)
 +            {
 +                logger.warn(String.format("[repair #%s] %s sync failed", session.getId(), desc.columnFamily));
 +                setException(t);
 +            }
 +        }, taskExecutor);
 +
 +        // Wait for validation to complete
 +        Futures.getUnchecked(validations);
      }
  
      /**
-      * Creates {@link ValidationTask} and submit them to task executor.
-      * If isSequential flag is true, wait previous ValidationTask to complete before submitting the next.
 -     * Add a new received tree and return the number of remaining tree to
 -     * be received for the job to be complete.
++     * Creates {@link ValidationTask} and submit them to task executor in parallel.
       *
 -     * Callers may assume exactly one addTree call will result in zero remaining endpoints.
 -     *
 -     * @param endpoint address of the endpoint that sent response
 -     * @param tree sent Merkle tree or null if validation failed on endpoint
 -     * @return the number of responses waiting to receive
 +     * @param endpoints Endpoint addresses to send validation request
 +     * @return Future that can get all {@link TreeResponse} from replica, if all validation succeed.
       */
 -    public synchronized int addTree(InetAddress endpoint, MerkleTree tree)
 +    private ListenableFuture<List<TreeResponse>> sendValidationRequest(Collection<InetAddress> endpoints)
      {
-         logger.info(String.format("[repair #%s] requesting merkle trees for %s (to %s)", desc.sessionId, desc.columnFamily, endpoints));
 -        // Wait for all request to have been performed (see #3400)
 -        try
 -        {
 -            requestsSent.await();
 -        }
 -        catch (InterruptedException e)
 +        int gcBefore = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).gcBefore(System.currentTimeMillis());
 +        List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
 +        for (InetAddress endpoint : endpoints)
          {
 -            throw new AssertionError("Interrupted while waiting for requests to be sent");
 +            ValidationTask task = new ValidationTask(desc, endpoint, gcBefore);
 +            tasks.add(task);
 +            session.waitForValidation(Pair.create(desc, endpoint), task);
 +            taskExecutor.execute(task);
-             if (isSequential)
+         }
 -
 -        if (tree == null)
 -            failed = true;
 -        else
 -            trees.add(new TreeResponse(endpoint, tree));
 -        return treeRequests.completed(endpoint);
++        return Futures.allAsList(tasks);
+     }
+ 
+     /**
 -     * Submit differencers for running.
 -     * All tree *must* have been received before this is called.
++     * Creates {@link ValidationTask} and submit them to task executor so that tasks run sequentially.
+      */
 -    public void submitDifferencers()
++    private ListenableFuture<List<TreeResponse>> sendSequentialValidationRequest(Collection<InetAddress> endpoints)
+     {
 -        assert !failed;
 -        List<Differencer> differencers = new ArrayList<>();
 -        // We need to difference all trees one against another
 -        for (int i = 0; i < trees.size() - 1; ++i)
++        int gcBefore = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).gcBefore(System.currentTimeMillis());
++        List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
++
++        Queue<InetAddress> requests = new LinkedList<>(endpoints);
++        InetAddress address = requests.poll();
++        ValidationTask firstTask = new ValidationTask(desc, address, gcBefore);
++        logger.info("Validating " + address);
++        session.waitForValidation(Pair.create(desc, address), firstTask);
++        tasks.add(firstTask);
++        ValidationTask currentTask = firstTask;
++        while (requests.size() > 0)
+         {
 -            TreeResponse r1 = trees.get(i);
 -            for (int j = i + 1; j < trees.size(); ++j)
++            final InetAddress nextAddress = requests.poll();
++            final ValidationTask nextTask = new ValidationTask(desc, nextAddress, gcBefore);
++            tasks.add(nextTask);
++            Futures.addCallback(currentTask, new FutureCallback<TreeResponse>()
+             {
 -                TreeResponse r2 = trees.get(j);
 -                Differencer differencer = new Differencer(desc, r1, r2);
 -                differencers.add(differencer);
 -                logger.debug("Queueing comparison {}", differencer);
 -            }
 -        }
 -        waitForSync = new AtomicInteger(differencers.size());
 -        for (Differencer differencer : differencers)
 -            taskExecutor.submit(differencer);
++                public void onSuccess(TreeResponse result)
++                {
++                    logger.info("Validating " + nextAddress);
++                    session.waitForValidation(Pair.create(desc, nextAddress), nextTask);
++                    taskExecutor.execute(nextTask);
++                }
+ 
 -        trees.clear(); // allows gc to do its thing
++                // failure is handled at root of job chain
++                public void onFailure(Throwable t) {}
++            });
++            currentTask = nextTask;
++        }
++        // start running tasks
++        taskExecutor.execute(firstTask);
++        return Futures.allAsList(tasks);
+     }
+ 
+     /**
 -     * @return true if the given node pair was the last remaining
++     * Creates {@link ValidationTask} and submit them to task executor so that tasks run sequentially within each dc.
+      */
 -    boolean completedSynchronization()
++    private ListenableFuture<List<TreeResponse>> sendDCAwareValidationRequest(Collection<InetAddress> endpoints)
+     {
 -        return waitForSync.decrementAndGet() == 0;
++        int gcBefore = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).gcBefore(System.currentTimeMillis());
++        List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
++
++        Map<String, Queue<InetAddress>> requestsByDatacenter = new HashMap<>();
++        for (InetAddress endpoint : endpoints)
++        {
++            String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(endpoint);
++            Queue<InetAddress> queue = requestsByDatacenter.get(dc);
++            if (queue == null)
 +            {
-                 // tasks are sequentially sent so wait until current validation is done.
-                 // NOTE: Wait happens on taskExecutor thread
-                 Futures.getUnchecked(task);
++                queue = new LinkedList<>();
++                requestsByDatacenter.put(dc, queue);
++            }
++            queue.add(endpoint);
++        }
++
++        for (Map.Entry<String, Queue<InetAddress>> entry : requestsByDatacenter.entrySet())
++        {
++            Queue<InetAddress> requests = entry.getValue();
++            InetAddress address = requests.poll();
++            ValidationTask firstTask = new ValidationTask(desc, address, gcBefore);
++            logger.info("Validating " + address);
++            session.waitForValidation(Pair.create(desc, address), firstTask);
++            tasks.add(firstTask);
++            ValidationTask currentTask = firstTask;
++            while (requests.size() > 0)
++            {
++                final InetAddress nextAddress = requests.poll();
++                final ValidationTask nextTask = new ValidationTask(desc, nextAddress, gcBefore);
++                tasks.add(nextTask);
++                Futures.addCallback(currentTask, new FutureCallback<TreeResponse>()
++                {
++                    public void onSuccess(TreeResponse result)
++                    {
++                        logger.info("Validating " + nextAddress);
++                        session.waitForValidation(Pair.create(desc, nextAddress), nextTask);
++                        taskExecutor.execute(nextTask);
++                    }
++
++                    // failure is handled at root of job chain
++                    public void onFailure(Throwable t) {}
++                });
++                currentTask = nextTask;
 +            }
++            // start running tasks
++            taskExecutor.execute(firstTask);
 +        }
 +        return Futures.allAsList(tasks);
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/src/java/org/apache/cassandra/repair/RepairParallelism.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/repair/RepairParallelism.java
index 0000000,12c22ca..2408cc1
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/repair/RepairParallelism.java
+++ b/src/java/org/apache/cassandra/repair/RepairParallelism.java
@@@ -1,0 -1,22 +1,57 @@@
+ package org.apache.cassandra.repair;
+ 
+ /**
+  * Specify the degree of parallelism when calculating the merkle trees in a repair job.
+  */
+ public enum RepairParallelism
+ {
+     /**
+      * One node at a time
+      */
 -    SEQUENTIAL,
++    SEQUENTIAL("sequential"),
+ 
+     /**
+      * All nodes at the same time
+      */
 -    PARALLEL,
++    PARALLEL("parallel"),
+ 
+     /**
+      * One node per data center at a time
+      */
 -    DATACENTER_AWARE
++    DATACENTER_AWARE("dc_parallel");
++
++    private final String name;
++
++    /**
++     * Return RepairParallelism that match given name.
++     * If name is null, or does not match any, this returns default "sequential" parallelism,
++     *
++     * @param name name of repair parallelism
++     * @return RepairParallelism that match given name
++     */
++    public static RepairParallelism fromName(String name)
++    {
++        if (PARALLEL.getName().equals(name))
++            return PARALLEL;
++        else if (DATACENTER_AWARE.getName().equals(name))
++            return DATACENTER_AWARE;
++        else
++            return SEQUENTIAL;
++    }
++
++    private RepairParallelism(String name)
++    {
++        this.name = name;
++    }
++
++    public String getName()
++    {
++        return name;
++    }
++
++    @Override
++    public String toString()
++    {
++        return getName();
++    }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/src/java/org/apache/cassandra/repair/RepairSession.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/repair/RepairSession.java
index cc46dbe,0580ebb..d9787e2
--- a/src/java/org/apache/cassandra/repair/RepairSession.java
+++ b/src/java/org/apache/cassandra/repair/RepairSession.java
@@@ -106,34 -112,27 +106,34 @@@ public class RepairSession extends Abst
      /**
       * Create new repair session.
       *
 +     * @param parentRepairSession the parent sessions id
 +     * @param id this sessions id
       * @param range range to repair
       * @param keyspace name of keyspace
-      * @param isSequential true if performing repair on snapshots sequentially
+      * @param parallelismDegree specifies the degree of parallelism when calculating the merkle trees
       * @param endpoints the data centers that should be part of the repair; null for all DCs
 +     * @param repairedAt when the repair occurred (millis)
       * @param cfnames names of columnfamilies
       */
 -    public RepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String... cfnames)
 +    public RepairSession(UUID parentRepairSession,
 +                         UUID id,
 +                         Range<Token> range,
 +                         String keyspace,
-                          boolean isSequential,
++                         RepairParallelism parallelismDegree,
 +                         Set<InetAddress> endpoints,
 +                         long repairedAt,
 +                         String... cfnames)
      {
 -        this(parentRepairSession, UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, endpoints, cfnames);
 -    }
 +        assert cfnames.length > 0 : "Repairing no column families seems pointless, doesn't it";
  
 -    public RepairSession(UUID parentRepairSession, UUID id, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String[] cfnames)
 -    {
          this.parentRepairSession = parentRepairSession;
          this.id = id;
-         this.isSequential = isSequential;
+         this.parallelismDegree = parallelismDegree;
          this.keyspace = keyspace;
          this.cfnames = cfnames;
 -        assert cfnames.length > 0 : "Repairing no column families seems pointless, doesn't it";
          this.range = range;
          this.endpoints = endpoints;
 +        this.repairedAt = repairedAt;
      }
  
      public UUID getId()
@@@ -239,34 -278,48 +239,34 @@@
              }
          }
  
 -        ActiveRepairService.instance.addToActiveSessions(this);
 -        try
 +        // Create and submit RepairJob for each ColumnFamily
 +        List<ListenableFuture<RepairResult>> jobs = new ArrayList<>(cfnames.length);
 +        for (String cfname : cfnames)
          {
-             RepairJob job = new RepairJob(this, cfname, isSequential, repairedAt, taskExecutor);
 -            // Create and queue a RepairJob for each column family
 -            for (String cfname : cfnames)
 -            {
 -                RepairJob job = new RepairJob(this, parentRepairSession, id, keyspace, cfname, range, parallelismDegree, taskExecutor);
 -                jobs.offer(job);
 -            }
 -            logger.debug("Sending tree requests to endpoints {}", endpoints);
 -            jobs.peek().sendTreeRequests(endpoints);
 -
 -            // block whatever thread started this session until all requests have been returned:
 -            // if this thread dies, the session will still complete in the background
 -            completed.await();
++            RepairJob job = new RepairJob(this, cfname, parallelismDegree, repairedAt, taskExecutor);
 +            executor.execute(job);
 +            jobs.add(job);
 +        }
  
 -            if (exception == null)
 +        // When all RepairJobs are done without error, cleanup and set the final result
 +        Futures.addCallback(Futures.allAsList(jobs), new FutureCallback<List<RepairResult>>()
 +        {
 +            public void onSuccess(List<RepairResult> results)
              {
 +                // this repair session is completed
                  logger.info(String.format("[repair #%s] session completed successfully", getId()));
 +                set(new RepairSessionResult(id, keyspace, range, results));
 +                taskExecutor.shutdown();
 +                // mark this session as terminated
 +                terminate();
              }
 -            else
 +
 +            public void onFailure(Throwable t)
              {
 -                logger.error(String.format("[repair #%s] session completed with the following error", getId()), exception);
 -                throw exception;
 +                logger.error("Repair job failed", t);
 +                setException(t);
              }
 -        }
 -        catch (InterruptedException e)
 -        {
 -            throw new RuntimeException("Interrupted while waiting for repair.");
 -        }
 -        finally
 -        {
 -            // mark this session as terminated
 -            terminate();
 -
 -            ActiveRepairService.instance.removeFromActiveSessions(this);
 -
 -            // If we've reached here in an exception state without completing Merkle Tree sync, we'll still be registered
 -            // with the FailureDetector.
 -            if (fdUnregistered.compareAndSet(false, true))
 -                FailureDetector.instance.unregisterFailureDetectionEventListener(this);
 -        }
 +        });
      }
  
      public void terminate()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/src/java/org/apache/cassandra/repair/messages/RepairOption.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/repair/messages/RepairOption.java
index 3f59d7b,0000000..d11f94f
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/repair/messages/RepairOption.java
+++ b/src/java/org/apache/cassandra/repair/messages/RepairOption.java
@@@ -1,275 -1,0 +1,276 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.repair.messages;
 +
 +import java.util.*;
 +
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.dht.IPartitioner;
 +import org.apache.cassandra.dht.Range;
 +import org.apache.cassandra.dht.Token;
- import org.apache.cassandra.utils.FBUtilities;
++import org.apache.cassandra.repair.RepairParallelism;
 +
 +/**
 + * Repair options.
 + */
 +public class RepairOption
 +{
-     public static final String SEQUENTIAL_KEY = "sequential";
++    public static final String PARALLELISM_KEY = "parallelism";
 +    public static final String PRIMARY_RANGE_KEY = "primaryRange";
 +    public static final String INCREMENTAL_KEY = "incremental";
 +    public static final String JOB_THREADS_KEY = "jobThreads";
 +    public static final String RANGES_KEY = "ranges";
 +    public static final String COLUMNFAMILIES_KEY = "columnFamilies";
 +    public static final String DATACENTERS_KEY = "dataCenters";
 +    public static final String HOSTS_KEY = "hosts";
 +
 +    // we don't want to push nodes too much for repair
 +    public static final int MAX_JOB_THREADS = 4;
 +
 +    private static final Logger logger = LoggerFactory.getLogger(RepairOption.class);
 +
 +    /**
 +     * Construct RepairOptions object from given map of Strings.
 +     * <p>
 +     * Available options are:
 +     *
 +     * <table>
 +     *     <thead>
 +     *         <tr>
 +     *             <th>key</th>
 +     *             <th>value</th>
 +     *             <th>default (when key not given)</th>
 +     *         </tr>
 +     *     </thead>
 +     *     <tbody>
 +     *         <tr>
-      *             <td>sequential</td>
-      *             <td>"true" if perform sequential repair.</td>
-      *             <td>true</td>
++     *             <td>parallelism</td>
++     *             <td>"sequential", "parallel" or "dc_parallel"</td>
++     *             <td>"sequential"</td>
 +     *         </tr>
 +     *         <tr>
 +     *             <td>primaryRange</td>
 +     *             <td>"true" if perform repair only on primary range.</td>
 +     *             <td>false</td>
 +     *         </tr>
 +     *         <tr>
 +     *             <td>incremental</td>
 +     *             <td>"true" if perform incremental repair.</td>
 +     *             <td>false</td>
 +     *         </tr>
 +     *         <tr>
 +     *             <td>jobThreads</td>
 +     *             <td>Number of threads to use to run repair job.</td>
 +     *             <td>1</td>
 +     *         </tr>
 +     *         <tr>
 +     *             <td>ranges</td>
 +     *             <td>Ranges to repair. A range is expressed as &lt;start token&gt;:&lt;end token&gt;
 +     *             and multiple ranges can be given as comma separated ranges(e.g. aaa:bbb,ccc:ddd).</td>
 +     *             <td></td>
 +     *         </tr>
 +     *         <tr>
 +     *             <td>columnFamilies</td>
 +     *             <td>Specify names of ColumnFamilies to repair.
 +     *             Multiple ColumnFamilies can be given as comma separated values(e.g. cf1,cf2,cf3).</td>
 +     *             <td></td>
 +     *         </tr>
 +     *         <tr>
 +     *             <td>dataCenters</td>
 +     *             <td>Specify names of data centers who participate in this repair.
 +     *             Multiple data centers can be given as comma separated values(e.g. dc1,dc2,dc3).</td>
 +     *             <td></td>
 +     *         </tr>
 +     *         <tr>
 +     *             <td>hosts</td>
 +     *             <td>Specify names of hosts who participate in this repair.
 +     *             Multiple hosts can be given as comma separated values(e.g. cass1,cass2).</td>
 +     *             <td></td>
 +     *         </tr>
 +     *     </tbody>
 +     * </table>
 +     *
 +     * @param options options to parse
 +     * @param partitioner partitioner is used to construct token ranges
 +     * @return RepairOptions object
 +     */
 +    public static RepairOption parse(Map<String, String> options, IPartitioner partitioner)
 +    {
-         boolean sequential = !options.containsKey(SEQUENTIAL_KEY) || Boolean.parseBoolean(options.get(SEQUENTIAL_KEY));
++        // if no parallel option is given, then this will be "sequential" by default.
++        RepairParallelism parallelism = RepairParallelism.fromName(options.get(PARALLELISM_KEY));
 +        boolean primaryRange = Boolean.parseBoolean(options.get(PRIMARY_RANGE_KEY));
 +        boolean incremental = Boolean.parseBoolean(options.get(INCREMENTAL_KEY));
 +
 +        int jobThreads = 1;
 +        if (options.containsKey(JOB_THREADS_KEY))
 +        {
 +            try
 +            {
 +                jobThreads = Integer.parseInt(options.get(JOB_THREADS_KEY));
 +            }
 +            catch (NumberFormatException ignore) {}
 +        }
 +        // ranges
 +        String rangesStr = options.get(RANGES_KEY);
 +        Set<Range<Token>> ranges = new HashSet<>();
 +        if (rangesStr != null)
 +        {
 +            StringTokenizer tokenizer = new StringTokenizer(rangesStr, ",");
 +            while (tokenizer.hasMoreTokens())
 +            {
 +                String[] rangeStr = tokenizer.nextToken().split(":", 2);
 +                if (rangeStr.length < 2)
 +                {
 +                    continue;
 +                }
 +                Token parsedBeginToken = partitioner.getTokenFactory().fromString(rangeStr[0].trim());
 +                Token parsedEndToken = partitioner.getTokenFactory().fromString(rangeStr[1].trim());
 +                ranges.add(new Range<>(parsedBeginToken, parsedEndToken));
 +            }
 +        }
 +
-         RepairOption option = new RepairOption(sequential, primaryRange, incremental, jobThreads, ranges);
++        RepairOption option = new RepairOption(parallelism, primaryRange, incremental, jobThreads, ranges);
 +
 +        // data centers
 +        String dataCentersStr = options.get(DATACENTERS_KEY);
 +        Collection<String> dataCenters = new HashSet<>();
 +        if (dataCentersStr != null)
 +        {
 +            StringTokenizer tokenizer = new StringTokenizer(dataCentersStr, ",");
 +            while (tokenizer.hasMoreTokens())
 +            {
 +                dataCenters.add(tokenizer.nextToken().trim());
 +            }
 +            option.getDataCenters().addAll(dataCenters);
 +        }
 +
 +        // hosts
 +        String hostsStr = options.get(HOSTS_KEY);
 +        Collection<String> hosts = new HashSet<>();
 +        if (hostsStr != null)
 +        {
 +            StringTokenizer tokenizer = new StringTokenizer(hostsStr, ",");
 +            while (tokenizer.hasMoreTokens())
 +            {
 +                hosts.add(tokenizer.nextToken().trim());
 +            }
 +            option.getHosts().addAll(hosts);
 +        }
 +
 +        // columnfamilies
 +        String cfStr = options.get(COLUMNFAMILIES_KEY);
 +        if (cfStr != null)
 +        {
 +            Collection<String> columnFamilies = new HashSet<>();
 +            StringTokenizer tokenizer = new StringTokenizer(cfStr, ",");
 +            while (tokenizer.hasMoreTokens())
 +            {
 +                columnFamilies.add(tokenizer.nextToken().trim());
 +            }
 +            option.getColumnFamilies().addAll(columnFamilies);
 +        }
 +
 +        // validate options
 +        if (jobThreads > MAX_JOB_THREADS)
 +        {
 +            throw new IllegalArgumentException("Too many job threads. Max is " + MAX_JOB_THREADS);
 +        }
 +        if (primaryRange && (!dataCenters.isEmpty() || !hosts.isEmpty()))
 +        {
 +            throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
 +        }
 +
 +        return option;
 +    }
 +
-     private final boolean sequential;
++    private final RepairParallelism parallelism;
 +    private final boolean primaryRange;
 +    private final boolean incremental;
 +    private final int jobThreads;
 +
 +    private final Collection<String> columnFamilies = new HashSet<>();
 +    private final Collection<String> dataCenters = new HashSet<>();
 +    private final Collection<String> hosts = new HashSet<>();
 +    private final Collection<Range<Token>> ranges = new HashSet<>();
 +
-     public RepairOption(boolean sequential, boolean primaryRange, boolean incremental, int jobThreads, Collection<Range<Token>> ranges)
++    public RepairOption(RepairParallelism parallelism, boolean primaryRange, boolean incremental, int jobThreads, Collection<Range<Token>> ranges)
 +    {
-         this.sequential = sequential;
++        this.parallelism = parallelism;
 +        this.primaryRange = primaryRange;
 +        this.incremental = incremental;
 +        this.jobThreads = jobThreads;
 +        this.ranges.addAll(ranges);
 +    }
 +
-     public boolean isSequential()
++    public RepairParallelism getParallelism()
 +    {
-         return sequential;
++        return parallelism;
 +    }
 +
 +    public boolean isPrimaryRange()
 +    {
 +        return primaryRange;
 +    }
 +
 +    public boolean isIncremental()
 +    {
 +        return incremental;
 +    }
 +
 +    public int getJobThreads()
 +    {
 +        return jobThreads;
 +    }
 +
 +    public Collection<String> getColumnFamilies()
 +    {
 +        return columnFamilies;
 +    }
 +
 +    public Collection<Range<Token>> getRanges()
 +    {
 +        return ranges;
 +    }
 +
 +    public Collection<String> getDataCenters()
 +    {
 +        return dataCenters;
 +    }
 +
 +    public Collection<String> getHosts()
 +    {
 +        return hosts;
 +    }
 +
 +    @Override
 +    public String toString()
 +    {
 +        return "repair options (" +
-                        "sequential: " + sequential +
++                       "parallelism: " + parallelism +
 +                       ", primary range: " + primaryRange +
 +                       ", incremental: " + incremental +
 +                       ", job threads: " + jobThreads +
 +                       ", ColumnFamilies: " + columnFamilies +
 +                       ", dataCenters: " + dataCenters +
 +                       ", hosts: " + hosts +
 +                       ", # of ranges: " + ranges.size() +
 +                       ')';
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/ActiveRepairService.java
index 3c1cc48,d43143e..15d786e
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@@ -48,9 -48,12 +48,10 @@@ import org.apache.cassandra.net.IAsyncC
  import org.apache.cassandra.net.MessageIn;
  import org.apache.cassandra.net.MessageOut;
  import org.apache.cassandra.net.MessagingService;
 -import org.apache.cassandra.repair.*;
 -import org.apache.cassandra.repair.messages.AnticompactionRequest;
 -import org.apache.cassandra.repair.messages.PrepareMessage;
 -import org.apache.cassandra.repair.messages.RepairMessage;
 -import org.apache.cassandra.repair.messages.SyncComplete;
 -import org.apache.cassandra.repair.messages.ValidationComplete;
 +import org.apache.cassandra.repair.RepairJobDesc;
++import org.apache.cassandra.repair.RepairParallelism;
 +import org.apache.cassandra.repair.RepairSession;
 +import org.apache.cassandra.repair.messages.*;
  import org.apache.cassandra.utils.FBUtilities;
  import org.apache.cassandra.utils.UUIDGen;
  
@@@ -102,40 -116,27 +103,40 @@@ public class ActiveRepairServic
       *
       * @return Future for asynchronous call or null if there is no need to repair
       */
 -    public RepairFuture submitRepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String... cfnames)
 +    public RepairSession submitRepairSession(UUID parentRepairSession,
 +                                             Range<Token> range,
 +                                             String keyspace,
-                                              boolean isSequential,
++                                             RepairParallelism parallelismDegree,
 +                                             Set<InetAddress> endpoints,
 +                                             long repairedAt,
 +                                             ListeningExecutorService executor,
 +                                             String... cfnames)
      {
 -        RepairSession session = new RepairSession(parentRepairSession, range, keyspace, parallelismDegree, endpoints, cfnames);
 -        if (session.endpoints.isEmpty())
 +        if (endpoints.isEmpty())
              return null;
 -        RepairFuture futureTask = new RepairFuture(session);
 -        executor.execute(futureTask);
 -        return futureTask;
 -    }
  
-         final RepairSession session = new RepairSession(parentRepairSession, UUIDGen.getTimeUUID(), range, keyspace, isSequential, endpoints, repairedAt, cfnames);
 -    public void addToActiveSessions(RepairSession session)
 -    {
++        final RepairSession session = new RepairSession(parentRepairSession, UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, endpoints, repairedAt, cfnames);
 +
          sessions.put(session.getId(), session);
 -        Gossiper.instance.register(session);
 -        FailureDetector.instance.registerFailureDetectionEventListener(session);
 -    }
 +        // register listeners
 +        gossiper.register(session);
 +        failureDetector.registerFailureDetectionEventListener(session);
  
 -    public void removeFromActiveSessions(RepairSession session)
 -    {
 -        Gossiper.instance.unregister(session);
 -        sessions.remove(session.getId());
 +        // unregister listeners at completion
 +        session.addListener(new Runnable()
 +        {
 +            /**
 +             * When repair finished, do clean up
 +             */
 +            public void run()
 +            {
 +                failureDetector.unregisterFailureDetectionEventListener(session);
 +                gossiper.unregister(session);
 +                sessions.remove(session.getId());
 +            }
 +        }, MoreExecutors.sameThreadExecutor());
 +        session.start(executor);
 +        return session;
      }
  
      public void terminateSessions()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index d5fa22c,38cca10..fc9e4e5
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -74,10 -80,9 +74,11 @@@ import org.apache.cassandra.net.AsyncOn
  import org.apache.cassandra.net.MessageOut;
  import org.apache.cassandra.net.MessagingService;
  import org.apache.cassandra.net.ResponseVerbHandler;
 -import org.apache.cassandra.repair.RepairFuture;
  import org.apache.cassandra.repair.RepairMessageVerbHandler;
 +import org.apache.cassandra.repair.RepairSessionResult;
 +import org.apache.cassandra.repair.messages.RepairOption;
 +import org.apache.cassandra.repair.RepairSession;
+ import org.apache.cassandra.repair.RepairParallelism;
  import org.apache.cassandra.service.paxos.CommitVerbHandler;
  import org.apache.cassandra.service.paxos.PrepareVerbHandler;
  import org.apache.cassandra.service.paxos.ProposeVerbHandler;
@@@ -2417,128 -2490,112 +2418,153 @@@ public class StorageService extends Not
          sendNotification(jmxNotification);
      }
  
 -    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies) throws IOException
 +    public int repairAsync(String keyspace, Map<String, String> repairSpec)
      {
 -        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, fullRepair, columnFamilies);
 -    }
 -
 -    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies)
 -    {
 -        Collection<Range<Token>> ranges;
 -        if (primaryRange)
 +        RepairOption option = RepairOption.parse(repairSpec, getPartitioner());
 +        // if ranges are not specified
 +        if (option.getRanges().isEmpty())
          {
 -            // when repairing only primary range, neither dataCenters nor hosts can be set
 -            if (dataCenters == null && hosts == null)
 -                ranges = getPrimaryRanges(keyspace);
 -            // except dataCenters only contain local DC (i.e. -local)
 -            else if (dataCenters != null && dataCenters.size() == 1 && dataCenters.contains(DatabaseDescriptor.getLocalDataCenter()))
 -                ranges = getPrimaryRangesWithinDC(keyspace);
 +            if (option.isPrimaryRange())
 +            {
 +                // when repairing only primary range, neither dataCenters nor hosts can be set
 +                if (option.getDataCenters().isEmpty() && option.getHosts().isEmpty())
 +                    option.getRanges().addAll(getPrimaryRanges(keyspace));
 +                    // except dataCenters only contain local DC (i.e. -local)
 +                else if (option.getDataCenters().size() == 1 && option.getDataCenters().contains(DatabaseDescriptor.getLocalDataCenter()))
 +                    option.getRanges().addAll(getPrimaryRangesWithinDC(keyspace));
 +                else
 +                    throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
 +            }
              else
 -                throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
 -        }
 -        else
 -        {
 -             ranges = getLocalRanges(keyspace);
 +            {
 +                option.getRanges().addAll(getLocalRanges(keyspace));
 +            }
          }
 -
 -        return forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, ranges, fullRepair, columnFamilies);
 +        return forceRepairAsync(keyspace, option);
      }
  
 -    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
++    @Deprecated
 +    public int forceRepairAsync(String keyspace,
 +                                boolean isSequential,
 +                                Collection<String> dataCenters,
 +                                Collection<String> hosts,
 +                                boolean primaryRange,
 +                                boolean fullRepair,
 +                                String... columnFamilies)
      {
-         if (!FBUtilities.isUnix() && isSequential)
 -        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, ranges, fullRepair, columnFamilies);
++        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, fullRepair, columnFamilies);
+     }
+ 
 -    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
++    @Deprecated
++    public int forceRepairAsync(String keyspace,
++                                RepairParallelism parallelismDegree,
++                                Collection<String> dataCenters,
++                                Collection<String> hosts,
++                                boolean primaryRange,
++                                boolean fullRepair,
++                                String... columnFamilies)
+     {
 -        if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
 -            return 0;
++        if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
 +        {
 +            logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-             isSequential = false;
++            parallelismDegree = RepairParallelism.PARALLEL;
 +        }
  
-         RepairOption options = new RepairOption(isSequential, primaryRange, !fullRepair, 1, Collections.<Range<Token>>emptyList());
 -        int cmd = nextRepairCommand.incrementAndGet();
 -        if (ranges.size() > 0)
++        RepairOption options = new RepairOption(parallelismDegree, primaryRange, !fullRepair, 1, Collections.<Range<Token>>emptyList());
 +        if (dataCenters != null)
          {
 -            if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
 +            options.getDataCenters().addAll(dataCenters);
 +        }
 +        if (hosts != null)
 +        {
 +            options.getHosts().addAll(hosts);
 +        }
 +        if (columnFamilies != null)
 +        {
 +            for (String columnFamily : columnFamilies)
              {
 -                logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
 -                parallelismDegree = RepairParallelism.PARALLEL;
 +                options.getColumnFamilies().add(columnFamily);
              }
 -            new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, hosts, fullRepair, columnFamilies)).start();
          }
 -        return cmd;
 +        return forceRepairAsync(keyspace, options);
      }
  
 -    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, boolean primaryRange, boolean fullRepair, String... columnFamilies)
 +    public int forceRepairAsync(String keyspace,
 +                                boolean isSequential,
 +                                boolean isLocal,
 +                                boolean primaryRange,
 +                                boolean fullRepair,
 +                                String... columnFamilies)
      {
 -        Collection<Range<Token>> ranges;
 -        if (primaryRange)
 -        {
 -            ranges = isLocal ? getPrimaryRangesWithinDC(keyspace) : getPrimaryRanges(keyspace);
 -        }
 -        else
 +        Set<String> dataCenters = null;
 +        if (isLocal)
          {
 -            ranges = getLocalRanges(keyspace);
 +            dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
          }
 -
 -        return forceRepairAsync(keyspace, isSequential, isLocal, ranges, fullRepair, columnFamilies);
 +        return forceRepairAsync(keyspace, isSequential, dataCenters, null, primaryRange, fullRepair, columnFamilies);
      }
  
 -    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
 +    public int forceRepairRangeAsync(String beginToken,
 +                                     String endToken,
 +                                     String keyspaceName,
 +                                     boolean isSequential,
 +                                     Collection<String> dataCenters,
 +                                     Collection<String> hosts,
 +                                     boolean fullRepair,
 +                                     String... columnFamilies)
      {
-         if (!FBUtilities.isUnix() && isSequential)
 -        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, ranges, fullRepair, columnFamilies);
++        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, fullRepair, columnFamilies);
+     }
+ 
 -    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, boolean isLocal, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
++    public int forceRepairRangeAsync(String beginToken,
++                                     String endToken,
++                                     String keyspaceName,
++                                     RepairParallelism parallelismDegree,
++                                     Collection<String> dataCenters,
++                                     Collection<String> hosts,
++                                     boolean fullRepair,
++                                     String... columnFamilies)
+     {
 -        if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
 -            return 0;
 -
 -        int cmd = nextRepairCommand.incrementAndGet();
+         if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
          {
              logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-             isSequential = false;
+             parallelismDegree = RepairParallelism.PARALLEL;
          }
 -        new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, isLocal, fullRepair, columnFamilies)).start();
 -        return cmd;
 -    }
 -
 -    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies) throws IOException
 -    {
 -        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, fullRepair, columnFamilies);
 -    }
 -
 -    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies)
 -    {
          Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
  
-         RepairOption options = new RepairOption(isSequential, false, !fullRepair, 1, repairingRange);
++        RepairOption options = new RepairOption(parallelismDegree, false, !fullRepair, 1, repairingRange);
 +        options.getDataCenters().addAll(dataCenters);
 +        if (hosts != null)
 +        {
 +            options.getHosts().addAll(hosts);
 +        }
 +        if (columnFamilies != null)
 +        {
 +            for (String columnFamily : columnFamilies)
 +            {
 +                options.getColumnFamilies().add(columnFamily);
 +            }
 +        }
 +
          logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
--                           repairingRange, keyspaceName, columnFamilies);
 -        return forceRepairAsync(keyspaceName, parallelismDegree, dataCenters, hosts, repairingRange, fullRepair, columnFamilies);
++                    repairingRange, keyspaceName, columnFamilies);
 +        return forceRepairAsync(keyspaceName, options);
      }
  
 -    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, boolean isLocal, boolean fullRepair, String... columnFamilies)
 +    public int forceRepairRangeAsync(String beginToken,
 +                                     String endToken,
 +                                     String keyspaceName,
 +                                     boolean isSequential,
 +                                     boolean isLocal,
 +                                     boolean fullRepair,
 +                                     String... columnFamilies)
      {
 -        Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
 -
 -        logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
 -                           repairingRange, keyspaceName, columnFamilies);
 -        return forceRepairAsync(keyspaceName, isSequential, isLocal, repairingRange, fullRepair, columnFamilies);
 +        Set<String> dataCenters = null;
 +        if (isLocal)
 +        {
 +            dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
 +        }
 +        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential, dataCenters, null, fullRepair, columnFamilies);
      }
  
      /**
@@@ -2664,81 -2733,53 +2690,81 @@@
                  {
                      cfnames[i] = columnFamilyStores.get(i).name;
                  }
 -                for (Range<Token> range : ranges)
 +                for (Range<Token> range : options.getRanges())
                  {
 -                    RepairFuture future = ActiveRepairService.instance.submitRepairSession(parentSession, range, keyspace, parallelismDegree, rangeToNeighbors.get(range), cfnames);
 -                    if (future == null)
 +                    final RepairSession session = ActiveRepairService.instance.submitRepairSession(parentSession,
 +                                                                      range,
 +                                                                      keyspace,
-                                                                       options.isSequential(),
++                                                                      options.getParallelism(),
 +                                                                      rangeToNeighbors.get(range),
 +                                                                      repairedAt,
 +                                                                      executor,
 +                                                                      cfnames);
 +                    if (session == null)
                          continue;
 -                    futures.add(future);
 -                    // wait for a session to be done with its differencing before starting the next one
 -                    try
 -                    {
 -                        future.session.differencingDone.await();
 -                    }
 -                    catch (InterruptedException e)
 +                    // After repair session completes, notify client its result
 +                    Futures.addCallback(session, new FutureCallback<RepairSessionResult>()
                      {
 -                        message = "Interrupted while waiting for the differencing of repair session " + future.session + " to be done. Repair may be imprecise.";
 -                        logger.error(message, e);
 -                        sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.SESSION_FAILED.ordinal()});
 -                    }
 +                        public void onSuccess(RepairSessionResult result)
 +                        {
 +                            String message = String.format("Repair session %s for range %s finished", session.getId(), session.getRange().toString());
 +                            logger.info(message);
 +                            sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.SESSION_SUCCESS.ordinal()});
 +                        }
 +
 +                        public void onFailure(Throwable t)
 +                        {
 +                            String message = String.format("Repair session %s for range %s failed with error %s", session.getId(), session.getRange().toString(), t.getMessage());
 +                            logger.error(message, t);
 +                            sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.SESSION_FAILED.ordinal()});
 +                        }
 +                    });
 +                    futures.add(session);
                  }
  
 -                boolean successful = true;
 -                for (RepairFuture future : futures)
 +                // After all repair sessions completes(successful or not),
 +                // run anticompaction if necessary and send finish notice back to client
 +                final ListenableFuture<List<RepairSessionResult>> allSessions = Futures.successfulAsList(futures);
 +                Futures.addCallback(allSessions, new FutureCallback<List<RepairSessionResult>>()
                  {
 -                    try
 +                    public void onSuccess(List<RepairSessionResult> result)
                      {
 -                        future.get();
 -                        message = String.format("Repair session %s for range %s finished", future.session.getId(), future.session.getRange().toString());
 -                        logger.info(message);
 -                        sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.SESSION_SUCCESS.ordinal()});
 +                        // filter out null(=failed) results and get successful ranges
 +                        Collection<Range<Token>> successfulRanges = new ArrayList<>();
 +                        for (RepairSessionResult sessionResult : result)
 +                        {
 +                            if (sessionResult != null)
 +                            {
 +                                successfulRanges.add(sessionResult.range);
 +                            }
 +                        }
 +                        try
 +                        {
 +                            ActiveRepairService.instance.finishParentSession(parentSession, allNeighbors, successfulRanges);
 +                        }
 +                        catch (Exception e)
 +                        {
 +                            logger.error("Error in incremental repair", e);
 +                        }
 +                        repairComplete();
                      }
 -                    catch (ExecutionException e)
 +
 +                    public void onFailure(Throwable t)
                      {
 -                        successful = false;
 -                        message = String.format("Repair session %s for range %s failed with error %s", future.session.getId(), future.session.getRange().toString(), e.getCause().getMessage());
 -                        logger.error(message, e);
 -                        sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.SESSION_FAILED.ordinal()});
 +                        repairComplete();
                      }
 -                    catch (Exception e)
 +
 +                    private void repairComplete()
                      {
 -                        successful = false;
 -                        message = String.format("Repair session %s for range %s failed with error %s", future.session.getId(), future.session.getRange().toString(), e.getMessage());
 -                        logger.error(message, e);
 -                        sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.SESSION_FAILED.ordinal()});
 +                        String duration = DurationFormatUtils.formatDurationWords(System.currentTimeMillis() - startTime,
 +                                                                                  true, true);
 +                        String message = String.format("Repair command #%d finished in %s", cmd, duration);
 +                        sendNotification("repair", message,
 +                                         new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
 +                        logger.info(message);
 +                        executor.shutdownNow();
                      }
 -                }
 -                if (!fullRepair)
 -                    ActiveRepairService.instance.finishParentSession(parentSession, allNeighbors, successful);
 -                sendNotification("repair", String.format("Repair command #%d finished", cmd), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
 +                });
              }
          }, null);
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 8aa2421,e7d6f14..6545527
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -270,22 -272,45 +272,28 @@@ public interface StorageServiceMBean ex
       *   type: "repair"
       *   userObject: int array of length 2, [0]=command number, [1]=ordinal of AntiEntropyService.Status
       *
 +     * @param keyspace Keyspace name to repair. Should not be null.
 +     * @param options repair option.
       * @return Repair command number, or 0 if nothing to repair
       */
 +    public int repairAsync(String keyspace, Map<String, String> options);
 +
 +    @Deprecated
      public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts,  boolean primaryRange, boolean repairedAt, String... columnFamilies) throws IOException;
  
 -    /**
 -     * Invoke repair asynchronously.
 -     * You can track repair progress by subscribing JMX notification sent from this StorageServiceMBean.
 -     * Notification format is:
 -     *   type: "repair"
 -     *   userObject: int array of length 2, [0]=command number, [1]=ordinal of AntiEntropyService.Status
 -     *
 -     * @return Repair command number, or 0 if nothing to repair
 -     */
 +    @Deprecated
+     public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies);
+ 
 -    /**
 -     * Same as forceRepairAsync, but handles a specified range
 -     */
++    @Deprecated
      public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean repairedAt, String... columnFamilies) throws IOException;
  
 -    /**
 -     * Same as forceRepairAsync, but handles a specified range
 -     */
 +    @Deprecated
+     public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies);
+ 
 -    /**
 -     * Invoke repair asynchronously.
 -     * You can track repair progress by subscribing JMX notification sent from this StorageServiceMBean.
 -     * Notification format is:
 -     *   type: "repair"
 -     *   userObject: int array of length 2, [0]=command number, [1]=ordinal of AntiEntropyService.Status
 -     *
 -     * @return Repair command number, or 0 if nothing to repair
 -     */
++    @Deprecated
      public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, boolean primaryRange, boolean fullRepair, String... columnFamilies);
  
 -    /**
 -     * Same as forceRepairAsync, but handles a specified range
 -     */
 +    @Deprecated
      public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, boolean isLocal, boolean repairedAt, String... columnFamilies);
  
      public void forceTerminateAllRepairSessions();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeTool.java
index 207da5e,1db0245..4ed9977
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@@ -46,8 -46,9 +46,9 @@@ import org.apache.cassandra.db.compacti
  import org.apache.cassandra.db.compaction.OperationType;
  import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 -import org.apache.cassandra.locator.LocalStrategy;
  import org.apache.cassandra.net.MessagingServiceMBean;
 +import org.apache.cassandra.repair.messages.RepairOption;
+ import org.apache.cassandra.repair.RepairParallelism;
  import org.apache.cassandra.service.CacheServiceMBean;
  import org.apache.cassandra.streaming.ProgressInfo;
  import org.apache.cassandra.streaming.SessionInfo;
@@@ -1655,35 -1667,18 +1656,38 @@@ public class NodeToo
          }
      }
  
 -    @Command(name = "repair", description = "Repair one or more column families")
 +    @Command(name = "assassinate", description = "Forcefully remove a dead node without re-replicating any data.  Use as a last resort if you cannot removenode")
 +    public static class Assassinate extends NodeToolCmd
 +    {
 +        @Arguments(title = "ip address", usage = "<ip_address>", description = "IP address of the endpoint to assassinate", required = true)
 +        private String endpoint = EMPTY;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            try
 +            {
 +                probe.assassinateEndpoint(endpoint);
 +            }
 +            catch (UnknownHostException e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "repair", description = "Repair one or more tables")
      public static class Repair extends NodeToolCmd
      {
 -        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
          private List<String> args = new ArrayList<>();
  
 -        @Option(title = "parallel", name = {"-par", "--parallel"}, description = "Use -par to carry out a parallel repair")
 -        private boolean parallel = false;
 +        @Option(title = "seqential", name = {"-seq", "--sequential"}, description = "Use -seq to carry out a sequential repair")
 +        private boolean sequential = false;
  
+         @Option(title = "dc parallel", name = {"-dcpar", "--dc-parallel"}, description = "Use -dcpar to repair data centers in parallel.")
+         private boolean dcParallel = false;
+ 
          @Option(title = "local_dc", name = {"-local", "--in-local-dc"}, description = "Use -local to only repair against nodes in the same datacenter")
          private boolean localDC = false;
  
@@@ -1721,28 -1711,26 +1725,33 @@@
  
              for (String keyspace : keyspaces)
              {
 +                Map<String, String> options = new HashMap<>();
-                 options.put(RepairOption.SEQUENTIAL_KEY, Boolean.toString(sequential));
++                RepairParallelism parallelismDegree = RepairParallelism.PARALLEL;
++                if (sequential)
++                    parallelismDegree = RepairParallelism.SEQUENTIAL;
++                else if (dcParallel)
++                    parallelismDegree = RepairParallelism.DATACENTER_AWARE;
++                options.put(RepairOption.PARALLELISM_KEY, parallelismDegree.getName());
 +                options.put(RepairOption.PRIMARY_RANGE_KEY, Boolean.toString(primaryRange));
 +                options.put(RepairOption.INCREMENTAL_KEY, Boolean.toString(!fullRepair));
 +                options.put(RepairOption.JOB_THREADS_KEY, Integer.toString(numJobThreads));
 +                options.put(RepairOption.COLUMNFAMILIES_KEY, StringUtils.join(cfnames, ","));
 +                if (!startToken.isEmpty() || !endToken.isEmpty())
 +                {
 +                    options.put(RepairOption.RANGES_KEY, startToken + ":" + endToken);
 +                }
 +                if (localDC)
 +                {
 +                    options.put(RepairOption.DATACENTERS_KEY, StringUtils.join(newArrayList(probe.getDataCenter()), ","));
 +                }
 +                else
 +                {
 +                    options.put(RepairOption.DATACENTERS_KEY, StringUtils.join(specificDataCenters, ","));
 +                }
 +                options.put(RepairOption.HOSTS_KEY, StringUtils.join(specificHosts, ","));
                  try
                  {
 -                    RepairParallelism parallelismDegree = RepairParallelism.SEQUENTIAL;
 -                    if (parallel)
 -                        parallelismDegree = RepairParallelism.PARALLEL;
 -                    else if (dcParallel)
 -                        parallelismDegree = RepairParallelism.DATACENTER_AWARE;
 -
 -                    Collection<String> dataCenters = null;
 -                    Collection<String> hosts = null;
 -                    if (!specificDataCenters.isEmpty())
 -                        dataCenters = newArrayList(specificDataCenters);
 -                    else if (localDC)
 -                        dataCenters = newArrayList(probe.getDataCenter());
 -                    else if(!specificHosts.isEmpty())
 -                        hosts = newArrayList(specificHosts);
 -                    if (!startToken.isEmpty() || !endToken.isEmpty())
 -                        probe.forceRepairRangeAsync(System.out, keyspace, parallelismDegree, dataCenters,hosts, startToken, endToken, !incrementalRepair);
 -                    else
 -                        probe.forceRepairAsync(System.out, keyspace, parallelismDegree, dataCenters, hosts, primaryRange, !incrementalRepair, cfnames);
 +                    probe.repairAsync(System.out, keyspace, options);
                  } catch (Exception e)
                  {
                      throw new RuntimeException("Error occurred during repair", e);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/repair/RepairSessionTest.java
index e21e5d4,0000000..8ea2bfa
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
+++ b/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
@@@ -1,71 -1,0 +1,71 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.repair;
 +
 +import java.io.IOException;
 +import java.net.InetAddress;
 +import java.util.Set;
 +import java.util.UUID;
 +import java.util.concurrent.ExecutionException;
 +
 +import com.google.common.collect.Sets;
 +import org.junit.Test;
 +
 +import org.apache.cassandra.dht.IPartitioner;
 +import org.apache.cassandra.dht.Murmur3Partitioner;
 +import org.apache.cassandra.dht.Range;
 +import org.apache.cassandra.dht.Token;
 +import org.apache.cassandra.gms.Gossiper;
 +import org.apache.cassandra.service.ActiveRepairService;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.UUIDGen;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.fail;
 +
 +public class RepairSessionTest
 +{
 +    @Test
 +    public void testConviction() throws Exception
 +    {
 +        InetAddress remote = InetAddress.getByName("127.0.0.2");
 +        Gossiper.instance.initializeNodeUnsafe(remote, UUID.randomUUID(), 1);
 +
 +        // Set up RepairSession
 +        UUID parentSessionId = UUIDGen.getTimeUUID();
 +        UUID sessionId = UUID.randomUUID();
 +        IPartitioner p = Murmur3Partitioner.instance;
 +        Range<Token> repairRange = new Range<>(p.getToken(ByteBufferUtil.bytes(0)), p.getToken(ByteBufferUtil.bytes(100)));
 +        Set<InetAddress> endpoints = Sets.newHashSet(remote);
-         RepairSession session = new RepairSession(parentSessionId, sessionId, repairRange, "Keyspace1", true, endpoints, ActiveRepairService.UNREPAIRED_SSTABLE, "Standard1");
++        RepairSession session = new RepairSession(parentSessionId, sessionId, repairRange, "Keyspace1", RepairParallelism.SEQUENTIAL, endpoints, ActiveRepairService.UNREPAIRED_SSTABLE, "Standard1");
 +
 +        // perform convict
 +        session.convict(remote, Double.MAX_VALUE);
 +
 +        // RepairSession should throw ExecutorException with the cause of IOException when getting its value
 +        try
 +        {
 +            session.get();
 +            fail();
 +        }
 +        catch (ExecutionException ex)
 +        {
 +            assertEquals(IOException.class, ex.getCause().getClass());
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c023d492/test/unit/org/apache/cassandra/repair/messages/RepairOptionTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/repair/messages/RepairOptionTest.java
index 03e6fa2,0000000..557d0b1
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/repair/messages/RepairOptionTest.java
+++ b/test/unit/org/apache/cassandra/repair/messages/RepairOptionTest.java
@@@ -1,88 -1,0 +1,88 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.repair.messages;
 +
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Map;
 +import java.util.Set;
 +
- import com.google.common.collect.Iterables;
 +import org.junit.Test;
 +
 +import org.apache.cassandra.dht.IPartitioner;
 +import org.apache.cassandra.dht.Murmur3Partitioner;
 +import org.apache.cassandra.dht.Range;
 +import org.apache.cassandra.dht.Token;
++import org.apache.cassandra.repair.RepairParallelism;
 +
 +import static org.junit.Assert.*;
 +
 +public class RepairOptionTest
 +{
 +    @Test
 +    public void testParseOptions()
 +    {
 +        IPartitioner partitioner = Murmur3Partitioner.instance;
 +        Token.TokenFactory tokenFactory = partitioner.getTokenFactory();
 +
 +        // parse with empty options
 +        RepairOption option = RepairOption.parse(new HashMap<String, String>(), partitioner);
-         assertTrue(option.isSequential());
++        assertTrue(option.getParallelism() == RepairParallelism.SEQUENTIAL);
 +        assertFalse(option.isPrimaryRange());
 +        assertFalse(option.isIncremental());
 +
 +        // parse everything
 +        Map<String, String> options = new HashMap<>();
-         options.put(RepairOption.SEQUENTIAL_KEY, "false");
++        options.put(RepairOption.PARALLELISM_KEY, "parallel");
 +        options.put(RepairOption.PRIMARY_RANGE_KEY, "false");
 +        options.put(RepairOption.INCREMENTAL_KEY, "true");
 +        options.put(RepairOption.RANGES_KEY, "0:10,11:20,21:30");
 +        options.put(RepairOption.COLUMNFAMILIES_KEY, "cf1,cf2,cf3");
 +        options.put(RepairOption.DATACENTERS_KEY, "dc1,dc2,dc3");
 +        options.put(RepairOption.HOSTS_KEY, "127.0.0.1,127.0.0.2,127.0.0.3");
 +
 +        option = RepairOption.parse(options, partitioner);
-         assertFalse(option.isSequential());
++        assertTrue(option.getParallelism() == RepairParallelism.PARALLEL);
 +        assertFalse(option.isPrimaryRange());
 +        assertTrue(option.isIncremental());
 +
 +        Set<Range<Token>> expectedRanges = new HashSet<>(3);
 +        expectedRanges.add(new Range<>(tokenFactory.fromString("0"), tokenFactory.fromString("10")));
 +        expectedRanges.add(new Range<>(tokenFactory.fromString("11"), tokenFactory.fromString("20")));
 +        expectedRanges.add(new Range<>(tokenFactory.fromString("21"), tokenFactory.fromString("30")));
 +        assertEquals(expectedRanges, option.getRanges());
 +
 +        Set<String> expectedCFs = new HashSet<>(3);
 +        expectedCFs.add("cf1");
 +        expectedCFs.add("cf2");
 +        expectedCFs.add("cf3");
 +        assertEquals(expectedCFs, option.getColumnFamilies());
 +
 +        Set<String> expectedDCs = new HashSet<>(3);
 +        expectedDCs.add("dc1");
 +        expectedDCs.add("dc2");
 +        expectedDCs.add("dc3");
 +        assertEquals(expectedDCs, option.getDataCenters());
 +
 +        Set<String> expectedHosts = new HashSet<>(3);
 +        expectedHosts.add("127.0.0.1");
 +        expectedHosts.add("127.0.0.2");
 +        expectedHosts.add("127.0.0.3");
 +        assertEquals(expectedHosts, option.getHosts());
 +    }
 +}


[4/8] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeTool.java
index 8a59e8d,0000000..1db0245
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@@ -1,2466 -1,0 +1,2476 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools;
 +
 +import java.io.*;
 +import java.lang.management.MemoryUsage;
 +import java.net.InetAddress;
 +import java.net.UnknownHostException;
 +import java.text.DecimalFormat;
 +import java.text.SimpleDateFormat;
 +import java.util.*;
 +import java.util.Map.Entry;
 +import java.util.concurrent.ExecutionException;
 +
 +import javax.management.openmbean.TabularData;
 +
 +import com.google.common.base.Joiner;
 +import com.google.common.base.Throwables;
 +import com.google.common.collect.ArrayListMultimap;
 +import com.google.common.collect.LinkedHashMultimap;
 +import com.google.common.collect.Maps;
 +import com.yammer.metrics.reporting.JmxReporter;
 +
 +import io.airlift.command.*;
 +
 +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
 +import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 +import org.apache.cassandra.db.Keyspace;
 +import org.apache.cassandra.db.compaction.CompactionManagerMBean;
 +import org.apache.cassandra.db.compaction.OperationType;
 +import org.apache.cassandra.io.util.FileUtils;
 +import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 +import org.apache.cassandra.locator.LocalStrategy;
 +import org.apache.cassandra.net.MessagingServiceMBean;
++import org.apache.cassandra.repair.RepairParallelism;
 +import org.apache.cassandra.service.CacheServiceMBean;
 +import org.apache.cassandra.streaming.ProgressInfo;
 +import org.apache.cassandra.streaming.SessionInfo;
 +import org.apache.cassandra.streaming.StreamState;
 +import org.apache.cassandra.utils.EstimatedHistogram;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.JVMStabilityInspector;
 +
 +import static com.google.common.base.Preconditions.checkArgument;
 +import static com.google.common.base.Preconditions.checkState;
 +import static com.google.common.base.Throwables.getStackTraceAsString;
 +import static com.google.common.collect.Iterables.toArray;
 +import static com.google.common.collect.Lists.newArrayList;
 +import static java.lang.Integer.parseInt;
 +import static java.lang.String.format;
 +import static org.apache.commons.lang3.ArrayUtils.EMPTY_STRING_ARRAY;
 +import static org.apache.commons.lang3.StringUtils.*;
 +
 +public class NodeTool
 +{
 +    private static final String HISTORYFILE = "nodetool.history";
 +
 +    public static void main(String... args)
 +    {
 +        List<Class<? extends Runnable>> commands = newArrayList(
 +                Help.class,
 +                Info.class,
 +                Ring.class,
 +                NetStats.class,
 +                CfStats.class,
 +                CfHistograms.class,
 +                Cleanup.class,
 +                ClearSnapshot.class,
 +                Compact.class,
 +                Scrub.class,
 +                Flush.class,
 +                UpgradeSSTable.class,
 +                DisableAutoCompaction.class,
 +                EnableAutoCompaction.class,
 +                CompactionStats.class,
 +                CompactionHistory.class,
 +                Decommission.class,
 +                DescribeCluster.class,
 +                DisableBinary.class,
 +                EnableBinary.class,
 +                EnableGossip.class,
 +                DisableGossip.class,
 +                EnableHandoff.class,
 +                EnableThrift.class,
 +                GcStats.class,
 +                GetCompactionThreshold.class,
 +                GetCompactionThroughput.class,
 +                GetStreamThroughput.class,
 +                GetEndpoints.class,
 +                GetSSTables.class,
 +                GossipInfo.class,
 +                InvalidateKeyCache.class,
 +                InvalidateRowCache.class,
 +                InvalidateCounterCache.class,
 +                Join.class,
 +                Move.class,
 +                PauseHandoff.class,
 +                ResumeHandoff.class,
 +                ProxyHistograms.class,
 +                Rebuild.class,
 +                Refresh.class,
 +                RemoveToken.class,
 +                RemoveNode.class,
 +                Repair.class,
 +                SetCacheCapacity.class,
 +                SetHintedHandoffThrottleInKB.class,
 +                SetCompactionThreshold.class,
 +                SetCompactionThroughput.class,
 +                SetStreamThroughput.class,
 +                SetTraceProbability.class,
 +                Snapshot.class,
 +                ListSnapshots.class,
 +                Status.class,
 +                StatusBinary.class,
 +                StatusGossip.class,
 +                StatusThrift.class,
 +                Stop.class,
 +                StopDaemon.class,
 +                Version.class,
 +                DescribeRing.class,
 +                RebuildIndex.class,
 +                RangeKeySample.class,
 +                EnableBackup.class,
 +                DisableBackup.class,
 +                ResetLocalSchema.class,
 +                ReloadTriggers.class,
 +                SetCacheKeysToSave.class,
 +                DisableThrift.class,
 +                DisableHandoff.class,
 +                Drain.class,
 +                TruncateHints.class,
 +                TpStats.class,
 +                SetLoggingLevel.class,
 +                GetLoggingLevels.class
 +        );
 +
 +        Cli<Runnable> parser = Cli.<Runnable>builder("nodetool")
 +                .withDescription("Manage your Cassandra cluster")
 +                .withDefaultCommand(Help.class)
 +                .withCommands(commands)
 +                .build();
 +
 +        int status = 0;
 +        try
 +        {
 +            Runnable parse = parser.parse(args);
 +            printHistory(args);
 +            parse.run();
 +        } catch (IllegalArgumentException |
 +                IllegalStateException |
 +                ParseArgumentsMissingException |
 +                ParseArgumentsUnexpectedException |
 +                ParseOptionConversionException |
 +                ParseOptionMissingException |
 +                ParseOptionMissingValueException |
 +                ParseCommandMissingException |
 +                ParseCommandUnrecognizedException e)
 +        {
 +            badUse(e);
 +            status = 1;
 +        } catch (Throwable throwable)
 +        {
 +            err(Throwables.getRootCause(throwable));
 +            status = 2;
 +        }
 +
 +        System.exit(status);
 +    }
 +
 +    private static void printHistory(String... args)
 +    {
 +        //don't bother to print if no args passed (meaning, nodetool is just printing out the sub-commands list)
 +        if (args.length == 0)
 +            return;
 +
 +        String cmdLine = Joiner.on(" ").skipNulls().join(args);
 +        cmdLine = cmdLine.replaceFirst("(?<=(-pw|--password))\\s+\\S+", " <hidden>");
 +
 +        try (FileWriter writer = new FileWriter(new File(FBUtilities.getToolsOutputDirectory(), HISTORYFILE), true))
 +        {
 +            SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
 +            writer.append(sdf.format(new Date())).append(": ").append(cmdLine).append(System.lineSeparator());
 +        }
 +        catch (IOException | IOError ioe)
 +        {
 +            //quietly ignore any errors about not being able to write out history
 +        }
 +    }
 +
 +    private static void badUse(Exception e)
 +    {
 +        System.out.println("nodetool: " + e.getMessage());
 +        System.out.println("See 'nodetool help' or 'nodetool help <command>'.");
 +    }
 +
 +    private static void err(Throwable e)
 +    {
 +        System.err.println("error: " + e.getMessage());
 +        System.err.println("-- StackTrace --");
 +        System.err.println(getStackTraceAsString(e));
 +    }
 +
 +    public static abstract class NodeToolCmd implements Runnable
 +    {
 +
 +        @Option(type = OptionType.GLOBAL, name = {"-h", "--host"}, description = "Node hostname or ip address")
 +        private String host = "127.0.0.1";
 +
 +        @Option(type = OptionType.GLOBAL, name = {"-p", "--port"}, description = "Remote jmx agent port number")
 +        private String port = "7199";
 +
 +        @Option(type = OptionType.GLOBAL, name = {"-u", "--username"}, description = "Remote jmx agent username")
 +        private String username = EMPTY;
 +
 +        @Option(type = OptionType.GLOBAL, name = {"-pw", "--password"}, description = "Remote jmx agent password")
 +        private String password = EMPTY;
 +
 +        @Option(type = OptionType.GLOBAL, name = {"-pwf", "--password-file"}, description = "Path to the JMX password file")
 +        private String passwordFilePath = EMPTY;
 +
 +        @Override
 +        public void run()
 +        {
 +            if (isNotEmpty(username)) {
 +                if (isNotEmpty(passwordFilePath))
 +                    password = readUserPasswordFromFile(username, passwordFilePath);
 +
 +                if (isEmpty(password))
 +                    password = promptAndReadPassword();
 +            }
 +
 +            try (NodeProbe probe = connect())
 +            {
 +                execute(probe);
 +            } 
 +            catch (IOException e)
 +            {
 +                throw new RuntimeException("Error while closing JMX connection", e);
 +            }
 +
 +        }
 +
 +        private String readUserPasswordFromFile(String username, String passwordFilePath) {
 +            String password = EMPTY;
 +
 +            File passwordFile = new File(passwordFilePath);
 +            try (Scanner scanner = new Scanner(passwordFile).useDelimiter("\\s+"))
 +            {
 +                while (scanner.hasNextLine())
 +                {
 +                    if (scanner.hasNext())
 +                    {
 +                        String jmxRole = scanner.next();
 +                        if (jmxRole.equals(username) && scanner.hasNext())
 +                        {
 +                            password = scanner.next();
 +                            break;
 +                        }
 +                    }
 +                    scanner.nextLine();
 +                }
 +            } catch (FileNotFoundException e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +
 +            return password;
 +        }
 +
 +        private String promptAndReadPassword()
 +        {
 +            String password = EMPTY;
 +
 +            Console console = System.console();
 +            if (console != null)
 +                password = String.valueOf(console.readPassword("Password:"));
 +
 +            return password;
 +        }
 +
 +        protected abstract void execute(NodeProbe probe);
 +
 +        private NodeProbe connect()
 +        {
 +            NodeProbe nodeClient = null;
 +
 +            try
 +            {
 +                if (username.isEmpty())
 +                    nodeClient = new NodeProbe(host, parseInt(port));
 +                else
 +                    nodeClient = new NodeProbe(host, parseInt(port), username, password);
 +            } catch (IOException e)
 +            {
 +                Throwable rootCause = Throwables.getRootCause(e);
 +                System.err.println(format("nodetool: Failed to connect to '%s:%s' - %s: '%s'.", host, port, rootCause.getClass().getSimpleName(), rootCause.getMessage()));
 +                System.exit(1);
 +            }
 +
 +            return nodeClient;
 +        }
 +
 +        protected List<String> parseOptionalKeyspace(List<String> cmdArgs, NodeProbe nodeProbe)
 +        {
 +            List<String> keyspaces = new ArrayList<>();
 +
 +            if (cmdArgs == null || cmdArgs.isEmpty())
 +                keyspaces.addAll(nodeProbe.getKeyspaces());
 +            else
 +                keyspaces.add(cmdArgs.get(0));
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                if (!nodeProbe.getKeyspaces().contains(keyspace))
 +                    throw new IllegalArgumentException("Keyspace [" + keyspace + "] does not exist.");
 +            }
 +
 +            return Collections.unmodifiableList(keyspaces);
 +        }
 +
 +        protected String[] parseOptionalColumnFamilies(List<String> cmdArgs)
 +        {
 +            return cmdArgs.size() <= 1 ? EMPTY_STRING_ARRAY : toArray(cmdArgs.subList(1, cmdArgs.size()), String.class);
 +        }
 +    }
 +
 +    @Command(name = "info", description = "Print node information (uptime, load, ...)")
 +    public static class Info extends NodeToolCmd
 +    {
 +        @Option(name = {"-T", "--tokens"}, description = "Display all tokens")
 +        private boolean tokens = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            boolean gossipInitialized = probe.isInitialized();
 +
 +            System.out.printf("%-17s: %s%n", "ID", probe.getLocalHostId());
 +            System.out.printf("%-17s: %s%n", "Gossip active", gossipInitialized);
 +            System.out.printf("%-17s: %s%n", "Thrift active", probe.isThriftServerRunning());
 +            System.out.printf("%-17s: %s%n", "Native Transport active", probe.isNativeTransportRunning());
 +            System.out.printf("%-17s: %s%n", "Load", probe.getLoadString());
 +            if (gossipInitialized)
 +                System.out.printf("%-17s: %s%n", "Generation No", probe.getCurrentGenerationNumber());
 +            else
 +                System.out.printf("%-17s: %s%n", "Generation No", 0);
 +
 +            // Uptime
 +            long secondsUp = probe.getUptime() / 1000;
 +            System.out.printf("%-17s: %d%n", "Uptime (seconds)", secondsUp);
 +
 +            // Memory usage
 +            MemoryUsage heapUsage = probe.getHeapMemoryUsage();
 +            double memUsed = (double) heapUsage.getUsed() / (1024 * 1024);
 +            double memMax = (double) heapUsage.getMax() / (1024 * 1024);
 +            System.out.printf("%-17s: %.2f / %.2f%n", "Heap Memory (MB)", memUsed, memMax);
 +
 +            // Data Center/Rack
 +            System.out.printf("%-17s: %s%n", "Data Center", probe.getDataCenter());
 +            System.out.printf("%-17s: %s%n", "Rack", probe.getRack());
 +
 +            // Exceptions
 +            System.out.printf("%-17s: %s%n", "Exceptions", probe.getStorageMetric("Exceptions"));
 +
 +            CacheServiceMBean cacheService = probe.getCacheServiceMBean();
 +
 +            // Key Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +            System.out.printf("%-17s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                    "Key Cache",
 +                    probe.getCacheMetric("KeyCache", "Entries"),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Size")),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Capacity")),
 +                    probe.getCacheMetric("KeyCache", "Hits"),
 +                    probe.getCacheMetric("KeyCache", "Requests"),
 +                    probe.getCacheMetric("KeyCache", "HitRate"),
 +                    cacheService.getKeyCacheSavePeriodInSeconds());
 +
 +            // Row Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +            System.out.printf("%-17s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                    "Row Cache",
 +                    probe.getCacheMetric("RowCache", "Entries"),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Size")),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Capacity")),
 +                    probe.getCacheMetric("RowCache", "Hits"),
 +                    probe.getCacheMetric("RowCache", "Requests"),
 +                    probe.getCacheMetric("RowCache", "HitRate"),
 +                    cacheService.getRowCacheSavePeriodInSeconds());
 +
 +            // Counter Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +            System.out.printf("%-17s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                    "Counter Cache",
 +                    probe.getCacheMetric("CounterCache", "Entries"),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Size")),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Capacity")),
 +                    probe.getCacheMetric("CounterCache", "Hits"),
 +                    probe.getCacheMetric("CounterCache", "Requests"),
 +                    probe.getCacheMetric("CounterCache", "HitRate"),
 +                    cacheService.getCounterCacheSavePeriodInSeconds());
 +
 +            // Tokens
 +            List<String> tokens = probe.getTokens();
 +            if (tokens.size() == 1 || this.tokens)
 +                for (String token : tokens)
 +                    System.out.printf("%-17s: %s%n", "Token", token);
 +            else
 +                System.out.printf("%-17s: (invoke with -T/--tokens to see all %d tokens)%n", "Token", tokens.size());
 +        }
 +    }
 +
 +    @Command(name = "ring", description = "Print information about the token ring")
 +    public static class Ring extends NodeToolCmd
 +    {
 +        @Arguments(description = "Specify a keyspace for accurate ownership information (topology awareness)")
 +        private String keyspace = null;
 +
 +        @Option(title = "resolve_ip", name = {"-r", "--resolve-ip"}, description = "Show node domain names instead of IPs")
 +        private boolean resolveIp = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            Map<String, String> tokensToEndpoints = probe.getTokenToEndpointMap();
 +            LinkedHashMultimap<String, String> endpointsToTokens = LinkedHashMultimap.create();
 +            boolean haveVnodes = false;
 +            for (Map.Entry<String, String> entry : tokensToEndpoints.entrySet())
 +            {
 +                haveVnodes |= endpointsToTokens.containsKey(entry.getValue());
 +                endpointsToTokens.put(entry.getValue(), entry.getKey());
 +            }
 +
 +            int maxAddressLength = Collections.max(endpointsToTokens.keys(), new Comparator<String>()
 +            {
 +                @Override
 +                public int compare(String first, String second)
 +                {
 +                    return ((Integer) first.length()).compareTo(second.length());
 +                }
 +            }).length();
 +
 +            String formatPlaceholder = "%%-%ds  %%-12s%%-7s%%-8s%%-16s%%-20s%%-44s%%n";
 +            String format = format(formatPlaceholder, maxAddressLength);
 +
 +            StringBuffer errors = new StringBuffer();
 +            boolean showEffectiveOwnership = true;
 +            // Calculate per-token ownership of the ring
 +            Map<InetAddress, Float> ownerships;
 +            try
 +            {
 +                ownerships = probe.effectiveOwnership(keyspace);
 +            } 
 +            catch (IllegalStateException ex)
 +            {
 +                ownerships = probe.getOwnership();
 +                errors.append("Note: " + ex.getMessage() + "%n");
 +                showEffectiveOwnership = false;
 +            } 
 +            catch (IllegalArgumentException ex)
 +            {
 +                System.out.printf("%nError: " + ex.getMessage() + "%n");
 +                return;
 +            }
 +
 +            
 +            System.out.println();
 +            for (Entry<String, SetHostStat> entry : getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
 +                printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
 +
 +            if (haveVnodes)
 +            {
 +                System.out.println("  Warning: \"nodetool ring\" is used to output all the tokens of a node.");
 +                System.out.println("  To view status related info of a node use \"nodetool status\" instead.\n");
 +            }
 +
 +            System.out.printf("%n  " + errors.toString());
 +        }
 +
 +        private void printDc(NodeProbe probe, String format,
 +                             String dc,
 +                             LinkedHashMultimap<String, String> endpointsToTokens,
 +                             SetHostStat hoststats,boolean showEffectiveOwnership)
 +        {
 +            Collection<String> liveNodes = probe.getLiveNodes();
 +            Collection<String> deadNodes = probe.getUnreachableNodes();
 +            Collection<String> joiningNodes = probe.getJoiningNodes();
 +            Collection<String> leavingNodes = probe.getLeavingNodes();
 +            Collection<String> movingNodes = probe.getMovingNodes();
 +            Map<String, String> loadMap = probe.getLoadMap();
 +
 +            System.out.println("Datacenter: " + dc);
 +            System.out.println("==========");
 +
 +            // get the total amount of replicas for this dc and the last token in this dc's ring
 +            List<String> tokens = new ArrayList<>();
 +            String lastToken = "";
 +
 +            for (HostStat stat : hoststats)
 +            {
 +                tokens.addAll(endpointsToTokens.get(stat.endpoint.getHostAddress()));
 +                lastToken = tokens.get(tokens.size() - 1);
 +            }
 +
 +            System.out.printf(format, "Address", "Rack", "Status", "State", "Load", "Owns", "Token");
 +
 +            if (hoststats.size() > 1)
 +                System.out.printf(format, "", "", "", "", "", "", lastToken);
 +            else
 +                System.out.println();
 +
 +            for (HostStat stat : hoststats)
 +            {
 +                String endpoint = stat.endpoint.getHostAddress();
 +                String rack;
 +                try
 +                {
 +                    rack = probe.getEndpointSnitchInfoProxy().getRack(endpoint);
 +                }
 +                catch (UnknownHostException e)
 +                {
 +                    rack = "Unknown";
 +                }
 +
 +                String status = liveNodes.contains(endpoint)
 +                        ? "Up"
 +                        : deadNodes.contains(endpoint)
 +                                ? "Down"
 +                                : "?";
 +
 +                String state = "Normal";
 +
 +                if (joiningNodes.contains(endpoint))
 +                    state = "Joining";
 +                else if (leavingNodes.contains(endpoint))
 +                    state = "Leaving";
 +                else if (movingNodes.contains(endpoint))
 +                    state = "Moving";
 +
 +                String load = loadMap.containsKey(endpoint)
 +                        ? loadMap.get(endpoint)
 +                        : "?";
 +                String owns = stat.owns != null && showEffectiveOwnership? new DecimalFormat("##0.00%").format(stat.owns) : "?";
 +                System.out.printf(format, stat.ipOrDns(), rack, status, state, load, owns, stat.token);
 +            }
 +            System.out.println();
 +        }
 +    }
 +
 +    @Command(name = "netstats", description = "Print network information on provided host (connecting node by default)")
 +    public static class NetStats extends NodeToolCmd
 +    {
 +        @Option(title = "human_readable",
 +                name = {"-H", "--human-readable"},
 +                description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
 +        private boolean humanReadable = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.out.printf("Mode: %s%n", probe.getOperationMode());
 +            Set<StreamState> statuses = probe.getStreamStatus();
 +            if (statuses.isEmpty())
 +                System.out.println("Not sending any streams.");
 +            for (StreamState status : statuses)
 +            {
 +                System.out.printf("%s %s%n", status.description, status.planId.toString());
 +                for (SessionInfo info : status.sessions)
 +                {
 +                    System.out.printf("    %s", info.peer.toString());
 +                    // print private IP when it is used
 +                    if (!info.peer.equals(info.connecting))
 +                    {
 +                        System.out.printf(" (using %s)", info.connecting.toString());
 +                    }
 +                    System.out.printf("%n");
 +                    if (!info.receivingSummaries.isEmpty())
 +                    {
 +                        if (humanReadable)
 +                            System.out.printf("        Receiving %d files, %s total%n", info.getTotalFilesToReceive(), FileUtils.stringifyFileSize(info.getTotalSizeToReceive()));
 +                        else
 +                            System.out.printf("        Receiving %d files, %d bytes total%n", info.getTotalFilesToReceive(), info.getTotalSizeToReceive());
 +                        for (ProgressInfo progress : info.getReceivingFiles())
 +                        {
 +                            System.out.printf("            %s%n", progress.toString());
 +                        }
 +                    }
 +                    if (!info.sendingSummaries.isEmpty())
 +                    {
 +                        if (humanReadable)
 +                            System.out.printf("        Sending %d files, %s total%n", info.getTotalFilesToSend(), FileUtils.stringifyFileSize(info.getTotalSizeToSend()));
 +                        else
 +                            System.out.printf("        Sending %d files, %d bytes total%n", info.getTotalFilesToSend(), info.getTotalSizeToSend());
 +                        for (ProgressInfo progress : info.getSendingFiles())
 +                        {
 +                            System.out.printf("            %s%n", progress.toString());
 +                        }
 +                    }
 +                }
 +            }
 +
 +            System.out.printf("Read Repair Statistics:%nAttempted: %d%nMismatch (Blocking): %d%nMismatch (Background): %d%n", probe.getReadRepairAttempted(), probe.getReadRepairRepairedBlocking(), probe.getReadRepairRepairedBackground());
 +
 +            MessagingServiceMBean ms = probe.msProxy;
 +            System.out.printf("%-25s", "Pool Name");
 +            System.out.printf("%10s", "Active");
 +            System.out.printf("%10s", "Pending");
 +            System.out.printf("%15s%n", "Completed");
 +
 +            int pending;
 +            long completed;
 +
 +            pending = 0;
 +            for (int n : ms.getCommandPendingTasks().values())
 +                pending += n;
 +            completed = 0;
 +            for (long n : ms.getCommandCompletedTasks().values())
 +                completed += n;
 +            System.out.printf("%-25s%10s%10s%15s%n", "Commands", "n/a", pending, completed);
 +
 +            pending = 0;
 +            for (int n : ms.getResponsePendingTasks().values())
 +                pending += n;
 +            completed = 0;
 +            for (long n : ms.getResponseCompletedTasks().values())
 +                completed += n;
 +            System.out.printf("%-25s%10s%10s%15s%n", "Responses", "n/a", pending, completed);
 +        }
 +    }
 +
 +    @Command(name = "cfstats", description = "Print statistics on column families")
 +    public static class CfStats extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace.cfname>...]", description = "List of column families (or keyspace) names")
 +        private List<String> cfnames = new ArrayList<>();
 +
 +        @Option(name = "-i", description = "Ignore the list of column families and display the remaining cfs")
 +        private boolean ignore = false;
 +
 +        @Option(title = "human_readable",
 +                name = {"-H", "--human-readable"},
 +                description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
 +        private boolean humanReadable = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            OptionFilter filter = new OptionFilter(ignore, cfnames);
 +            Map<String, List<ColumnFamilyStoreMBean>> cfstoreMap = new HashMap<>();
 +
 +            // get a list of column family stores
 +            Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> cfamilies = probe.getColumnFamilyStoreMBeanProxies();
 +
 +            while (cfamilies.hasNext())
 +            {
 +                Map.Entry<String, ColumnFamilyStoreMBean> entry = cfamilies.next();
 +                String keyspaceName = entry.getKey();
 +                ColumnFamilyStoreMBean cfsProxy = entry.getValue();
 +
 +                if (!cfstoreMap.containsKey(keyspaceName) && filter.isColumnFamilyIncluded(entry.getKey(), cfsProxy.getColumnFamilyName()))
 +                {
 +                    List<ColumnFamilyStoreMBean> columnFamilies = new ArrayList<>();
 +                    columnFamilies.add(cfsProxy);
 +                    cfstoreMap.put(keyspaceName, columnFamilies);
 +                } else if (filter.isColumnFamilyIncluded(entry.getKey(), cfsProxy.getColumnFamilyName()))
 +                {
 +                    cfstoreMap.get(keyspaceName).add(cfsProxy);
 +                }
 +            }
 +
 +            // make sure all specified kss and cfs exist
 +            filter.verifyKeyspaces(probe.getKeyspaces());
 +            filter.verifyColumnFamilies();
 +
 +            // print out the table statistics
 +            for (Map.Entry<String, List<ColumnFamilyStoreMBean>> entry : cfstoreMap.entrySet())
 +            {
 +                String keyspaceName = entry.getKey();
 +                List<ColumnFamilyStoreMBean> columnFamilies = entry.getValue();
 +                long keyspaceReadCount = 0;
 +                long keyspaceWriteCount = 0;
 +                int keyspacePendingFlushes = 0;
 +                double keyspaceTotalReadTime = 0.0f;
 +                double keyspaceTotalWriteTime = 0.0f;
 +
 +                System.out.println("Keyspace: " + keyspaceName);
 +                for (ColumnFamilyStoreMBean cfstore : columnFamilies)
 +                {
 +                    String cfName = cfstore.getColumnFamilyName();
 +                    long writeCount = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getCount();
 +                    long readCount = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getCount();
 +
 +                    if (readCount > 0)
 +                    {
 +                        keyspaceReadCount += readCount;
 +                        keyspaceTotalReadTime += (long) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadTotalLatency");
 +                    }
 +                    if (writeCount > 0)
 +                    {
 +                        keyspaceWriteCount += writeCount;
 +                        keyspaceTotalWriteTime += (long) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteTotalLatency");
 +                    }
 +                    keyspacePendingFlushes += (long) probe.getColumnFamilyMetric(keyspaceName, cfName, "PendingFlushes");
 +                }
 +
 +                double keyspaceReadLatency = keyspaceReadCount > 0
 +                                             ? keyspaceTotalReadTime / keyspaceReadCount / 1000
 +                                             : Double.NaN;
 +                double keyspaceWriteLatency = keyspaceWriteCount > 0
 +                                              ? keyspaceTotalWriteTime / keyspaceWriteCount / 1000
 +                                              : Double.NaN;
 +
 +                System.out.println("\tRead Count: " + keyspaceReadCount);
 +                System.out.println("\tRead Latency: " + String.format("%s", keyspaceReadLatency) + " ms.");
 +                System.out.println("\tWrite Count: " + keyspaceWriteCount);
 +                System.out.println("\tWrite Latency: " + String.format("%s", keyspaceWriteLatency) + " ms.");
 +                System.out.println("\tPending Flushes: " + keyspacePendingFlushes);
 +
 +                // print out column family statistics for this keyspace
 +                for (ColumnFamilyStoreMBean cfstore : columnFamilies)
 +                {
 +                    String cfName = cfstore.getColumnFamilyName();
 +                    if (cfName.contains("."))
 +                        System.out.println("\t\tTable (index): " + cfName);
 +                    else
 +                        System.out.println("\t\tTable: " + cfName);
 +
 +                    System.out.println("\t\tSSTable count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveSSTableCount"));
 +
 +                    int[] leveledSStables = cfstore.getSSTableCountPerLevel();
 +                    if (leveledSStables != null)
 +                    {
 +                        System.out.print("\t\tSSTables in each level: [");
 +                        for (int level = 0; level < leveledSStables.length; level++)
 +                        {
 +                            int count = leveledSStables[level];
 +                            System.out.print(count);
 +                            long maxCount = 4L; // for L0
 +                            if (level > 0)
 +                                maxCount = (long) Math.pow(10, level);
 +                            //  show max threshold for level when exceeded
 +                            if (count > maxCount)
 +                                System.out.print("/" + maxCount);
 +
 +                            if (level < leveledSStables.length - 1)
 +                                System.out.print(", ");
 +                            else
 +                                System.out.println("]");
 +                        }
 +                    }
 +                    System.out.println("\t\tSpace used (live): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveDiskSpaceUsed"), humanReadable));
 +                    System.out.println("\t\tSpace used (total): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "TotalDiskSpaceUsed"), humanReadable));
 +                    System.out.println("\t\tSpace used by snapshots (total): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "SnapshotsSize"), humanReadable));
 +                    System.out.println("\t\tSSTable Compression Ratio: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "CompressionRatio"));
 +                    System.out.println("\t\tMemtable cell count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableColumnsCount"));
 +                    System.out.println("\t\tMemtable data size: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableLiveDataSize"), humanReadable));
 +                    System.out.println("\t\tMemtable switch count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableSwitchCount"));
 +                    System.out.println("\t\tLocal read count: " + ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getCount());
 +                    double localReadLatency = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getMean() / 1000;
 +                    double localRLatency = localReadLatency > 0 ? localReadLatency : Double.NaN;
 +                    System.out.printf("\t\tLocal read latency: %01.3f ms%n", localRLatency);
 +                    System.out.println("\t\tLocal write count: " + ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getCount());
 +                    double localWriteLatency = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getMean() / 1000;
 +                    double localWLatency = localWriteLatency > 0 ? localWriteLatency : Double.NaN;
 +                    System.out.printf("\t\tLocal write latency: %01.3f ms%n", localWLatency);
 +                    System.out.println("\t\tPending flushes: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "PendingFlushes"));
 +                    System.out.println("\t\tBloom filter false positives: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterFalsePositives"));
 +                    System.out.printf("\t\tBloom filter false ratio: %s%n", String.format("%01.5f", probe.getColumnFamilyMetric(keyspaceName, cfName, "RecentBloomFilterFalseRatio")));
 +                    System.out.println("\t\tBloom filter space used: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterDiskSpaceUsed"), humanReadable));
 +                    System.out.println("\t\tCompacted partition minimum bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MinRowSize"), humanReadable));
 +                    System.out.println("\t\tCompacted partition maximum bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MaxRowSize"), humanReadable));
 +                    System.out.println("\t\tCompacted partition mean bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MeanRowSize"), humanReadable));
 +                    JmxReporter.HistogramMBean histogram = (JmxReporter.HistogramMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveScannedHistogram");
 +                    System.out.println("\t\tAverage live cells per slice (last five minutes): " + histogram.getMean());
 +                    System.out.println("\t\tMaximum live cells per slice (last five minutes): " + histogram.getMax());
 +                    histogram = (JmxReporter.HistogramMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "TombstoneScannedHistogram");
 +                    System.out.println("\t\tAverage tombstones per slice (last five minutes): " + histogram.getMean());
 +                    System.out.println("\t\tMaximum tombstones per slice (last five minutes): " + histogram.getMax());
 +
 +                    System.out.println("");
 +                }
 +                System.out.println("----------------");
 +            }
 +        }
 +
 +        private String format(long bytes, boolean humanReadable) {
 +            return humanReadable ? FileUtils.stringifyFileSize(bytes) : Long.toString(bytes);
 +        }
 +
 +        /**
 +         * Used for filtering keyspaces and columnfamilies to be displayed using the cfstats command.
 +         */
 +        private static class OptionFilter
 +        {
 +            private Map<String, List<String>> filter = new HashMap<>();
 +            private Map<String, List<String>> verifier = new HashMap<>();
 +            private List<String> filterList = new ArrayList<>();
 +            private boolean ignoreMode;
 +
 +            public OptionFilter(boolean ignoreMode, List<String> filterList)
 +            {
 +                this.filterList.addAll(filterList);
 +                this.ignoreMode = ignoreMode;
 +
 +                for (String s : filterList)
 +                {
 +                    String[] keyValues = s.split("\\.", 2);
 +
 +                    // build the map that stores the ks' and cfs to use
 +                    if (!filter.containsKey(keyValues[0]))
 +                    {
 +                        filter.put(keyValues[0], new ArrayList<String>());
 +                        verifier.put(keyValues[0], new ArrayList<String>());
 +
 +                        if (keyValues.length == 2)
 +                        {
 +                            filter.get(keyValues[0]).add(keyValues[1]);
 +                            verifier.get(keyValues[0]).add(keyValues[1]);
 +                        }
 +                    } else
 +                    {
 +                        if (keyValues.length == 2)
 +                        {
 +                            filter.get(keyValues[0]).add(keyValues[1]);
 +                            verifier.get(keyValues[0]).add(keyValues[1]);
 +                        }
 +                    }
 +                }
 +            }
 +
 +            public boolean isColumnFamilyIncluded(String keyspace, String columnFamily)
 +            {
 +                // supplying empty params list is treated as wanting to display all kss & cfs
 +                if (filterList.isEmpty())
 +                    return !ignoreMode;
 +
 +                List<String> cfs = filter.get(keyspace);
 +
 +                // no such keyspace is in the map
 +                if (cfs == null)
 +                    return ignoreMode;
 +                    // only a keyspace with no cfs was supplied
 +                    // so ignore or include (based on the flag) every column family in specified keyspace
 +                else if (cfs.size() == 0)
 +                    return !ignoreMode;
 +
 +                // keyspace exists, and it contains specific cfs
 +                verifier.get(keyspace).remove(columnFamily);
 +                return ignoreMode ^ cfs.contains(columnFamily);
 +            }
 +
 +            public void verifyKeyspaces(List<String> keyspaces)
 +            {
 +                for (String ks : verifier.keySet())
 +                    if (!keyspaces.contains(ks))
 +                        throw new IllegalArgumentException("Unknown keyspace: " + ks);
 +            }
 +
 +            public void verifyColumnFamilies()
 +            {
 +                for (String ks : filter.keySet())
 +                    if (verifier.get(ks).size() > 0)
 +                        throw new IllegalArgumentException("Unknown column families: " + verifier.get(ks).toString() + " in keyspace: " + ks);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "cfhistograms", description = "Print statistic histograms for a given column family")
 +    public static class CfHistograms extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<keyspace> <cfname>", description = "The keyspace and column family name")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 2, "cfhistograms requires ks and cf args");
 +
 +            String keyspace = args.get(0);
 +            String cfname = args.get(1);
 +
 +            // calculate percentile of row size and column count
 +            long[] estimatedRowSize = (long[]) probe.getColumnFamilyMetric(keyspace, cfname, "EstimatedRowSizeHistogram");
 +            long[] estimatedColumnCount = (long[]) probe.getColumnFamilyMetric(keyspace, cfname, "EstimatedColumnCountHistogram");
 +
 +            long[] rowSizeBucketOffsets = new EstimatedHistogram(estimatedRowSize.length).getBucketOffsets();
 +            long[] columnCountBucketOffsets = new EstimatedHistogram(estimatedColumnCount.length).getBucketOffsets();
 +            EstimatedHistogram rowSizeHist = new EstimatedHistogram(rowSizeBucketOffsets, estimatedRowSize);
 +            EstimatedHistogram columnCountHist = new EstimatedHistogram(columnCountBucketOffsets, estimatedColumnCount);
 +
 +            // build arrays to store percentile values
 +            double[] estimatedRowSizePercentiles = new double[7];
 +            double[] estimatedColumnCountPercentiles = new double[7];
 +            double[] offsetPercentiles = new double[]{0.5, 0.75, 0.95, 0.98, 0.99};
 +
 +            if (rowSizeHist.isOverflowed())
 +            {
 +                System.err.println(String.format("Row sizes are larger than %s, unable to calculate percentiles", rowSizeBucketOffsets[rowSizeBucketOffsets.length - 1]));
 +                for (int i = 0; i < offsetPercentiles.length; i++)
 +                        estimatedRowSizePercentiles[i] = Double.NaN;
 +            }
 +            else
 +            {
 +                for (int i = 0; i < offsetPercentiles.length; i++)
 +                    estimatedRowSizePercentiles[i] = rowSizeHist.percentile(offsetPercentiles[i]);
 +            }
 +
 +            if (columnCountHist.isOverflowed())
 +            {
 +                System.err.println(String.format("Column counts are larger than %s, unable to calculate percentiles", columnCountBucketOffsets[columnCountBucketOffsets.length - 1]));
 +                for (int i = 0; i < estimatedColumnCountPercentiles.length; i++)
 +                    estimatedColumnCountPercentiles[i] = Double.NaN;
 +            }
 +            else
 +            {
 +                for (int i = 0; i < offsetPercentiles.length; i++)
 +                    estimatedColumnCountPercentiles[i] = columnCountHist.percentile(offsetPercentiles[i]);
 +            }
 +
 +            // min value
 +            estimatedRowSizePercentiles[5] = rowSizeHist.min();
 +            estimatedColumnCountPercentiles[5] = columnCountHist.min();
 +            // max value
 +            estimatedRowSizePercentiles[6] = rowSizeHist.max();
 +            estimatedColumnCountPercentiles[6] = columnCountHist.max();
 +
 +            String[] percentiles = new String[]{"50%", "75%", "95%", "98%", "99%", "Min", "Max"};
 +            double[] readLatency = probe.metricPercentilesAsArray((JmxReporter.HistogramMBean) probe.getColumnFamilyMetric(keyspace, cfname, "ReadLatency"));
 +            double[] writeLatency = probe.metricPercentilesAsArray((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspace, cfname, "WriteLatency"));
 +            double[] sstablesPerRead = probe.metricPercentilesAsArray((JmxReporter.HistogramMBean) probe.getColumnFamilyMetric(keyspace, cfname, "SSTablesPerReadHistogram"));
 +
 +            System.out.println(format("%s/%s histograms", keyspace, cfname));
 +            System.out.println(format("%-10s%10s%18s%18s%18s%18s",
 +                    "Percentile", "SSTables", "Write Latency", "Read Latency", "Partition Size", "Cell Count"));
 +            System.out.println(format("%-10s%10s%18s%18s%18s%18s",
 +                    "", "", "(micros)", "(micros)", "(bytes)", ""));
 +
 +            for (int i = 0; i < percentiles.length; i++)
 +            {
 +                System.out.println(format("%-10s%10.2f%18.2f%18.2f%18.0f%18.0f",
 +                        percentiles[i],
 +                        sstablesPerRead[i],
 +                        writeLatency[i],
 +                        readLatency[i],
 +                        estimatedRowSizePercentiles[i],
 +                        estimatedColumnCountPercentiles[i]));
 +            }
 +            System.out.println();
 +        }
 +    }
 +
 +    @Command(name = "cleanup", description = "Triggers the immediate cleanup of keys no longer belonging to a node. By default, clean all keyspaces")
 +    public static class Cleanup extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                if (Keyspace.SYSTEM_KS.equals(keyspace))
 +                    continue;
 +
 +                try
 +                {
 +                    probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during cleanup", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "clearsnapshot", description = "Remove the snapshot with the given name from the given keyspaces. If no snapshotName is specified we will remove all snapshots")
 +    public static class ClearSnapshot extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspaces>...] ", description = "Remove snapshots from the given keyspaces")
 +        private List<String> keyspaces = new ArrayList<>();
 +
 +        @Option(title = "snapshot_name", name = "-t", description = "Remove the snapshot with a given name")
 +        private String snapshotName = EMPTY;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            StringBuilder sb = new StringBuilder();
 +
 +            sb.append("Requested clearing snapshot(s) for ");
 +
 +            if (keyspaces.isEmpty())
 +                sb.append("[all keyspaces]");
 +            else
 +                sb.append("[").append(join(keyspaces, ", ")).append("]");
 +
 +            if (!snapshotName.isEmpty())
 +                sb.append(" with snapshot name [").append(snapshotName).append("]");
 +
 +            System.out.println(sb.toString());
 +
 +            try
 +            {
 +                probe.clearSnapshot(snapshotName, toArray(keyspaces, String.class));
 +            } catch (IOException e)
 +            {
 +                throw new RuntimeException("Error during clearing snapshots", e);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "compact", description = "Force a (major) compaction on one or more column families")
 +    public static class Compact extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.forceKeyspaceCompaction(keyspace, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during compaction", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "flush", description = "Flush one or more column families")
 +    public static class Flush extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.forceKeyspaceFlush(keyspace, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during flushing", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more column families")
 +    public static class Scrub extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Option(title = "disable_snapshot",
 +                name = {"-ns", "--no-snapshot"},
 +                description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
 +        private boolean disableSnapshot = false;
 +
 +        @Option(title = "skip_corrupted",
 +                name = {"-s", "--skip-corrupted"},
 +                description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
 +        private boolean skipCorrupted = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.scrub(System.out, disableSnapshot, skipCorrupted, keyspace, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during flushing", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "disableautocompaction", description = "Disable autocompaction for the given keyspace and column family")
 +    public static class DisableAutoCompaction extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.disableAutoCompaction(keyspace, cfnames);
 +                } catch (IOException e)
 +                {
 +                    throw new RuntimeException("Error occurred during disabling auto-compaction", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "enableautocompaction", description = "Enable autocompaction for the given keyspace and column family")
 +    public static class EnableAutoCompaction extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.enableAutoCompaction(keyspace, cfnames);
 +                } catch (IOException e)
 +                {
 +                    throw new RuntimeException("Error occurred during enabling auto-compaction", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "upgradesstables", description = "Rewrite sstables (for the requested column families) that are not on the current version (thus upgrading them to said current version)")
 +    public static class UpgradeSSTable extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
 +        private boolean includeAll = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.upgradeSSTables(System.out, keyspace, !includeAll, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during enabling auto-compaction", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "compactionstats", description = "Print statistics on compactions")
 +    public static class CompactionStats extends NodeToolCmd
 +    {
 +        @Option(title = "human_readable",
 +                name = {"-H", "--human-readable"},
 +                description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
 +        private boolean humanReadable = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            int compactionThroughput = probe.getCompactionThroughput();
 +            CompactionManagerMBean cm = probe.getCompactionManagerProxy();
 +            System.out.println("pending tasks: " + probe.getCompactionMetric("PendingTasks"));
 +            long remainingBytes = 0;
 +            List<Map<String, String>> compactions = cm.getCompactions();
 +            if (!compactions.isEmpty())
 +            {
 +                List<String[]> lines = new ArrayList<>();
 +                int[] columnSizes = new int[] { 0, 0, 0, 0, 0, 0, 0 };
 +
 +                addLine(lines, columnSizes, "compaction type", "keyspace", "table", "completed", "total", "unit", "progress");
 +                for (Map<String, String> c : compactions)
 +                {
 +                    long total = Long.parseLong(c.get("total"));
 +                    long completed = Long.parseLong(c.get("completed"));
 +                    String taskType = c.get("taskType");
 +                    String keyspace = c.get("keyspace");
 +                    String columnFamily = c.get("columnfamily");
 +                    String completedStr = humanReadable ? FileUtils.stringifyFileSize(completed) : Long.toString(completed);
 +                    String totalStr = humanReadable ? FileUtils.stringifyFileSize(total) : Long.toString(total);
 +                    String unit = c.get("unit");
 +                    String percentComplete = total == 0 ? "n/a" : new DecimalFormat("0.00").format((double) completed / total * 100) + "%";
 +                    addLine(lines, columnSizes, taskType, keyspace, columnFamily, completedStr, totalStr, unit, percentComplete);
 +                    if (taskType.equals(OperationType.COMPACTION.toString()))
 +                        remainingBytes += total - completed;
 +                }
 +
 +                StringBuilder buffer = new StringBuilder();
 +                for (int columnSize : columnSizes) {
 +                    buffer.append("%");
 +                    buffer.append(columnSize + 3);
 +                    buffer.append("s");
 +                }
 +                buffer.append("%n");
 +                String format = buffer.toString();
 +
 +                for (String[] line : lines)
 +                {
 +                    System.out.printf(format, line[0], line[1], line[2], line[3], line[4], line[5], line[6]);
 +                }
 +
 +                String remainingTime = "n/a";
 +                if (compactionThroughput != 0)
 +                {
 +                    long remainingTimeInSecs = remainingBytes / (1024L * 1024L * compactionThroughput);
 +                    remainingTime = format("%dh%02dm%02ds", remainingTimeInSecs / 3600, (remainingTimeInSecs % 3600) / 60, (remainingTimeInSecs % 60));
 +                }
 +                System.out.printf("%25s%10s%n", "Active compaction remaining time : ", remainingTime);
 +            }
 +        }
 +
 +        private void addLine(List<String[]> lines, int[] columnSizes, String... columns) {
 +            lines.add(columns);
 +            for (int i = 0; i < columns.length; i++) {
 +                columnSizes[i] = Math.max(columnSizes[i], columns[i].length());
 +            }
 +        }
 +    }
 +
 +    @Command(name = "compactionhistory", description = "Print history of compaction")
 +    public static class CompactionHistory extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.out.println("Compaction History: ");
 +
 +            TabularData tabularData = probe.getCompactionHistory();
 +            if (tabularData.isEmpty())
 +            {
 +                System.out.printf("There is no compaction history");
 +                return;
 +            }
 +
 +            String format = "%-41s%-19s%-29s%-26s%-15s%-15s%s%n";
 +            List<String> indexNames = tabularData.getTabularType().getIndexNames();
 +            System.out.printf(format, toArray(indexNames, Object.class));
 +
 +            Set<?> values = tabularData.keySet();
 +            for (Object eachValue : values)
 +            {
 +                List<?> value = (List<?>) eachValue;
 +                System.out.printf(format, toArray(value, Object.class));
 +            }
 +        }
 +    }
 +
 +    @Command(name = "decommission", description = "Decommission the *node I am connecting to*")
 +    public static class Decommission extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            try
 +            {
 +                probe.decommission();
 +            } catch (InterruptedException e)
 +            {
 +                throw new RuntimeException("Error decommissioning node", e);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "describecluster", description = "Print the name, snitch, partitioner and schema version of a cluster")
 +    public static class DescribeCluster extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            // display cluster name, snitch and partitioner
 +            System.out.println("Cluster Information:");
 +            System.out.println("\tName: " + probe.getClusterName());
 +            System.out.println("\tSnitch: " + probe.getEndpointSnitchInfoProxy().getSnitchName());
 +            System.out.println("\tPartitioner: " + probe.getPartitioner());
 +
 +            // display schema version for each node
 +            System.out.println("\tSchema versions:");
 +            Map<String, List<String>> schemaVersions = probe.getSpProxy().getSchemaVersions();
 +            for (String version : schemaVersions.keySet())
 +            {
 +                System.out.println(format("\t\t%s: %s%n", version, schemaVersions.get(version)));
 +            }
 +        }
 +    }
 +
 +    @Command(name = "disablebinary", description = "Disable native transport (binary protocol)")
 +    public static class DisableBinary extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.stopNativeTransport();
 +        }
 +    }
 +
 +    @Command(name = "enablebinary", description = "Reenable native transport (binary protocol)")
 +    public static class EnableBinary extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.startNativeTransport();
 +        }
 +    }
 +
 +    @Command(name = "enablegossip", description = "Reenable gossip")
 +    public static class EnableGossip extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.startGossiping();
 +        }
 +    }
 +
 +    @Command(name = "disablegossip", description = "Disable gossip (effectively marking the node down)")
 +    public static class DisableGossip extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.stopGossiping();
 +        }
 +    }
 +
 +    @Command(name = "enablehandoff", description = "Reenable the future hints storing on the current node")
 +    public static class EnableHandoff extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<dc-name>,<dc-name>", description = "Enable hinted handoff only for these DCs")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() <= 1, "enablehandoff does not accept two args");
 +            if(args.size() == 1)
 +                probe.enableHintedHandoff(args.get(0));
 +            else
 +                probe.enableHintedHandoff();
 +        }
 +    }
 +
 +    @Command(name = "enablethrift", description = "Reenable thrift server")
 +    public static class EnableThrift extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.startThriftServer();
 +        }
 +    }
 +
 +    @Command(name = "getcompactionthreshold", description = "Print min and max compaction thresholds for a given column family")
 +    public static class GetCompactionThreshold extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<keyspace> <cfname>", description = "The keyspace with a column family")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 2, "getcompactionthreshold requires ks and cf args");
 +            String ks = args.get(0);
 +            String cf = args.get(1);
 +
 +            ColumnFamilyStoreMBean cfsProxy = probe.getCfsProxy(ks, cf);
 +            System.out.println("Current compaction thresholds for " + ks + "/" + cf + ": \n" +
 +                    " min = " + cfsProxy.getMinimumCompactionThreshold() + ", " +
 +                    " max = " + cfsProxy.getMaximumCompactionThreshold());
 +        }
 +    }
 +
 +    @Command(name = "getcompactionthroughput", description = "Print the MB/s throughput cap for compaction in the system")
 +    public static class GetCompactionThroughput extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.out.println("Current compaction throughput: " + probe.getCompactionThroughput() + " MB/s");
 +        }
 +    }
 +
 +    @Command(name = "getstreamthroughput", description = "Print the Mb/s throughput cap for streaming in the system")
 +    public static class GetStreamThroughput extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.out.println("Current stream throughput: " + probe.getStreamThroughput() + " Mb/s");
 +        }
 +    }
 +
 +    @Command(name = "getendpoints", description = "Print the end points that owns the key")
 +    public static class GetEndpoints extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<keyspace> <cfname> <key>", description = "The keyspace, the column family, and the key for which we need to find the endpoint")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 3, "getendpoints requires ks, cf and key args");
 +            String ks = args.get(0);
 +            String cf = args.get(1);
 +            String key = args.get(2);
 +
 +            List<InetAddress> endpoints = probe.getEndpoints(ks, cf, key);
 +            for (InetAddress endpoint : endpoints)
 +            {
 +                System.out.println(endpoint.getHostAddress());
 +            }
 +        }
 +    }
 +
 +    @Command(name = "getsstables", description = "Print the sstable filenames that own the key")
 +    public static class GetSSTables extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<keyspace> <cfname> <key>", description = "The keyspace, the column family, and the key")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 3, "getsstables requires ks, cf and key args");
 +            String ks = args.get(0);
 +            String cf = args.get(1);
 +            String key = args.get(2);
 +
 +            List<String> sstables = probe.getSSTables(ks, cf, key);
 +            for (String sstable : sstables)
 +            {
 +                System.out.println(sstable);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "gossipinfo", description = "Shows the gossip information for the cluster")
 +    public static class GossipInfo extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.out.println(probe.getGossipInfo());
 +        }
 +    }
 +
 +    @Command(name = "invalidatekeycache", description = "Invalidate the key cache")
 +    public static class InvalidateKeyCache extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.invalidateKeyCache();
 +        }
 +    }
 +
 +    @Command(name = "invalidaterowcache", description = "Invalidate the row cache")
 +    public static class InvalidateRowCache extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.invalidateRowCache();
 +        }
 +    }
 +
 +    @Command(name = "invalidatecountercache", description = "Invalidate the counter cache")
 +    public static class InvalidateCounterCache extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.invalidateCounterCache();
 +        }
 +    }
 +
 +    @Command(name = "join", description = "Join the ring")
 +    public static class Join extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkState(!probe.isJoined(), "This node has already joined the ring.");
 +
 +            try
 +            {
 +                probe.joinRing();
 +            } catch (IOException e)
 +            {
 +                throw new RuntimeException("Error during joining the ring", e);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "move", description = "Move node on the token ring to a new token")
 +    public static class Move extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<new token>", description = "The new token.", required = true)
 +        private String newToken = EMPTY;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            try
 +            {
 +                probe.move(newToken);
 +            } catch (IOException e)
 +            {
 +                throw new RuntimeException("Error during moving node", e);
 +            }
 +        }
 +    }
 +
 +
 +
 +    @Command(name = "pausehandoff", description = "Pause hints delivery process")
 +    public static class PauseHandoff extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.pauseHintsDelivery();
 +        }
 +    }
 +
 +    @Command(name = "resumehandoff", description = "Resume hints delivery process")
 +    public static class ResumeHandoff extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.resumeHintsDelivery();
 +        }
 +    }
 +
 +
 +    @Command(name = "proxyhistograms", description = "Print statistic histograms for network operations")
 +    public static class ProxyHistograms extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            String[] percentiles = new String[]{"50%", "75%", "95%", "98%", "99%", "Min", "Max"};
 +            double[] readLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("Read"));
 +            double[] writeLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("Write"));
 +            double[] rangeLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("RangeSlice"));
 +
 +            System.out.println("proxy histograms");
 +            System.out.println(format("%-10s%18s%18s%18s",
 +                    "Percentile", "Read Latency", "Write Latency", "Range Latency"));
 +            System.out.println(format("%-10s%18s%18s%18s",
 +                    "", "(micros)", "(micros)", "(micros)"));
 +            for (int i = 0; i < percentiles.length; i++)
 +            {
 +                System.out.println(format("%-10s%18.2f%18.2f%18.2f",
 +                        percentiles[i],
 +                        readLatency[i],
 +                        writeLatency[i],
 +                        rangeLatency[i]));
 +            }
 +            System.out.println();
 +        }
 +    }
 +
 +    @Command(name = "rebuild", description = "Rebuild data by streaming from other nodes (similarly to bootstrap)")
 +    public static class Rebuild extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<src-dc-name>", description = "Name of DC from which to select sources for streaming. By default, pick any DC")
 +        private String sourceDataCenterName = null;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.rebuild(sourceDataCenterName);
 +        }
 +    }
 +
 +    @Command(name = "refresh", description = "Load newly placed SSTables to the system without restart")
 +    public static class Refresh extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<keyspace> <cfname>", description = "The keyspace and column family name")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 2, "refresh requires ks and cf args");
 +            probe.loadNewSSTables(args.get(0), args.get(1));
 +        }
 +    }
 +
 +    @Deprecated
 +    @Command(name = "removetoken", description = "DEPRECATED (see removenode)", hidden = true)
 +    public static class RemoveToken extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.err.println("Warn: removetoken is deprecated, please use removenode instead");
 +        }
 +    }
 +
 +    @Command(name = "removenode", description = "Show status of current node removal, force completion of pending removal or remove provided ID")
 +    public static class RemoveNode extends NodeToolCmd
 +    {
 +        @Arguments(title = "remove_operation", usage = "<status>|<force>|<ID>", description = "Show status of current node removal, force completion of pending removal, or remove provided ID", required = true)
 +        private String removeOperation = EMPTY;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            switch (removeOperation)
 +            {
 +                case "status":
 +                    System.out.println("RemovalStatus: " + probe.getRemovalStatus());
 +                    break;
 +                case "force":
 +                    System.out.println("RemovalStatus: " + probe.getRemovalStatus());
 +                    probe.forceRemoveCompletion();
 +                    break;
 +                default:
 +                    probe.removeNode(removeOperation);
 +                    break;
 +            }
 +        }
 +    }
 +
 +    @Command(name = "repair", description = "Repair one or more column families")
 +    public static class Repair extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Option(title = "parallel", name = {"-par", "--parallel"}, description = "Use -par to carry out a parallel repair")
 +        private boolean parallel = false;
 +
++        @Option(title = "dc parallel", name = {"-dcpar", "--dc-parallel"}, description = "Use -dcpar to repair data centers in parallel.")
++        private boolean dcParallel = false;
++
 +        @Option(title = "local_dc", name = {"-local", "--in-local-dc"}, description = "Use -local to only repair against nodes in the same datacenter")
 +        private boolean localDC = false;
 +
 +        @Option(title = "specific_dc", name = {"-dc", "--in-dc"}, description = "Use -dc to repair specific datacenters")
 +        private List<String> specificDataCenters = new ArrayList<>();
 +
 +        @Option(title = "specific_host", name = {"-hosts", "--in-hosts"}, description = "Use -hosts to repair specific hosts")
 +        private List<String> specificHosts = new ArrayList<>();
 +
 +        @Option(title = "start_token", name = {"-st", "--start-token"}, description = "Use -st to specify a token at which the repair range starts")
 +        private String startToken = EMPTY;
 +
 +        @Option(title = "end_token", name = {"-et", "--end-token"}, description = "Use -et to specify a token at which repair range ends")
 +        private String endToken = EMPTY;
 +
 +        @Option(title = "primary_range", name = {"-pr", "--partitioner-range"}, description = "Use -pr to repair only the first range returned by the partitioner")
 +        private boolean primaryRange = false;
 +
 +        @Option(title = "incremental_repair", name = {"-inc", "--incremental"}, description = "Use -inc to use the new incremental repair")
 +        private boolean incrementalRepair = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            if (primaryRange && (!specificDataCenters.isEmpty() || !specificHosts.isEmpty()))
 +                throw new RuntimeException("Primary range repair should be performed on all nodes in the cluster.");
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
++                    RepairParallelism parallelismDegree = RepairParallelism.SEQUENTIAL;
++                    if (parallel)
++                        parallelismDegree = RepairParallelism.PARALLEL;
++                    else if (dcParallel)
++                        parallelismDegree = RepairParallelism.DATACENTER_AWARE;
++
 +                    Collection<String> dataCenters = null;
 +                    Collection<String> hosts = null;
 +                    if (!specificDataCenters.isEmpty())
 +                        dataCenters = newArrayList(specificDataCenters);
 +                    else if (localDC)
 +                        dataCenters = newArrayList(probe.getDataCenter());
 +                    else if(!specificHosts.isEmpty())
 +                        hosts = newArrayList(specificHosts);
 +                    if (!startToken.isEmpty() || !endToken.isEmpty())
-                         probe.forceRepairRangeAsync(System.out, keyspace, !parallel, dataCenters,hosts, startToken, endToken, !incrementalRepair);
++                        probe.forceRepairRangeAsync(System.out, keyspace, parallelismDegree, dataCenters,hosts, startToken, endToken, !incrementalRepair);
 +                    else
-                         probe.forceRepairAsync(System.out, keyspace, !parallel, dataCenters, hosts, primaryRange, !incrementalRepair, cfnames);
++                        probe.forceRepairAsync(System.out, keyspace, parallelismDegree, dataCenters, hosts, primaryRange, !incrementalRepair, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during repair", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "setcachecapacity", description = "Set global key, row, and counter cache capacities (in MB units)")
 +    public static class SetCacheCapacity extends NodeToolCmd
 +    {
 +        @Arguments(title = "<key-cache-capacity> <row-cache-capacity> <counter-cache-capacity>",
 +                   usage = "<key-cache-capacity> <row-cache-capacity> <counter-cache-capacity>",
 +                   description = "Key cache, row cache, and counter cache (in MB)",
 +                   required = true)
 +        private List<Integer> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 3, "setcachecapacity requires key-cache-capacity, row-cache-capacity, and counter-cache-capacity args.");
 +            probe.setCacheCapacities(args.get(0), args.get(1), args.get(2));
 +        }
 +    }
 +
 +    @Command(name = "setcompactionthreshold", description = "Set min and max compaction thresholds for a given column family")
 +    public static class SetCompactionThreshold extends NodeToolCmd
 +    {
 +        @Arguments(title = "<keyspace> <cfname> <minthreshold> <maxthreshold>", usage = "<keyspace> <cfname> <minthreshold> <maxthreshold>", description = "The keyspace, the column family, min and max threshold", required = true)
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 4, "setcompactionthreshold requires ks, cf, min, and max threshold args.");
 +
 +            int minthreshold = parseInt(args.get(2));
 +            int maxthreshold = parseInt(args.get(3));
 +            checkArgument(minthreshold >= 0 && maxthreshold >= 0, "Thresholds must be positive integers");
 +            checkArgument(minthreshold <= maxthreshold, "Min threshold cannot be greater than max.");
 +            checkArgument(minthreshold >= 2 || maxthreshold == 0, "Min threshold must be at least 2");
 +
 +            probe.setCompactionThreshold(args.get(0), args.get(1), minthreshold, maxthreshold);
 +        }
 +    }
 +
 +    @Command(name = "setcompactionthroughput", description = "Set the MB/s throughput cap for compaction in the system, or 0 to disable throttling")
 +    public static class SetCompactionThroughput extends NodeToolCmd
 +    {
 +        @Arguments(title = "compaction_throughput", usage = "<value_in_mb>", description = "Value in MB, 0 to disable throttling", required = true)
 +        private Integer compactionThroughput = null;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.setCompactionThroughput(compactionThroughput);
 +        }
 +    }
 +
 +    @Command(name = "sethintedhandoffthrottlekb", description =  "Set hinted handoff throttle in kb per second, per delivery thread.")
 +    public static class SetHintedHandoffThrottleInKB extends NodeToolCmd
 +    {
 +        @Arguments(title = "throttle_in_kb", usage = "<value_in_kb_per_sec>", description = "Value in KB per second", required = true)
 +        private Integer throttleInKB = null;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.setHintedHandoffThrottleInKB(throttleInKB);
 +        }
 +    }
 +
 +    @Command(name = "setstreamthroughput", description = "Set the Mb/s throughput cap for streaming in the system, or 0 to disable throttling")
 +    public static class SetStreamThroughput extends NodeToolCmd
 +    {
 +        @Arguments(title = "stream_throughput", usage = "<value_in_mb>", description = "Value in Mb, 0 to disable throttling", required = true)
 +        private Integer streamThroughput = null;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.setStreamThroughput(streamThroughput);
 +        }
 +    }
 +
 +    @Command(name = "settraceprobability", description = "Sets the probability for tracing any given request to value. 0 disables, 1 enables for all requests, 0 is the default")
 +    public static class SetTraceProbability extends NodeToolCmd
 +    {
 +        @Arguments(title = "trace_probability", usage = "<value>", description = "Trace probability between 0 and 1 (ex: 0.2)", required = true)
 +        private Double traceProbability = null;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(traceProbability >= 0 && traceProbability <= 1, "Trace probability must be between 0 and 1");
 +            probe.setTraceProbability(traceProbability);
 +        }
 +    }
 +
 +    @Command(name = "snapshot", description = "Take a snapshot of specified keyspaces or a snapshot of the specified column family")
 +    public static class Snapshot extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspaces...>]", description = "List of keyspaces. By default, all keyspaces")
 +        private List<String> keyspaces = new ArrayList<>();
 +
 +        @Option(title = "cfname", name = {"-cf", "--column-family"}, description = "The column family name (you must specify one and only one keyspace for using this option)")
 +        private String columnFamily = null;
 +
 +        @Option(title = "tag", name = {"-t", "--tag"}, description = "The name of the snapshot")
 +        private String snapshotName = Long.toString(System.currentTimeMillis());
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            try
 +            {
 +                StringBuilder sb = new StringBuilder();
 +
 +                sb.append("Requested creating snapshot(s) for ");
 +
 +                if (keyspaces.isEmpty())
 +                    sb.append("[all keyspaces]");
 +                else
 +                    sb.append("[").append(join(keyspaces, ", ")).append("]");
 +
 +                if (!snapshotName.isEmpty())
 +                    sb.append(" with snapshot name [").append(snapshotName).append("]");
 +
 +                System.out.println(sb.toString());
 +
 +                probe.takeSnapshot(snapshotName, columnFamily, toArray(keyspaces, String.class));
 +                System.out.println("Snapshot directory: " + snapshotName);
 +            } catch (IOException e)
 +            {
 +                throw new RuntimeException("Error during taking a snapshot", e);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "listsnapshots", description = "Lists all the snapshots along with the size on disk and true size.")
 +    public static class ListSnapshots extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            try
 +            {
 +                System.out.println("Snapshot Details: ");
 +
 +                final Map<String,TabularData> snapshotDetails = probe.getSnapshotDetails();
 +                if (snapshotDetails.isEmpty())
 +                {
 +                    System.out.printf("There are no snapshots");
 +                    return;
 +                }
 +
 +                final long trueSnapshotsSize = probe.trueSnapshotsSize();
 +                final String format = "%-20s%-29s%-29s%-19s%-19s%n";
 +                // display column names only once
 +                final List<String> indexNames = snapshotDetails.entrySet().iterator().next().getValue().getTabularType().getIndexNames();
 +                System.out.printf(format, (Object[]) indexNames.toArray(new String[indexNames.size()]));
 +
 +                for (final Map.Entry<String, TabularData> snapshotDetail : snapshotDetails.entrySet())
 +                {
 +                    Set<?> values = snapshotDetail.getValue().keySet();
 +                    for (Object eachValue : values)
 +                    {
 +                        final List<?> value = (List<?>) eachValue;
 +                        System.out.printf(format, value.toArray(new Object[value.size()]));
 +                    }
 +                }
 +
 +                System.out.println("\nTotal TrueDiskSpaceUsed: " + FileUtils.stringifyFileSize(trueSnapshotsSize) + "\n");
 +            }
 +            catch (Exception e)
 +            {
 +                throw new RuntimeException("Error during list snapshot", e);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "status", description = "Print cluster information (state, load, IDs, ...)")
 +    public static class Status extends NodeToolCmd
 +    {
 +        @Arg

<TRUNCATED>

[2/8] cassandra git commit: Add DC-aware sequential repair

Posted by yu...@apache.org.
Add DC-aware sequential repair

patch by Jimmy Mårdell; reviewed by yukim for CASSANDRA-8193


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/41469ecf
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/41469ecf
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/41469ecf

Branch: refs/heads/cassandra-2.1
Commit: 41469ecf6a27e94441f96ef905ed3b5354c23987
Parents: 17de36f
Author: Jimmy Mårdell <ya...@spotify.com>
Authored: Mon Nov 24 15:07:33 2014 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Mon Nov 24 15:09:41 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../DatacenterAwareRequestCoordinator.java      |  73 +++++++++++
 .../cassandra/repair/IRequestCoordinator.java   |  28 ++++
 .../cassandra/repair/IRequestProcessor.java     |  23 ++++
 .../repair/ParallelRequestCoordinator.java      |  49 +++++++
 .../org/apache/cassandra/repair/RepairJob.java  |  32 ++++-
 .../cassandra/repair/RepairParallelism.java     |  22 ++++
 .../apache/cassandra/repair/RepairSession.java  |  14 +-
 .../cassandra/repair/RequestCoordinator.java    | 128 -------------------
 .../repair/SequentialRequestCoordinator.java    |  58 +++++++++
 .../cassandra/service/ActiveRepairService.java  |   6 +-
 .../cassandra/service/StorageService.java       |  64 ++++++----
 .../cassandra/service/StorageServiceMBean.java  |  19 ++-
 .../org/apache/cassandra/tools/NodeCmd.java     |  21 ++-
 .../org/apache/cassandra/tools/NodeProbe.java   |  30 +++--
 .../apache/cassandra/tools/NodeToolHelp.yaml    |   1 +
 .../repair/RequestCoordinatorTest.java          | 124 ++++++++++++++++++
 17 files changed, 506 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index fe23248..7519653 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -24,6 +24,7 @@
  * Allow concurrent writing of the same table in the same JVM using
    CQLSSTableWriter (CASSANDRA-7463)
  * Fix totalDiskSpaceUsed calculation (CASSANDRA-8205)
+ * Add DC-aware sequential repair (CASSANDRA-8193)
 
 
 2.0.11:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java b/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java
new file mode 100644
index 0000000..ab3e03e
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+import java.net.InetAddress;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Queue;
+
+public class DatacenterAwareRequestCoordinator implements IRequestCoordinator<InetAddress>
+{
+    private Map<String, Queue<InetAddress>> requestsByDatacenter = new HashMap<>();
+    private int remaining = 0;
+    private final IRequestProcessor<InetAddress> processor;
+
+    protected DatacenterAwareRequestCoordinator(IRequestProcessor<InetAddress> processor)
+    {
+        this.processor = processor;
+    }
+
+    public void add(InetAddress request)
+    {
+        String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(request);
+        Queue<InetAddress> queue = requestsByDatacenter.get(dc);
+        if (queue == null)
+        {
+            queue = new LinkedList<>();
+            requestsByDatacenter.put(dc, queue);
+        }
+        queue.add(request);
+        remaining++;
+    }
+
+    public void start()
+    {
+        for (Queue<InetAddress> requests : requestsByDatacenter.values())
+        {
+            if (!requests.isEmpty())
+              processor.process(requests.peek());
+        }
+    }
+
+    // Returns how many request remains
+    public int completed(InetAddress request)
+    {
+        String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(request);
+        Queue<InetAddress> requests = requestsByDatacenter.get(dc);
+        assert requests != null;
+        assert request.equals(requests.peek());
+        requests.poll();
+        if (!requests.isEmpty())
+            processor.process(requests.peek());
+        return --remaining;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/IRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/IRequestCoordinator.java b/src/java/org/apache/cassandra/repair/IRequestCoordinator.java
new file mode 100644
index 0000000..db2ffe3
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/IRequestCoordinator.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+public interface IRequestCoordinator<R>
+{
+    void add(R request);
+
+    void start();
+
+    // Returns how many request remains
+    int completed(R request);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/IRequestProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/IRequestProcessor.java b/src/java/org/apache/cassandra/repair/IRequestProcessor.java
new file mode 100644
index 0000000..c7b49d7
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/IRequestProcessor.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+public interface IRequestProcessor<R>
+{
+    void process(R request);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java b/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java
new file mode 100644
index 0000000..839bb43
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class ParallelRequestCoordinator<R> implements IRequestCoordinator<R>
+{
+    private final Set<R> requests = new HashSet<>();
+    private final IRequestProcessor<R> processor;
+
+    public ParallelRequestCoordinator(IRequestProcessor<R> processor)
+    {
+        this.processor = processor;
+    }
+
+    @Override
+    public void add(R request) { requests.add(request); }
+
+    @Override
+    public void start()
+    {
+        for (R request : requests)
+            processor.process(request);
+    }
+
+    @Override
+    public int completed(R request)
+    {
+        requests.remove(request);
+        return requests.size();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RepairJob.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java
index 931f95a..7c791aa 100644
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@ -43,9 +43,9 @@ public class RepairJob
     private static Logger logger = LoggerFactory.getLogger(RepairJob.class);
 
     public final RepairJobDesc desc;
-    private final boolean isSequential;
+    private final RepairParallelism parallelismDegree;
     // first we send tree requests. this tracks the endpoints remaining to hear from
-    private final RequestCoordinator<InetAddress> treeRequests;
+    private final IRequestCoordinator<InetAddress> treeRequests;
     // tree responses are then tracked here
     private final List<TreeResponse> trees = new ArrayList<>();
     // once all responses are received, each tree is compared with each other, and differencer tasks
@@ -68,21 +68,38 @@ public class RepairJob
                      String keyspace,
                      String columnFamily,
                      Range<Token> range,
-                     boolean isSequential,
+                     RepairParallelism parallelismDegree,
                      ListeningExecutorService taskExecutor)
     {
         this.listener = listener;
         this.desc = new RepairJobDesc(sessionId, keyspace, columnFamily, range);
-        this.isSequential = isSequential;
+        this.parallelismDegree = parallelismDegree;
         this.taskExecutor = taskExecutor;
-        this.treeRequests = new RequestCoordinator<InetAddress>(isSequential)
+
+        IRequestProcessor<InetAddress> processor = new IRequestProcessor<InetAddress>()
         {
-            public void send(InetAddress endpoint)
+            @Override
+            public void process(InetAddress endpoint)
             {
                 ValidationRequest request = new ValidationRequest(desc, gcBefore);
                 MessagingService.instance().sendOneWay(request.createMessage(), endpoint);
             }
         };
+
+        switch (parallelismDegree)
+        {
+            case SEQUENTIAL:
+                this.treeRequests = new SequentialRequestCoordinator<>(processor);
+                break;
+            case PARALLEL:
+                this.treeRequests = new ParallelRequestCoordinator<>(processor);
+                break;
+            case DATACENTER_AWARE:
+                this.treeRequests = new DatacenterAwareRequestCoordinator(processor);
+                break;
+            default:
+                throw new AssertionError("Unknown degree of parallelism specified");
+        }
     }
 
     /**
@@ -102,7 +119,8 @@ public class RepairJob
         List<InetAddress> allEndpoints = new ArrayList<>(endpoints);
         allEndpoints.add(FBUtilities.getBroadcastAddress());
 
-        if (isSequential)
+        // Create a snapshot at all nodes unless we're using pure parallel repairs
+        if (parallelismDegree != RepairParallelism.PARALLEL)
         {
             List<ListenableFuture<InetAddress>> snapshotTasks = new ArrayList<>(allEndpoints.size());
             for (InetAddress endpoint : allEndpoints)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RepairParallelism.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairParallelism.java b/src/java/org/apache/cassandra/repair/RepairParallelism.java
new file mode 100644
index 0000000..12c22ca
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/RepairParallelism.java
@@ -0,0 +1,22 @@
+package org.apache.cassandra.repair;
+
+/**
+ * Specify the degree of parallelism when calculating the merkle trees in a repair job.
+ */
+public enum RepairParallelism
+{
+    /**
+     * One node at a time
+     */
+    SEQUENTIAL,
+
+    /**
+     * All nodes at the same time
+     */
+    PARALLEL,
+
+    /**
+     * One node per data center at a time
+     */
+    DATACENTER_AWARE
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RepairSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairSession.java b/src/java/org/apache/cassandra/repair/RepairSession.java
index c9a9671..f2b95eb 100644
--- a/src/java/org/apache/cassandra/repair/RepairSession.java
+++ b/src/java/org/apache/cassandra/repair/RepairSession.java
@@ -83,7 +83,7 @@ public class RepairSession extends WrappedRunnable implements IEndpointStateChan
     private final UUID id;
     public final String keyspace;
     private final String[] cfnames;
-    public final boolean isSequential;
+    public final RepairParallelism parallelismDegree;
     /** Range to repair */
     public final Range<Token> range;
     public final Set<InetAddress> endpoints;
@@ -110,19 +110,19 @@ public class RepairSession extends WrappedRunnable implements IEndpointStateChan
      *
      * @param range range to repair
      * @param keyspace name of keyspace
-     * @param isSequential true if performing repair on snapshots sequentially
+     * @param parallelismDegree specifies the degree of parallelism when calculating the merkle trees
      * @param dataCenters the data centers that should be part of the repair; null for all DCs
      * @param cfnames names of columnfamilies
      */
-    public RepairSession(Range<Token> range, String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
+    public RepairSession(Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
     {
-        this(UUIDGen.getTimeUUID(), range, keyspace, isSequential, dataCenters, hosts, cfnames);
+        this(UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, dataCenters, hosts, cfnames);
     }
 
-    public RepairSession(UUID id, Range<Token> range, String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, String[] cfnames)
+    public RepairSession(UUID id, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String[] cfnames)
     {
         this.id = id;
-        this.isSequential = isSequential;
+        this.parallelismDegree = parallelismDegree;
         this.keyspace = keyspace;
         this.cfnames = cfnames;
         assert cfnames.length > 0 : "Repairing no column families seems pointless, doesn't it";
@@ -270,7 +270,7 @@ public class RepairSession extends WrappedRunnable implements IEndpointStateChan
             // Create and queue a RepairJob for each column family
             for (String cfname : cfnames)
             {
-                RepairJob job = new RepairJob(this, id, keyspace, cfname, range, isSequential, taskExecutor);
+                RepairJob job = new RepairJob(this, id, keyspace, cfname, range, parallelismDegree, taskExecutor);
                 jobs.offer(job);
             }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RequestCoordinator.java b/src/java/org/apache/cassandra/repair/RequestCoordinator.java
deleted file mode 100644
index ed089ef..0000000
--- a/src/java/org/apache/cassandra/repair/RequestCoordinator.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.repair;
-
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Queue;
-import java.util.Set;
-
-/**
-*/
-public abstract class RequestCoordinator<R>
-{
-    private final Order<R> orderer;
-
-    public RequestCoordinator(boolean isSequential)
-    {
-        this.orderer = isSequential ? new SequentialOrder(this) : new ParallelOrder(this);
-    }
-
-    public abstract void send(R request);
-
-    public void add(R request)
-    {
-        orderer.add(request);
-    }
-
-    public void start()
-    {
-        orderer.start();
-    }
-
-    // Returns how many request remains
-    public int completed(R request)
-    {
-        return orderer.completed(request);
-    }
-
-    private static abstract class Order<R>
-    {
-        protected final RequestCoordinator<R> coordinator;
-
-        Order(RequestCoordinator<R> coordinator)
-        {
-            this.coordinator = coordinator;
-        }
-
-        public abstract void add(R request);
-        public abstract void start();
-        public abstract int completed(R request);
-    }
-
-    private static class SequentialOrder<R> extends Order<R>
-    {
-        private final Queue<R> requests = new LinkedList<>();
-
-        SequentialOrder(RequestCoordinator<R> coordinator)
-        {
-            super(coordinator);
-        }
-
-        public void add(R request)
-        {
-            requests.add(request);
-        }
-
-        public void start()
-        {
-            if (requests.isEmpty())
-                return;
-
-            coordinator.send(requests.peek());
-        }
-
-        public int completed(R request)
-        {
-            assert request.equals(requests.peek());
-            requests.poll();
-            int remaining = requests.size();
-            if (remaining != 0)
-                coordinator.send(requests.peek());
-            return remaining;
-        }
-    }
-
-    private static class ParallelOrder<R> extends Order<R>
-    {
-        private final Set<R> requests = new HashSet<>();
-
-        ParallelOrder(RequestCoordinator<R> coordinator)
-        {
-            super(coordinator);
-        }
-
-        public void add(R request)
-        {
-            requests.add(request);
-        }
-
-        public void start()
-        {
-            for (R request : requests)
-                coordinator.send(request);
-        }
-
-        public int completed(R request)
-        {
-            requests.remove(request);
-            return requests.size();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java b/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java
new file mode 100644
index 0000000..6bb5de9
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import java.util.LinkedList;
+import java.util.Queue;
+
+public class SequentialRequestCoordinator<R> implements IRequestCoordinator<R>
+{
+    private final Queue<R> requests = new LinkedList<>();
+    private final IRequestProcessor<R> processor;
+
+    public SequentialRequestCoordinator(IRequestProcessor<R> processor)
+    {
+        this.processor = processor;
+    }
+
+    @Override
+    public void add(R request)
+    {
+        requests.add(request);
+    }
+
+    @Override
+    public void start()
+    {
+        if (requests.isEmpty())
+            return;
+
+        processor.process(requests.peek());
+    }
+
+    @Override
+    public int completed(R request)
+    {
+        assert request.equals(requests.peek());
+        requests.poll();
+        int remaining = requests.size();
+        if (remaining != 0)
+            processor.process(requests.peek());
+        return remaining;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index aac9f9a..da81e8f 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -92,9 +92,9 @@ public class ActiveRepairService
      *
      * @return Future for asynchronous call or null if there is no need to repair
      */
-    public RepairFuture submitRepairSession(Range<Token> range, String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
+    public RepairFuture submitRepairSession(Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
     {
-        RepairSession session = new RepairSession(range, keyspace, isSequential, dataCenters, hosts, cfnames);
+        RepairSession session = new RepairSession(range, keyspace, parallelismDegree, dataCenters, hosts, cfnames);
         if (session.endpoints.isEmpty())
             return null;
         RepairFuture futureTask = new RepairFuture(session);
@@ -128,7 +128,7 @@ public class ActiveRepairService
     // add it to the sessions (avoid NPE in tests)
     RepairFuture submitArtificialRepairSession(RepairJobDesc desc)
     {
-        RepairSession session = new RepairSession(desc.sessionId, desc.range, desc.keyspace, false, null, null, new String[]{desc.columnFamily});
+        RepairSession session = new RepairSession(desc.sessionId, desc.range, desc.keyspace, RepairParallelism.PARALLEL, null, null, new String[]{desc.columnFamily});
         sessions.put(session.getId(), session);
         RepairFuture futureTask = new RepairFuture(session);
         executor.execute(futureTask);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 4bc1eee..3d42d1c 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -77,6 +77,7 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.ResponseVerbHandler;
 import org.apache.cassandra.repair.RepairFuture;
 import org.apache.cassandra.repair.RepairMessageVerbHandler;
+import org.apache.cassandra.repair.RepairParallelism;
 import org.apache.cassandra.service.paxos.CommitVerbHandler;
 import org.apache.cassandra.service.paxos.PrepareVerbHandler;
 import org.apache.cassandra.service.paxos.ProposeVerbHandler;
@@ -334,7 +335,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         {
             throw new IllegalStateException("No configured daemon");
         }
-        
+
         try
         {
             daemon.nativeServer.start();
@@ -432,10 +433,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             if (Gossiper.instance.getEndpointStateForEndpoint(DatabaseDescriptor.getReplaceAddress()).getApplicationState(ApplicationState.TOKENS) == null)
                 throw new RuntimeException("Could not find tokens for " + DatabaseDescriptor.getReplaceAddress() + " to replace");
             Collection<Token> tokens = TokenSerializer.deserialize(getPartitioner(), new DataInputStream(new ByteArrayInputStream(getApplicationStateValue(DatabaseDescriptor.getReplaceAddress(), ApplicationState.TOKENS))));
-            
+
             SystemKeyspace.setLocalHostId(hostId); // use the replacee's host Id as our own so we receive hints, etc
             Gossiper.instance.resetEndpointStateMap(); // clean up since we have what we need
-            return tokens;        
+            return tokens;
         }
         catch (IOException e)
         {
@@ -2408,16 +2409,21 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int forceRepairAsync(final String keyspace, final boolean isSequential, final Collection<String> dataCenters, final Collection<String> hosts, final boolean primaryRange, final String... columnFamilies)
     {
+        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, columnFamilies);
+    }
+
+    public int forceRepairAsync(final String keyspace, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts, final boolean primaryRange, final String... columnFamilies)
+    {
         // when repairing only primary range, dataCenter nor hosts can be set
         if (primaryRange && (dataCenters != null || hosts != null))
         {
             throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
         }
         final Collection<Range<Token>> ranges = primaryRange ? getLocalPrimaryRanges(keyspace) : getLocalRanges(keyspace);
-        return forceRepairAsync(keyspace, isSequential, dataCenters, hosts, ranges, columnFamilies);
+        return forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, ranges, columnFamilies);
     }
 
-    public int forceRepairAsync(final String keyspace, final boolean isSequential, final Collection<String> dataCenters, final Collection<String> hosts,  final Collection<Range<Token>> ranges, final String... columnFamilies)
+    public int forceRepairAsync(final String keyspace, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts,  final Collection<Range<Token>> ranges, final String... columnFamilies)
     {
         if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
             return 0;
@@ -2425,7 +2431,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         final int cmd = nextRepairCommand.incrementAndGet();
         if (ranges.size() > 0)
         {
-            new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, hosts, columnFamilies)).start();
+            new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, hosts, columnFamilies)).start();
         }
         return cmd;
     }
@@ -2438,37 +2444,42 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
         }
         final Collection<Range<Token>> ranges = primaryRange ? getLocalPrimaryRanges(keyspace) : getLocalRanges(keyspace);
-        return forceRepairAsync(keyspace, isSequential, isLocal, ranges, columnFamilies);
+        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, ranges, columnFamilies);
     }
 
-    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, Collection<Range<Token>> ranges, String... columnFamilies)
+    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, boolean isLocal, Collection<Range<Token>> ranges, String... columnFamilies)
     {
         if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
             return 0;
 
         final int cmd = nextRepairCommand.incrementAndGet();
-        if (!FBUtilities.isUnix() && isSequential)
+        if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
         {
             logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-            isSequential = false;
+            parallelismDegree = RepairParallelism.PARALLEL;
         }
-        new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, isLocal, columnFamilies)).start();
+        new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, isLocal, columnFamilies)).start();
         return cmd;
     }
 
     public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
     {
+        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, columnFamilies);
+    }
+
+    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
+    {
         Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
 
         logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
                     repairingRange, keyspaceName, columnFamilies);
 
-        if (!FBUtilities.isUnix() && isSequential)
+        if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
         {
             logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-            isSequential = false;
+            parallelismDegree = RepairParallelism.PARALLEL;
         }
-        return forceRepairAsync(keyspaceName, isSequential, dataCenters, hosts, repairingRange, columnFamilies);
+        return forceRepairAsync(keyspaceName, parallelismDegree, dataCenters, hosts, repairingRange, columnFamilies);
     }
 
     public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies)
@@ -2486,7 +2497,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public void forceKeyspaceRepair(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
     {
-        forceKeyspaceRepairRange(keyspaceName, getLocalRanges(keyspaceName), isSequential, isLocal, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, getLocalRanges(keyspaceName), isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, columnFamilies);
     }
 
     public void forceKeyspaceRepairPrimaryRange(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
@@ -2498,7 +2509,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
         }
 
-        forceKeyspaceRepairRange(keyspaceName, getLocalPrimaryRanges(keyspaceName), isSequential, false, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, getLocalPrimaryRanges(keyspaceName), isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, false, columnFamilies);
     }
 
     public void forceKeyspaceRepairRange(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
@@ -2507,14 +2518,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
                            repairingRange, keyspaceName, columnFamilies);
-        forceKeyspaceRepairRange(keyspaceName, repairingRange, isSequential, isLocal, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, repairingRange, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, columnFamilies);
     }
 
-    public void forceKeyspaceRepairRange(final String keyspaceName, final Collection<Range<Token>> ranges, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
+    public void forceKeyspaceRepairRange(final String keyspaceName, final Collection<Range<Token>> ranges, RepairParallelism parallelismDegree, boolean isLocal, final String... columnFamilies) throws IOException
     {
         if (ranges.isEmpty() || Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor() < 2)
             return;
-        createRepairTask(nextRepairCommand.incrementAndGet(), keyspaceName, ranges, isSequential, isLocal, columnFamilies).run();
+        createRepairTask(nextRepairCommand.incrementAndGet(), keyspaceName, ranges, parallelismDegree, isLocal, columnFamilies).run();
     }
 
     /**
@@ -2556,17 +2567,17 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return repairingRange;
     }
 
-    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final boolean isSequential, final boolean isLocal, final String... columnFamilies)
+    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final RepairParallelism parallelismDegree, final boolean isLocal, final String... columnFamilies)
     {
         Set<String> dataCenters = null;
         if (isLocal)
         {
             dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
         }
-        return createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, null, columnFamilies);
+        return createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, null, columnFamilies);
     }
 
-    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final boolean isSequential, final Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
+    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
     {
         if (dataCenters != null && !dataCenters.contains(DatabaseDescriptor.getLocalDataCenter()))
         {
@@ -2587,7 +2598,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                     RepairFuture future;
                     try
                     {
-                        future = forceKeyspaceRepair(range, keyspace, isSequential, dataCenters, hosts, columnFamilies);
+                        future = forceKeyspaceRepair(range, keyspace, parallelismDegree, dataCenters, hosts, columnFamilies);
                     }
                     catch (IllegalArgumentException e)
                     {
@@ -2639,6 +2650,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public RepairFuture forceKeyspaceRepair(final Range<Token> range, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, final String... columnFamilies) throws IOException
     {
+        return forceKeyspaceRepair(range, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, columnFamilies);
+    }
+
+    public RepairFuture forceKeyspaceRepair(final Range<Token> range, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, final String... columnFamilies) throws IOException
+    {
         ArrayList<String> names = new ArrayList<String>();
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
@@ -2651,7 +2667,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             return null;
         }
 
-        return ActiveRepairService.instance.submitRepairSession(range, keyspaceName, isSequential, dataCenters, hosts, names.toArray(new String[names.size()]));
+        return ActiveRepairService.instance.submitRepairSession(range, keyspaceName, parallelismDegree, dataCenters, hosts, names.toArray(new String[names.size()]));
     }
 
     public void forceTerminateAllRepairSessions() {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 0e2f55d..2386fc8 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -17,6 +17,8 @@
  */
 package org.apache.cassandra.service;
 
+import org.apache.cassandra.repair.RepairParallelism;
+
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -260,10 +262,25 @@ public interface StorageServiceMBean extends NotificationEmitter
     public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, String... columnFamilies);
 
     /**
+     * Invoke repair asynchronously.
+     * You can track repair progress by subscribing JMX notification sent from this StorageServiceMBean.
+     * Notification format is:
+     *   type: "repair"
+     *   userObject: int array of length 2, [0]=command number, [1]=ordinal of AntiEntropyService.Status
+     *
+     * @return Repair command number, or 0 if nothing to repair
+     */
+    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, String... columnFamilies);
+
+    /**
      * Same as forceRepairAsync, but handles a specified range
      */
     public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts,  final String... columnFamilies);
 
+    /**
+     * Same as forceRepairAsync, but handles a specified range
+     */
+    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts,  final String... columnFamilies);
 
     /**
      * Invoke repair asynchronously.
@@ -477,7 +494,7 @@ public interface StorageServiceMBean extends NotificationEmitter
 
     /**
      * Enables/Disables tracing for the whole system. Only thrift requests can start tracing currently.
-     * 
+     *
      * @param probability
      *            ]0,1[ will enable tracing on a partial number of requests with the provided probability. 0 will
      *            disable tracing and 1 will enable tracing for all requests (which mich severely cripple the system)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/tools/NodeCmd.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeCmd.java b/src/java/org/apache/cassandra/tools/NodeCmd.java
index a397244..2d7809a 100644
--- a/src/java/org/apache/cassandra/tools/NodeCmd.java
+++ b/src/java/org/apache/cassandra/tools/NodeCmd.java
@@ -47,6 +47,7 @@ import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.net.MessagingServiceMBean;
+import org.apache.cassandra.repair.RepairParallelism;
 import org.apache.cassandra.service.CacheServiceMBean;
 import org.apache.cassandra.service.StorageProxyMBean;
 import org.apache.cassandra.streaming.StreamState;
@@ -70,6 +71,7 @@ public class NodeCmd
     private static final Pair<String, String> TOKENS_OPT = Pair.create("T", "tokens");
     private static final Pair<String, String> PRIMARY_RANGE_OPT = Pair.create("pr", "partitioner-range");
     private static final Pair<String, String> PARALLEL_REPAIR_OPT = Pair.create("par", "parallel");
+    private static final Pair<String, String> DCPARALLEL_REPAIR_OPT = Pair.create("dcpar", "dcparallel");
     private static final Pair<String, String> LOCAL_DC_REPAIR_OPT = Pair.create("local", "in-local-dc");
     private static final Pair<String, String> HOST_REPAIR_OPT = Pair.create("hosts", "in-host");
     private static final Pair<String, String> DC_REPAIR_OPT = Pair.create("dc", "in-dc");
@@ -100,6 +102,7 @@ public class NodeCmd
         options.addOption(TOKENS_OPT,   false, "display all tokens");
         options.addOption(PRIMARY_RANGE_OPT, false, "only repair the first range returned by the partitioner for the node");
         options.addOption(PARALLEL_REPAIR_OPT, false, "repair nodes in parallel.");
+        options.addOption(DCPARALLEL_REPAIR_OPT, false, "repair data centers in parallel.");
         options.addOption(LOCAL_DC_REPAIR_OPT, false, "only repair against nodes in the same datacenter");
         options.addOption(DC_REPAIR_OPT, true, "only repair against nodes in the specified datacenters (comma separated)");
         options.addOption(HOST_REPAIR_OPT, true, "only repair against specified nodes (comma separated)");
@@ -203,10 +206,10 @@ public class NodeCmd
         StringBuilder header = new StringBuilder(512);
         header.append("\nAvailable commands\n");
         final NodeToolHelp ntHelp = loadHelp();
-        Collections.sort(ntHelp.commands, new Comparator<NodeToolHelp.NodeToolCommand>() 
+        Collections.sort(ntHelp.commands, new Comparator<NodeToolHelp.NodeToolCommand>()
         {
             @Override
-            public int compare(NodeToolHelp.NodeToolCommand o1, NodeToolHelp.NodeToolCommand o2) 
+            public int compare(NodeToolHelp.NodeToolCommand o1, NodeToolHelp.NodeToolCommand o2)
             {
                 return o1.name.compareTo(o2.name);
             }
@@ -525,7 +528,7 @@ public class NodeCmd
         }
     }
 
-    private Map<String, SetHostStat> getOwnershipByDc(boolean resolveIp, Map<String, String> tokenToEndpoint, 
+    private Map<String, SetHostStat> getOwnershipByDc(boolean resolveIp, Map<String, String> tokenToEndpoint,
                                                       Map<InetAddress, Float> ownerships) throws UnknownHostException
     {
         Map<String, SetHostStat> ownershipByDc = Maps.newLinkedHashMap();
@@ -574,7 +577,7 @@ public class NodeCmd
         public final Float owns;
         public final String token;
 
-        public HostStat(String token, InetAddress endpoint, boolean resolveIp, Float owns) 
+        public HostStat(String token, InetAddress endpoint, boolean resolveIp, Float owns)
         {
             this.token = token;
             this.endpoint = endpoint;
@@ -1668,7 +1671,11 @@ public class NodeCmd
             switch (nc)
             {
                 case REPAIR  :
-                    boolean sequential = !cmd.hasOption(PARALLEL_REPAIR_OPT.left);
+                    RepairParallelism parallelismDegree = RepairParallelism.SEQUENTIAL;
+                    if (cmd.hasOption(PARALLEL_REPAIR_OPT.left))
+                        parallelismDegree = RepairParallelism.PARALLEL;
+                    else if (cmd.hasOption(DCPARALLEL_REPAIR_OPT.left))
+                        parallelismDegree = RepairParallelism.DATACENTER_AWARE;
                     boolean localDC = cmd.hasOption(LOCAL_DC_REPAIR_OPT.left);
                     boolean specificDC = cmd.hasOption(DC_REPAIR_OPT.left);
                     boolean specificHosts = cmd.hasOption(HOST_REPAIR_OPT.left);
@@ -1686,9 +1693,9 @@ public class NodeCmd
                     else if(specificHosts)
                         hosts  = Arrays.asList(cmd.getOptionValue(HOST_REPAIR_OPT.left).split(","));
                     if (cmd.hasOption(START_TOKEN_OPT.left) || cmd.hasOption(END_TOKEN_OPT.left))
-                        probe.forceRepairRangeAsync(System.out, keyspace, sequential, dataCenters, hosts, cmd.getOptionValue(START_TOKEN_OPT.left), cmd.getOptionValue(END_TOKEN_OPT.left), columnFamilies);
+                        probe.forceRepairRangeAsync(System.out, keyspace, parallelismDegree, dataCenters, hosts, cmd.getOptionValue(START_TOKEN_OPT.left), cmd.getOptionValue(END_TOKEN_OPT.left), columnFamilies);
                     else
-                        probe.forceRepairAsync(System.out, keyspace, sequential, dataCenters, hosts, primaryRange, columnFamilies);
+                        probe.forceRepairAsync(System.out, keyspace, parallelismDegree, dataCenters, hosts, primaryRange, columnFamilies);
                     break;
                 case FLUSH   :
                     try { probe.forceKeyspaceFlush(keyspace, columnFamilies); }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 849e368..261d416 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -54,6 +54,7 @@ import org.apache.cassandra.gms.FailureDetectorMBean;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.MessagingServiceMBean;
+import org.apache.cassandra.repair.RepairParallelism;
 import org.apache.cassandra.service.*;
 import org.apache.cassandra.streaming.StreamState;
 import org.apache.cassandra.streaming.StreamManagerMBean;
@@ -217,12 +218,17 @@ public class NodeProbe
 
     public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts,  boolean primaryRange, String... columnFamilies) throws IOException
     {
+        forceRepairAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, columnFamilies);
+    }
+
+    public void forceRepairAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts,  boolean primaryRange, String... columnFamilies) throws IOException
+    {
         RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
         try
         {
             jmxc.addConnectionNotificationListener(runner, null, null);
             ssProxy.addNotificationListener(runner, null, null);
-            if (!runner.repairAndWait(ssProxy, isSequential, dataCenters, hosts, primaryRange))
+            if (!runner.repairAndWait(ssProxy, parallelismDegree, dataCenters, hosts, primaryRange))
                 failed = true;
         }
         catch (Exception e)
@@ -239,15 +245,19 @@ public class NodeProbe
             catch (Throwable ignored) {}
         }
     }
-
     public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, String... columnFamilies) throws IOException
     {
+        forceRepairRangeAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, startToken, endToken, columnFamilies);
+    }
+
+    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, String... columnFamilies) throws IOException
+    {
         RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
         try
         {
             jmxc.addConnectionNotificationListener(runner, null, null);
             ssProxy.addNotificationListener(runner, null, null);
-            if (!runner.repairRangeAndWait(ssProxy,  isSequential, dataCenters, hosts, startToken, endToken))
+            if (!runner.repairRangeAndWait(ssProxy, parallelismDegree, dataCenters, hosts, startToken, endToken))
                 failed = true;
         }
         catch (Exception e)
@@ -896,17 +906,17 @@ public class NodeProbe
     {
         return failed;
     }
-    
+
     public long getReadRepairAttempted()
     {
         return spProxy.getReadRepairAttempted();
     }
-    
+
     public long getReadRepairRepairedBlocking()
     {
         return spProxy.getReadRepairRepairedBlocking();
     }
-    
+
     public long getReadRepairRepairedBackground()
     {
         return spProxy.getReadRepairRepairedBackground();
@@ -1060,16 +1070,16 @@ class RepairRunner implements NotificationListener
         this.columnFamilies = columnFamilies;
     }
 
-    public boolean repairAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRangeOnly) throws Exception
+    public boolean repairAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRangeOnly) throws Exception
     {
-        cmd = ssProxy.forceRepairAsync(keyspace, isSequential, dataCenters, hosts, primaryRangeOnly, columnFamilies);
+        cmd = ssProxy.forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, primaryRangeOnly, columnFamilies);
         waitForRepair();
         return success;
     }
 
-    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, String startToken, String endToken) throws Exception
+    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, String startToken, String endToken) throws Exception
     {
-        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, isSequential, dataCenters, hosts, columnFamilies);
+        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, parallelismDegree, dataCenters, hosts, columnFamilies);
         waitForRepair();
         return success;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml b/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
index 523335e..b254bac 100644
--- a/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
+++ b/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
@@ -155,6 +155,7 @@ commands:
     help: |
       Repair one or more column families
          Use -dc to repair specific datacenters (csv list).
+         Use -dcpar to repair datacenters in parallel.
          Use -et to specify a token at which repair range ends.
          Use -local to only repair against nodes in the same datacenter.
          Use -pr to repair only the first range returned by the partitioner.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java b/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java
new file mode 100644
index 0000000..ad6eabe
--- /dev/null
+++ b/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+
+public class RequestCoordinatorTest implements IRequestProcessor<InetAddress>
+{
+    private InetAddress[] endpoints;
+    private List<InetAddress> activeRequests;
+    private static Random random = new Random(0);
+
+    @Before
+    public void setup() throws UnknownHostException
+    {
+        endpoints = new InetAddress[12];
+        for (int i = 0; i < 12; i++)
+            endpoints[i] = InetAddress.getByName("127.0.0." + (i + 1));
+        activeRequests = new ArrayList<>();
+        DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
+        {
+            @Override
+            public String getRack(InetAddress endpoint)
+            {
+                return "rack1";
+            }
+
+            @Override
+            public String getDatacenter(InetAddress endpoint)
+            {
+                // 127.0.0.1, 127.0.0.2, 127.0.0.3 -> datacenter1
+                // 127.0.0.4, 127.0.0.5, 127.0.0.6 -> datacenter2 etc
+                int no = endpoint.getAddress()[3] - 1;
+                return "datacenter" + (no / 3 + 1);
+            }
+
+            @Override
+            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+            {
+                return 0;
+            }
+        });
+    }
+
+    @Override
+    public void process(InetAddress request)
+    {
+        activeRequests.add(request);
+    }
+
+    @Test
+    public void testSequentialRequestCoordinator()
+    {
+        SequentialRequestCoordinator<InetAddress> coordinator = new SequentialRequestCoordinator<>(this);
+        for (InetAddress endpoint : endpoints)
+            coordinator.add(endpoint);
+        coordinator.start();
+        int max = finishRequests(coordinator);
+        assertEquals(1, max);
+    }
+
+    @Test
+    public void testParallelRequestCoordinator()
+    {
+        ParallelRequestCoordinator<InetAddress> coordinator = new ParallelRequestCoordinator<>(this);
+        for (InetAddress endpoint : endpoints)
+            coordinator.add(endpoint);
+        coordinator.start();
+        int max = finishRequests(coordinator);
+        assertEquals(endpoints.length, max);
+    }
+
+    @Test
+    public void testDatacenterAwareRequestCoordinator()
+    {
+        DatacenterAwareRequestCoordinator coordinator = new DatacenterAwareRequestCoordinator(this);
+        for (InetAddress endpoint : endpoints)
+            coordinator.add(endpoint);
+        coordinator.start();
+        int max = finishRequests(coordinator);
+        assertEquals(4, max);
+    }
+
+    private int finishRequests(IRequestCoordinator<InetAddress> requestCoordinator)
+    {
+        int max = 0;
+        while (activeRequests.size() > 0)
+        {
+            max = Math.max(max, activeRequests.size());
+            // Finish a request
+            int ix = random.nextInt(activeRequests.size());
+            InetAddress finished = activeRequests.get(ix);
+            activeRequests.remove(ix);
+            requestCoordinator.completed(finished);
+        }
+        return max;
+    }
+}


[6/8] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeTool.java
index 8a59e8d,0000000..1db0245
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@@ -1,2466 -1,0 +1,2476 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.tools;
 +
 +import java.io.*;
 +import java.lang.management.MemoryUsage;
 +import java.net.InetAddress;
 +import java.net.UnknownHostException;
 +import java.text.DecimalFormat;
 +import java.text.SimpleDateFormat;
 +import java.util.*;
 +import java.util.Map.Entry;
 +import java.util.concurrent.ExecutionException;
 +
 +import javax.management.openmbean.TabularData;
 +
 +import com.google.common.base.Joiner;
 +import com.google.common.base.Throwables;
 +import com.google.common.collect.ArrayListMultimap;
 +import com.google.common.collect.LinkedHashMultimap;
 +import com.google.common.collect.Maps;
 +import com.yammer.metrics.reporting.JmxReporter;
 +
 +import io.airlift.command.*;
 +
 +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
 +import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 +import org.apache.cassandra.db.Keyspace;
 +import org.apache.cassandra.db.compaction.CompactionManagerMBean;
 +import org.apache.cassandra.db.compaction.OperationType;
 +import org.apache.cassandra.io.util.FileUtils;
 +import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 +import org.apache.cassandra.locator.LocalStrategy;
 +import org.apache.cassandra.net.MessagingServiceMBean;
++import org.apache.cassandra.repair.RepairParallelism;
 +import org.apache.cassandra.service.CacheServiceMBean;
 +import org.apache.cassandra.streaming.ProgressInfo;
 +import org.apache.cassandra.streaming.SessionInfo;
 +import org.apache.cassandra.streaming.StreamState;
 +import org.apache.cassandra.utils.EstimatedHistogram;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.JVMStabilityInspector;
 +
 +import static com.google.common.base.Preconditions.checkArgument;
 +import static com.google.common.base.Preconditions.checkState;
 +import static com.google.common.base.Throwables.getStackTraceAsString;
 +import static com.google.common.collect.Iterables.toArray;
 +import static com.google.common.collect.Lists.newArrayList;
 +import static java.lang.Integer.parseInt;
 +import static java.lang.String.format;
 +import static org.apache.commons.lang3.ArrayUtils.EMPTY_STRING_ARRAY;
 +import static org.apache.commons.lang3.StringUtils.*;
 +
 +public class NodeTool
 +{
 +    private static final String HISTORYFILE = "nodetool.history";
 +
 +    public static void main(String... args)
 +    {
 +        List<Class<? extends Runnable>> commands = newArrayList(
 +                Help.class,
 +                Info.class,
 +                Ring.class,
 +                NetStats.class,
 +                CfStats.class,
 +                CfHistograms.class,
 +                Cleanup.class,
 +                ClearSnapshot.class,
 +                Compact.class,
 +                Scrub.class,
 +                Flush.class,
 +                UpgradeSSTable.class,
 +                DisableAutoCompaction.class,
 +                EnableAutoCompaction.class,
 +                CompactionStats.class,
 +                CompactionHistory.class,
 +                Decommission.class,
 +                DescribeCluster.class,
 +                DisableBinary.class,
 +                EnableBinary.class,
 +                EnableGossip.class,
 +                DisableGossip.class,
 +                EnableHandoff.class,
 +                EnableThrift.class,
 +                GcStats.class,
 +                GetCompactionThreshold.class,
 +                GetCompactionThroughput.class,
 +                GetStreamThroughput.class,
 +                GetEndpoints.class,
 +                GetSSTables.class,
 +                GossipInfo.class,
 +                InvalidateKeyCache.class,
 +                InvalidateRowCache.class,
 +                InvalidateCounterCache.class,
 +                Join.class,
 +                Move.class,
 +                PauseHandoff.class,
 +                ResumeHandoff.class,
 +                ProxyHistograms.class,
 +                Rebuild.class,
 +                Refresh.class,
 +                RemoveToken.class,
 +                RemoveNode.class,
 +                Repair.class,
 +                SetCacheCapacity.class,
 +                SetHintedHandoffThrottleInKB.class,
 +                SetCompactionThreshold.class,
 +                SetCompactionThroughput.class,
 +                SetStreamThroughput.class,
 +                SetTraceProbability.class,
 +                Snapshot.class,
 +                ListSnapshots.class,
 +                Status.class,
 +                StatusBinary.class,
 +                StatusGossip.class,
 +                StatusThrift.class,
 +                Stop.class,
 +                StopDaemon.class,
 +                Version.class,
 +                DescribeRing.class,
 +                RebuildIndex.class,
 +                RangeKeySample.class,
 +                EnableBackup.class,
 +                DisableBackup.class,
 +                ResetLocalSchema.class,
 +                ReloadTriggers.class,
 +                SetCacheKeysToSave.class,
 +                DisableThrift.class,
 +                DisableHandoff.class,
 +                Drain.class,
 +                TruncateHints.class,
 +                TpStats.class,
 +                SetLoggingLevel.class,
 +                GetLoggingLevels.class
 +        );
 +
 +        Cli<Runnable> parser = Cli.<Runnable>builder("nodetool")
 +                .withDescription("Manage your Cassandra cluster")
 +                .withDefaultCommand(Help.class)
 +                .withCommands(commands)
 +                .build();
 +
 +        int status = 0;
 +        try
 +        {
 +            Runnable parse = parser.parse(args);
 +            printHistory(args);
 +            parse.run();
 +        } catch (IllegalArgumentException |
 +                IllegalStateException |
 +                ParseArgumentsMissingException |
 +                ParseArgumentsUnexpectedException |
 +                ParseOptionConversionException |
 +                ParseOptionMissingException |
 +                ParseOptionMissingValueException |
 +                ParseCommandMissingException |
 +                ParseCommandUnrecognizedException e)
 +        {
 +            badUse(e);
 +            status = 1;
 +        } catch (Throwable throwable)
 +        {
 +            err(Throwables.getRootCause(throwable));
 +            status = 2;
 +        }
 +
 +        System.exit(status);
 +    }
 +
 +    private static void printHistory(String... args)
 +    {
 +        //don't bother to print if no args passed (meaning, nodetool is just printing out the sub-commands list)
 +        if (args.length == 0)
 +            return;
 +
 +        String cmdLine = Joiner.on(" ").skipNulls().join(args);
 +        cmdLine = cmdLine.replaceFirst("(?<=(-pw|--password))\\s+\\S+", " <hidden>");
 +
 +        try (FileWriter writer = new FileWriter(new File(FBUtilities.getToolsOutputDirectory(), HISTORYFILE), true))
 +        {
 +            SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
 +            writer.append(sdf.format(new Date())).append(": ").append(cmdLine).append(System.lineSeparator());
 +        }
 +        catch (IOException | IOError ioe)
 +        {
 +            //quietly ignore any errors about not being able to write out history
 +        }
 +    }
 +
 +    private static void badUse(Exception e)
 +    {
 +        System.out.println("nodetool: " + e.getMessage());
 +        System.out.println("See 'nodetool help' or 'nodetool help <command>'.");
 +    }
 +
 +    private static void err(Throwable e)
 +    {
 +        System.err.println("error: " + e.getMessage());
 +        System.err.println("-- StackTrace --");
 +        System.err.println(getStackTraceAsString(e));
 +    }
 +
 +    public static abstract class NodeToolCmd implements Runnable
 +    {
 +
 +        @Option(type = OptionType.GLOBAL, name = {"-h", "--host"}, description = "Node hostname or ip address")
 +        private String host = "127.0.0.1";
 +
 +        @Option(type = OptionType.GLOBAL, name = {"-p", "--port"}, description = "Remote jmx agent port number")
 +        private String port = "7199";
 +
 +        @Option(type = OptionType.GLOBAL, name = {"-u", "--username"}, description = "Remote jmx agent username")
 +        private String username = EMPTY;
 +
 +        @Option(type = OptionType.GLOBAL, name = {"-pw", "--password"}, description = "Remote jmx agent password")
 +        private String password = EMPTY;
 +
 +        @Option(type = OptionType.GLOBAL, name = {"-pwf", "--password-file"}, description = "Path to the JMX password file")
 +        private String passwordFilePath = EMPTY;
 +
 +        @Override
 +        public void run()
 +        {
 +            if (isNotEmpty(username)) {
 +                if (isNotEmpty(passwordFilePath))
 +                    password = readUserPasswordFromFile(username, passwordFilePath);
 +
 +                if (isEmpty(password))
 +                    password = promptAndReadPassword();
 +            }
 +
 +            try (NodeProbe probe = connect())
 +            {
 +                execute(probe);
 +            } 
 +            catch (IOException e)
 +            {
 +                throw new RuntimeException("Error while closing JMX connection", e);
 +            }
 +
 +        }
 +
 +        private String readUserPasswordFromFile(String username, String passwordFilePath) {
 +            String password = EMPTY;
 +
 +            File passwordFile = new File(passwordFilePath);
 +            try (Scanner scanner = new Scanner(passwordFile).useDelimiter("\\s+"))
 +            {
 +                while (scanner.hasNextLine())
 +                {
 +                    if (scanner.hasNext())
 +                    {
 +                        String jmxRole = scanner.next();
 +                        if (jmxRole.equals(username) && scanner.hasNext())
 +                        {
 +                            password = scanner.next();
 +                            break;
 +                        }
 +                    }
 +                    scanner.nextLine();
 +                }
 +            } catch (FileNotFoundException e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +
 +            return password;
 +        }
 +
 +        private String promptAndReadPassword()
 +        {
 +            String password = EMPTY;
 +
 +            Console console = System.console();
 +            if (console != null)
 +                password = String.valueOf(console.readPassword("Password:"));
 +
 +            return password;
 +        }
 +
 +        protected abstract void execute(NodeProbe probe);
 +
 +        private NodeProbe connect()
 +        {
 +            NodeProbe nodeClient = null;
 +
 +            try
 +            {
 +                if (username.isEmpty())
 +                    nodeClient = new NodeProbe(host, parseInt(port));
 +                else
 +                    nodeClient = new NodeProbe(host, parseInt(port), username, password);
 +            } catch (IOException e)
 +            {
 +                Throwable rootCause = Throwables.getRootCause(e);
 +                System.err.println(format("nodetool: Failed to connect to '%s:%s' - %s: '%s'.", host, port, rootCause.getClass().getSimpleName(), rootCause.getMessage()));
 +                System.exit(1);
 +            }
 +
 +            return nodeClient;
 +        }
 +
 +        protected List<String> parseOptionalKeyspace(List<String> cmdArgs, NodeProbe nodeProbe)
 +        {
 +            List<String> keyspaces = new ArrayList<>();
 +
 +            if (cmdArgs == null || cmdArgs.isEmpty())
 +                keyspaces.addAll(nodeProbe.getKeyspaces());
 +            else
 +                keyspaces.add(cmdArgs.get(0));
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                if (!nodeProbe.getKeyspaces().contains(keyspace))
 +                    throw new IllegalArgumentException("Keyspace [" + keyspace + "] does not exist.");
 +            }
 +
 +            return Collections.unmodifiableList(keyspaces);
 +        }
 +
 +        protected String[] parseOptionalColumnFamilies(List<String> cmdArgs)
 +        {
 +            return cmdArgs.size() <= 1 ? EMPTY_STRING_ARRAY : toArray(cmdArgs.subList(1, cmdArgs.size()), String.class);
 +        }
 +    }
 +
 +    @Command(name = "info", description = "Print node information (uptime, load, ...)")
 +    public static class Info extends NodeToolCmd
 +    {
 +        @Option(name = {"-T", "--tokens"}, description = "Display all tokens")
 +        private boolean tokens = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            boolean gossipInitialized = probe.isInitialized();
 +
 +            System.out.printf("%-17s: %s%n", "ID", probe.getLocalHostId());
 +            System.out.printf("%-17s: %s%n", "Gossip active", gossipInitialized);
 +            System.out.printf("%-17s: %s%n", "Thrift active", probe.isThriftServerRunning());
 +            System.out.printf("%-17s: %s%n", "Native Transport active", probe.isNativeTransportRunning());
 +            System.out.printf("%-17s: %s%n", "Load", probe.getLoadString());
 +            if (gossipInitialized)
 +                System.out.printf("%-17s: %s%n", "Generation No", probe.getCurrentGenerationNumber());
 +            else
 +                System.out.printf("%-17s: %s%n", "Generation No", 0);
 +
 +            // Uptime
 +            long secondsUp = probe.getUptime() / 1000;
 +            System.out.printf("%-17s: %d%n", "Uptime (seconds)", secondsUp);
 +
 +            // Memory usage
 +            MemoryUsage heapUsage = probe.getHeapMemoryUsage();
 +            double memUsed = (double) heapUsage.getUsed() / (1024 * 1024);
 +            double memMax = (double) heapUsage.getMax() / (1024 * 1024);
 +            System.out.printf("%-17s: %.2f / %.2f%n", "Heap Memory (MB)", memUsed, memMax);
 +
 +            // Data Center/Rack
 +            System.out.printf("%-17s: %s%n", "Data Center", probe.getDataCenter());
 +            System.out.printf("%-17s: %s%n", "Rack", probe.getRack());
 +
 +            // Exceptions
 +            System.out.printf("%-17s: %s%n", "Exceptions", probe.getStorageMetric("Exceptions"));
 +
 +            CacheServiceMBean cacheService = probe.getCacheServiceMBean();
 +
 +            // Key Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +            System.out.printf("%-17s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                    "Key Cache",
 +                    probe.getCacheMetric("KeyCache", "Entries"),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Size")),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Capacity")),
 +                    probe.getCacheMetric("KeyCache", "Hits"),
 +                    probe.getCacheMetric("KeyCache", "Requests"),
 +                    probe.getCacheMetric("KeyCache", "HitRate"),
 +                    cacheService.getKeyCacheSavePeriodInSeconds());
 +
 +            // Row Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +            System.out.printf("%-17s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                    "Row Cache",
 +                    probe.getCacheMetric("RowCache", "Entries"),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Size")),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Capacity")),
 +                    probe.getCacheMetric("RowCache", "Hits"),
 +                    probe.getCacheMetric("RowCache", "Requests"),
 +                    probe.getCacheMetric("RowCache", "HitRate"),
 +                    cacheService.getRowCacheSavePeriodInSeconds());
 +
 +            // Counter Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +            System.out.printf("%-17s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                    "Counter Cache",
 +                    probe.getCacheMetric("CounterCache", "Entries"),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Size")),
 +                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Capacity")),
 +                    probe.getCacheMetric("CounterCache", "Hits"),
 +                    probe.getCacheMetric("CounterCache", "Requests"),
 +                    probe.getCacheMetric("CounterCache", "HitRate"),
 +                    cacheService.getCounterCacheSavePeriodInSeconds());
 +
 +            // Tokens
 +            List<String> tokens = probe.getTokens();
 +            if (tokens.size() == 1 || this.tokens)
 +                for (String token : tokens)
 +                    System.out.printf("%-17s: %s%n", "Token", token);
 +            else
 +                System.out.printf("%-17s: (invoke with -T/--tokens to see all %d tokens)%n", "Token", tokens.size());
 +        }
 +    }
 +
 +    @Command(name = "ring", description = "Print information about the token ring")
 +    public static class Ring extends NodeToolCmd
 +    {
 +        @Arguments(description = "Specify a keyspace for accurate ownership information (topology awareness)")
 +        private String keyspace = null;
 +
 +        @Option(title = "resolve_ip", name = {"-r", "--resolve-ip"}, description = "Show node domain names instead of IPs")
 +        private boolean resolveIp = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            Map<String, String> tokensToEndpoints = probe.getTokenToEndpointMap();
 +            LinkedHashMultimap<String, String> endpointsToTokens = LinkedHashMultimap.create();
 +            boolean haveVnodes = false;
 +            for (Map.Entry<String, String> entry : tokensToEndpoints.entrySet())
 +            {
 +                haveVnodes |= endpointsToTokens.containsKey(entry.getValue());
 +                endpointsToTokens.put(entry.getValue(), entry.getKey());
 +            }
 +
 +            int maxAddressLength = Collections.max(endpointsToTokens.keys(), new Comparator<String>()
 +            {
 +                @Override
 +                public int compare(String first, String second)
 +                {
 +                    return ((Integer) first.length()).compareTo(second.length());
 +                }
 +            }).length();
 +
 +            String formatPlaceholder = "%%-%ds  %%-12s%%-7s%%-8s%%-16s%%-20s%%-44s%%n";
 +            String format = format(formatPlaceholder, maxAddressLength);
 +
 +            StringBuffer errors = new StringBuffer();
 +            boolean showEffectiveOwnership = true;
 +            // Calculate per-token ownership of the ring
 +            Map<InetAddress, Float> ownerships;
 +            try
 +            {
 +                ownerships = probe.effectiveOwnership(keyspace);
 +            } 
 +            catch (IllegalStateException ex)
 +            {
 +                ownerships = probe.getOwnership();
 +                errors.append("Note: " + ex.getMessage() + "%n");
 +                showEffectiveOwnership = false;
 +            } 
 +            catch (IllegalArgumentException ex)
 +            {
 +                System.out.printf("%nError: " + ex.getMessage() + "%n");
 +                return;
 +            }
 +
 +            
 +            System.out.println();
 +            for (Entry<String, SetHostStat> entry : getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
 +                printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
 +
 +            if (haveVnodes)
 +            {
 +                System.out.println("  Warning: \"nodetool ring\" is used to output all the tokens of a node.");
 +                System.out.println("  To view status related info of a node use \"nodetool status\" instead.\n");
 +            }
 +
 +            System.out.printf("%n  " + errors.toString());
 +        }
 +
 +        private void printDc(NodeProbe probe, String format,
 +                             String dc,
 +                             LinkedHashMultimap<String, String> endpointsToTokens,
 +                             SetHostStat hoststats,boolean showEffectiveOwnership)
 +        {
 +            Collection<String> liveNodes = probe.getLiveNodes();
 +            Collection<String> deadNodes = probe.getUnreachableNodes();
 +            Collection<String> joiningNodes = probe.getJoiningNodes();
 +            Collection<String> leavingNodes = probe.getLeavingNodes();
 +            Collection<String> movingNodes = probe.getMovingNodes();
 +            Map<String, String> loadMap = probe.getLoadMap();
 +
 +            System.out.println("Datacenter: " + dc);
 +            System.out.println("==========");
 +
 +            // get the total amount of replicas for this dc and the last token in this dc's ring
 +            List<String> tokens = new ArrayList<>();
 +            String lastToken = "";
 +
 +            for (HostStat stat : hoststats)
 +            {
 +                tokens.addAll(endpointsToTokens.get(stat.endpoint.getHostAddress()));
 +                lastToken = tokens.get(tokens.size() - 1);
 +            }
 +
 +            System.out.printf(format, "Address", "Rack", "Status", "State", "Load", "Owns", "Token");
 +
 +            if (hoststats.size() > 1)
 +                System.out.printf(format, "", "", "", "", "", "", lastToken);
 +            else
 +                System.out.println();
 +
 +            for (HostStat stat : hoststats)
 +            {
 +                String endpoint = stat.endpoint.getHostAddress();
 +                String rack;
 +                try
 +                {
 +                    rack = probe.getEndpointSnitchInfoProxy().getRack(endpoint);
 +                }
 +                catch (UnknownHostException e)
 +                {
 +                    rack = "Unknown";
 +                }
 +
 +                String status = liveNodes.contains(endpoint)
 +                        ? "Up"
 +                        : deadNodes.contains(endpoint)
 +                                ? "Down"
 +                                : "?";
 +
 +                String state = "Normal";
 +
 +                if (joiningNodes.contains(endpoint))
 +                    state = "Joining";
 +                else if (leavingNodes.contains(endpoint))
 +                    state = "Leaving";
 +                else if (movingNodes.contains(endpoint))
 +                    state = "Moving";
 +
 +                String load = loadMap.containsKey(endpoint)
 +                        ? loadMap.get(endpoint)
 +                        : "?";
 +                String owns = stat.owns != null && showEffectiveOwnership? new DecimalFormat("##0.00%").format(stat.owns) : "?";
 +                System.out.printf(format, stat.ipOrDns(), rack, status, state, load, owns, stat.token);
 +            }
 +            System.out.println();
 +        }
 +    }
 +
 +    @Command(name = "netstats", description = "Print network information on provided host (connecting node by default)")
 +    public static class NetStats extends NodeToolCmd
 +    {
 +        @Option(title = "human_readable",
 +                name = {"-H", "--human-readable"},
 +                description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
 +        private boolean humanReadable = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.out.printf("Mode: %s%n", probe.getOperationMode());
 +            Set<StreamState> statuses = probe.getStreamStatus();
 +            if (statuses.isEmpty())
 +                System.out.println("Not sending any streams.");
 +            for (StreamState status : statuses)
 +            {
 +                System.out.printf("%s %s%n", status.description, status.planId.toString());
 +                for (SessionInfo info : status.sessions)
 +                {
 +                    System.out.printf("    %s", info.peer.toString());
 +                    // print private IP when it is used
 +                    if (!info.peer.equals(info.connecting))
 +                    {
 +                        System.out.printf(" (using %s)", info.connecting.toString());
 +                    }
 +                    System.out.printf("%n");
 +                    if (!info.receivingSummaries.isEmpty())
 +                    {
 +                        if (humanReadable)
 +                            System.out.printf("        Receiving %d files, %s total%n", info.getTotalFilesToReceive(), FileUtils.stringifyFileSize(info.getTotalSizeToReceive()));
 +                        else
 +                            System.out.printf("        Receiving %d files, %d bytes total%n", info.getTotalFilesToReceive(), info.getTotalSizeToReceive());
 +                        for (ProgressInfo progress : info.getReceivingFiles())
 +                        {
 +                            System.out.printf("            %s%n", progress.toString());
 +                        }
 +                    }
 +                    if (!info.sendingSummaries.isEmpty())
 +                    {
 +                        if (humanReadable)
 +                            System.out.printf("        Sending %d files, %s total%n", info.getTotalFilesToSend(), FileUtils.stringifyFileSize(info.getTotalSizeToSend()));
 +                        else
 +                            System.out.printf("        Sending %d files, %d bytes total%n", info.getTotalFilesToSend(), info.getTotalSizeToSend());
 +                        for (ProgressInfo progress : info.getSendingFiles())
 +                        {
 +                            System.out.printf("            %s%n", progress.toString());
 +                        }
 +                    }
 +                }
 +            }
 +
 +            System.out.printf("Read Repair Statistics:%nAttempted: %d%nMismatch (Blocking): %d%nMismatch (Background): %d%n", probe.getReadRepairAttempted(), probe.getReadRepairRepairedBlocking(), probe.getReadRepairRepairedBackground());
 +
 +            MessagingServiceMBean ms = probe.msProxy;
 +            System.out.printf("%-25s", "Pool Name");
 +            System.out.printf("%10s", "Active");
 +            System.out.printf("%10s", "Pending");
 +            System.out.printf("%15s%n", "Completed");
 +
 +            int pending;
 +            long completed;
 +
 +            pending = 0;
 +            for (int n : ms.getCommandPendingTasks().values())
 +                pending += n;
 +            completed = 0;
 +            for (long n : ms.getCommandCompletedTasks().values())
 +                completed += n;
 +            System.out.printf("%-25s%10s%10s%15s%n", "Commands", "n/a", pending, completed);
 +
 +            pending = 0;
 +            for (int n : ms.getResponsePendingTasks().values())
 +                pending += n;
 +            completed = 0;
 +            for (long n : ms.getResponseCompletedTasks().values())
 +                completed += n;
 +            System.out.printf("%-25s%10s%10s%15s%n", "Responses", "n/a", pending, completed);
 +        }
 +    }
 +
 +    @Command(name = "cfstats", description = "Print statistics on column families")
 +    public static class CfStats extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace.cfname>...]", description = "List of column families (or keyspace) names")
 +        private List<String> cfnames = new ArrayList<>();
 +
 +        @Option(name = "-i", description = "Ignore the list of column families and display the remaining cfs")
 +        private boolean ignore = false;
 +
 +        @Option(title = "human_readable",
 +                name = {"-H", "--human-readable"},
 +                description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
 +        private boolean humanReadable = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            OptionFilter filter = new OptionFilter(ignore, cfnames);
 +            Map<String, List<ColumnFamilyStoreMBean>> cfstoreMap = new HashMap<>();
 +
 +            // get a list of column family stores
 +            Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> cfamilies = probe.getColumnFamilyStoreMBeanProxies();
 +
 +            while (cfamilies.hasNext())
 +            {
 +                Map.Entry<String, ColumnFamilyStoreMBean> entry = cfamilies.next();
 +                String keyspaceName = entry.getKey();
 +                ColumnFamilyStoreMBean cfsProxy = entry.getValue();
 +
 +                if (!cfstoreMap.containsKey(keyspaceName) && filter.isColumnFamilyIncluded(entry.getKey(), cfsProxy.getColumnFamilyName()))
 +                {
 +                    List<ColumnFamilyStoreMBean> columnFamilies = new ArrayList<>();
 +                    columnFamilies.add(cfsProxy);
 +                    cfstoreMap.put(keyspaceName, columnFamilies);
 +                } else if (filter.isColumnFamilyIncluded(entry.getKey(), cfsProxy.getColumnFamilyName()))
 +                {
 +                    cfstoreMap.get(keyspaceName).add(cfsProxy);
 +                }
 +            }
 +
 +            // make sure all specified kss and cfs exist
 +            filter.verifyKeyspaces(probe.getKeyspaces());
 +            filter.verifyColumnFamilies();
 +
 +            // print out the table statistics
 +            for (Map.Entry<String, List<ColumnFamilyStoreMBean>> entry : cfstoreMap.entrySet())
 +            {
 +                String keyspaceName = entry.getKey();
 +                List<ColumnFamilyStoreMBean> columnFamilies = entry.getValue();
 +                long keyspaceReadCount = 0;
 +                long keyspaceWriteCount = 0;
 +                int keyspacePendingFlushes = 0;
 +                double keyspaceTotalReadTime = 0.0f;
 +                double keyspaceTotalWriteTime = 0.0f;
 +
 +                System.out.println("Keyspace: " + keyspaceName);
 +                for (ColumnFamilyStoreMBean cfstore : columnFamilies)
 +                {
 +                    String cfName = cfstore.getColumnFamilyName();
 +                    long writeCount = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getCount();
 +                    long readCount = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getCount();
 +
 +                    if (readCount > 0)
 +                    {
 +                        keyspaceReadCount += readCount;
 +                        keyspaceTotalReadTime += (long) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadTotalLatency");
 +                    }
 +                    if (writeCount > 0)
 +                    {
 +                        keyspaceWriteCount += writeCount;
 +                        keyspaceTotalWriteTime += (long) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteTotalLatency");
 +                    }
 +                    keyspacePendingFlushes += (long) probe.getColumnFamilyMetric(keyspaceName, cfName, "PendingFlushes");
 +                }
 +
 +                double keyspaceReadLatency = keyspaceReadCount > 0
 +                                             ? keyspaceTotalReadTime / keyspaceReadCount / 1000
 +                                             : Double.NaN;
 +                double keyspaceWriteLatency = keyspaceWriteCount > 0
 +                                              ? keyspaceTotalWriteTime / keyspaceWriteCount / 1000
 +                                              : Double.NaN;
 +
 +                System.out.println("\tRead Count: " + keyspaceReadCount);
 +                System.out.println("\tRead Latency: " + String.format("%s", keyspaceReadLatency) + " ms.");
 +                System.out.println("\tWrite Count: " + keyspaceWriteCount);
 +                System.out.println("\tWrite Latency: " + String.format("%s", keyspaceWriteLatency) + " ms.");
 +                System.out.println("\tPending Flushes: " + keyspacePendingFlushes);
 +
 +                // print out column family statistics for this keyspace
 +                for (ColumnFamilyStoreMBean cfstore : columnFamilies)
 +                {
 +                    String cfName = cfstore.getColumnFamilyName();
 +                    if (cfName.contains("."))
 +                        System.out.println("\t\tTable (index): " + cfName);
 +                    else
 +                        System.out.println("\t\tTable: " + cfName);
 +
 +                    System.out.println("\t\tSSTable count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveSSTableCount"));
 +
 +                    int[] leveledSStables = cfstore.getSSTableCountPerLevel();
 +                    if (leveledSStables != null)
 +                    {
 +                        System.out.print("\t\tSSTables in each level: [");
 +                        for (int level = 0; level < leveledSStables.length; level++)
 +                        {
 +                            int count = leveledSStables[level];
 +                            System.out.print(count);
 +                            long maxCount = 4L; // for L0
 +                            if (level > 0)
 +                                maxCount = (long) Math.pow(10, level);
 +                            //  show max threshold for level when exceeded
 +                            if (count > maxCount)
 +                                System.out.print("/" + maxCount);
 +
 +                            if (level < leveledSStables.length - 1)
 +                                System.out.print(", ");
 +                            else
 +                                System.out.println("]");
 +                        }
 +                    }
 +                    System.out.println("\t\tSpace used (live): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveDiskSpaceUsed"), humanReadable));
 +                    System.out.println("\t\tSpace used (total): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "TotalDiskSpaceUsed"), humanReadable));
 +                    System.out.println("\t\tSpace used by snapshots (total): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "SnapshotsSize"), humanReadable));
 +                    System.out.println("\t\tSSTable Compression Ratio: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "CompressionRatio"));
 +                    System.out.println("\t\tMemtable cell count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableColumnsCount"));
 +                    System.out.println("\t\tMemtable data size: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableLiveDataSize"), humanReadable));
 +                    System.out.println("\t\tMemtable switch count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableSwitchCount"));
 +                    System.out.println("\t\tLocal read count: " + ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getCount());
 +                    double localReadLatency = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getMean() / 1000;
 +                    double localRLatency = localReadLatency > 0 ? localReadLatency : Double.NaN;
 +                    System.out.printf("\t\tLocal read latency: %01.3f ms%n", localRLatency);
 +                    System.out.println("\t\tLocal write count: " + ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getCount());
 +                    double localWriteLatency = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getMean() / 1000;
 +                    double localWLatency = localWriteLatency > 0 ? localWriteLatency : Double.NaN;
 +                    System.out.printf("\t\tLocal write latency: %01.3f ms%n", localWLatency);
 +                    System.out.println("\t\tPending flushes: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "PendingFlushes"));
 +                    System.out.println("\t\tBloom filter false positives: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterFalsePositives"));
 +                    System.out.printf("\t\tBloom filter false ratio: %s%n", String.format("%01.5f", probe.getColumnFamilyMetric(keyspaceName, cfName, "RecentBloomFilterFalseRatio")));
 +                    System.out.println("\t\tBloom filter space used: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterDiskSpaceUsed"), humanReadable));
 +                    System.out.println("\t\tCompacted partition minimum bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MinRowSize"), humanReadable));
 +                    System.out.println("\t\tCompacted partition maximum bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MaxRowSize"), humanReadable));
 +                    System.out.println("\t\tCompacted partition mean bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MeanRowSize"), humanReadable));
 +                    JmxReporter.HistogramMBean histogram = (JmxReporter.HistogramMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveScannedHistogram");
 +                    System.out.println("\t\tAverage live cells per slice (last five minutes): " + histogram.getMean());
 +                    System.out.println("\t\tMaximum live cells per slice (last five minutes): " + histogram.getMax());
 +                    histogram = (JmxReporter.HistogramMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "TombstoneScannedHistogram");
 +                    System.out.println("\t\tAverage tombstones per slice (last five minutes): " + histogram.getMean());
 +                    System.out.println("\t\tMaximum tombstones per slice (last five minutes): " + histogram.getMax());
 +
 +                    System.out.println("");
 +                }
 +                System.out.println("----------------");
 +            }
 +        }
 +
 +        private String format(long bytes, boolean humanReadable) {
 +            return humanReadable ? FileUtils.stringifyFileSize(bytes) : Long.toString(bytes);
 +        }
 +
 +        /**
 +         * Used for filtering keyspaces and columnfamilies to be displayed using the cfstats command.
 +         */
 +        private static class OptionFilter
 +        {
 +            private Map<String, List<String>> filter = new HashMap<>();
 +            private Map<String, List<String>> verifier = new HashMap<>();
 +            private List<String> filterList = new ArrayList<>();
 +            private boolean ignoreMode;
 +
 +            public OptionFilter(boolean ignoreMode, List<String> filterList)
 +            {
 +                this.filterList.addAll(filterList);
 +                this.ignoreMode = ignoreMode;
 +
 +                for (String s : filterList)
 +                {
 +                    String[] keyValues = s.split("\\.", 2);
 +
 +                    // build the map that stores the ks' and cfs to use
 +                    if (!filter.containsKey(keyValues[0]))
 +                    {
 +                        filter.put(keyValues[0], new ArrayList<String>());
 +                        verifier.put(keyValues[0], new ArrayList<String>());
 +
 +                        if (keyValues.length == 2)
 +                        {
 +                            filter.get(keyValues[0]).add(keyValues[1]);
 +                            verifier.get(keyValues[0]).add(keyValues[1]);
 +                        }
 +                    } else
 +                    {
 +                        if (keyValues.length == 2)
 +                        {
 +                            filter.get(keyValues[0]).add(keyValues[1]);
 +                            verifier.get(keyValues[0]).add(keyValues[1]);
 +                        }
 +                    }
 +                }
 +            }
 +
 +            public boolean isColumnFamilyIncluded(String keyspace, String columnFamily)
 +            {
 +                // supplying empty params list is treated as wanting to display all kss & cfs
 +                if (filterList.isEmpty())
 +                    return !ignoreMode;
 +
 +                List<String> cfs = filter.get(keyspace);
 +
 +                // no such keyspace is in the map
 +                if (cfs == null)
 +                    return ignoreMode;
 +                    // only a keyspace with no cfs was supplied
 +                    // so ignore or include (based on the flag) every column family in specified keyspace
 +                else if (cfs.size() == 0)
 +                    return !ignoreMode;
 +
 +                // keyspace exists, and it contains specific cfs
 +                verifier.get(keyspace).remove(columnFamily);
 +                return ignoreMode ^ cfs.contains(columnFamily);
 +            }
 +
 +            public void verifyKeyspaces(List<String> keyspaces)
 +            {
 +                for (String ks : verifier.keySet())
 +                    if (!keyspaces.contains(ks))
 +                        throw new IllegalArgumentException("Unknown keyspace: " + ks);
 +            }
 +
 +            public void verifyColumnFamilies()
 +            {
 +                for (String ks : filter.keySet())
 +                    if (verifier.get(ks).size() > 0)
 +                        throw new IllegalArgumentException("Unknown column families: " + verifier.get(ks).toString() + " in keyspace: " + ks);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "cfhistograms", description = "Print statistic histograms for a given column family")
 +    public static class CfHistograms extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<keyspace> <cfname>", description = "The keyspace and column family name")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 2, "cfhistograms requires ks and cf args");
 +
 +            String keyspace = args.get(0);
 +            String cfname = args.get(1);
 +
 +            // calculate percentile of row size and column count
 +            long[] estimatedRowSize = (long[]) probe.getColumnFamilyMetric(keyspace, cfname, "EstimatedRowSizeHistogram");
 +            long[] estimatedColumnCount = (long[]) probe.getColumnFamilyMetric(keyspace, cfname, "EstimatedColumnCountHistogram");
 +
 +            long[] rowSizeBucketOffsets = new EstimatedHistogram(estimatedRowSize.length).getBucketOffsets();
 +            long[] columnCountBucketOffsets = new EstimatedHistogram(estimatedColumnCount.length).getBucketOffsets();
 +            EstimatedHistogram rowSizeHist = new EstimatedHistogram(rowSizeBucketOffsets, estimatedRowSize);
 +            EstimatedHistogram columnCountHist = new EstimatedHistogram(columnCountBucketOffsets, estimatedColumnCount);
 +
 +            // build arrays to store percentile values
 +            double[] estimatedRowSizePercentiles = new double[7];
 +            double[] estimatedColumnCountPercentiles = new double[7];
 +            double[] offsetPercentiles = new double[]{0.5, 0.75, 0.95, 0.98, 0.99};
 +
 +            if (rowSizeHist.isOverflowed())
 +            {
 +                System.err.println(String.format("Row sizes are larger than %s, unable to calculate percentiles", rowSizeBucketOffsets[rowSizeBucketOffsets.length - 1]));
 +                for (int i = 0; i < offsetPercentiles.length; i++)
 +                        estimatedRowSizePercentiles[i] = Double.NaN;
 +            }
 +            else
 +            {
 +                for (int i = 0; i < offsetPercentiles.length; i++)
 +                    estimatedRowSizePercentiles[i] = rowSizeHist.percentile(offsetPercentiles[i]);
 +            }
 +
 +            if (columnCountHist.isOverflowed())
 +            {
 +                System.err.println(String.format("Column counts are larger than %s, unable to calculate percentiles", columnCountBucketOffsets[columnCountBucketOffsets.length - 1]));
 +                for (int i = 0; i < estimatedColumnCountPercentiles.length; i++)
 +                    estimatedColumnCountPercentiles[i] = Double.NaN;
 +            }
 +            else
 +            {
 +                for (int i = 0; i < offsetPercentiles.length; i++)
 +                    estimatedColumnCountPercentiles[i] = columnCountHist.percentile(offsetPercentiles[i]);
 +            }
 +
 +            // min value
 +            estimatedRowSizePercentiles[5] = rowSizeHist.min();
 +            estimatedColumnCountPercentiles[5] = columnCountHist.min();
 +            // max value
 +            estimatedRowSizePercentiles[6] = rowSizeHist.max();
 +            estimatedColumnCountPercentiles[6] = columnCountHist.max();
 +
 +            String[] percentiles = new String[]{"50%", "75%", "95%", "98%", "99%", "Min", "Max"};
 +            double[] readLatency = probe.metricPercentilesAsArray((JmxReporter.HistogramMBean) probe.getColumnFamilyMetric(keyspace, cfname, "ReadLatency"));
 +            double[] writeLatency = probe.metricPercentilesAsArray((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspace, cfname, "WriteLatency"));
 +            double[] sstablesPerRead = probe.metricPercentilesAsArray((JmxReporter.HistogramMBean) probe.getColumnFamilyMetric(keyspace, cfname, "SSTablesPerReadHistogram"));
 +
 +            System.out.println(format("%s/%s histograms", keyspace, cfname));
 +            System.out.println(format("%-10s%10s%18s%18s%18s%18s",
 +                    "Percentile", "SSTables", "Write Latency", "Read Latency", "Partition Size", "Cell Count"));
 +            System.out.println(format("%-10s%10s%18s%18s%18s%18s",
 +                    "", "", "(micros)", "(micros)", "(bytes)", ""));
 +
 +            for (int i = 0; i < percentiles.length; i++)
 +            {
 +                System.out.println(format("%-10s%10.2f%18.2f%18.2f%18.0f%18.0f",
 +                        percentiles[i],
 +                        sstablesPerRead[i],
 +                        writeLatency[i],
 +                        readLatency[i],
 +                        estimatedRowSizePercentiles[i],
 +                        estimatedColumnCountPercentiles[i]));
 +            }
 +            System.out.println();
 +        }
 +    }
 +
 +    @Command(name = "cleanup", description = "Triggers the immediate cleanup of keys no longer belonging to a node. By default, clean all keyspaces")
 +    public static class Cleanup extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                if (Keyspace.SYSTEM_KS.equals(keyspace))
 +                    continue;
 +
 +                try
 +                {
 +                    probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during cleanup", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "clearsnapshot", description = "Remove the snapshot with the given name from the given keyspaces. If no snapshotName is specified we will remove all snapshots")
 +    public static class ClearSnapshot extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspaces>...] ", description = "Remove snapshots from the given keyspaces")
 +        private List<String> keyspaces = new ArrayList<>();
 +
 +        @Option(title = "snapshot_name", name = "-t", description = "Remove the snapshot with a given name")
 +        private String snapshotName = EMPTY;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            StringBuilder sb = new StringBuilder();
 +
 +            sb.append("Requested clearing snapshot(s) for ");
 +
 +            if (keyspaces.isEmpty())
 +                sb.append("[all keyspaces]");
 +            else
 +                sb.append("[").append(join(keyspaces, ", ")).append("]");
 +
 +            if (!snapshotName.isEmpty())
 +                sb.append(" with snapshot name [").append(snapshotName).append("]");
 +
 +            System.out.println(sb.toString());
 +
 +            try
 +            {
 +                probe.clearSnapshot(snapshotName, toArray(keyspaces, String.class));
 +            } catch (IOException e)
 +            {
 +                throw new RuntimeException("Error during clearing snapshots", e);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "compact", description = "Force a (major) compaction on one or more column families")
 +    public static class Compact extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.forceKeyspaceCompaction(keyspace, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during compaction", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "flush", description = "Flush one or more column families")
 +    public static class Flush extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.forceKeyspaceFlush(keyspace, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during flushing", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more column families")
 +    public static class Scrub extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Option(title = "disable_snapshot",
 +                name = {"-ns", "--no-snapshot"},
 +                description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
 +        private boolean disableSnapshot = false;
 +
 +        @Option(title = "skip_corrupted",
 +                name = {"-s", "--skip-corrupted"},
 +                description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
 +        private boolean skipCorrupted = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.scrub(System.out, disableSnapshot, skipCorrupted, keyspace, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during flushing", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "disableautocompaction", description = "Disable autocompaction for the given keyspace and column family")
 +    public static class DisableAutoCompaction extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.disableAutoCompaction(keyspace, cfnames);
 +                } catch (IOException e)
 +                {
 +                    throw new RuntimeException("Error occurred during disabling auto-compaction", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "enableautocompaction", description = "Enable autocompaction for the given keyspace and column family")
 +    public static class EnableAutoCompaction extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.enableAutoCompaction(keyspace, cfnames);
 +                } catch (IOException e)
 +                {
 +                    throw new RuntimeException("Error occurred during enabling auto-compaction", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "upgradesstables", description = "Rewrite sstables (for the requested column families) that are not on the current version (thus upgrading them to said current version)")
 +    public static class UpgradeSSTable extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
 +        private boolean includeAll = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
 +                    probe.upgradeSSTables(System.out, keyspace, !includeAll, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during enabling auto-compaction", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "compactionstats", description = "Print statistics on compactions")
 +    public static class CompactionStats extends NodeToolCmd
 +    {
 +        @Option(title = "human_readable",
 +                name = {"-H", "--human-readable"},
 +                description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
 +        private boolean humanReadable = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            int compactionThroughput = probe.getCompactionThroughput();
 +            CompactionManagerMBean cm = probe.getCompactionManagerProxy();
 +            System.out.println("pending tasks: " + probe.getCompactionMetric("PendingTasks"));
 +            long remainingBytes = 0;
 +            List<Map<String, String>> compactions = cm.getCompactions();
 +            if (!compactions.isEmpty())
 +            {
 +                List<String[]> lines = new ArrayList<>();
 +                int[] columnSizes = new int[] { 0, 0, 0, 0, 0, 0, 0 };
 +
 +                addLine(lines, columnSizes, "compaction type", "keyspace", "table", "completed", "total", "unit", "progress");
 +                for (Map<String, String> c : compactions)
 +                {
 +                    long total = Long.parseLong(c.get("total"));
 +                    long completed = Long.parseLong(c.get("completed"));
 +                    String taskType = c.get("taskType");
 +                    String keyspace = c.get("keyspace");
 +                    String columnFamily = c.get("columnfamily");
 +                    String completedStr = humanReadable ? FileUtils.stringifyFileSize(completed) : Long.toString(completed);
 +                    String totalStr = humanReadable ? FileUtils.stringifyFileSize(total) : Long.toString(total);
 +                    String unit = c.get("unit");
 +                    String percentComplete = total == 0 ? "n/a" : new DecimalFormat("0.00").format((double) completed / total * 100) + "%";
 +                    addLine(lines, columnSizes, taskType, keyspace, columnFamily, completedStr, totalStr, unit, percentComplete);
 +                    if (taskType.equals(OperationType.COMPACTION.toString()))
 +                        remainingBytes += total - completed;
 +                }
 +
 +                StringBuilder buffer = new StringBuilder();
 +                for (int columnSize : columnSizes) {
 +                    buffer.append("%");
 +                    buffer.append(columnSize + 3);
 +                    buffer.append("s");
 +                }
 +                buffer.append("%n");
 +                String format = buffer.toString();
 +
 +                for (String[] line : lines)
 +                {
 +                    System.out.printf(format, line[0], line[1], line[2], line[3], line[4], line[5], line[6]);
 +                }
 +
 +                String remainingTime = "n/a";
 +                if (compactionThroughput != 0)
 +                {
 +                    long remainingTimeInSecs = remainingBytes / (1024L * 1024L * compactionThroughput);
 +                    remainingTime = format("%dh%02dm%02ds", remainingTimeInSecs / 3600, (remainingTimeInSecs % 3600) / 60, (remainingTimeInSecs % 60));
 +                }
 +                System.out.printf("%25s%10s%n", "Active compaction remaining time : ", remainingTime);
 +            }
 +        }
 +
 +        private void addLine(List<String[]> lines, int[] columnSizes, String... columns) {
 +            lines.add(columns);
 +            for (int i = 0; i < columns.length; i++) {
 +                columnSizes[i] = Math.max(columnSizes[i], columns[i].length());
 +            }
 +        }
 +    }
 +
 +    @Command(name = "compactionhistory", description = "Print history of compaction")
 +    public static class CompactionHistory extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.out.println("Compaction History: ");
 +
 +            TabularData tabularData = probe.getCompactionHistory();
 +            if (tabularData.isEmpty())
 +            {
 +                System.out.printf("There is no compaction history");
 +                return;
 +            }
 +
 +            String format = "%-41s%-19s%-29s%-26s%-15s%-15s%s%n";
 +            List<String> indexNames = tabularData.getTabularType().getIndexNames();
 +            System.out.printf(format, toArray(indexNames, Object.class));
 +
 +            Set<?> values = tabularData.keySet();
 +            for (Object eachValue : values)
 +            {
 +                List<?> value = (List<?>) eachValue;
 +                System.out.printf(format, toArray(value, Object.class));
 +            }
 +        }
 +    }
 +
 +    @Command(name = "decommission", description = "Decommission the *node I am connecting to*")
 +    public static class Decommission extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            try
 +            {
 +                probe.decommission();
 +            } catch (InterruptedException e)
 +            {
 +                throw new RuntimeException("Error decommissioning node", e);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "describecluster", description = "Print the name, snitch, partitioner and schema version of a cluster")
 +    public static class DescribeCluster extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            // display cluster name, snitch and partitioner
 +            System.out.println("Cluster Information:");
 +            System.out.println("\tName: " + probe.getClusterName());
 +            System.out.println("\tSnitch: " + probe.getEndpointSnitchInfoProxy().getSnitchName());
 +            System.out.println("\tPartitioner: " + probe.getPartitioner());
 +
 +            // display schema version for each node
 +            System.out.println("\tSchema versions:");
 +            Map<String, List<String>> schemaVersions = probe.getSpProxy().getSchemaVersions();
 +            for (String version : schemaVersions.keySet())
 +            {
 +                System.out.println(format("\t\t%s: %s%n", version, schemaVersions.get(version)));
 +            }
 +        }
 +    }
 +
 +    @Command(name = "disablebinary", description = "Disable native transport (binary protocol)")
 +    public static class DisableBinary extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.stopNativeTransport();
 +        }
 +    }
 +
 +    @Command(name = "enablebinary", description = "Reenable native transport (binary protocol)")
 +    public static class EnableBinary extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.startNativeTransport();
 +        }
 +    }
 +
 +    @Command(name = "enablegossip", description = "Reenable gossip")
 +    public static class EnableGossip extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.startGossiping();
 +        }
 +    }
 +
 +    @Command(name = "disablegossip", description = "Disable gossip (effectively marking the node down)")
 +    public static class DisableGossip extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.stopGossiping();
 +        }
 +    }
 +
 +    @Command(name = "enablehandoff", description = "Reenable the future hints storing on the current node")
 +    public static class EnableHandoff extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<dc-name>,<dc-name>", description = "Enable hinted handoff only for these DCs")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() <= 1, "enablehandoff does not accept two args");
 +            if(args.size() == 1)
 +                probe.enableHintedHandoff(args.get(0));
 +            else
 +                probe.enableHintedHandoff();
 +        }
 +    }
 +
 +    @Command(name = "enablethrift", description = "Reenable thrift server")
 +    public static class EnableThrift extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.startThriftServer();
 +        }
 +    }
 +
 +    @Command(name = "getcompactionthreshold", description = "Print min and max compaction thresholds for a given column family")
 +    public static class GetCompactionThreshold extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<keyspace> <cfname>", description = "The keyspace with a column family")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 2, "getcompactionthreshold requires ks and cf args");
 +            String ks = args.get(0);
 +            String cf = args.get(1);
 +
 +            ColumnFamilyStoreMBean cfsProxy = probe.getCfsProxy(ks, cf);
 +            System.out.println("Current compaction thresholds for " + ks + "/" + cf + ": \n" +
 +                    " min = " + cfsProxy.getMinimumCompactionThreshold() + ", " +
 +                    " max = " + cfsProxy.getMaximumCompactionThreshold());
 +        }
 +    }
 +
 +    @Command(name = "getcompactionthroughput", description = "Print the MB/s throughput cap for compaction in the system")
 +    public static class GetCompactionThroughput extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.out.println("Current compaction throughput: " + probe.getCompactionThroughput() + " MB/s");
 +        }
 +    }
 +
 +    @Command(name = "getstreamthroughput", description = "Print the Mb/s throughput cap for streaming in the system")
 +    public static class GetStreamThroughput extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.out.println("Current stream throughput: " + probe.getStreamThroughput() + " Mb/s");
 +        }
 +    }
 +
 +    @Command(name = "getendpoints", description = "Print the end points that owns the key")
 +    public static class GetEndpoints extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<keyspace> <cfname> <key>", description = "The keyspace, the column family, and the key for which we need to find the endpoint")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 3, "getendpoints requires ks, cf and key args");
 +            String ks = args.get(0);
 +            String cf = args.get(1);
 +            String key = args.get(2);
 +
 +            List<InetAddress> endpoints = probe.getEndpoints(ks, cf, key);
 +            for (InetAddress endpoint : endpoints)
 +            {
 +                System.out.println(endpoint.getHostAddress());
 +            }
 +        }
 +    }
 +
 +    @Command(name = "getsstables", description = "Print the sstable filenames that own the key")
 +    public static class GetSSTables extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<keyspace> <cfname> <key>", description = "The keyspace, the column family, and the key")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 3, "getsstables requires ks, cf and key args");
 +            String ks = args.get(0);
 +            String cf = args.get(1);
 +            String key = args.get(2);
 +
 +            List<String> sstables = probe.getSSTables(ks, cf, key);
 +            for (String sstable : sstables)
 +            {
 +                System.out.println(sstable);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "gossipinfo", description = "Shows the gossip information for the cluster")
 +    public static class GossipInfo extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.out.println(probe.getGossipInfo());
 +        }
 +    }
 +
 +    @Command(name = "invalidatekeycache", description = "Invalidate the key cache")
 +    public static class InvalidateKeyCache extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.invalidateKeyCache();
 +        }
 +    }
 +
 +    @Command(name = "invalidaterowcache", description = "Invalidate the row cache")
 +    public static class InvalidateRowCache extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.invalidateRowCache();
 +        }
 +    }
 +
 +    @Command(name = "invalidatecountercache", description = "Invalidate the counter cache")
 +    public static class InvalidateCounterCache extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.invalidateCounterCache();
 +        }
 +    }
 +
 +    @Command(name = "join", description = "Join the ring")
 +    public static class Join extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkState(!probe.isJoined(), "This node has already joined the ring.");
 +
 +            try
 +            {
 +                probe.joinRing();
 +            } catch (IOException e)
 +            {
 +                throw new RuntimeException("Error during joining the ring", e);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "move", description = "Move node on the token ring to a new token")
 +    public static class Move extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<new token>", description = "The new token.", required = true)
 +        private String newToken = EMPTY;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            try
 +            {
 +                probe.move(newToken);
 +            } catch (IOException e)
 +            {
 +                throw new RuntimeException("Error during moving node", e);
 +            }
 +        }
 +    }
 +
 +
 +
 +    @Command(name = "pausehandoff", description = "Pause hints delivery process")
 +    public static class PauseHandoff extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.pauseHintsDelivery();
 +        }
 +    }
 +
 +    @Command(name = "resumehandoff", description = "Resume hints delivery process")
 +    public static class ResumeHandoff extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.resumeHintsDelivery();
 +        }
 +    }
 +
 +
 +    @Command(name = "proxyhistograms", description = "Print statistic histograms for network operations")
 +    public static class ProxyHistograms extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            String[] percentiles = new String[]{"50%", "75%", "95%", "98%", "99%", "Min", "Max"};
 +            double[] readLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("Read"));
 +            double[] writeLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("Write"));
 +            double[] rangeLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("RangeSlice"));
 +
 +            System.out.println("proxy histograms");
 +            System.out.println(format("%-10s%18s%18s%18s",
 +                    "Percentile", "Read Latency", "Write Latency", "Range Latency"));
 +            System.out.println(format("%-10s%18s%18s%18s",
 +                    "", "(micros)", "(micros)", "(micros)"));
 +            for (int i = 0; i < percentiles.length; i++)
 +            {
 +                System.out.println(format("%-10s%18.2f%18.2f%18.2f",
 +                        percentiles[i],
 +                        readLatency[i],
 +                        writeLatency[i],
 +                        rangeLatency[i]));
 +            }
 +            System.out.println();
 +        }
 +    }
 +
 +    @Command(name = "rebuild", description = "Rebuild data by streaming from other nodes (similarly to bootstrap)")
 +    public static class Rebuild extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<src-dc-name>", description = "Name of DC from which to select sources for streaming. By default, pick any DC")
 +        private String sourceDataCenterName = null;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.rebuild(sourceDataCenterName);
 +        }
 +    }
 +
 +    @Command(name = "refresh", description = "Load newly placed SSTables to the system without restart")
 +    public static class Refresh extends NodeToolCmd
 +    {
 +        @Arguments(usage = "<keyspace> <cfname>", description = "The keyspace and column family name")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 2, "refresh requires ks and cf args");
 +            probe.loadNewSSTables(args.get(0), args.get(1));
 +        }
 +    }
 +
 +    @Deprecated
 +    @Command(name = "removetoken", description = "DEPRECATED (see removenode)", hidden = true)
 +    public static class RemoveToken extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            System.err.println("Warn: removetoken is deprecated, please use removenode instead");
 +        }
 +    }
 +
 +    @Command(name = "removenode", description = "Show status of current node removal, force completion of pending removal or remove provided ID")
 +    public static class RemoveNode extends NodeToolCmd
 +    {
 +        @Arguments(title = "remove_operation", usage = "<status>|<force>|<ID>", description = "Show status of current node removal, force completion of pending removal, or remove provided ID", required = true)
 +        private String removeOperation = EMPTY;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            switch (removeOperation)
 +            {
 +                case "status":
 +                    System.out.println("RemovalStatus: " + probe.getRemovalStatus());
 +                    break;
 +                case "force":
 +                    System.out.println("RemovalStatus: " + probe.getRemovalStatus());
 +                    probe.forceRemoveCompletion();
 +                    break;
 +                default:
 +                    probe.removeNode(removeOperation);
 +                    break;
 +            }
 +        }
 +    }
 +
 +    @Command(name = "repair", description = "Repair one or more column families")
 +    public static class Repair extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
 +        private List<String> args = new ArrayList<>();
 +
 +        @Option(title = "parallel", name = {"-par", "--parallel"}, description = "Use -par to carry out a parallel repair")
 +        private boolean parallel = false;
 +
++        @Option(title = "dc parallel", name = {"-dcpar", "--dc-parallel"}, description = "Use -dcpar to repair data centers in parallel.")
++        private boolean dcParallel = false;
++
 +        @Option(title = "local_dc", name = {"-local", "--in-local-dc"}, description = "Use -local to only repair against nodes in the same datacenter")
 +        private boolean localDC = false;
 +
 +        @Option(title = "specific_dc", name = {"-dc", "--in-dc"}, description = "Use -dc to repair specific datacenters")
 +        private List<String> specificDataCenters = new ArrayList<>();
 +
 +        @Option(title = "specific_host", name = {"-hosts", "--in-hosts"}, description = "Use -hosts to repair specific hosts")
 +        private List<String> specificHosts = new ArrayList<>();
 +
 +        @Option(title = "start_token", name = {"-st", "--start-token"}, description = "Use -st to specify a token at which the repair range starts")
 +        private String startToken = EMPTY;
 +
 +        @Option(title = "end_token", name = {"-et", "--end-token"}, description = "Use -et to specify a token at which repair range ends")
 +        private String endToken = EMPTY;
 +
 +        @Option(title = "primary_range", name = {"-pr", "--partitioner-range"}, description = "Use -pr to repair only the first range returned by the partitioner")
 +        private boolean primaryRange = false;
 +
 +        @Option(title = "incremental_repair", name = {"-inc", "--incremental"}, description = "Use -inc to use the new incremental repair")
 +        private boolean incrementalRepair = false;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +            String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +            if (primaryRange && (!specificDataCenters.isEmpty() || !specificHosts.isEmpty()))
 +                throw new RuntimeException("Primary range repair should be performed on all nodes in the cluster.");
 +
 +            for (String keyspace : keyspaces)
 +            {
 +                try
 +                {
++                    RepairParallelism parallelismDegree = RepairParallelism.SEQUENTIAL;
++                    if (parallel)
++                        parallelismDegree = RepairParallelism.PARALLEL;
++                    else if (dcParallel)
++                        parallelismDegree = RepairParallelism.DATACENTER_AWARE;
++
 +                    Collection<String> dataCenters = null;
 +                    Collection<String> hosts = null;
 +                    if (!specificDataCenters.isEmpty())
 +                        dataCenters = newArrayList(specificDataCenters);
 +                    else if (localDC)
 +                        dataCenters = newArrayList(probe.getDataCenter());
 +                    else if(!specificHosts.isEmpty())
 +                        hosts = newArrayList(specificHosts);
 +                    if (!startToken.isEmpty() || !endToken.isEmpty())
-                         probe.forceRepairRangeAsync(System.out, keyspace, !parallel, dataCenters,hosts, startToken, endToken, !incrementalRepair);
++                        probe.forceRepairRangeAsync(System.out, keyspace, parallelismDegree, dataCenters,hosts, startToken, endToken, !incrementalRepair);
 +                    else
-                         probe.forceRepairAsync(System.out, keyspace, !parallel, dataCenters, hosts, primaryRange, !incrementalRepair, cfnames);
++                        probe.forceRepairAsync(System.out, keyspace, parallelismDegree, dataCenters, hosts, primaryRange, !incrementalRepair, cfnames);
 +                } catch (Exception e)
 +                {
 +                    throw new RuntimeException("Error occurred during repair", e);
 +                }
 +            }
 +        }
 +    }
 +
 +    @Command(name = "setcachecapacity", description = "Set global key, row, and counter cache capacities (in MB units)")
 +    public static class SetCacheCapacity extends NodeToolCmd
 +    {
 +        @Arguments(title = "<key-cache-capacity> <row-cache-capacity> <counter-cache-capacity>",
 +                   usage = "<key-cache-capacity> <row-cache-capacity> <counter-cache-capacity>",
 +                   description = "Key cache, row cache, and counter cache (in MB)",
 +                   required = true)
 +        private List<Integer> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 3, "setcachecapacity requires key-cache-capacity, row-cache-capacity, and counter-cache-capacity args.");
 +            probe.setCacheCapacities(args.get(0), args.get(1), args.get(2));
 +        }
 +    }
 +
 +    @Command(name = "setcompactionthreshold", description = "Set min and max compaction thresholds for a given column family")
 +    public static class SetCompactionThreshold extends NodeToolCmd
 +    {
 +        @Arguments(title = "<keyspace> <cfname> <minthreshold> <maxthreshold>", usage = "<keyspace> <cfname> <minthreshold> <maxthreshold>", description = "The keyspace, the column family, min and max threshold", required = true)
 +        private List<String> args = new ArrayList<>();
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(args.size() == 4, "setcompactionthreshold requires ks, cf, min, and max threshold args.");
 +
 +            int minthreshold = parseInt(args.get(2));
 +            int maxthreshold = parseInt(args.get(3));
 +            checkArgument(minthreshold >= 0 && maxthreshold >= 0, "Thresholds must be positive integers");
 +            checkArgument(minthreshold <= maxthreshold, "Min threshold cannot be greater than max.");
 +            checkArgument(minthreshold >= 2 || maxthreshold == 0, "Min threshold must be at least 2");
 +
 +            probe.setCompactionThreshold(args.get(0), args.get(1), minthreshold, maxthreshold);
 +        }
 +    }
 +
 +    @Command(name = "setcompactionthroughput", description = "Set the MB/s throughput cap for compaction in the system, or 0 to disable throttling")
 +    public static class SetCompactionThroughput extends NodeToolCmd
 +    {
 +        @Arguments(title = "compaction_throughput", usage = "<value_in_mb>", description = "Value in MB, 0 to disable throttling", required = true)
 +        private Integer compactionThroughput = null;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.setCompactionThroughput(compactionThroughput);
 +        }
 +    }
 +
 +    @Command(name = "sethintedhandoffthrottlekb", description =  "Set hinted handoff throttle in kb per second, per delivery thread.")
 +    public static class SetHintedHandoffThrottleInKB extends NodeToolCmd
 +    {
 +        @Arguments(title = "throttle_in_kb", usage = "<value_in_kb_per_sec>", description = "Value in KB per second", required = true)
 +        private Integer throttleInKB = null;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.setHintedHandoffThrottleInKB(throttleInKB);
 +        }
 +    }
 +
 +    @Command(name = "setstreamthroughput", description = "Set the Mb/s throughput cap for streaming in the system, or 0 to disable throttling")
 +    public static class SetStreamThroughput extends NodeToolCmd
 +    {
 +        @Arguments(title = "stream_throughput", usage = "<value_in_mb>", description = "Value in Mb, 0 to disable throttling", required = true)
 +        private Integer streamThroughput = null;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            probe.setStreamThroughput(streamThroughput);
 +        }
 +    }
 +
 +    @Command(name = "settraceprobability", description = "Sets the probability for tracing any given request to value. 0 disables, 1 enables for all requests, 0 is the default")
 +    public static class SetTraceProbability extends NodeToolCmd
 +    {
 +        @Arguments(title = "trace_probability", usage = "<value>", description = "Trace probability between 0 and 1 (ex: 0.2)", required = true)
 +        private Double traceProbability = null;
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            checkArgument(traceProbability >= 0 && traceProbability <= 1, "Trace probability must be between 0 and 1");
 +            probe.setTraceProbability(traceProbability);
 +        }
 +    }
 +
 +    @Command(name = "snapshot", description = "Take a snapshot of specified keyspaces or a snapshot of the specified column family")
 +    public static class Snapshot extends NodeToolCmd
 +    {
 +        @Arguments(usage = "[<keyspaces...>]", description = "List of keyspaces. By default, all keyspaces")
 +        private List<String> keyspaces = new ArrayList<>();
 +
 +        @Option(title = "cfname", name = {"-cf", "--column-family"}, description = "The column family name (you must specify one and only one keyspace for using this option)")
 +        private String columnFamily = null;
 +
 +        @Option(title = "tag", name = {"-t", "--tag"}, description = "The name of the snapshot")
 +        private String snapshotName = Long.toString(System.currentTimeMillis());
 +
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            try
 +            {
 +                StringBuilder sb = new StringBuilder();
 +
 +                sb.append("Requested creating snapshot(s) for ");
 +
 +                if (keyspaces.isEmpty())
 +                    sb.append("[all keyspaces]");
 +                else
 +                    sb.append("[").append(join(keyspaces, ", ")).append("]");
 +
 +                if (!snapshotName.isEmpty())
 +                    sb.append(" with snapshot name [").append(snapshotName).append("]");
 +
 +                System.out.println(sb.toString());
 +
 +                probe.takeSnapshot(snapshotName, columnFamily, toArray(keyspaces, String.class));
 +                System.out.println("Snapshot directory: " + snapshotName);
 +            } catch (IOException e)
 +            {
 +                throw new RuntimeException("Error during taking a snapshot", e);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "listsnapshots", description = "Lists all the snapshots along with the size on disk and true size.")
 +    public static class ListSnapshots extends NodeToolCmd
 +    {
 +        @Override
 +        public void execute(NodeProbe probe)
 +        {
 +            try
 +            {
 +                System.out.println("Snapshot Details: ");
 +
 +                final Map<String,TabularData> snapshotDetails = probe.getSnapshotDetails();
 +                if (snapshotDetails.isEmpty())
 +                {
 +                    System.out.printf("There are no snapshots");
 +                    return;
 +                }
 +
 +                final long trueSnapshotsSize = probe.trueSnapshotsSize();
 +                final String format = "%-20s%-29s%-29s%-19s%-19s%n";
 +                // display column names only once
 +                final List<String> indexNames = snapshotDetails.entrySet().iterator().next().getValue().getTabularType().getIndexNames();
 +                System.out.printf(format, (Object[]) indexNames.toArray(new String[indexNames.size()]));
 +
 +                for (final Map.Entry<String, TabularData> snapshotDetail : snapshotDetails.entrySet())
 +                {
 +                    Set<?> values = snapshotDetail.getValue().keySet();
 +                    for (Object eachValue : values)
 +                    {
 +                        final List<?> value = (List<?>) eachValue;
 +                        System.out.printf(format, value.toArray(new Object[value.size()]));
 +                    }
 +                }
 +
 +                System.out.println("\nTotal TrueDiskSpaceUsed: " + FileUtils.stringifyFileSize(trueSnapshotsSize) + "\n");
 +            }
 +            catch (Exception e)
 +            {
 +                throw new RuntimeException("Error during list snapshot", e);
 +            }
 +        }
 +    }
 +
 +    @Command(name = "status", description = "Print cluster information (state, load, IDs, ...)")
 +    public static class Status extends NodeToolCmd
 +    {
 +        @Arg

<TRUNCATED>

[3/8] cassandra git commit: Add DC-aware sequential repair

Posted by yu...@apache.org.
Add DC-aware sequential repair

patch by Jimmy Mårdell; reviewed by yukim for CASSANDRA-8193


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/41469ecf
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/41469ecf
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/41469ecf

Branch: refs/heads/trunk
Commit: 41469ecf6a27e94441f96ef905ed3b5354c23987
Parents: 17de36f
Author: Jimmy Mårdell <ya...@spotify.com>
Authored: Mon Nov 24 15:07:33 2014 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Mon Nov 24 15:09:41 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../DatacenterAwareRequestCoordinator.java      |  73 +++++++++++
 .../cassandra/repair/IRequestCoordinator.java   |  28 ++++
 .../cassandra/repair/IRequestProcessor.java     |  23 ++++
 .../repair/ParallelRequestCoordinator.java      |  49 +++++++
 .../org/apache/cassandra/repair/RepairJob.java  |  32 ++++-
 .../cassandra/repair/RepairParallelism.java     |  22 ++++
 .../apache/cassandra/repair/RepairSession.java  |  14 +-
 .../cassandra/repair/RequestCoordinator.java    | 128 -------------------
 .../repair/SequentialRequestCoordinator.java    |  58 +++++++++
 .../cassandra/service/ActiveRepairService.java  |   6 +-
 .../cassandra/service/StorageService.java       |  64 ++++++----
 .../cassandra/service/StorageServiceMBean.java  |  19 ++-
 .../org/apache/cassandra/tools/NodeCmd.java     |  21 ++-
 .../org/apache/cassandra/tools/NodeProbe.java   |  30 +++--
 .../apache/cassandra/tools/NodeToolHelp.yaml    |   1 +
 .../repair/RequestCoordinatorTest.java          | 124 ++++++++++++++++++
 17 files changed, 506 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index fe23248..7519653 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -24,6 +24,7 @@
  * Allow concurrent writing of the same table in the same JVM using
    CQLSSTableWriter (CASSANDRA-7463)
  * Fix totalDiskSpaceUsed calculation (CASSANDRA-8205)
+ * Add DC-aware sequential repair (CASSANDRA-8193)
 
 
 2.0.11:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java b/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java
new file mode 100644
index 0000000..ab3e03e
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+import java.net.InetAddress;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Queue;
+
+public class DatacenterAwareRequestCoordinator implements IRequestCoordinator<InetAddress>
+{
+    private Map<String, Queue<InetAddress>> requestsByDatacenter = new HashMap<>();
+    private int remaining = 0;
+    private final IRequestProcessor<InetAddress> processor;
+
+    protected DatacenterAwareRequestCoordinator(IRequestProcessor<InetAddress> processor)
+    {
+        this.processor = processor;
+    }
+
+    public void add(InetAddress request)
+    {
+        String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(request);
+        Queue<InetAddress> queue = requestsByDatacenter.get(dc);
+        if (queue == null)
+        {
+            queue = new LinkedList<>();
+            requestsByDatacenter.put(dc, queue);
+        }
+        queue.add(request);
+        remaining++;
+    }
+
+    public void start()
+    {
+        for (Queue<InetAddress> requests : requestsByDatacenter.values())
+        {
+            if (!requests.isEmpty())
+              processor.process(requests.peek());
+        }
+    }
+
+    // Returns how many request remains
+    public int completed(InetAddress request)
+    {
+        String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(request);
+        Queue<InetAddress> requests = requestsByDatacenter.get(dc);
+        assert requests != null;
+        assert request.equals(requests.peek());
+        requests.poll();
+        if (!requests.isEmpty())
+            processor.process(requests.peek());
+        return --remaining;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/IRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/IRequestCoordinator.java b/src/java/org/apache/cassandra/repair/IRequestCoordinator.java
new file mode 100644
index 0000000..db2ffe3
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/IRequestCoordinator.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+public interface IRequestCoordinator<R>
+{
+    void add(R request);
+
+    void start();
+
+    // Returns how many request remains
+    int completed(R request);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/IRequestProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/IRequestProcessor.java b/src/java/org/apache/cassandra/repair/IRequestProcessor.java
new file mode 100644
index 0000000..c7b49d7
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/IRequestProcessor.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+public interface IRequestProcessor<R>
+{
+    void process(R request);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java b/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java
new file mode 100644
index 0000000..839bb43
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class ParallelRequestCoordinator<R> implements IRequestCoordinator<R>
+{
+    private final Set<R> requests = new HashSet<>();
+    private final IRequestProcessor<R> processor;
+
+    public ParallelRequestCoordinator(IRequestProcessor<R> processor)
+    {
+        this.processor = processor;
+    }
+
+    @Override
+    public void add(R request) { requests.add(request); }
+
+    @Override
+    public void start()
+    {
+        for (R request : requests)
+            processor.process(request);
+    }
+
+    @Override
+    public int completed(R request)
+    {
+        requests.remove(request);
+        return requests.size();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RepairJob.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java
index 931f95a..7c791aa 100644
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@ -43,9 +43,9 @@ public class RepairJob
     private static Logger logger = LoggerFactory.getLogger(RepairJob.class);
 
     public final RepairJobDesc desc;
-    private final boolean isSequential;
+    private final RepairParallelism parallelismDegree;
     // first we send tree requests. this tracks the endpoints remaining to hear from
-    private final RequestCoordinator<InetAddress> treeRequests;
+    private final IRequestCoordinator<InetAddress> treeRequests;
     // tree responses are then tracked here
     private final List<TreeResponse> trees = new ArrayList<>();
     // once all responses are received, each tree is compared with each other, and differencer tasks
@@ -68,21 +68,38 @@ public class RepairJob
                      String keyspace,
                      String columnFamily,
                      Range<Token> range,
-                     boolean isSequential,
+                     RepairParallelism parallelismDegree,
                      ListeningExecutorService taskExecutor)
     {
         this.listener = listener;
         this.desc = new RepairJobDesc(sessionId, keyspace, columnFamily, range);
-        this.isSequential = isSequential;
+        this.parallelismDegree = parallelismDegree;
         this.taskExecutor = taskExecutor;
-        this.treeRequests = new RequestCoordinator<InetAddress>(isSequential)
+
+        IRequestProcessor<InetAddress> processor = new IRequestProcessor<InetAddress>()
         {
-            public void send(InetAddress endpoint)
+            @Override
+            public void process(InetAddress endpoint)
             {
                 ValidationRequest request = new ValidationRequest(desc, gcBefore);
                 MessagingService.instance().sendOneWay(request.createMessage(), endpoint);
             }
         };
+
+        switch (parallelismDegree)
+        {
+            case SEQUENTIAL:
+                this.treeRequests = new SequentialRequestCoordinator<>(processor);
+                break;
+            case PARALLEL:
+                this.treeRequests = new ParallelRequestCoordinator<>(processor);
+                break;
+            case DATACENTER_AWARE:
+                this.treeRequests = new DatacenterAwareRequestCoordinator(processor);
+                break;
+            default:
+                throw new AssertionError("Unknown degree of parallelism specified");
+        }
     }
 
     /**
@@ -102,7 +119,8 @@ public class RepairJob
         List<InetAddress> allEndpoints = new ArrayList<>(endpoints);
         allEndpoints.add(FBUtilities.getBroadcastAddress());
 
-        if (isSequential)
+        // Create a snapshot at all nodes unless we're using pure parallel repairs
+        if (parallelismDegree != RepairParallelism.PARALLEL)
         {
             List<ListenableFuture<InetAddress>> snapshotTasks = new ArrayList<>(allEndpoints.size());
             for (InetAddress endpoint : allEndpoints)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RepairParallelism.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairParallelism.java b/src/java/org/apache/cassandra/repair/RepairParallelism.java
new file mode 100644
index 0000000..12c22ca
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/RepairParallelism.java
@@ -0,0 +1,22 @@
+package org.apache.cassandra.repair;
+
+/**
+ * Specify the degree of parallelism when calculating the merkle trees in a repair job.
+ */
+public enum RepairParallelism
+{
+    /**
+     * One node at a time
+     */
+    SEQUENTIAL,
+
+    /**
+     * All nodes at the same time
+     */
+    PARALLEL,
+
+    /**
+     * One node per data center at a time
+     */
+    DATACENTER_AWARE
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RepairSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairSession.java b/src/java/org/apache/cassandra/repair/RepairSession.java
index c9a9671..f2b95eb 100644
--- a/src/java/org/apache/cassandra/repair/RepairSession.java
+++ b/src/java/org/apache/cassandra/repair/RepairSession.java
@@ -83,7 +83,7 @@ public class RepairSession extends WrappedRunnable implements IEndpointStateChan
     private final UUID id;
     public final String keyspace;
     private final String[] cfnames;
-    public final boolean isSequential;
+    public final RepairParallelism parallelismDegree;
     /** Range to repair */
     public final Range<Token> range;
     public final Set<InetAddress> endpoints;
@@ -110,19 +110,19 @@ public class RepairSession extends WrappedRunnable implements IEndpointStateChan
      *
      * @param range range to repair
      * @param keyspace name of keyspace
-     * @param isSequential true if performing repair on snapshots sequentially
+     * @param parallelismDegree specifies the degree of parallelism when calculating the merkle trees
      * @param dataCenters the data centers that should be part of the repair; null for all DCs
      * @param cfnames names of columnfamilies
      */
-    public RepairSession(Range<Token> range, String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
+    public RepairSession(Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
     {
-        this(UUIDGen.getTimeUUID(), range, keyspace, isSequential, dataCenters, hosts, cfnames);
+        this(UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, dataCenters, hosts, cfnames);
     }
 
-    public RepairSession(UUID id, Range<Token> range, String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, String[] cfnames)
+    public RepairSession(UUID id, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String[] cfnames)
     {
         this.id = id;
-        this.isSequential = isSequential;
+        this.parallelismDegree = parallelismDegree;
         this.keyspace = keyspace;
         this.cfnames = cfnames;
         assert cfnames.length > 0 : "Repairing no column families seems pointless, doesn't it";
@@ -270,7 +270,7 @@ public class RepairSession extends WrappedRunnable implements IEndpointStateChan
             // Create and queue a RepairJob for each column family
             for (String cfname : cfnames)
             {
-                RepairJob job = new RepairJob(this, id, keyspace, cfname, range, isSequential, taskExecutor);
+                RepairJob job = new RepairJob(this, id, keyspace, cfname, range, parallelismDegree, taskExecutor);
                 jobs.offer(job);
             }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/RequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RequestCoordinator.java b/src/java/org/apache/cassandra/repair/RequestCoordinator.java
deleted file mode 100644
index ed089ef..0000000
--- a/src/java/org/apache/cassandra/repair/RequestCoordinator.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.repair;
-
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Queue;
-import java.util.Set;
-
-/**
-*/
-public abstract class RequestCoordinator<R>
-{
-    private final Order<R> orderer;
-
-    public RequestCoordinator(boolean isSequential)
-    {
-        this.orderer = isSequential ? new SequentialOrder(this) : new ParallelOrder(this);
-    }
-
-    public abstract void send(R request);
-
-    public void add(R request)
-    {
-        orderer.add(request);
-    }
-
-    public void start()
-    {
-        orderer.start();
-    }
-
-    // Returns how many request remains
-    public int completed(R request)
-    {
-        return orderer.completed(request);
-    }
-
-    private static abstract class Order<R>
-    {
-        protected final RequestCoordinator<R> coordinator;
-
-        Order(RequestCoordinator<R> coordinator)
-        {
-            this.coordinator = coordinator;
-        }
-
-        public abstract void add(R request);
-        public abstract void start();
-        public abstract int completed(R request);
-    }
-
-    private static class SequentialOrder<R> extends Order<R>
-    {
-        private final Queue<R> requests = new LinkedList<>();
-
-        SequentialOrder(RequestCoordinator<R> coordinator)
-        {
-            super(coordinator);
-        }
-
-        public void add(R request)
-        {
-            requests.add(request);
-        }
-
-        public void start()
-        {
-            if (requests.isEmpty())
-                return;
-
-            coordinator.send(requests.peek());
-        }
-
-        public int completed(R request)
-        {
-            assert request.equals(requests.peek());
-            requests.poll();
-            int remaining = requests.size();
-            if (remaining != 0)
-                coordinator.send(requests.peek());
-            return remaining;
-        }
-    }
-
-    private static class ParallelOrder<R> extends Order<R>
-    {
-        private final Set<R> requests = new HashSet<>();
-
-        ParallelOrder(RequestCoordinator<R> coordinator)
-        {
-            super(coordinator);
-        }
-
-        public void add(R request)
-        {
-            requests.add(request);
-        }
-
-        public void start()
-        {
-            for (R request : requests)
-                coordinator.send(request);
-        }
-
-        public int completed(R request)
-        {
-            requests.remove(request);
-            return requests.size();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java b/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java
new file mode 100644
index 0000000..6bb5de9
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import java.util.LinkedList;
+import java.util.Queue;
+
+public class SequentialRequestCoordinator<R> implements IRequestCoordinator<R>
+{
+    private final Queue<R> requests = new LinkedList<>();
+    private final IRequestProcessor<R> processor;
+
+    public SequentialRequestCoordinator(IRequestProcessor<R> processor)
+    {
+        this.processor = processor;
+    }
+
+    @Override
+    public void add(R request)
+    {
+        requests.add(request);
+    }
+
+    @Override
+    public void start()
+    {
+        if (requests.isEmpty())
+            return;
+
+        processor.process(requests.peek());
+    }
+
+    @Override
+    public int completed(R request)
+    {
+        assert request.equals(requests.peek());
+        requests.poll();
+        int remaining = requests.size();
+        if (remaining != 0)
+            processor.process(requests.peek());
+        return remaining;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index aac9f9a..da81e8f 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -92,9 +92,9 @@ public class ActiveRepairService
      *
      * @return Future for asynchronous call or null if there is no need to repair
      */
-    public RepairFuture submitRepairSession(Range<Token> range, String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
+    public RepairFuture submitRepairSession(Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
     {
-        RepairSession session = new RepairSession(range, keyspace, isSequential, dataCenters, hosts, cfnames);
+        RepairSession session = new RepairSession(range, keyspace, parallelismDegree, dataCenters, hosts, cfnames);
         if (session.endpoints.isEmpty())
             return null;
         RepairFuture futureTask = new RepairFuture(session);
@@ -128,7 +128,7 @@ public class ActiveRepairService
     // add it to the sessions (avoid NPE in tests)
     RepairFuture submitArtificialRepairSession(RepairJobDesc desc)
     {
-        RepairSession session = new RepairSession(desc.sessionId, desc.range, desc.keyspace, false, null, null, new String[]{desc.columnFamily});
+        RepairSession session = new RepairSession(desc.sessionId, desc.range, desc.keyspace, RepairParallelism.PARALLEL, null, null, new String[]{desc.columnFamily});
         sessions.put(session.getId(), session);
         RepairFuture futureTask = new RepairFuture(session);
         executor.execute(futureTask);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 4bc1eee..3d42d1c 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -77,6 +77,7 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.ResponseVerbHandler;
 import org.apache.cassandra.repair.RepairFuture;
 import org.apache.cassandra.repair.RepairMessageVerbHandler;
+import org.apache.cassandra.repair.RepairParallelism;
 import org.apache.cassandra.service.paxos.CommitVerbHandler;
 import org.apache.cassandra.service.paxos.PrepareVerbHandler;
 import org.apache.cassandra.service.paxos.ProposeVerbHandler;
@@ -334,7 +335,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         {
             throw new IllegalStateException("No configured daemon");
         }
-        
+
         try
         {
             daemon.nativeServer.start();
@@ -432,10 +433,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             if (Gossiper.instance.getEndpointStateForEndpoint(DatabaseDescriptor.getReplaceAddress()).getApplicationState(ApplicationState.TOKENS) == null)
                 throw new RuntimeException("Could not find tokens for " + DatabaseDescriptor.getReplaceAddress() + " to replace");
             Collection<Token> tokens = TokenSerializer.deserialize(getPartitioner(), new DataInputStream(new ByteArrayInputStream(getApplicationStateValue(DatabaseDescriptor.getReplaceAddress(), ApplicationState.TOKENS))));
-            
+
             SystemKeyspace.setLocalHostId(hostId); // use the replacee's host Id as our own so we receive hints, etc
             Gossiper.instance.resetEndpointStateMap(); // clean up since we have what we need
-            return tokens;        
+            return tokens;
         }
         catch (IOException e)
         {
@@ -2408,16 +2409,21 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int forceRepairAsync(final String keyspace, final boolean isSequential, final Collection<String> dataCenters, final Collection<String> hosts, final boolean primaryRange, final String... columnFamilies)
     {
+        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, columnFamilies);
+    }
+
+    public int forceRepairAsync(final String keyspace, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts, final boolean primaryRange, final String... columnFamilies)
+    {
         // when repairing only primary range, dataCenter nor hosts can be set
         if (primaryRange && (dataCenters != null || hosts != null))
         {
             throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
         }
         final Collection<Range<Token>> ranges = primaryRange ? getLocalPrimaryRanges(keyspace) : getLocalRanges(keyspace);
-        return forceRepairAsync(keyspace, isSequential, dataCenters, hosts, ranges, columnFamilies);
+        return forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, ranges, columnFamilies);
     }
 
-    public int forceRepairAsync(final String keyspace, final boolean isSequential, final Collection<String> dataCenters, final Collection<String> hosts,  final Collection<Range<Token>> ranges, final String... columnFamilies)
+    public int forceRepairAsync(final String keyspace, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts,  final Collection<Range<Token>> ranges, final String... columnFamilies)
     {
         if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
             return 0;
@@ -2425,7 +2431,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         final int cmd = nextRepairCommand.incrementAndGet();
         if (ranges.size() > 0)
         {
-            new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, hosts, columnFamilies)).start();
+            new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, hosts, columnFamilies)).start();
         }
         return cmd;
     }
@@ -2438,37 +2444,42 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
         }
         final Collection<Range<Token>> ranges = primaryRange ? getLocalPrimaryRanges(keyspace) : getLocalRanges(keyspace);
-        return forceRepairAsync(keyspace, isSequential, isLocal, ranges, columnFamilies);
+        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, ranges, columnFamilies);
     }
 
-    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, Collection<Range<Token>> ranges, String... columnFamilies)
+    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, boolean isLocal, Collection<Range<Token>> ranges, String... columnFamilies)
     {
         if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
             return 0;
 
         final int cmd = nextRepairCommand.incrementAndGet();
-        if (!FBUtilities.isUnix() && isSequential)
+        if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
         {
             logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-            isSequential = false;
+            parallelismDegree = RepairParallelism.PARALLEL;
         }
-        new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, isLocal, columnFamilies)).start();
+        new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, isLocal, columnFamilies)).start();
         return cmd;
     }
 
     public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
     {
+        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, columnFamilies);
+    }
+
+    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
+    {
         Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
 
         logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
                     repairingRange, keyspaceName, columnFamilies);
 
-        if (!FBUtilities.isUnix() && isSequential)
+        if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
         {
             logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-            isSequential = false;
+            parallelismDegree = RepairParallelism.PARALLEL;
         }
-        return forceRepairAsync(keyspaceName, isSequential, dataCenters, hosts, repairingRange, columnFamilies);
+        return forceRepairAsync(keyspaceName, parallelismDegree, dataCenters, hosts, repairingRange, columnFamilies);
     }
 
     public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies)
@@ -2486,7 +2497,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public void forceKeyspaceRepair(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
     {
-        forceKeyspaceRepairRange(keyspaceName, getLocalRanges(keyspaceName), isSequential, isLocal, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, getLocalRanges(keyspaceName), isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, columnFamilies);
     }
 
     public void forceKeyspaceRepairPrimaryRange(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
@@ -2498,7 +2509,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
         }
 
-        forceKeyspaceRepairRange(keyspaceName, getLocalPrimaryRanges(keyspaceName), isSequential, false, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, getLocalPrimaryRanges(keyspaceName), isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, false, columnFamilies);
     }
 
     public void forceKeyspaceRepairRange(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
@@ -2507,14 +2518,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
                            repairingRange, keyspaceName, columnFamilies);
-        forceKeyspaceRepairRange(keyspaceName, repairingRange, isSequential, isLocal, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, repairingRange, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, columnFamilies);
     }
 
-    public void forceKeyspaceRepairRange(final String keyspaceName, final Collection<Range<Token>> ranges, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
+    public void forceKeyspaceRepairRange(final String keyspaceName, final Collection<Range<Token>> ranges, RepairParallelism parallelismDegree, boolean isLocal, final String... columnFamilies) throws IOException
     {
         if (ranges.isEmpty() || Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor() < 2)
             return;
-        createRepairTask(nextRepairCommand.incrementAndGet(), keyspaceName, ranges, isSequential, isLocal, columnFamilies).run();
+        createRepairTask(nextRepairCommand.incrementAndGet(), keyspaceName, ranges, parallelismDegree, isLocal, columnFamilies).run();
     }
 
     /**
@@ -2556,17 +2567,17 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return repairingRange;
     }
 
-    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final boolean isSequential, final boolean isLocal, final String... columnFamilies)
+    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final RepairParallelism parallelismDegree, final boolean isLocal, final String... columnFamilies)
     {
         Set<String> dataCenters = null;
         if (isLocal)
         {
             dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
         }
-        return createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, null, columnFamilies);
+        return createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, null, columnFamilies);
     }
 
-    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final boolean isSequential, final Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
+    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
     {
         if (dataCenters != null && !dataCenters.contains(DatabaseDescriptor.getLocalDataCenter()))
         {
@@ -2587,7 +2598,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                     RepairFuture future;
                     try
                     {
-                        future = forceKeyspaceRepair(range, keyspace, isSequential, dataCenters, hosts, columnFamilies);
+                        future = forceKeyspaceRepair(range, keyspace, parallelismDegree, dataCenters, hosts, columnFamilies);
                     }
                     catch (IllegalArgumentException e)
                     {
@@ -2639,6 +2650,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public RepairFuture forceKeyspaceRepair(final Range<Token> range, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, final String... columnFamilies) throws IOException
     {
+        return forceKeyspaceRepair(range, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, columnFamilies);
+    }
+
+    public RepairFuture forceKeyspaceRepair(final Range<Token> range, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, final String... columnFamilies) throws IOException
+    {
         ArrayList<String> names = new ArrayList<String>();
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
@@ -2651,7 +2667,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             return null;
         }
 
-        return ActiveRepairService.instance.submitRepairSession(range, keyspaceName, isSequential, dataCenters, hosts, names.toArray(new String[names.size()]));
+        return ActiveRepairService.instance.submitRepairSession(range, keyspaceName, parallelismDegree, dataCenters, hosts, names.toArray(new String[names.size()]));
     }
 
     public void forceTerminateAllRepairSessions() {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 0e2f55d..2386fc8 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -17,6 +17,8 @@
  */
 package org.apache.cassandra.service;
 
+import org.apache.cassandra.repair.RepairParallelism;
+
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -260,10 +262,25 @@ public interface StorageServiceMBean extends NotificationEmitter
     public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, String... columnFamilies);
 
     /**
+     * Invoke repair asynchronously.
+     * You can track repair progress by subscribing JMX notification sent from this StorageServiceMBean.
+     * Notification format is:
+     *   type: "repair"
+     *   userObject: int array of length 2, [0]=command number, [1]=ordinal of AntiEntropyService.Status
+     *
+     * @return Repair command number, or 0 if nothing to repair
+     */
+    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, String... columnFamilies);
+
+    /**
      * Same as forceRepairAsync, but handles a specified range
      */
     public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts,  final String... columnFamilies);
 
+    /**
+     * Same as forceRepairAsync, but handles a specified range
+     */
+    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts,  final String... columnFamilies);
 
     /**
      * Invoke repair asynchronously.
@@ -477,7 +494,7 @@ public interface StorageServiceMBean extends NotificationEmitter
 
     /**
      * Enables/Disables tracing for the whole system. Only thrift requests can start tracing currently.
-     * 
+     *
      * @param probability
      *            ]0,1[ will enable tracing on a partial number of requests with the provided probability. 0 will
      *            disable tracing and 1 will enable tracing for all requests (which mich severely cripple the system)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/tools/NodeCmd.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeCmd.java b/src/java/org/apache/cassandra/tools/NodeCmd.java
index a397244..2d7809a 100644
--- a/src/java/org/apache/cassandra/tools/NodeCmd.java
+++ b/src/java/org/apache/cassandra/tools/NodeCmd.java
@@ -47,6 +47,7 @@ import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.net.MessagingServiceMBean;
+import org.apache.cassandra.repair.RepairParallelism;
 import org.apache.cassandra.service.CacheServiceMBean;
 import org.apache.cassandra.service.StorageProxyMBean;
 import org.apache.cassandra.streaming.StreamState;
@@ -70,6 +71,7 @@ public class NodeCmd
     private static final Pair<String, String> TOKENS_OPT = Pair.create("T", "tokens");
     private static final Pair<String, String> PRIMARY_RANGE_OPT = Pair.create("pr", "partitioner-range");
     private static final Pair<String, String> PARALLEL_REPAIR_OPT = Pair.create("par", "parallel");
+    private static final Pair<String, String> DCPARALLEL_REPAIR_OPT = Pair.create("dcpar", "dcparallel");
     private static final Pair<String, String> LOCAL_DC_REPAIR_OPT = Pair.create("local", "in-local-dc");
     private static final Pair<String, String> HOST_REPAIR_OPT = Pair.create("hosts", "in-host");
     private static final Pair<String, String> DC_REPAIR_OPT = Pair.create("dc", "in-dc");
@@ -100,6 +102,7 @@ public class NodeCmd
         options.addOption(TOKENS_OPT,   false, "display all tokens");
         options.addOption(PRIMARY_RANGE_OPT, false, "only repair the first range returned by the partitioner for the node");
         options.addOption(PARALLEL_REPAIR_OPT, false, "repair nodes in parallel.");
+        options.addOption(DCPARALLEL_REPAIR_OPT, false, "repair data centers in parallel.");
         options.addOption(LOCAL_DC_REPAIR_OPT, false, "only repair against nodes in the same datacenter");
         options.addOption(DC_REPAIR_OPT, true, "only repair against nodes in the specified datacenters (comma separated)");
         options.addOption(HOST_REPAIR_OPT, true, "only repair against specified nodes (comma separated)");
@@ -203,10 +206,10 @@ public class NodeCmd
         StringBuilder header = new StringBuilder(512);
         header.append("\nAvailable commands\n");
         final NodeToolHelp ntHelp = loadHelp();
-        Collections.sort(ntHelp.commands, new Comparator<NodeToolHelp.NodeToolCommand>() 
+        Collections.sort(ntHelp.commands, new Comparator<NodeToolHelp.NodeToolCommand>()
         {
             @Override
-            public int compare(NodeToolHelp.NodeToolCommand o1, NodeToolHelp.NodeToolCommand o2) 
+            public int compare(NodeToolHelp.NodeToolCommand o1, NodeToolHelp.NodeToolCommand o2)
             {
                 return o1.name.compareTo(o2.name);
             }
@@ -525,7 +528,7 @@ public class NodeCmd
         }
     }
 
-    private Map<String, SetHostStat> getOwnershipByDc(boolean resolveIp, Map<String, String> tokenToEndpoint, 
+    private Map<String, SetHostStat> getOwnershipByDc(boolean resolveIp, Map<String, String> tokenToEndpoint,
                                                       Map<InetAddress, Float> ownerships) throws UnknownHostException
     {
         Map<String, SetHostStat> ownershipByDc = Maps.newLinkedHashMap();
@@ -574,7 +577,7 @@ public class NodeCmd
         public final Float owns;
         public final String token;
 
-        public HostStat(String token, InetAddress endpoint, boolean resolveIp, Float owns) 
+        public HostStat(String token, InetAddress endpoint, boolean resolveIp, Float owns)
         {
             this.token = token;
             this.endpoint = endpoint;
@@ -1668,7 +1671,11 @@ public class NodeCmd
             switch (nc)
             {
                 case REPAIR  :
-                    boolean sequential = !cmd.hasOption(PARALLEL_REPAIR_OPT.left);
+                    RepairParallelism parallelismDegree = RepairParallelism.SEQUENTIAL;
+                    if (cmd.hasOption(PARALLEL_REPAIR_OPT.left))
+                        parallelismDegree = RepairParallelism.PARALLEL;
+                    else if (cmd.hasOption(DCPARALLEL_REPAIR_OPT.left))
+                        parallelismDegree = RepairParallelism.DATACENTER_AWARE;
                     boolean localDC = cmd.hasOption(LOCAL_DC_REPAIR_OPT.left);
                     boolean specificDC = cmd.hasOption(DC_REPAIR_OPT.left);
                     boolean specificHosts = cmd.hasOption(HOST_REPAIR_OPT.left);
@@ -1686,9 +1693,9 @@ public class NodeCmd
                     else if(specificHosts)
                         hosts  = Arrays.asList(cmd.getOptionValue(HOST_REPAIR_OPT.left).split(","));
                     if (cmd.hasOption(START_TOKEN_OPT.left) || cmd.hasOption(END_TOKEN_OPT.left))
-                        probe.forceRepairRangeAsync(System.out, keyspace, sequential, dataCenters, hosts, cmd.getOptionValue(START_TOKEN_OPT.left), cmd.getOptionValue(END_TOKEN_OPT.left), columnFamilies);
+                        probe.forceRepairRangeAsync(System.out, keyspace, parallelismDegree, dataCenters, hosts, cmd.getOptionValue(START_TOKEN_OPT.left), cmd.getOptionValue(END_TOKEN_OPT.left), columnFamilies);
                     else
-                        probe.forceRepairAsync(System.out, keyspace, sequential, dataCenters, hosts, primaryRange, columnFamilies);
+                        probe.forceRepairAsync(System.out, keyspace, parallelismDegree, dataCenters, hosts, primaryRange, columnFamilies);
                     break;
                 case FLUSH   :
                     try { probe.forceKeyspaceFlush(keyspace, columnFamilies); }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 849e368..261d416 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -54,6 +54,7 @@ import org.apache.cassandra.gms.FailureDetectorMBean;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.MessagingServiceMBean;
+import org.apache.cassandra.repair.RepairParallelism;
 import org.apache.cassandra.service.*;
 import org.apache.cassandra.streaming.StreamState;
 import org.apache.cassandra.streaming.StreamManagerMBean;
@@ -217,12 +218,17 @@ public class NodeProbe
 
     public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts,  boolean primaryRange, String... columnFamilies) throws IOException
     {
+        forceRepairAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, columnFamilies);
+    }
+
+    public void forceRepairAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts,  boolean primaryRange, String... columnFamilies) throws IOException
+    {
         RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
         try
         {
             jmxc.addConnectionNotificationListener(runner, null, null);
             ssProxy.addNotificationListener(runner, null, null);
-            if (!runner.repairAndWait(ssProxy, isSequential, dataCenters, hosts, primaryRange))
+            if (!runner.repairAndWait(ssProxy, parallelismDegree, dataCenters, hosts, primaryRange))
                 failed = true;
         }
         catch (Exception e)
@@ -239,15 +245,19 @@ public class NodeProbe
             catch (Throwable ignored) {}
         }
     }
-
     public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, String... columnFamilies) throws IOException
     {
+        forceRepairRangeAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, startToken, endToken, columnFamilies);
+    }
+
+    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, String... columnFamilies) throws IOException
+    {
         RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
         try
         {
             jmxc.addConnectionNotificationListener(runner, null, null);
             ssProxy.addNotificationListener(runner, null, null);
-            if (!runner.repairRangeAndWait(ssProxy,  isSequential, dataCenters, hosts, startToken, endToken))
+            if (!runner.repairRangeAndWait(ssProxy, parallelismDegree, dataCenters, hosts, startToken, endToken))
                 failed = true;
         }
         catch (Exception e)
@@ -896,17 +906,17 @@ public class NodeProbe
     {
         return failed;
     }
-    
+
     public long getReadRepairAttempted()
     {
         return spProxy.getReadRepairAttempted();
     }
-    
+
     public long getReadRepairRepairedBlocking()
     {
         return spProxy.getReadRepairRepairedBlocking();
     }
-    
+
     public long getReadRepairRepairedBackground()
     {
         return spProxy.getReadRepairRepairedBackground();
@@ -1060,16 +1070,16 @@ class RepairRunner implements NotificationListener
         this.columnFamilies = columnFamilies;
     }
 
-    public boolean repairAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRangeOnly) throws Exception
+    public boolean repairAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRangeOnly) throws Exception
     {
-        cmd = ssProxy.forceRepairAsync(keyspace, isSequential, dataCenters, hosts, primaryRangeOnly, columnFamilies);
+        cmd = ssProxy.forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, primaryRangeOnly, columnFamilies);
         waitForRepair();
         return success;
     }
 
-    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, String startToken, String endToken) throws Exception
+    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, String startToken, String endToken) throws Exception
     {
-        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, isSequential, dataCenters, hosts, columnFamilies);
+        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, parallelismDegree, dataCenters, hosts, columnFamilies);
         waitForRepair();
         return success;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml b/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
index 523335e..b254bac 100644
--- a/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
+++ b/src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml
@@ -155,6 +155,7 @@ commands:
     help: |
       Repair one or more column families
          Use -dc to repair specific datacenters (csv list).
+         Use -dcpar to repair datacenters in parallel.
          Use -et to specify a token at which repair range ends.
          Use -local to only repair against nodes in the same datacenter.
          Use -pr to repair only the first range returned by the partitioner.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/41469ecf/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java b/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java
new file mode 100644
index 0000000..ad6eabe
--- /dev/null
+++ b/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.repair;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+
+public class RequestCoordinatorTest implements IRequestProcessor<InetAddress>
+{
+    private InetAddress[] endpoints;
+    private List<InetAddress> activeRequests;
+    private static Random random = new Random(0);
+
+    @Before
+    public void setup() throws UnknownHostException
+    {
+        endpoints = new InetAddress[12];
+        for (int i = 0; i < 12; i++)
+            endpoints[i] = InetAddress.getByName("127.0.0." + (i + 1));
+        activeRequests = new ArrayList<>();
+        DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
+        {
+            @Override
+            public String getRack(InetAddress endpoint)
+            {
+                return "rack1";
+            }
+
+            @Override
+            public String getDatacenter(InetAddress endpoint)
+            {
+                // 127.0.0.1, 127.0.0.2, 127.0.0.3 -> datacenter1
+                // 127.0.0.4, 127.0.0.5, 127.0.0.6 -> datacenter2 etc
+                int no = endpoint.getAddress()[3] - 1;
+                return "datacenter" + (no / 3 + 1);
+            }
+
+            @Override
+            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
+            {
+                return 0;
+            }
+        });
+    }
+
+    @Override
+    public void process(InetAddress request)
+    {
+        activeRequests.add(request);
+    }
+
+    @Test
+    public void testSequentialRequestCoordinator()
+    {
+        SequentialRequestCoordinator<InetAddress> coordinator = new SequentialRequestCoordinator<>(this);
+        for (InetAddress endpoint : endpoints)
+            coordinator.add(endpoint);
+        coordinator.start();
+        int max = finishRequests(coordinator);
+        assertEquals(1, max);
+    }
+
+    @Test
+    public void testParallelRequestCoordinator()
+    {
+        ParallelRequestCoordinator<InetAddress> coordinator = new ParallelRequestCoordinator<>(this);
+        for (InetAddress endpoint : endpoints)
+            coordinator.add(endpoint);
+        coordinator.start();
+        int max = finishRequests(coordinator);
+        assertEquals(endpoints.length, max);
+    }
+
+    @Test
+    public void testDatacenterAwareRequestCoordinator()
+    {
+        DatacenterAwareRequestCoordinator coordinator = new DatacenterAwareRequestCoordinator(this);
+        for (InetAddress endpoint : endpoints)
+            coordinator.add(endpoint);
+        coordinator.start();
+        int max = finishRequests(coordinator);
+        assertEquals(4, max);
+    }
+
+    private int finishRequests(IRequestCoordinator<InetAddress> requestCoordinator)
+    {
+        int max = 0;
+        while (activeRequests.size() > 0)
+        {
+            max = Math.max(max, activeRequests.size());
+            // Finish a request
+            int ix = random.nextInt(activeRequests.size());
+            InetAddress finished = activeRequests.get(ix);
+            activeRequests.remove(ix);
+            requestCoordinator.completed(finished);
+        }
+        return max;
+    }
+}


[7/8] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by yu...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
	src/java/org/apache/cassandra/repair/RepairJob.java
	src/java/org/apache/cassandra/repair/RepairSession.java
	src/java/org/apache/cassandra/service/ActiveRepairService.java
	src/java/org/apache/cassandra/service/StorageService.java
	src/java/org/apache/cassandra/service/StorageServiceMBean.java
	src/java/org/apache/cassandra/tools/NodeCmd.java
	src/java/org/apache/cassandra/tools/NodeProbe.java
	src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/326a9ff2
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/326a9ff2
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/326a9ff2

Branch: refs/heads/cassandra-2.1
Commit: 326a9ff2f831eeafedbc37b7a4b8f8f4a709e399
Parents: eac7781 41469ec
Author: Yuki Morishita <yu...@apache.org>
Authored: Mon Nov 24 15:21:34 2014 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Mon Nov 24 15:21:34 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../DatacenterAwareRequestCoordinator.java      |  73 +++++++++++
 .../cassandra/repair/IRequestCoordinator.java   |  28 ++++
 .../cassandra/repair/IRequestProcessor.java     |  23 ++++
 .../repair/ParallelRequestCoordinator.java      |  49 +++++++
 .../org/apache/cassandra/repair/RepairJob.java  |  32 ++++-
 .../cassandra/repair/RepairParallelism.java     |  22 ++++
 .../apache/cassandra/repair/RepairSession.java  |  14 +-
 .../cassandra/repair/RequestCoordinator.java    | 128 -------------------
 .../repair/SequentialRequestCoordinator.java    |  58 +++++++++
 .../cassandra/service/ActiveRepairService.java  |   6 +-
 .../cassandra/service/StorageService.java       |  49 +++++--
 .../cassandra/service/StorageServiceMBean.java  |  20 ++-
 .../org/apache/cassandra/tools/NodeProbe.java   |  29 +++--
 .../org/apache/cassandra/tools/NodeTool.java    |  14 +-
 .../repair/RequestCoordinatorTest.java          | 124 ++++++++++++++++++
 16 files changed, 499 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c9e35d5,7519653..fa3ce8a
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -26,34 -12,7 +26,35 @@@ Merged from 2.0
   * Avoid overlap in L1 when L0 contains many nonoverlapping
     sstables (CASSANDRA-8211)
   * Improve PropertyFileSnitch logging (CASSANDRA-8183)
 - * Abort liveRatio calculation if the memtable is flushed (CASSANDRA-8164)
++ * Add DC-aware sequential repair (CASSANDRA-8193)
 +
 +
 +2.1.2
 + * (cqlsh) parse_for_table_meta errors out on queries with undefined
 +   grammars (CASSANDRA-8262)
 + * (cqlsh) Fix SELECT ... TOKEN() function broken in C* 2.1.1 (CASSANDRA-8258)
 + * Fix Cassandra crash when running on JDK8 update 40 (CASSANDRA-8209)
 + * Optimize partitioner tokens (CASSANDRA-8230)
 + * Improve compaction of repaired/unrepaired sstables (CASSANDRA-8004)
 + * Make cache serializers pluggable (CASSANDRA-8096)
 + * Fix issues with CONTAINS (KEY) queries on secondary indexes
 +   (CASSANDRA-8147)
 + * Fix read-rate tracking of sstables for some queries (CASSANDRA-8239)
 + * Fix default timestamp in QueryOptions (CASSANDRA-8246)
 + * Set socket timeout when reading remote version (CASSANDRA-8188)
 + * Refactor how we track live size (CASSANDRA-7852)
 + * Make sure unfinished compaction files are removed (CASSANDRA-8124)
 + * Fix shutdown when run as Windows service (CASSANDRA-8136)
 + * Fix DESCRIBE TABLE with custom indexes (CASSANDRA-8031)
 + * Fix race in RecoveryManagerTest (CASSANDRA-8176)
 + * Avoid IllegalArgumentException while sorting sstables in
 +   IndexSummaryManager (CASSANDRA-8182)
 + * Shutdown JVM on file descriptor exhaustion (CASSANDRA-7579)
 + * Add 'die' policy for commit log and disk failure (CASSANDRA-7927)
 + * Fix installing as service on Windows (CASSANDRA-8115)
 + * Fix CREATE TABLE for CQL2 (CASSANDRA-8144)
 + * Avoid boxing in ColumnStats min/max trackers (CASSANDRA-8109)
 +Merged from 2.0:
   * Correctly handle non-text column names in cql3 (CASSANDRA-8178)
   * Fix deletion for indexes on primary key columns (CASSANDRA-8206)
   * Add 'nodetool statusgossip' (CASSANDRA-8125)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/repair/RepairJob.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/repair/RepairJob.java
index 8057ed5,7c791aa..20d5d97
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@@ -73,12 -72,14 +73,14 @@@ public class RepairJo
                       ListeningExecutorService taskExecutor)
      {
          this.listener = listener;
 -        this.desc = new RepairJobDesc(sessionId, keyspace, columnFamily, range);
 +        this.desc = new RepairJobDesc(parentSessionId, sessionId, keyspace, columnFamily, range);
-         this.isSequential = isSequential;
+         this.parallelismDegree = parallelismDegree;
          this.taskExecutor = taskExecutor;
-         this.treeRequests = new RequestCoordinator<InetAddress>(isSequential)
+ 
+         IRequestProcessor<InetAddress> processor = new IRequestProcessor<InetAddress>()
          {
-             public void send(InetAddress endpoint)
+             @Override
+             public void process(InetAddress endpoint)
              {
                  ValidationRequest request = new ValidationRequest(desc, gcBefore);
                  MessagingService.instance().sendOneWay(request.createMessage(), endpoint);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/repair/RepairSession.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/repair/RepairSession.java
index 346f3f4,f2b95eb..0580ebb
--- a/src/java/org/apache/cassandra/repair/RepairSession.java
+++ b/src/java/org/apache/cassandra/repair/RepairSession.java
@@@ -114,20 -110,19 +114,20 @@@ public class RepairSession extends Wrap
       *
       * @param range range to repair
       * @param keyspace name of keyspace
-      * @param isSequential true if performing repair on snapshots sequentially
+      * @param parallelismDegree specifies the degree of parallelism when calculating the merkle trees
 -     * @param dataCenters the data centers that should be part of the repair; null for all DCs
 +     * @param endpoints the data centers that should be part of the repair; null for all DCs
       * @param cfnames names of columnfamilies
       */
-     public RepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, boolean isSequential, Set<InetAddress> endpoints, String... cfnames)
 -    public RepairSession(Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
++    public RepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String... cfnames)
      {
-         this(parentRepairSession, UUIDGen.getTimeUUID(), range, keyspace, isSequential, endpoints, cfnames);
 -        this(UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, dataCenters, hosts, cfnames);
++        this(parentRepairSession, UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, endpoints, cfnames);
      }
  
-     public RepairSession(UUID parentRepairSession, UUID id, Range<Token> range, String keyspace, boolean isSequential, Set<InetAddress> endpoints, String[] cfnames)
 -    public RepairSession(UUID id, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String[] cfnames)
++    public RepairSession(UUID parentRepairSession, UUID id, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String[] cfnames)
      {
 +        this.parentRepairSession = parentRepairSession;
          this.id = id;
-         this.isSequential = isSequential;
+         this.parallelismDegree = parallelismDegree;
          this.keyspace = keyspace;
          this.cfnames = cfnames;
          assert cfnames.length > 0 : "Repairing no column families seems pointless, doesn't it";
@@@ -284,10 -270,10 +284,10 @@@
              // Create and queue a RepairJob for each column family
              for (String cfname : cfnames)
              {
-                 RepairJob job = new RepairJob(this, parentRepairSession, id, keyspace, cfname, range, isSequential, taskExecutor);
 -                RepairJob job = new RepairJob(this, id, keyspace, cfname, range, parallelismDegree, taskExecutor);
++                RepairJob job = new RepairJob(this, parentRepairSession, id, keyspace, cfname, range, parallelismDegree, taskExecutor);
                  jobs.offer(job);
              }
 -
 +            logger.debug("Sending tree requests to endpoints {}", endpoints);
              jobs.peek().sendTreeRequests(endpoints);
  
              // block whatever thread started this session until all requests have been returned:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/ActiveRepairService.java
index 68c2fae,da81e8f..d43143e
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@@ -116,9 -92,9 +116,9 @@@ public class ActiveRepairServic
       *
       * @return Future for asynchronous call or null if there is no need to repair
       */
-     public RepairFuture submitRepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, boolean isSequential, Set<InetAddress> endpoints, String... cfnames)
 -    public RepairFuture submitRepairSession(Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
++    public RepairFuture submitRepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String... cfnames)
      {
-         RepairSession session = new RepairSession(parentRepairSession, range, keyspace, isSequential, endpoints, cfnames);
 -        RepairSession session = new RepairSession(range, keyspace, parallelismDegree, dataCenters, hosts, cfnames);
++        RepairSession session = new RepairSession(parentRepairSession, range, keyspace, parallelismDegree, endpoints, cfnames);
          if (session.endpoints.isEmpty())
              return null;
          RepairFuture futureTask = new RepairFuture(session);
@@@ -152,9 -128,7 +152,9 @@@
      // add it to the sessions (avoid NPE in tests)
      RepairFuture submitArtificialRepairSession(RepairJobDesc desc)
      {
 -        RepairSession session = new RepairSession(desc.sessionId, desc.range, desc.keyspace, RepairParallelism.PARALLEL, null, null, new String[]{desc.columnFamily});
 +        Set<InetAddress> neighbours = new HashSet<>();
 +        neighbours.addAll(ActiveRepairService.getNeighbors(desc.keyspace, desc.range, null, null));
-         RepairSession session = new RepairSession(desc.parentSessionId, desc.sessionId, desc.range, desc.keyspace, false, neighbours, new String[]{desc.columnFamily});
++        RepairSession session = new RepairSession(desc.parentSessionId, desc.sessionId, desc.range, desc.keyspace, RepairParallelism.PARALLEL, neighbours, new String[]{desc.columnFamily});
          sessions.put(session.getId(), session);
          RepairFuture futureTask = new RepairFuture(session);
          executor.execute(futureTask);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 79cea8e,3d42d1c..38cca10
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2489,86 -2407,112 +2490,106 @@@ public class StorageService extends Not
          sendNotification(jmxNotification);
      }
  
 -    public int forceRepairAsync(final String keyspace, final boolean isSequential, final Collection<String> dataCenters, final Collection<String> hosts, final boolean primaryRange, final String... columnFamilies)
 +    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies) throws IOException
      {
 -        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, columnFamilies);
++        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, fullRepair, columnFamilies);
+     }
+ 
 -    public int forceRepairAsync(final String keyspace, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts, final boolean primaryRange, final String... columnFamilies)
++    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies)
+     {
 -        // when repairing only primary range, dataCenter nor hosts can be set
 -        if (primaryRange && (dataCenters != null || hosts != null))
 +        Collection<Range<Token>> ranges;
 +        if (primaryRange)
          {
 -            throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
 +            // when repairing only primary range, neither dataCenters nor hosts can be set
 +            if (dataCenters == null && hosts == null)
 +                ranges = getPrimaryRanges(keyspace);
 +            // except dataCenters only contain local DC (i.e. -local)
 +            else if (dataCenters != null && dataCenters.size() == 1 && dataCenters.contains(DatabaseDescriptor.getLocalDataCenter()))
 +                ranges = getPrimaryRangesWithinDC(keyspace);
 +            else
 +                throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
          }
 -        final Collection<Range<Token>> ranges = primaryRange ? getLocalPrimaryRanges(keyspace) : getLocalRanges(keyspace);
 -        return forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, ranges, columnFamilies);
 +        else
 +        {
 +             ranges = getLocalRanges(keyspace);
 +        }
 +
-         return forceRepairAsync(keyspace, isSequential, dataCenters, hosts, ranges, fullRepair, columnFamilies);
++        return forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, ranges, fullRepair, columnFamilies);
      }
  
 -    public int forceRepairAsync(final String keyspace, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts,  final Collection<Range<Token>> ranges, final String... columnFamilies)
 +    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
 +    {
++        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, ranges, fullRepair, columnFamilies);
++    }
++
++    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
+     {
          if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
              return 0;
  
 -        final int cmd = nextRepairCommand.incrementAndGet();
 +        int cmd = nextRepairCommand.incrementAndGet();
          if (ranges.size() > 0)
          {
-             if (!FBUtilities.isUnix() && isSequential)
 -            new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, hosts, columnFamilies)).start();
++            if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
 +            {
 +                logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-                 isSequential = false;
++                parallelismDegree = RepairParallelism.PARALLEL;
 +            }
-             new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, hosts, fullRepair, columnFamilies)).start();
++            new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, hosts, fullRepair, columnFamilies)).start();
          }
          return cmd;
      }
  
 -    public int forceRepairAsync(final String keyspace, final boolean isSequential, final boolean isLocal, final boolean primaryRange, final String... columnFamilies)
 +    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, boolean primaryRange, boolean fullRepair, String... columnFamilies)
      {
 -        // when repairing only primary range, you cannot repair only on local DC
 -        if (primaryRange && isLocal)
 +        Collection<Range<Token>> ranges;
 +        if (primaryRange)
          {
 -            throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
 +            ranges = isLocal ? getPrimaryRangesWithinDC(keyspace) : getPrimaryRanges(keyspace);
          }
 -        final Collection<Range<Token>> ranges = primaryRange ? getLocalPrimaryRanges(keyspace) : getLocalRanges(keyspace);
 -        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, ranges, columnFamilies);
 -    }
 -
 -    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, boolean isLocal, Collection<Range<Token>> ranges, String... columnFamilies)
 -    {
 -        if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
 -            return 0;
 -
 -        final int cmd = nextRepairCommand.incrementAndGet();
 -        if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
 +        else
          {
 -            logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
 -            parallelismDegree = RepairParallelism.PARALLEL;
 +            ranges = getLocalRanges(keyspace);
          }
 -        new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, isLocal, columnFamilies)).start();
 -        return cmd;
 +
 +        return forceRepairAsync(keyspace, isSequential, isLocal, ranges, fullRepair, columnFamilies);
      }
  
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
 +    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
      {
 -        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, columnFamilies);
++        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, ranges, fullRepair, columnFamilies);
+     }
+ 
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
++    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, boolean isLocal, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
+     {
 -        Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
 -
 -        logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
 -                    repairingRange, keyspaceName, columnFamilies);
 +        if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
 +            return 0;
  
 +        int cmd = nextRepairCommand.incrementAndGet();
-         if (!FBUtilities.isUnix() && isSequential)
+         if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
          {
              logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-             isSequential = false;
+             parallelismDegree = RepairParallelism.PARALLEL;
          }
-         new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, isLocal, fullRepair, columnFamilies)).start();
 -        return forceRepairAsync(keyspaceName, parallelismDegree, dataCenters, hosts, repairingRange, columnFamilies);
 -    }
 -
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies)
 -    {
 -        Set<String> dataCenters = null;
 -        if (isLocal)
 -        {
 -            dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
 -        }
 -        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential, dataCenters, null, columnFamilies);
++        new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, isLocal, fullRepair, columnFamilies)).start();
 +        return cmd;
      }
  
 -    /**
 -     * Trigger proactive repair for a keyspace and column families.
 -     */
 -    public void forceKeyspaceRepair(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
 +    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies) throws IOException
      {
 -        forceKeyspaceRepairRange(keyspaceName, getLocalRanges(keyspaceName), isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, columnFamilies);
++        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, fullRepair, columnFamilies);
+     }
+ 
 -    public void forceKeyspaceRepairPrimaryRange(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
++    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies)
+     {
 -        // primary range repair can only be performed for whole cluster.
 -        // NOTE: we should omit the param but keep API as is for now.
 -        if (isLocal)
 -        {
 -            throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
 -        }
 +        Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
  
 -        forceKeyspaceRepairRange(keyspaceName, getLocalPrimaryRanges(keyspaceName), isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, false, columnFamilies);
 +        logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
 +                           repairingRange, keyspaceName, columnFamilies);
-         return forceRepairAsync(keyspaceName, isSequential, dataCenters, hosts, repairingRange, fullRepair, columnFamilies);
++        return forceRepairAsync(keyspaceName, parallelismDegree, dataCenters, hosts, repairingRange, fullRepair, columnFamilies);
      }
  
 -    public void forceKeyspaceRepairRange(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
 +    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, boolean isLocal, boolean fullRepair, String... columnFamilies)
      {
          Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
  
@@@ -2616,30 -2567,17 +2637,30 @@@
          return repairingRange;
      }
  
 -    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final RepairParallelism parallelismDegree, final boolean isLocal, final String... columnFamilies)
 +    private FutureTask<Object> createRepairTask(int cmd,
 +                                                String keyspace,
 +                                                Collection<Range<Token>> ranges,
-                                                 boolean isSequential,
++                                                RepairParallelism parallelismDegree,
 +                                                boolean isLocal,
 +                                                boolean fullRepair,
 +                                                String... columnFamilies)
      {
          Set<String> dataCenters = null;
          if (isLocal)
          {
              dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
          }
-         return createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, null, fullRepair, columnFamilies);
 -        return createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, null, columnFamilies);
++        return createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, null, fullRepair, columnFamilies);
      }
  
 -    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
 +    private FutureTask<Object> createRepairTask(final int cmd,
 +                                                final String keyspace,
 +                                                final Collection<Range<Token>> ranges,
-                                                 final boolean isSequential,
++                                                final RepairParallelism parallelismDegree,
 +                                                final Collection<String> dataCenters,
 +                                                final Collection<String> hosts,
 +                                                final boolean fullRepair,
 +                                                final String... columnFamilies)
      {
          if (dataCenters != null && !dataCenters.contains(DatabaseDescriptor.getLocalDataCenter()))
          {
@@@ -2650,71 -2588,24 +2671,71 @@@
          {
              protected void runMayThrow() throws Exception
              {
-                 String message = String.format("Starting repair command #%d, repairing %d ranges for keyspace %s (seq=%b, full=%b)", cmd, ranges.size(), keyspace, isSequential, fullRepair);
 -                String message = String.format("Starting repair command #%d, repairing %d ranges for keyspace %s", cmd, ranges.size(), keyspace);
++                String message = String.format("Starting repair command #%d, repairing %d ranges for keyspace %s (parallelism=%s, full=%b)", cmd, ranges.size(), keyspace, parallelismDegree, fullRepair);
                  logger.info(message);
                  sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.STARTED.ordinal()});
  
-                 if (isSequential && !fullRepair)
 -                List<RepairFuture> futures = new ArrayList<>(ranges.size());
++                if (parallelismDegree != RepairParallelism.PARALLEL && !fullRepair)
 +                {
 +                    message = "It is not possible to mix sequential repair and incremental repairs.";
 +                    logger.error(message);
 +                    sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
 +                    return;
 +                }
 +
 +                Set<InetAddress> allNeighbors = new HashSet<>();
 +                Map<Range, Set<InetAddress>> rangeToNeighbors = new HashMap<>();
                  for (Range<Token> range : ranges)
                  {
 -                    RepairFuture future;
                      try
                      {
 -                        future = forceKeyspaceRepair(range, keyspace, parallelismDegree, dataCenters, hosts, columnFamilies);
 +                        Set<InetAddress> neighbors = ActiveRepairService.getNeighbors(keyspace, range, dataCenters, hosts);
 +                        rangeToNeighbors.put(range, neighbors);
 +                        allNeighbors.addAll(neighbors);
                      }
                      catch (IllegalArgumentException e)
                      {
 -                        logger.error("Repair session failed:", e);
 -                        sendNotification("repair", e.getMessage(), new int[]{cmd, ActiveRepairService.Status.SESSION_FAILED.ordinal()});
 -                        continue;
 +                        logger.error("Repair failed:", e);
 +                        sendNotification("repair", e.getMessage(), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
 +                        return;
                      }
 +                }
 +
 +                // Validate columnfamilies
 +                List<ColumnFamilyStore> columnFamilyStores = new ArrayList<>();
 +                try
 +                {
 +                    Iterables.addAll(columnFamilyStores, getValidColumnFamilies(false, false, keyspace, columnFamilies));
 +                }
 +                catch (IllegalArgumentException e)
 +                {
 +                    sendNotification("repair", e.getMessage(), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
 +                    return;
 +                }
 +
 +                UUID parentSession = null;
 +                if (!fullRepair)
 +                {
 +                    try
 +                    {
 +                        parentSession = ActiveRepairService.instance.prepareForRepair(allNeighbors, ranges, columnFamilyStores);
 +                    }
 +                    catch (Throwable t)
 +                    {
 +                        sendNotification("repair", String.format("Repair failed with error %s", t.getMessage()), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
 +                        return;
 +                    }
 +                }
 +
 +                List<RepairFuture> futures = new ArrayList<>(ranges.size());
 +                String[] cfnames = new String[columnFamilyStores.size()];
 +                for (int i = 0; i < columnFamilyStores.size(); i++)
 +                {
 +                    cfnames[i] = columnFamilyStores.get(i).name;
 +                }
 +                for (Range<Token> range : ranges)
 +                {
-                     RepairFuture future = ActiveRepairService.instance.submitRepairSession(parentSession, range, keyspace, isSequential, rangeToNeighbors.get(range), cfnames);
++                    RepairFuture future = ActiveRepairService.instance.submitRepairSession(parentSession, range, keyspace, parallelismDegree, rangeToNeighbors.get(range), cfnames);
                      if (future == null)
                          continue;
                      futures.add(future);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 8ae44ff,2386fc8..e7d6f14
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -272,14 -259,30 +274,30 @@@ public interface StorageServiceMBean ex
       *
       * @return Repair command number, or 0 if nothing to repair
       */
 -    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, String... columnFamilies);
 +    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts,  boolean primaryRange, boolean repairedAt, String... columnFamilies) throws IOException;
  
      /**
+      * Invoke repair asynchronously.
+      * You can track repair progress by subscribing JMX notification sent from this StorageServiceMBean.
+      * Notification format is:
+      *   type: "repair"
+      *   userObject: int array of length 2, [0]=command number, [1]=ordinal of AntiEntropyService.Status
+      *
+      * @return Repair command number, or 0 if nothing to repair
+      */
 -    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, String... columnFamilies);
++    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies);
+ 
+     /**
       * Same as forceRepairAsync, but handles a specified range
       */
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts,  final String... columnFamilies);
 +    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean repairedAt, String... columnFamilies) throws IOException;
  
      /**
+      * Same as forceRepairAsync, but handles a specified range
+      */
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts,  final String... columnFamilies);
++    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies);
+ 
+     /**
       * Invoke repair asynchronously.
       * You can track repair progress by subscribing JMX notification sent from this StorageServiceMBean.
       * Notification format is:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index d495786,261d416..1d05887
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -244,14 -211,24 +245,19 @@@ public class NodeProbe implements AutoC
          ssProxy.forceKeyspaceFlush(keyspaceName, columnFamilies);
      }
  
 -    public void forceKeyspaceRepair(String keyspaceName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException
 -    {
 -        ssProxy.forceKeyspaceRepair(keyspaceName, isSequential, isLocal, columnFamilies);
 -    }
 -
 -    public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts,  boolean primaryRange, String... columnFamilies) throws IOException
 +    public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies) throws IOException
      {
 -        forceRepairAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, columnFamilies);
++        forceRepairAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, fullRepair, columnFamilies);
+     }
+ 
 -    public void forceRepairAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts,  boolean primaryRange, String... columnFamilies) throws IOException
++    public void forceRepairAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies) throws IOException
+     {
          RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
          try
          {
              jmxc.addConnectionNotificationListener(runner, null, null);
              ssProxy.addNotificationListener(runner, null, null);
-             if (!runner.repairAndWait(ssProxy, isSequential, dataCenters, hosts, primaryRange, fullRepair))
 -            if (!runner.repairAndWait(ssProxy, parallelismDegree, dataCenters, hosts, primaryRange))
++            if (!runner.repairAndWait(ssProxy, parallelismDegree, dataCenters, hosts, primaryRange, fullRepair))
                  failed = true;
          }
          catch (Exception e)
@@@ -265,22 -242,22 +271,27 @@@
                  ssProxy.removeNotificationListener(runner);
                  jmxc.removeConnectionNotificationListener(runner);
              }
 -            catch (Throwable ignored) {}
 +            catch (Throwable t)
 +            {
 +                JVMStabilityInspector.inspectThrowable(t);
 +                out.println("Exception occurred during clean-up. " + t);
 +            }
          }
      }
 -    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, String... columnFamilies) throws IOException
 +
 +    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, final String startToken, final String endToken, boolean fullRepair, String... columnFamilies) throws IOException
      {
 -        forceRepairRangeAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, startToken, endToken, columnFamilies);
++        forceRepairRangeAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, startToken, endToken, fullRepair, columnFamilies);
+     }
+ 
 -    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, String... columnFamilies) throws IOException
++    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, boolean fullRepair, String... columnFamilies) throws IOException
+     {
          RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
          try
          {
              jmxc.addConnectionNotificationListener(runner, null, null);
              ssProxy.addNotificationListener(runner, null, null);
-             if (!runner.repairRangeAndWait(ssProxy,  isSequential, dataCenters, hosts, startToken, endToken, fullRepair))
 -            if (!runner.repairRangeAndWait(ssProxy, parallelismDegree, dataCenters, hosts, startToken, endToken))
++            if (!runner.repairRangeAndWait(ssProxy, parallelismDegree, dataCenters, hosts, startToken, endToken, fullRepair))
                  failed = true;
          }
          catch (Exception e)
@@@ -1287,16 -1070,16 +1298,16 @@@ class RepairRunner implements Notificat
          this.columnFamilies = columnFamilies;
      }
  
-     public boolean repairAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRangeOnly, boolean fullRepair) throws Exception
 -    public boolean repairAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRangeOnly) throws Exception
++    public boolean repairAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRangeOnly, boolean fullRepair) throws Exception
      {
-         cmd = ssProxy.forceRepairAsync(keyspace, isSequential, dataCenters, hosts, primaryRangeOnly, fullRepair, columnFamilies);
 -        cmd = ssProxy.forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, primaryRangeOnly, columnFamilies);
++        cmd = ssProxy.forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, primaryRangeOnly, fullRepair, columnFamilies);
          waitForRepair();
          return success;
      }
  
-     public boolean repairRangeAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, String startToken, String endToken, boolean fullRepair) throws Exception
 -    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, String startToken, String endToken) throws Exception
++    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String startToken, String endToken, boolean fullRepair) throws Exception
      {
-         cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, isSequential, dataCenters, hosts, fullRepair, columnFamilies);
 -        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, parallelismDegree, dataCenters, hosts, columnFamilies);
++        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, parallelismDegree, dataCenters, hosts, fullRepair, columnFamilies);
          waitForRepair();
          return success;
      }


[5/8] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by yu...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
	src/java/org/apache/cassandra/repair/RepairJob.java
	src/java/org/apache/cassandra/repair/RepairSession.java
	src/java/org/apache/cassandra/service/ActiveRepairService.java
	src/java/org/apache/cassandra/service/StorageService.java
	src/java/org/apache/cassandra/service/StorageServiceMBean.java
	src/java/org/apache/cassandra/tools/NodeCmd.java
	src/java/org/apache/cassandra/tools/NodeProbe.java
	src/resources/org/apache/cassandra/tools/NodeToolHelp.yaml


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/326a9ff2
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/326a9ff2
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/326a9ff2

Branch: refs/heads/trunk
Commit: 326a9ff2f831eeafedbc37b7a4b8f8f4a709e399
Parents: eac7781 41469ec
Author: Yuki Morishita <yu...@apache.org>
Authored: Mon Nov 24 15:21:34 2014 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Mon Nov 24 15:21:34 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../DatacenterAwareRequestCoordinator.java      |  73 +++++++++++
 .../cassandra/repair/IRequestCoordinator.java   |  28 ++++
 .../cassandra/repair/IRequestProcessor.java     |  23 ++++
 .../repair/ParallelRequestCoordinator.java      |  49 +++++++
 .../org/apache/cassandra/repair/RepairJob.java  |  32 ++++-
 .../cassandra/repair/RepairParallelism.java     |  22 ++++
 .../apache/cassandra/repair/RepairSession.java  |  14 +-
 .../cassandra/repair/RequestCoordinator.java    | 128 -------------------
 .../repair/SequentialRequestCoordinator.java    |  58 +++++++++
 .../cassandra/service/ActiveRepairService.java  |   6 +-
 .../cassandra/service/StorageService.java       |  49 +++++--
 .../cassandra/service/StorageServiceMBean.java  |  20 ++-
 .../org/apache/cassandra/tools/NodeProbe.java   |  29 +++--
 .../org/apache/cassandra/tools/NodeTool.java    |  14 +-
 .../repair/RequestCoordinatorTest.java          | 124 ++++++++++++++++++
 16 files changed, 499 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c9e35d5,7519653..fa3ce8a
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -26,34 -12,7 +26,35 @@@ Merged from 2.0
   * Avoid overlap in L1 when L0 contains many nonoverlapping
     sstables (CASSANDRA-8211)
   * Improve PropertyFileSnitch logging (CASSANDRA-8183)
 - * Abort liveRatio calculation if the memtable is flushed (CASSANDRA-8164)
++ * Add DC-aware sequential repair (CASSANDRA-8193)
 +
 +
 +2.1.2
 + * (cqlsh) parse_for_table_meta errors out on queries with undefined
 +   grammars (CASSANDRA-8262)
 + * (cqlsh) Fix SELECT ... TOKEN() function broken in C* 2.1.1 (CASSANDRA-8258)
 + * Fix Cassandra crash when running on JDK8 update 40 (CASSANDRA-8209)
 + * Optimize partitioner tokens (CASSANDRA-8230)
 + * Improve compaction of repaired/unrepaired sstables (CASSANDRA-8004)
 + * Make cache serializers pluggable (CASSANDRA-8096)
 + * Fix issues with CONTAINS (KEY) queries on secondary indexes
 +   (CASSANDRA-8147)
 + * Fix read-rate tracking of sstables for some queries (CASSANDRA-8239)
 + * Fix default timestamp in QueryOptions (CASSANDRA-8246)
 + * Set socket timeout when reading remote version (CASSANDRA-8188)
 + * Refactor how we track live size (CASSANDRA-7852)
 + * Make sure unfinished compaction files are removed (CASSANDRA-8124)
 + * Fix shutdown when run as Windows service (CASSANDRA-8136)
 + * Fix DESCRIBE TABLE with custom indexes (CASSANDRA-8031)
 + * Fix race in RecoveryManagerTest (CASSANDRA-8176)
 + * Avoid IllegalArgumentException while sorting sstables in
 +   IndexSummaryManager (CASSANDRA-8182)
 + * Shutdown JVM on file descriptor exhaustion (CASSANDRA-7579)
 + * Add 'die' policy for commit log and disk failure (CASSANDRA-7927)
 + * Fix installing as service on Windows (CASSANDRA-8115)
 + * Fix CREATE TABLE for CQL2 (CASSANDRA-8144)
 + * Avoid boxing in ColumnStats min/max trackers (CASSANDRA-8109)
 +Merged from 2.0:
   * Correctly handle non-text column names in cql3 (CASSANDRA-8178)
   * Fix deletion for indexes on primary key columns (CASSANDRA-8206)
   * Add 'nodetool statusgossip' (CASSANDRA-8125)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/repair/RepairJob.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/repair/RepairJob.java
index 8057ed5,7c791aa..20d5d97
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@@ -73,12 -72,14 +73,14 @@@ public class RepairJo
                       ListeningExecutorService taskExecutor)
      {
          this.listener = listener;
 -        this.desc = new RepairJobDesc(sessionId, keyspace, columnFamily, range);
 +        this.desc = new RepairJobDesc(parentSessionId, sessionId, keyspace, columnFamily, range);
-         this.isSequential = isSequential;
+         this.parallelismDegree = parallelismDegree;
          this.taskExecutor = taskExecutor;
-         this.treeRequests = new RequestCoordinator<InetAddress>(isSequential)
+ 
+         IRequestProcessor<InetAddress> processor = new IRequestProcessor<InetAddress>()
          {
-             public void send(InetAddress endpoint)
+             @Override
+             public void process(InetAddress endpoint)
              {
                  ValidationRequest request = new ValidationRequest(desc, gcBefore);
                  MessagingService.instance().sendOneWay(request.createMessage(), endpoint);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/repair/RepairSession.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/repair/RepairSession.java
index 346f3f4,f2b95eb..0580ebb
--- a/src/java/org/apache/cassandra/repair/RepairSession.java
+++ b/src/java/org/apache/cassandra/repair/RepairSession.java
@@@ -114,20 -110,19 +114,20 @@@ public class RepairSession extends Wrap
       *
       * @param range range to repair
       * @param keyspace name of keyspace
-      * @param isSequential true if performing repair on snapshots sequentially
+      * @param parallelismDegree specifies the degree of parallelism when calculating the merkle trees
 -     * @param dataCenters the data centers that should be part of the repair; null for all DCs
 +     * @param endpoints the data centers that should be part of the repair; null for all DCs
       * @param cfnames names of columnfamilies
       */
-     public RepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, boolean isSequential, Set<InetAddress> endpoints, String... cfnames)
 -    public RepairSession(Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
++    public RepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String... cfnames)
      {
-         this(parentRepairSession, UUIDGen.getTimeUUID(), range, keyspace, isSequential, endpoints, cfnames);
 -        this(UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, dataCenters, hosts, cfnames);
++        this(parentRepairSession, UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, endpoints, cfnames);
      }
  
-     public RepairSession(UUID parentRepairSession, UUID id, Range<Token> range, String keyspace, boolean isSequential, Set<InetAddress> endpoints, String[] cfnames)
 -    public RepairSession(UUID id, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String[] cfnames)
++    public RepairSession(UUID parentRepairSession, UUID id, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String[] cfnames)
      {
 +        this.parentRepairSession = parentRepairSession;
          this.id = id;
-         this.isSequential = isSequential;
+         this.parallelismDegree = parallelismDegree;
          this.keyspace = keyspace;
          this.cfnames = cfnames;
          assert cfnames.length > 0 : "Repairing no column families seems pointless, doesn't it";
@@@ -284,10 -270,10 +284,10 @@@
              // Create and queue a RepairJob for each column family
              for (String cfname : cfnames)
              {
-                 RepairJob job = new RepairJob(this, parentRepairSession, id, keyspace, cfname, range, isSequential, taskExecutor);
 -                RepairJob job = new RepairJob(this, id, keyspace, cfname, range, parallelismDegree, taskExecutor);
++                RepairJob job = new RepairJob(this, parentRepairSession, id, keyspace, cfname, range, parallelismDegree, taskExecutor);
                  jobs.offer(job);
              }
 -
 +            logger.debug("Sending tree requests to endpoints {}", endpoints);
              jobs.peek().sendTreeRequests(endpoints);
  
              // block whatever thread started this session until all requests have been returned:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/ActiveRepairService.java
index 68c2fae,da81e8f..d43143e
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@@ -116,9 -92,9 +116,9 @@@ public class ActiveRepairServic
       *
       * @return Future for asynchronous call or null if there is no need to repair
       */
-     public RepairFuture submitRepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, boolean isSequential, Set<InetAddress> endpoints, String... cfnames)
 -    public RepairFuture submitRepairSession(Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String... cfnames)
++    public RepairFuture submitRepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String... cfnames)
      {
-         RepairSession session = new RepairSession(parentRepairSession, range, keyspace, isSequential, endpoints, cfnames);
 -        RepairSession session = new RepairSession(range, keyspace, parallelismDegree, dataCenters, hosts, cfnames);
++        RepairSession session = new RepairSession(parentRepairSession, range, keyspace, parallelismDegree, endpoints, cfnames);
          if (session.endpoints.isEmpty())
              return null;
          RepairFuture futureTask = new RepairFuture(session);
@@@ -152,9 -128,7 +152,9 @@@
      // add it to the sessions (avoid NPE in tests)
      RepairFuture submitArtificialRepairSession(RepairJobDesc desc)
      {
 -        RepairSession session = new RepairSession(desc.sessionId, desc.range, desc.keyspace, RepairParallelism.PARALLEL, null, null, new String[]{desc.columnFamily});
 +        Set<InetAddress> neighbours = new HashSet<>();
 +        neighbours.addAll(ActiveRepairService.getNeighbors(desc.keyspace, desc.range, null, null));
-         RepairSession session = new RepairSession(desc.parentSessionId, desc.sessionId, desc.range, desc.keyspace, false, neighbours, new String[]{desc.columnFamily});
++        RepairSession session = new RepairSession(desc.parentSessionId, desc.sessionId, desc.range, desc.keyspace, RepairParallelism.PARALLEL, neighbours, new String[]{desc.columnFamily});
          sessions.put(session.getId(), session);
          RepairFuture futureTask = new RepairFuture(session);
          executor.execute(futureTask);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 79cea8e,3d42d1c..38cca10
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2489,86 -2407,112 +2490,106 @@@ public class StorageService extends Not
          sendNotification(jmxNotification);
      }
  
 -    public int forceRepairAsync(final String keyspace, final boolean isSequential, final Collection<String> dataCenters, final Collection<String> hosts, final boolean primaryRange, final String... columnFamilies)
 +    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies) throws IOException
      {
 -        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, columnFamilies);
++        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, fullRepair, columnFamilies);
+     }
+ 
 -    public int forceRepairAsync(final String keyspace, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts, final boolean primaryRange, final String... columnFamilies)
++    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies)
+     {
 -        // when repairing only primary range, dataCenter nor hosts can be set
 -        if (primaryRange && (dataCenters != null || hosts != null))
 +        Collection<Range<Token>> ranges;
 +        if (primaryRange)
          {
 -            throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
 +            // when repairing only primary range, neither dataCenters nor hosts can be set
 +            if (dataCenters == null && hosts == null)
 +                ranges = getPrimaryRanges(keyspace);
 +            // except dataCenters only contain local DC (i.e. -local)
 +            else if (dataCenters != null && dataCenters.size() == 1 && dataCenters.contains(DatabaseDescriptor.getLocalDataCenter()))
 +                ranges = getPrimaryRangesWithinDC(keyspace);
 +            else
 +                throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
          }
 -        final Collection<Range<Token>> ranges = primaryRange ? getLocalPrimaryRanges(keyspace) : getLocalRanges(keyspace);
 -        return forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, ranges, columnFamilies);
 +        else
 +        {
 +             ranges = getLocalRanges(keyspace);
 +        }
 +
-         return forceRepairAsync(keyspace, isSequential, dataCenters, hosts, ranges, fullRepair, columnFamilies);
++        return forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, ranges, fullRepair, columnFamilies);
      }
  
 -    public int forceRepairAsync(final String keyspace, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts,  final Collection<Range<Token>> ranges, final String... columnFamilies)
 +    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
 +    {
++        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, ranges, fullRepair, columnFamilies);
++    }
++
++    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
+     {
          if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
              return 0;
  
 -        final int cmd = nextRepairCommand.incrementAndGet();
 +        int cmd = nextRepairCommand.incrementAndGet();
          if (ranges.size() > 0)
          {
-             if (!FBUtilities.isUnix() && isSequential)
 -            new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, hosts, columnFamilies)).start();
++            if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
 +            {
 +                logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-                 isSequential = false;
++                parallelismDegree = RepairParallelism.PARALLEL;
 +            }
-             new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, hosts, fullRepair, columnFamilies)).start();
++            new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, hosts, fullRepair, columnFamilies)).start();
          }
          return cmd;
      }
  
 -    public int forceRepairAsync(final String keyspace, final boolean isSequential, final boolean isLocal, final boolean primaryRange, final String... columnFamilies)
 +    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, boolean primaryRange, boolean fullRepair, String... columnFamilies)
      {
 -        // when repairing only primary range, you cannot repair only on local DC
 -        if (primaryRange && isLocal)
 +        Collection<Range<Token>> ranges;
 +        if (primaryRange)
          {
 -            throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
 +            ranges = isLocal ? getPrimaryRangesWithinDC(keyspace) : getPrimaryRanges(keyspace);
          }
 -        final Collection<Range<Token>> ranges = primaryRange ? getLocalPrimaryRanges(keyspace) : getLocalRanges(keyspace);
 -        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, ranges, columnFamilies);
 -    }
 -
 -    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, boolean isLocal, Collection<Range<Token>> ranges, String... columnFamilies)
 -    {
 -        if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
 -            return 0;
 -
 -        final int cmd = nextRepairCommand.incrementAndGet();
 -        if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
 +        else
          {
 -            logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
 -            parallelismDegree = RepairParallelism.PARALLEL;
 +            ranges = getLocalRanges(keyspace);
          }
 -        new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, isLocal, columnFamilies)).start();
 -        return cmd;
 +
 +        return forceRepairAsync(keyspace, isSequential, isLocal, ranges, fullRepair, columnFamilies);
      }
  
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
 +    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
      {
 -        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, columnFamilies);
++        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, ranges, fullRepair, columnFamilies);
+     }
+ 
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
++    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, boolean isLocal, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
+     {
 -        Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
 -
 -        logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
 -                    repairingRange, keyspaceName, columnFamilies);
 +        if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
 +            return 0;
  
 +        int cmd = nextRepairCommand.incrementAndGet();
-         if (!FBUtilities.isUnix() && isSequential)
+         if (!FBUtilities.isUnix() && parallelismDegree != RepairParallelism.PARALLEL)
          {
              logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-             isSequential = false;
+             parallelismDegree = RepairParallelism.PARALLEL;
          }
-         new Thread(createRepairTask(cmd, keyspace, ranges, isSequential, isLocal, fullRepair, columnFamilies)).start();
 -        return forceRepairAsync(keyspaceName, parallelismDegree, dataCenters, hosts, repairingRange, columnFamilies);
 -    }
 -
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies)
 -    {
 -        Set<String> dataCenters = null;
 -        if (isLocal)
 -        {
 -            dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
 -        }
 -        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential, dataCenters, null, columnFamilies);
++        new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, isLocal, fullRepair, columnFamilies)).start();
 +        return cmd;
      }
  
 -    /**
 -     * Trigger proactive repair for a keyspace and column families.
 -     */
 -    public void forceKeyspaceRepair(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
 +    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies) throws IOException
      {
 -        forceKeyspaceRepairRange(keyspaceName, getLocalRanges(keyspaceName), isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, columnFamilies);
++        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, fullRepair, columnFamilies);
+     }
+ 
 -    public void forceKeyspaceRepairPrimaryRange(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
++    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies)
+     {
 -        // primary range repair can only be performed for whole cluster.
 -        // NOTE: we should omit the param but keep API as is for now.
 -        if (isLocal)
 -        {
 -            throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
 -        }
 +        Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
  
 -        forceKeyspaceRepairRange(keyspaceName, getLocalPrimaryRanges(keyspaceName), isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, false, columnFamilies);
 +        logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
 +                           repairingRange, keyspaceName, columnFamilies);
-         return forceRepairAsync(keyspaceName, isSequential, dataCenters, hosts, repairingRange, fullRepair, columnFamilies);
++        return forceRepairAsync(keyspaceName, parallelismDegree, dataCenters, hosts, repairingRange, fullRepair, columnFamilies);
      }
  
 -    public void forceKeyspaceRepairRange(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
 +    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, boolean isLocal, boolean fullRepair, String... columnFamilies)
      {
          Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
  
@@@ -2616,30 -2567,17 +2637,30 @@@
          return repairingRange;
      }
  
 -    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final RepairParallelism parallelismDegree, final boolean isLocal, final String... columnFamilies)
 +    private FutureTask<Object> createRepairTask(int cmd,
 +                                                String keyspace,
 +                                                Collection<Range<Token>> ranges,
-                                                 boolean isSequential,
++                                                RepairParallelism parallelismDegree,
 +                                                boolean isLocal,
 +                                                boolean fullRepair,
 +                                                String... columnFamilies)
      {
          Set<String> dataCenters = null;
          if (isLocal)
          {
              dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
          }
-         return createRepairTask(cmd, keyspace, ranges, isSequential, dataCenters, null, fullRepair, columnFamilies);
 -        return createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, null, columnFamilies);
++        return createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, null, fullRepair, columnFamilies);
      }
  
 -    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final RepairParallelism parallelismDegree, final Collection<String> dataCenters, final Collection<String> hosts, final String... columnFamilies)
 +    private FutureTask<Object> createRepairTask(final int cmd,
 +                                                final String keyspace,
 +                                                final Collection<Range<Token>> ranges,
-                                                 final boolean isSequential,
++                                                final RepairParallelism parallelismDegree,
 +                                                final Collection<String> dataCenters,
 +                                                final Collection<String> hosts,
 +                                                final boolean fullRepair,
 +                                                final String... columnFamilies)
      {
          if (dataCenters != null && !dataCenters.contains(DatabaseDescriptor.getLocalDataCenter()))
          {
@@@ -2650,71 -2588,24 +2671,71 @@@
          {
              protected void runMayThrow() throws Exception
              {
-                 String message = String.format("Starting repair command #%d, repairing %d ranges for keyspace %s (seq=%b, full=%b)", cmd, ranges.size(), keyspace, isSequential, fullRepair);
 -                String message = String.format("Starting repair command #%d, repairing %d ranges for keyspace %s", cmd, ranges.size(), keyspace);
++                String message = String.format("Starting repair command #%d, repairing %d ranges for keyspace %s (parallelism=%s, full=%b)", cmd, ranges.size(), keyspace, parallelismDegree, fullRepair);
                  logger.info(message);
                  sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.STARTED.ordinal()});
  
-                 if (isSequential && !fullRepair)
 -                List<RepairFuture> futures = new ArrayList<>(ranges.size());
++                if (parallelismDegree != RepairParallelism.PARALLEL && !fullRepair)
 +                {
 +                    message = "It is not possible to mix sequential repair and incremental repairs.";
 +                    logger.error(message);
 +                    sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
 +                    return;
 +                }
 +
 +                Set<InetAddress> allNeighbors = new HashSet<>();
 +                Map<Range, Set<InetAddress>> rangeToNeighbors = new HashMap<>();
                  for (Range<Token> range : ranges)
                  {
 -                    RepairFuture future;
                      try
                      {
 -                        future = forceKeyspaceRepair(range, keyspace, parallelismDegree, dataCenters, hosts, columnFamilies);
 +                        Set<InetAddress> neighbors = ActiveRepairService.getNeighbors(keyspace, range, dataCenters, hosts);
 +                        rangeToNeighbors.put(range, neighbors);
 +                        allNeighbors.addAll(neighbors);
                      }
                      catch (IllegalArgumentException e)
                      {
 -                        logger.error("Repair session failed:", e);
 -                        sendNotification("repair", e.getMessage(), new int[]{cmd, ActiveRepairService.Status.SESSION_FAILED.ordinal()});
 -                        continue;
 +                        logger.error("Repair failed:", e);
 +                        sendNotification("repair", e.getMessage(), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
 +                        return;
                      }
 +                }
 +
 +                // Validate columnfamilies
 +                List<ColumnFamilyStore> columnFamilyStores = new ArrayList<>();
 +                try
 +                {
 +                    Iterables.addAll(columnFamilyStores, getValidColumnFamilies(false, false, keyspace, columnFamilies));
 +                }
 +                catch (IllegalArgumentException e)
 +                {
 +                    sendNotification("repair", e.getMessage(), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
 +                    return;
 +                }
 +
 +                UUID parentSession = null;
 +                if (!fullRepair)
 +                {
 +                    try
 +                    {
 +                        parentSession = ActiveRepairService.instance.prepareForRepair(allNeighbors, ranges, columnFamilyStores);
 +                    }
 +                    catch (Throwable t)
 +                    {
 +                        sendNotification("repair", String.format("Repair failed with error %s", t.getMessage()), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
 +                        return;
 +                    }
 +                }
 +
 +                List<RepairFuture> futures = new ArrayList<>(ranges.size());
 +                String[] cfnames = new String[columnFamilyStores.size()];
 +                for (int i = 0; i < columnFamilyStores.size(); i++)
 +                {
 +                    cfnames[i] = columnFamilyStores.get(i).name;
 +                }
 +                for (Range<Token> range : ranges)
 +                {
-                     RepairFuture future = ActiveRepairService.instance.submitRepairSession(parentSession, range, keyspace, isSequential, rangeToNeighbors.get(range), cfnames);
++                    RepairFuture future = ActiveRepairService.instance.submitRepairSession(parentSession, range, keyspace, parallelismDegree, rangeToNeighbors.get(range), cfnames);
                      if (future == null)
                          continue;
                      futures.add(future);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 8ae44ff,2386fc8..e7d6f14
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -272,14 -259,30 +274,30 @@@ public interface StorageServiceMBean ex
       *
       * @return Repair command number, or 0 if nothing to repair
       */
 -    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, String... columnFamilies);
 +    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts,  boolean primaryRange, boolean repairedAt, String... columnFamilies) throws IOException;
  
      /**
+      * Invoke repair asynchronously.
+      * You can track repair progress by subscribing JMX notification sent from this StorageServiceMBean.
+      * Notification format is:
+      *   type: "repair"
+      *   userObject: int array of length 2, [0]=command number, [1]=ordinal of AntiEntropyService.Status
+      *
+      * @return Repair command number, or 0 if nothing to repair
+      */
 -    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, String... columnFamilies);
++    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies);
+ 
+     /**
       * Same as forceRepairAsync, but handles a specified range
       */
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts,  final String... columnFamilies);
 +    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean repairedAt, String... columnFamilies) throws IOException;
  
      /**
+      * Same as forceRepairAsync, but handles a specified range
+      */
 -    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts,  final String... columnFamilies);
++    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies);
+ 
+     /**
       * Invoke repair asynchronously.
       * You can track repair progress by subscribing JMX notification sent from this StorageServiceMBean.
       * Notification format is:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/326a9ff2/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index d495786,261d416..1d05887
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -244,14 -211,24 +245,19 @@@ public class NodeProbe implements AutoC
          ssProxy.forceKeyspaceFlush(keyspaceName, columnFamilies);
      }
  
 -    public void forceKeyspaceRepair(String keyspaceName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException
 -    {
 -        ssProxy.forceKeyspaceRepair(keyspaceName, isSequential, isLocal, columnFamilies);
 -    }
 -
 -    public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts,  boolean primaryRange, String... columnFamilies) throws IOException
 +    public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies) throws IOException
      {
 -        forceRepairAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, columnFamilies);
++        forceRepairAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, fullRepair, columnFamilies);
+     }
+ 
 -    public void forceRepairAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts,  boolean primaryRange, String... columnFamilies) throws IOException
++    public void forceRepairAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies) throws IOException
+     {
          RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
          try
          {
              jmxc.addConnectionNotificationListener(runner, null, null);
              ssProxy.addNotificationListener(runner, null, null);
-             if (!runner.repairAndWait(ssProxy, isSequential, dataCenters, hosts, primaryRange, fullRepair))
 -            if (!runner.repairAndWait(ssProxy, parallelismDegree, dataCenters, hosts, primaryRange))
++            if (!runner.repairAndWait(ssProxy, parallelismDegree, dataCenters, hosts, primaryRange, fullRepair))
                  failed = true;
          }
          catch (Exception e)
@@@ -265,22 -242,22 +271,27 @@@
                  ssProxy.removeNotificationListener(runner);
                  jmxc.removeConnectionNotificationListener(runner);
              }
 -            catch (Throwable ignored) {}
 +            catch (Throwable t)
 +            {
 +                JVMStabilityInspector.inspectThrowable(t);
 +                out.println("Exception occurred during clean-up. " + t);
 +            }
          }
      }
 -    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, String... columnFamilies) throws IOException
 +
 +    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, final String startToken, final String endToken, boolean fullRepair, String... columnFamilies) throws IOException
      {
 -        forceRepairRangeAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, startToken, endToken, columnFamilies);
++        forceRepairRangeAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, startToken, endToken, fullRepair, columnFamilies);
+     }
+ 
 -    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, String... columnFamilies) throws IOException
++    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, boolean fullRepair, String... columnFamilies) throws IOException
+     {
          RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
          try
          {
              jmxc.addConnectionNotificationListener(runner, null, null);
              ssProxy.addNotificationListener(runner, null, null);
-             if (!runner.repairRangeAndWait(ssProxy,  isSequential, dataCenters, hosts, startToken, endToken, fullRepair))
 -            if (!runner.repairRangeAndWait(ssProxy, parallelismDegree, dataCenters, hosts, startToken, endToken))
++            if (!runner.repairRangeAndWait(ssProxy, parallelismDegree, dataCenters, hosts, startToken, endToken, fullRepair))
                  failed = true;
          }
          catch (Exception e)
@@@ -1287,16 -1070,16 +1298,16 @@@ class RepairRunner implements Notificat
          this.columnFamilies = columnFamilies;
      }
  
-     public boolean repairAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRangeOnly, boolean fullRepair) throws Exception
 -    public boolean repairAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRangeOnly) throws Exception
++    public boolean repairAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRangeOnly, boolean fullRepair) throws Exception
      {
-         cmd = ssProxy.forceRepairAsync(keyspace, isSequential, dataCenters, hosts, primaryRangeOnly, fullRepair, columnFamilies);
 -        cmd = ssProxy.forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, primaryRangeOnly, columnFamilies);
++        cmd = ssProxy.forceRepairAsync(keyspace, parallelismDegree, dataCenters, hosts, primaryRangeOnly, fullRepair, columnFamilies);
          waitForRepair();
          return success;
      }
  
-     public boolean repairRangeAndWait(StorageServiceMBean ssProxy, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, String startToken, String endToken, boolean fullRepair) throws Exception
 -    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, String startToken, String endToken) throws Exception
++    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String startToken, String endToken, boolean fullRepair) throws Exception
      {
-         cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, isSequential, dataCenters, hosts, fullRepair, columnFamilies);
 -        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, parallelismDegree, dataCenters, hosts, columnFamilies);
++        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, parallelismDegree, dataCenters, hosts, fullRepair, columnFamilies);
          waitForRepair();
          return success;
      }