Remove C* specific class from JMX API

This is follow up 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/e20810c7
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e20810c7
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e20810c7

Branch: refs/heads/cassandra-2.1
Commit: e20810c7bdf81b2ea225d1356ac3237994fb5349
Parents: eb9c5bb
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Jan 6 14:24:37 2015 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Jan 6 14:24:37 2015 -0600

----------------------------------------------------------------------
 .../cassandra/service/StorageService.java       | 25 +++++++++++++-------
 .../cassandra/service/StorageServiceMBean.java  |  9 +++----
 .../org/apache/cassandra/tools/NodeProbe.java   |  4 ++--
 3 files changed, 24 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e20810c7/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 3daa66f..8085d7b 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2414,18 +2414,22 @@ 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);
+        return forceRepairAsync(keyspace, isSequential ? 
RepairParallelism.SEQUENTIAL.ordinal() : RepairParallelism.PARALLEL.ordinal(), 
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)
+    public int forceRepairAsync(final String keyspace, final int 
parallelismDegree, final Collection<String> dataCenters, final 
Collection<String> hosts, final boolean primaryRange, final String... 
columnFamilies)
     {
+        if (parallelismDegree < 0 || parallelismDegree > 
RepairParallelism.values().length - 1)
+        {
+            throw new IllegalArgumentException("Invalid parallelism degree 
specified: " + parallelismDegree);
+        }
         // 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, parallelismDegree, dataCenters, 
hosts, ranges, columnFamilies);
+        return forceRepairAsync(keyspace, 
RepairParallelism.values()[parallelismDegree], dataCenters, hosts, ranges, 
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)
@@ -2469,22 +2473,27 @@ public class StorageService extends 
NotificationBroadcasterSupport implements IE
 
     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);
+        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, 
isSequential ? RepairParallelism.SEQUENTIAL.ordinal() : 
RepairParallelism.PARALLEL.ordinal(), 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)
+    public int forceRepairRangeAsync(String beginToken, String endToken, final 
String keyspaceName, int parallelismDegree, Collection<String> dataCenters, 
final Collection<String> hosts, final String... columnFamilies)
     {
+        if (parallelismDegree < 0 || parallelismDegree > 
RepairParallelism.values().length - 1)
+        {
+            throw new IllegalArgumentException("Invalid parallelism degree 
specified: " + parallelismDegree);
+        }
         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() && parallelismDegree != 
RepairParallelism.PARALLEL)
+        RepairParallelism parallelism = 
RepairParallelism.values()[parallelismDegree];
+        if (!FBUtilities.isUnix() && parallelism != RepairParallelism.PARALLEL)
         {
             logger.warn("Snapshot-based repair is not yet supported on 
Windows.  Reverting to parallel repair.");
-            parallelismDegree = RepairParallelism.PARALLEL;
+            parallelism = RepairParallelism.PARALLEL;
         }
-        return forceRepairAsync(keyspaceName, parallelismDegree, dataCenters, 
hosts, repairingRange, columnFamilies);
+        return forceRepairAsync(keyspaceName, parallelism, dataCenters, hosts, 
repairingRange, columnFamilies);
     }
 
     public int forceRepairRangeAsync(String beginToken, String endToken, final 
String keyspaceName, boolean isSequential, boolean isLocal, final String... 
columnFamilies)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e20810c7/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 0ea08a2..10d17fd 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.service;
 
-import org.apache.cassandra.repair.RepairParallelism;
-
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -268,9 +266,10 @@ public interface StorageServiceMBean extends 
NotificationEmitter
      *   type: "repair"
      *   userObject: int array of length 2, [0]=command number, [1]=ordinal of 
AntiEntropyService.Status
      *
+     * @param parallelismDegree 0: sequential, 1: parallel, 2: DC parallel
      * @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, int parallelismDegree, 
Collection<String> dataCenters, final Collection<String> hosts, boolean 
primaryRange, String... columnFamilies);
 
     /**
      * Same as forceRepairAsync, but handles a specified range
@@ -279,8 +278,10 @@ public interface StorageServiceMBean extends 
NotificationEmitter
 
     /**
      * Same as forceRepairAsync, but handles a specified range
+     *
+     * @param parallelismDegree 0: sequential, 1: parallel, 2: DC parallel
      */
-    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, final 
String keyspaceName, int parallelismDegree, Collection<String> dataCenters, 
final Collection<String> hosts,  final String... columnFamilies);
 
     /**
      * Invoke repair asynchronously.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e20810c7/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 3736659..6b28f18 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -1077,14 +1077,14 @@ class RepairRunner implements NotificationListener
 
     public boolean repairAndWait(StorageServiceMBean ssProxy, 
RepairParallelism parallelismDegree, Collection<String> dataCenters, final 
Collection<String> hosts, boolean primaryRangeOnly) throws Exception
     {
-        cmd = ssProxy.forceRepairAsync(keyspace, parallelismDegree, 
dataCenters, hosts, primaryRangeOnly, columnFamilies);
+        cmd = ssProxy.forceRepairAsync(keyspace, parallelismDegree.ordinal(), 
dataCenters, hosts, primaryRangeOnly, columnFamilies);
         waitForRepair();
         return success;
     }
 
     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, 
parallelismDegree, dataCenters, hosts, columnFamilies);
+        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, 
parallelismDegree.ordinal(), dataCenters, hosts, columnFamilies);
         waitForRepair();
         return success;
     }

Reply via email to