diff --git a/src/main/java/org/apache/cassandra/service/StorageService.java b/src/main/java/org/apache/cassandra/service/StorageService.java index 6ddf665..4e34516 100644 --- a/src/main/java/org/apache/cassandra/service/StorageService.java +++ b/src/main/java/org/apache/cassandra/service/StorageService.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -93,6 +94,8 @@ public class StorageService extends MetricsMBean implements StorageServiceMBean, private static final CompositeType COUNTER_COMPOSITE_TYPE; private static final TabularType COUNTER_TYPE; + private static final String[] OPERATION_NAMES = new String[]{"read", "write"}; + private static final String[] SAMPLER_NAMES = new String[]{"cardinality", "partitions"}; private static final String[] SAMPLER_DESCS = new String[] { "cardinality of partitions", @@ -1799,6 +1802,11 @@ public class StorageService extends MetricsMBean implements StorageServiceMBean, @Override public CompositeData getToppartitions(String sampler, List keyspaceFilters, List tableFilters, int duration, int capacity, int count) throws OpenDataException { + return getToppartitions(Arrays.asList(sampler), keyspaceFilters, tableFilters, duration, capacity, count).get(sampler.toLowerCase()); + } + + @Override + public Map getToppartitions(List samplers, List keyspaceFilters, List tableFilters, int duration, int capacity, int count) throws OpenDataException { MultivaluedMap queryParams = new MultivaluedHashMap(); APIClient.set_query_param(queryParams, "duration", Integer.toString(duration)); APIClient.set_query_param(queryParams, "capacity", Integer.toString(capacity)); @@ -1806,23 +1814,29 @@ public class StorageService extends MetricsMBean implements StorageServiceMBean, APIClient.set_query_param(queryParams, "table_filters", tableFilters != null ? APIClient.join(tableFilters.toArray(new String[0])) : null); JsonObject result = client.getJsonObj("/storage_service/toppartitions", queryParams); - JsonArray counters = result.getJsonArray((sampler.equalsIgnoreCase("reads")) ? "read" : "write"); - long cardinality = result.getJsonNumber((sampler.equalsIgnoreCase("reads")) ? "read_cardinality" : "write_cardinality").longValue(); - long size = 0; - TabularDataSupport tabularResult = new TabularDataSupport(COUNTER_TYPE); + Map resultsMap = new HashMap<>(); - if (counters != null) { - size = (count > counters.size()) ? counters.size() : count; - for (int i = 0; i < size; i++) { - JsonObject counter = counters.getJsonObject(i); - tabularResult.put(new CompositeDataSupport(COUNTER_COMPOSITE_TYPE, COUNTER_NAMES, - new Object[] { counter.getString("partition"), // raw - counter.getJsonNumber("count").longValue(), // count - counter.getJsonNumber("error").longValue(), // error - counter.getString("partition") })); // string + for (String operation : OPERATION_NAMES) { + JsonArray counters = result.getJsonArray(operation); + long cardinality = result.getJsonNumber(operation + "_cardinality").longValue(); + long size = 0; + TabularDataSupport tabularResult = new TabularDataSupport(COUNTER_TYPE); + + if (counters != null) { + size = (count > counters.size()) ? counters.size() : count; + for (int i = 0; i < size; i++) { + JsonObject counter = counters.getJsonObject(i); + tabularResult.put(new CompositeDataSupport(COUNTER_COMPOSITE_TYPE, COUNTER_NAMES, + new Object[] { counter.getString("partition"), // raw + counter.getJsonNumber("count").longValue(), // count + counter.getJsonNumber("error").longValue(), // error + counter.getString("partition") })); // string + } } + + resultsMap.put(operation + "s", new CompositeDataSupport(SAMPLING_RESULT, SAMPLER_NAMES, new Object[] { cardinality, tabularResult })); } - return new CompositeDataSupport(SAMPLING_RESULT, SAMPLER_NAMES, new Object[] { cardinality, tabularResult }); + return resultsMap; } } diff --git a/src/main/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/main/java/org/apache/cassandra/service/StorageServiceMBean.java index 1463b41..a3dce1c 100644 --- a/src/main/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/main/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -890,4 +890,6 @@ public interface StorageServiceMBean extends NotificationEmitter { public long getUptime(); public CompositeData getToppartitions(String sampler, List keyspaceFilters, List tableFilters, int duration, int capacity, int count) throws OpenDataException; + + public Map getToppartitions(List samplers, List keyspaceFilters, List tableFilters, int duration, int capacity, int count) throws OpenDataException; }