Adding StorageService MBean implementation
This adds the following method implementation that are supported: getLiveNodes() getUnreachableNodes() getTokens() getTokens(String endpoint) getCommitLogLocation() getTokenToEndpointMap() After applying this series, those functionality will be available via jconsole The series also complete the API by adding an implementation based on the swagger storage_service.json definition. An implementation that will be added to the API should be reflected in the MBean. Signed-off-by: Amnon Heiman <amnon@cloudius-systems.com>
This commit is contained in:
parent
67abbeb060
commit
afac410a0d
@ -32,10 +32,12 @@ import java.util.concurrent.*;
|
|||||||
|
|
||||||
import javax.management.*;
|
import javax.management.*;
|
||||||
import javax.management.openmbean.TabularData;
|
import javax.management.openmbean.TabularData;
|
||||||
|
import javax.ws.rs.core.MultivaluedMap;
|
||||||
|
|
||||||
import org.apache.cassandra.repair.RepairParallelism;
|
import org.apache.cassandra.repair.RepairParallelism;
|
||||||
|
|
||||||
import com.cloudius.urchin.api.APIClient;
|
import com.cloudius.urchin.api.APIClient;
|
||||||
|
import com.sun.jersey.core.util.MultivaluedMapImpl;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This abstraction contains the token/identifier of this node on the identifier
|
* This abstraction contains the token/identifier of this node on the identifier
|
||||||
@ -84,7 +86,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public List<String> getLiveNodes() {
|
public List<String> getLiveNodes() {
|
||||||
log(" getLiveNodes()");
|
log(" getLiveNodes()");
|
||||||
return c.getListStrValue("");
|
return c.getListStrValue("/gossiper/endpoint/live");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -95,7 +97,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public List<String> getUnreachableNodes() {
|
public List<String> getUnreachableNodes() {
|
||||||
log(" getUnreachableNodes()");
|
log(" getUnreachableNodes()");
|
||||||
return c.getListStrValue("");
|
return c.getListStrValue("/gossiper/endpoint/down");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -105,7 +107,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public List<String> getJoiningNodes() {
|
public List<String> getJoiningNodes() {
|
||||||
log(" getJoiningNodes()");
|
log(" getJoiningNodes()");
|
||||||
return c.getListStrValue("");
|
return c.getListStrValue("/storage_service/nodes/joining");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -115,7 +117,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public List<String> getLeavingNodes() {
|
public List<String> getLeavingNodes() {
|
||||||
log(" getLeavingNodes()");
|
log(" getLeavingNodes()");
|
||||||
return c.getListStrValue("");
|
return c.getListStrValue("/storage_service/nodes/leaving");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -125,7 +127,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public List<String> getMovingNodes() {
|
public List<String> getMovingNodes() {
|
||||||
log(" getMovingNodes()");
|
log(" getMovingNodes()");
|
||||||
return c.getListStrValue("");
|
return c.getListStrValue("/storage_service/nodes/moving");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -135,7 +137,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public List<String> getTokens() {
|
public List<String> getTokens() {
|
||||||
log(" getTokens()");
|
log(" getTokens()");
|
||||||
return c.getListStrValue("");
|
return c.getListStrValue("/storage_service/tokens/");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -147,7 +149,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public List<String> getTokens(String endpoint) throws UnknownHostException {
|
public List<String> getTokens(String endpoint) throws UnknownHostException {
|
||||||
log(" getTokens(String endpoint) throws UnknownHostException");
|
log(" getTokens(String endpoint) throws UnknownHostException");
|
||||||
return c.getListStrValue("");
|
return c.getListStrValue("/storage_service/tokens/" + endpoint);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -157,7 +159,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public String getReleaseVersion() {
|
public String getReleaseVersion() {
|
||||||
log(" getReleaseVersion()");
|
log(" getReleaseVersion()");
|
||||||
return c.getStringValue("");
|
return c.getStringValue("/storage_service/release_version");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -167,7 +169,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public String getSchemaVersion() {
|
public String getSchemaVersion() {
|
||||||
log(" getSchemaVersion()");
|
log(" getSchemaVersion()");
|
||||||
return c.getStringValue("");
|
return c.getStringValue("/storage_service/schema_version");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -177,7 +179,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public String[] getAllDataFileLocations() {
|
public String[] getAllDataFileLocations() {
|
||||||
log(" getAllDataFileLocations()");
|
log(" getAllDataFileLocations()");
|
||||||
return c.getStringArrValue("");
|
return c.getStringArrValue("/storage_service/data_file/locations");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -187,7 +189,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public String getCommitLogLocation() {
|
public String getCommitLogLocation() {
|
||||||
log(" getCommitLogLocation()");
|
log(" getCommitLogLocation()");
|
||||||
return c.getStringValue("");
|
return c.getStringValue("/storage_service/commitlog");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -197,7 +199,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public String getSavedCachesLocation() {
|
public String getSavedCachesLocation() {
|
||||||
log(" getSavedCachesLocation()");
|
log(" getSavedCachesLocation()");
|
||||||
return c.getStringValue("");
|
return c.getStringValue("/storage_service/saved_caches/location");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -208,7 +210,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public Map<List<String>, List<String>> getRangeToEndpointMap(String keyspace) {
|
public Map<List<String>, List<String>> getRangeToEndpointMap(String keyspace) {
|
||||||
log(" getRangeToEndpointMap(String keyspace)");
|
log(" getRangeToEndpointMap(String keyspace)");
|
||||||
return c.getMapListStrValue("");
|
return c.getMapListStrValue("/storage_service/range/" + keyspace);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -220,7 +222,10 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
public Map<List<String>, List<String>> getRangeToRpcaddressMap(
|
public Map<List<String>, List<String>> getRangeToRpcaddressMap(
|
||||||
String keyspace) {
|
String keyspace) {
|
||||||
log(" getRangeToRpcaddressMap(String keyspace)");
|
log(" getRangeToRpcaddressMap(String keyspace)");
|
||||||
return c.getMapListStrValue("");
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("rpc", "true");
|
||||||
|
return c.getMapListStrValue("/storage_service/range/" + keyspace,
|
||||||
|
queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -235,7 +240,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public List<String> describeRingJMX(String keyspace) throws IOException {
|
public List<String> describeRingJMX(String keyspace) throws IOException {
|
||||||
log(" describeRingJMX(String keyspace) throws IOException");
|
log(" describeRingJMX(String keyspace) throws IOException");
|
||||||
return c.getListStrValue("");
|
return c.getListStrValue("/storage_service/describe_ring/" + keyspace);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -249,7 +254,8 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
public Map<List<String>, List<String>> getPendingRangeToEndpointMap(
|
public Map<List<String>, List<String>> getPendingRangeToEndpointMap(
|
||||||
String keyspace) {
|
String keyspace) {
|
||||||
log(" getPendingRangeToEndpointMap(String keyspace)");
|
log(" getPendingRangeToEndpointMap(String keyspace)");
|
||||||
return c.getMapListStrValue("");
|
return c.getMapListStrValue("/storage_service/pending_range/"
|
||||||
|
+ keyspace);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -259,19 +265,19 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public Map<String, String> getTokenToEndpointMap() {
|
public Map<String, String> getTokenToEndpointMap() {
|
||||||
log(" getTokenToEndpointMap()");
|
log(" getTokenToEndpointMap()");
|
||||||
return c.getMapStrValue("");
|
return c.getMapStrValue("/storage_service/tokens_endpoint");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Retrieve this hosts unique ID */
|
/** Retrieve this hosts unique ID */
|
||||||
public String getLocalHostId() {
|
public String getLocalHostId() {
|
||||||
log(" getLocalHostId()");
|
log(" getLocalHostId()");
|
||||||
return c.getStringValue("");
|
return c.getStringValue("/storage_service/hostid/local");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Retrieve the mapping of endpoint to host ID */
|
/** Retrieve the mapping of endpoint to host ID */
|
||||||
public Map<String, String> getHostIdMap() {
|
public Map<String, String> getHostIdMap() {
|
||||||
log(" getHostIdMap()");
|
log(" getHostIdMap()");
|
||||||
return c.getMapStrValue("");
|
return c.getMapStrValue("/storage_service/host_id");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -282,19 +288,19 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
@Deprecated
|
@Deprecated
|
||||||
public double getLoad() {
|
public double getLoad() {
|
||||||
log(" getLoad()");
|
log(" getLoad()");
|
||||||
return c.getDoubleValue("");
|
return c.getDoubleValue("/storage_service/load");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Human-readable load value */
|
/** Human-readable load value */
|
||||||
public String getLoadString() {
|
public String getLoadString() {
|
||||||
log(" getLoadString()");
|
log(" getLoadString()");
|
||||||
return c.getStringValue("");
|
return String.valueOf(getLoad());
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Human-readable load value. Keys are IP addresses. */
|
/** Human-readable load value. Keys are IP addresses. */
|
||||||
public Map<String, String> getLoadMap() {
|
public Map<String, String> getLoadMap() {
|
||||||
log(" getLoadMap()");
|
log(" getLoadMap()");
|
||||||
return c.getMapStrValue("");
|
return c.getMapStrValue("/storage_service/load_map");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -304,7 +310,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public int getCurrentGenerationNumber() {
|
public int getCurrentGenerationNumber() {
|
||||||
log(" getCurrentGenerationNumber()");
|
log(" getCurrentGenerationNumber()");
|
||||||
return c.getIntValue("");
|
return c.getIntValue("/storage_service/generation_number");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -322,7 +328,11 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
public List<InetAddress> getNaturalEndpoints(String keyspaceName,
|
public List<InetAddress> getNaturalEndpoints(String keyspaceName,
|
||||||
String cf, String key) {
|
String cf, String key) {
|
||||||
log(" getNaturalEndpoints(String keyspaceName, String cf, String key)");
|
log(" getNaturalEndpoints(String keyspaceName, String cf, String key)");
|
||||||
return c.getListInetAddressValue("");
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("cf", cf);
|
||||||
|
queryParams.add("key", key);
|
||||||
|
return c.getListInetAddressValue("/storage_service/natural_endpoints/"
|
||||||
|
+ keyspaceName, queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<InetAddress> getNaturalEndpoints(String keyspaceName,
|
public List<InetAddress> getNaturalEndpoints(String keyspaceName,
|
||||||
@ -343,6 +353,11 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
public void takeSnapshot(String tag, String... keyspaceNames)
|
public void takeSnapshot(String tag, String... keyspaceNames)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
log(" takeSnapshot(String tag, String... keyspaceNames) throws IOException");
|
log(" takeSnapshot(String tag, String... keyspaceNames) throws IOException");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "tag", tag);
|
||||||
|
APIClient.set_query_param(queryParams, "kn",
|
||||||
|
APIClient.join(keyspaceNames));
|
||||||
|
c.post("/storage_service/snapshot", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -359,6 +374,17 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
public void takeColumnFamilySnapshot(String keyspaceName,
|
public void takeColumnFamilySnapshot(String keyspaceName,
|
||||||
String columnFamilyName, String tag) throws IOException {
|
String columnFamilyName, String tag) throws IOException {
|
||||||
log(" takeColumnFamilySnapshot(String keyspaceName, String columnFamilyName, String tag) throws IOException");
|
log(" takeColumnFamilySnapshot(String keyspaceName, String columnFamilyName, String tag) throws IOException");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
if (keyspaceName == null)
|
||||||
|
throw new IOException("You must supply a keyspace name");
|
||||||
|
if (columnFamilyName == null)
|
||||||
|
throw new IOException("You must supply a table name");
|
||||||
|
if (tag == null || tag.equals(""))
|
||||||
|
throw new IOException("You must supply a snapshot name.");
|
||||||
|
queryParams.add("tag", tag);
|
||||||
|
queryParams.add("kn", keyspaceName);
|
||||||
|
queryParams.add("cf", columnFamilyName);
|
||||||
|
c.post("/storage_service/snapshots", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -368,6 +394,11 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
public void clearSnapshot(String tag, String... keyspaceNames)
|
public void clearSnapshot(String tag, String... keyspaceNames)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
log(" clearSnapshot(String tag, String... keyspaceNames) throws IOException");
|
log(" clearSnapshot(String tag, String... keyspaceNames) throws IOException");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "tag", tag);
|
||||||
|
APIClient.set_query_param(queryParams, "kn",
|
||||||
|
APIClient.join(keyspaceNames));
|
||||||
|
c.delete("/storage_service/snapshots", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -377,7 +408,8 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public Map<String, TabularData> getSnapshotDetails() {
|
public Map<String, TabularData> getSnapshotDetails() {
|
||||||
log(" getSnapshotDetails()");
|
log(" getSnapshotDetails()");
|
||||||
return c.getMapStringTabularDataValue("");
|
return c.getMapStringSnapshotTabularDataValue(
|
||||||
|
"/storage_service/snapshots", null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -387,7 +419,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public long trueSnapshotsSize() {
|
public long trueSnapshotsSize() {
|
||||||
log(" trueSnapshotsSize()");
|
log(" trueSnapshotsSize()");
|
||||||
return c.getLongValue("");
|
return c.getLongValue("/storage_service/snapshots/size/true");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -397,6 +429,11 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
String... columnFamilies) throws IOException, ExecutionException,
|
String... columnFamilies) throws IOException, ExecutionException,
|
||||||
InterruptedException {
|
InterruptedException {
|
||||||
log(" forceKeyspaceCompaction(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException");
|
log(" forceKeyspaceCompaction(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "cf",
|
||||||
|
APIClient.join(columnFamilies));
|
||||||
|
c.post("/storage_service/keyspace_compaction/" + keyspaceName,
|
||||||
|
queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -406,7 +443,11 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
String... columnFamilies) throws IOException, ExecutionException,
|
String... columnFamilies) throws IOException, ExecutionException,
|
||||||
InterruptedException {
|
InterruptedException {
|
||||||
log(" forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException");
|
log(" forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException");
|
||||||
return c.getIntValue("");
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "cf",
|
||||||
|
APIClient.join(columnFamilies));
|
||||||
|
return c.getIntValue("/storage_service/keyspace_compaction/"
|
||||||
|
+ keyspaceName, queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -420,7 +461,14 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
String keyspaceName, String... columnFamilies) throws IOException,
|
String keyspaceName, String... columnFamilies) throws IOException,
|
||||||
ExecutionException, InterruptedException {
|
ExecutionException, InterruptedException {
|
||||||
log(" scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException");
|
log(" scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException");
|
||||||
return c.getIntValue("");
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_bool_query_param(queryParams, "disable_snapshot",
|
||||||
|
disableSnapshot);
|
||||||
|
APIClient.set_bool_query_param(queryParams, "skip_corrupted",
|
||||||
|
skipCorrupted);
|
||||||
|
APIClient.set_query_param(queryParams, "cf",
|
||||||
|
APIClient.join(columnFamilies));
|
||||||
|
return c.getIntValue("/storage_service/keyspace_scrub/" + keyspaceName);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -431,7 +479,13 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
boolean excludeCurrentVersion, String... columnFamilies)
|
boolean excludeCurrentVersion, String... columnFamilies)
|
||||||
throws IOException, ExecutionException, InterruptedException {
|
throws IOException, ExecutionException, InterruptedException {
|
||||||
log(" upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException");
|
log(" upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException");
|
||||||
return c.getIntValue("");
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_bool_query_param(queryParams, "exclude_current_version",
|
||||||
|
excludeCurrentVersion);
|
||||||
|
APIClient.set_query_param(queryParams, "cf",
|
||||||
|
APIClient.join(columnFamilies));
|
||||||
|
return c.getIntValue("/storage_service/keyspace_upgrade_sstables/"
|
||||||
|
+ keyspaceName);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -446,6 +500,10 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
String... columnFamilies) throws IOException, ExecutionException,
|
String... columnFamilies) throws IOException, ExecutionException,
|
||||||
InterruptedException {
|
InterruptedException {
|
||||||
log(" forceKeyspaceFlush(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException");
|
log(" forceKeyspaceFlush(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "cf",
|
||||||
|
APIClient.join(columnFamilies));
|
||||||
|
c.post("/storage_service/keyspace_flush/" + keyspaceName, queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -462,7 +520,16 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public int repairAsync(String keyspace, Map<String, String> options) {
|
public int repairAsync(String keyspace, Map<String, String> options) {
|
||||||
log(" repairAsync(String keyspace, Map<String, String> options)");
|
log(" repairAsync(String keyspace, Map<String, String> options)");
|
||||||
return c.getIntValue("");
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
String opts = "";
|
||||||
|
for (String op : options.keySet()) {
|
||||||
|
if (!opts.equals("")) {
|
||||||
|
opts = opts + ",";
|
||||||
|
}
|
||||||
|
opts = opts + op + "=" + options.get(op);
|
||||||
|
}
|
||||||
|
APIClient.set_query_param(queryParams, "options", opts);
|
||||||
|
return c.getIntValue("/storage_service/repair_async/" + keyspace);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Deprecated
|
@Deprecated
|
||||||
@ -519,6 +586,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
|
|
||||||
public void forceTerminateAllRepairSessions() {
|
public void forceTerminateAllRepairSessions() {
|
||||||
log(" forceTerminateAllRepairSessions()");
|
log(" forceTerminateAllRepairSessions()");
|
||||||
|
c.post("/storage_service/force_terminate");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -526,6 +594,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public void decommission() throws InterruptedException {
|
public void decommission() throws InterruptedException {
|
||||||
log(" decommission() throws InterruptedException");
|
log(" decommission() throws InterruptedException");
|
||||||
|
c.post("/storage_service/decommission");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -535,6 +604,9 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public void move(String newToken) throws IOException {
|
public void move(String newToken) throws IOException {
|
||||||
log(" move(String newToken) throws IOException");
|
log(" move(String newToken) throws IOException");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "new_token", newToken);
|
||||||
|
c.post("/storage_service/move");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -543,6 +615,9 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public void removeNode(String token) {
|
public void removeNode(String token) {
|
||||||
log(" removeNode(String token)");
|
log(" removeNode(String token)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "new_token", token);
|
||||||
|
c.post("/storage_service/remove_node", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -550,7 +625,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public String getRemovalStatus() {
|
public String getRemovalStatus() {
|
||||||
log(" getRemovalStatus()");
|
log(" getRemovalStatus()");
|
||||||
return c.getStringValue("");
|
return c.getStringValue("/storage_service/removal_status");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -558,6 +633,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public void forceRemoveCompletion() {
|
public void forceRemoveCompletion() {
|
||||||
log(" forceRemoveCompletion()");
|
log(" forceRemoveCompletion()");
|
||||||
|
c.post("/storage_service/force_remove_completion");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -582,12 +658,17 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
public void setLoggingLevel(String classQualifier, String level)
|
public void setLoggingLevel(String classQualifier, String level)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
log(" setLoggingLevel(String classQualifier, String level) throws Exception");
|
log(" setLoggingLevel(String classQualifier, String level) throws Exception");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "level", level);
|
||||||
|
APIClient.set_query_param(queryParams, "class_qualifier",
|
||||||
|
classQualifier);
|
||||||
|
c.post("/storage_service/logging_level", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** get the runtime logging levels */
|
/** get the runtime logging levels */
|
||||||
public Map<String, String> getLoggingLevels() {
|
public Map<String, String> getLoggingLevels() {
|
||||||
log(" getLoggingLevels()");
|
log(" getLoggingLevels()");
|
||||||
return c.getMapStrValue("");
|
return c.getMapStrValue("/storage_service/logging_level");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -596,19 +677,19 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
**/
|
**/
|
||||||
public String getOperationMode() {
|
public String getOperationMode() {
|
||||||
log(" getOperationMode()");
|
log(" getOperationMode()");
|
||||||
return c.getStringValue("");
|
return c.getStringValue("/storage_service/operation_mode");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns whether the storage service is starting or not */
|
/** Returns whether the storage service is starting or not */
|
||||||
public boolean isStarting() {
|
public boolean isStarting() {
|
||||||
log(" isStarting()");
|
log(" isStarting()");
|
||||||
return c.getBooleanValue("");
|
return c.getBooleanValue("/storage_service/is_starting");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** get the progress of a drain operation */
|
/** get the progress of a drain operation */
|
||||||
public String getDrainProgress() {
|
public String getDrainProgress() {
|
||||||
log(" getDrainProgress()");
|
log(" getDrainProgress()");
|
||||||
return c.getStringValue("");
|
return c.getStringValue("/storage_service/drain");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -618,6 +699,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
public void drain() throws IOException, InterruptedException,
|
public void drain() throws IOException, InterruptedException,
|
||||||
ExecutionException {
|
ExecutionException {
|
||||||
log(" drain() throws IOException, InterruptedException, ExecutionException");
|
log(" drain() throws IOException, InterruptedException, ExecutionException");
|
||||||
|
c.post("/storage_service/drain");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -635,6 +717,9 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
public void truncate(String keyspace, String columnFamily)
|
public void truncate(String keyspace, String columnFamily)
|
||||||
throws TimeoutException, IOException {
|
throws TimeoutException, IOException {
|
||||||
log(" truncate(String keyspace, String columnFamily)throws TimeoutException, IOException");
|
log(" truncate(String keyspace, String columnFamily)throws TimeoutException, IOException");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "cf", columnFamily);
|
||||||
|
c.post("/storage_service/truncate/" + keyspace, queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -643,6 +728,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public Map<InetAddress, Float> getOwnership() {
|
public Map<InetAddress, Float> getOwnership() {
|
||||||
log(" getOwnership()");
|
log(" getOwnership()");
|
||||||
|
// TBD
|
||||||
return c.getMapInetAddressFloatValue("");
|
return c.getMapInetAddressFloatValue("");
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -656,17 +742,20 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
public Map<InetAddress, Float> effectiveOwnership(String keyspace)
|
public Map<InetAddress, Float> effectiveOwnership(String keyspace)
|
||||||
throws IllegalStateException {
|
throws IllegalStateException {
|
||||||
log(" effectiveOwnership(String keyspace) throws IllegalStateException");
|
log(" effectiveOwnership(String keyspace) throws IllegalStateException");
|
||||||
|
// TBD
|
||||||
return c.getMapInetAddressFloatValue("");
|
return c.getMapInetAddressFloatValue("");
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<String> getKeyspaces() {
|
public List<String> getKeyspaces() {
|
||||||
log(" getKeyspaces()");
|
log(" getKeyspaces()");
|
||||||
return c.getListStrValue("");
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("non_system", "true");
|
||||||
|
return c.getListStrValue("/storage_service/keyspaces", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<String> getNonSystemKeyspaces() {
|
public List<String> getNonSystemKeyspaces() {
|
||||||
log(" getNonSystemKeyspaces()");
|
log(" getNonSystemKeyspaces()");
|
||||||
return c.getListStrValue("");
|
return c.getListStrValue("/storage_service/keyspaces");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -689,72 +778,97 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
Integer dynamicUpdateInterval, Integer dynamicResetInterval,
|
Integer dynamicUpdateInterval, Integer dynamicResetInterval,
|
||||||
Double dynamicBadnessThreshold) throws ClassNotFoundException {
|
Double dynamicBadnessThreshold) throws ClassNotFoundException {
|
||||||
log(" updateSnitch(String epSnitchClassName, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException");
|
log(" updateSnitch(String epSnitchClassName, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_bool_query_param(queryParams, "dynamic", dynamic);
|
||||||
|
APIClient.set_query_param(queryParams, "epSnitchClassName",
|
||||||
|
epSnitchClassName);
|
||||||
|
if (dynamicUpdateInterval != null) {
|
||||||
|
queryParams.add("dynamic_update_interval",
|
||||||
|
dynamicUpdateInterval.toString());
|
||||||
|
}
|
||||||
|
if (dynamicResetInterval != null) {
|
||||||
|
queryParams.add("dynamic_reset_interval",
|
||||||
|
dynamicResetInterval.toString());
|
||||||
|
}
|
||||||
|
if (dynamicBadnessThreshold != null) {
|
||||||
|
queryParams.add("dynamic_badness_threshold",
|
||||||
|
dynamicBadnessThreshold.toString());
|
||||||
|
}
|
||||||
|
c.post("/storage_service/update_snitch", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
// allows a user to forcibly 'kill' a sick node
|
// allows a user to forcibly 'kill' a sick node
|
||||||
public void stopGossiping() {
|
public void stopGossiping() {
|
||||||
log(" stopGossiping()");
|
log(" stopGossiping()");
|
||||||
|
c.delete("/storage_service/gossiping");
|
||||||
}
|
}
|
||||||
|
|
||||||
// allows a user to recover a forcibly 'killed' node
|
// allows a user to recover a forcibly 'killed' node
|
||||||
public void startGossiping() {
|
public void startGossiping() {
|
||||||
log(" startGossiping()");
|
log(" startGossiping()");
|
||||||
|
c.post("/storage_service/gossiping");
|
||||||
}
|
}
|
||||||
|
|
||||||
// allows a user to see whether gossip is running or not
|
// allows a user to see whether gossip is running or not
|
||||||
public boolean isGossipRunning() {
|
public boolean isGossipRunning() {
|
||||||
log(" isGossipRunning()");
|
log(" isGossipRunning()");
|
||||||
return c.getBooleanValue("");
|
return c.getBooleanValue("/storage_service/gossiping");
|
||||||
}
|
}
|
||||||
|
|
||||||
// allows a user to forcibly completely stop cassandra
|
// allows a user to forcibly completely stop cassandra
|
||||||
public void stopDaemon() {
|
public void stopDaemon() {
|
||||||
log(" stopDaemon()");
|
log(" stopDaemon()");
|
||||||
|
c.post("/storage_service/stop_daemon");
|
||||||
}
|
}
|
||||||
|
|
||||||
// to determine if gossip is disabled
|
// to determine if gossip is disabled
|
||||||
public boolean isInitialized() {
|
public boolean isInitialized() {
|
||||||
log(" isInitialized()");
|
log(" isInitialized()");
|
||||||
return c.getBooleanValue("");
|
return c.getBooleanValue("/storage_service/is_initialized");
|
||||||
}
|
}
|
||||||
|
|
||||||
// allows a user to disable thrift
|
// allows a user to disable thrift
|
||||||
public void stopRPCServer() {
|
public void stopRPCServer() {
|
||||||
log(" stopRPCServer()");
|
log(" stopRPCServer()");
|
||||||
|
c.delete("/storage_service/rpc_server");
|
||||||
}
|
}
|
||||||
|
|
||||||
// allows a user to reenable thrift
|
// allows a user to reenable thrift
|
||||||
public void startRPCServer() {
|
public void startRPCServer() {
|
||||||
log(" startRPCServer()");
|
log(" startRPCServer()");
|
||||||
|
c.post("/storage_service/rpc_server");
|
||||||
}
|
}
|
||||||
|
|
||||||
// to determine if thrift is running
|
// to determine if thrift is running
|
||||||
public boolean isRPCServerRunning() {
|
public boolean isRPCServerRunning() {
|
||||||
log(" isRPCServerRunning()");
|
log(" isRPCServerRunning()");
|
||||||
return c.getBooleanValue("");
|
return c.getBooleanValue("/storage_service/rpc_server");
|
||||||
}
|
}
|
||||||
|
|
||||||
public void stopNativeTransport() {
|
public void stopNativeTransport() {
|
||||||
log(" stopNativeTransport()");
|
log(" stopNativeTransport()");
|
||||||
|
c.delete("/storage_service/native_transport");
|
||||||
}
|
}
|
||||||
|
|
||||||
public void startNativeTransport() {
|
public void startNativeTransport() {
|
||||||
log(" startNativeTransport()");
|
log(" startNativeTransport()");
|
||||||
|
c.post("/storage_service/native_transport");
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isNativeTransportRunning() {
|
public boolean isNativeTransportRunning() {
|
||||||
log(" isNativeTransportRunning()");
|
log(" isNativeTransportRunning()");
|
||||||
return c.getBooleanValue("");
|
return c.getBooleanValue("/storage_service/native_transport");
|
||||||
}
|
}
|
||||||
|
|
||||||
// allows a node that have been started without joining the ring to join it
|
// allows a node that have been started without joining the ring to join it
|
||||||
public void joinRing() throws IOException {
|
public void joinRing() throws IOException {
|
||||||
log(" joinRing() throws IOException");
|
log(" joinRing() throws IOException");
|
||||||
|
c.post("/storage_service/join_ring");
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isJoined() {
|
public boolean isJoined() {
|
||||||
log(" isJoined()");
|
log(" isJoined()");
|
||||||
return c.getBooleanValue("");
|
return c.getBooleanValue("/storage_service/join_ring");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Deprecated
|
@Deprecated
|
||||||
@ -765,29 +879,39 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
|
|
||||||
public void setStreamThroughputMbPerSec(int value) {
|
public void setStreamThroughputMbPerSec(int value) {
|
||||||
log(" setStreamThroughputMbPerSec(int value)");
|
log(" setStreamThroughputMbPerSec(int value)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("value", Integer.toString(value));
|
||||||
|
c.post("/storage_service/stream_throughput", queryParams);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getStreamThroughputMbPerSec() {
|
public int getStreamThroughputMbPerSec() {
|
||||||
log(" getStreamThroughputMbPerSec()");
|
log(" getStreamThroughputMbPerSec()");
|
||||||
return c.getIntValue("");
|
return c.getIntValue("/storage_service/stream_throughput");
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getCompactionThroughputMbPerSec() {
|
public int getCompactionThroughputMbPerSec() {
|
||||||
log(" getCompactionThroughputMbPerSec()");
|
log(" getCompactionThroughputMbPerSec()");
|
||||||
return c.getIntValue("");
|
return c.getIntValue("/storage_service/compaction_throughput");
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setCompactionThroughputMbPerSec(int value) {
|
public void setCompactionThroughputMbPerSec(int value) {
|
||||||
log(" setCompactionThroughputMbPerSec(int value)");
|
log(" setCompactionThroughputMbPerSec(int value)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("value", Integer.toString(value));
|
||||||
|
c.post("/storage_service/compaction_throughput", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isIncrementalBackupsEnabled() {
|
public boolean isIncrementalBackupsEnabled() {
|
||||||
log(" isIncrementalBackupsEnabled()");
|
log(" isIncrementalBackupsEnabled()");
|
||||||
return c.getBooleanValue("");
|
return c.getBooleanValue("/storage_service/incremental_backups");
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setIncrementalBackupsEnabled(boolean value) {
|
public void setIncrementalBackupsEnabled(boolean value) {
|
||||||
log(" setIncrementalBackupsEnabled(boolean value)");
|
log(" setIncrementalBackupsEnabled(boolean value)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("value", Boolean.toString(value));
|
||||||
|
c.post("/storage_service/incremental_backups", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -802,11 +926,15 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public void rebuild(String sourceDc) {
|
public void rebuild(String sourceDc) {
|
||||||
log(" rebuild(String sourceDc)");
|
log(" rebuild(String sourceDc)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "value", sourceDc);
|
||||||
|
c.post("/storage_service/rebuild", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Starts a bulk load and blocks until it completes. */
|
/** Starts a bulk load and blocks until it completes. */
|
||||||
public void bulkLoad(String directory) {
|
public void bulkLoad(String directory) {
|
||||||
log(" bulkLoad(String directory)");
|
log(" bulkLoad(String directory)");
|
||||||
|
c.post("/storage_service/bulk_load/" + directory);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -815,11 +943,12 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public String bulkLoadAsync(String directory) {
|
public String bulkLoadAsync(String directory) {
|
||||||
log(" bulkLoadAsync(String directory)");
|
log(" bulkLoadAsync(String directory)");
|
||||||
return c.getStringValue("");
|
return c.getStringValue("/storage_service/bulk_load_async/" + directory);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void rescheduleFailedDeletions() {
|
public void rescheduleFailedDeletions() {
|
||||||
log(" rescheduleFailedDeletions()");
|
log(" rescheduleFailedDeletions()");
|
||||||
|
c.post("/storage_service/reschedule_failed_deletions");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -832,6 +961,9 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public void loadNewSSTables(String ksName, String cfName) {
|
public void loadNewSSTables(String ksName, String cfName) {
|
||||||
log(" loadNewSSTables(String ksName, String cfName)");
|
log(" loadNewSSTables(String ksName, String cfName)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("cf", cfName);
|
||||||
|
c.post("/storage_service/sstables/", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -846,7 +978,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public List<String> sampleKeyRange() {
|
public List<String> sampleKeyRange() {
|
||||||
log(" sampleKeyRange()");
|
log(" sampleKeyRange()");
|
||||||
return c.getListStrValue("");
|
return c.getListStrValue("/storage_service/sample_key_range");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -859,6 +991,7 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
|
|
||||||
public void resetLocalSchema() throws IOException {
|
public void resetLocalSchema() throws IOException {
|
||||||
log(" resetLocalSchema() throws IOException");
|
log(" resetLocalSchema() throws IOException");
|
||||||
|
c.post("/storage_service/relocal_schema");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -873,6 +1006,9 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public void setTraceProbability(double probability) {
|
public void setTraceProbability(double probability) {
|
||||||
log(" setTraceProbability(double probability)");
|
log(" setTraceProbability(double probability)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("probability", Double.toString(probability));
|
||||||
|
c.post("/storage_service/trace_probability", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -880,70 +1016,97 @@ public class StorageService extends NotificationBroadcasterSupport implements
|
|||||||
*/
|
*/
|
||||||
public double getTraceProbability() {
|
public double getTraceProbability() {
|
||||||
log(" getTraceProbability()");
|
log(" getTraceProbability()");
|
||||||
return c.getDoubleValue("");
|
return c.getDoubleValue("/storage_service/trace_probability");
|
||||||
}
|
}
|
||||||
|
|
||||||
public void disableAutoCompaction(String ks, String... columnFamilies)
|
public void disableAutoCompaction(String ks, String... columnFamilies)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
log("disableAutoCompaction(String ks, String... columnFamilies)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "cf",
|
||||||
|
APIClient.join(columnFamilies));
|
||||||
|
c.delete("/storage_service/auto_compaction/", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void enableAutoCompaction(String ks, String... columnFamilies)
|
public void enableAutoCompaction(String ks, String... columnFamilies)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
log("enableAutoCompaction(String ks, String... columnFamilies)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
APIClient.set_query_param(queryParams, "cf",
|
||||||
|
APIClient.join(columnFamilies));
|
||||||
|
c.post("/storage_service/auto_compaction/", queryParams);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void deliverHints(String host) throws UnknownHostException {
|
public void deliverHints(String host) throws UnknownHostException {
|
||||||
log(" deliverHints(String host) throws UnknownHostException");
|
log(" deliverHints(String host) throws UnknownHostException");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("host", host);
|
||||||
|
c.post("/storage_service/deliver_hints", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns the name of the cluster */
|
/** Returns the name of the cluster */
|
||||||
public String getClusterName() {
|
public String getClusterName() {
|
||||||
log(" getClusterName()");
|
log(" getClusterName()");
|
||||||
return c.getStringValue("");
|
return c.getStringValue("/storage_service/cluster_name");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns the cluster partitioner */
|
/** Returns the cluster partitioner */
|
||||||
public String getPartitionerName() {
|
public String getPartitionerName() {
|
||||||
log(" getPartitionerName()");
|
log(" getPartitionerName()");
|
||||||
return c.getStringValue("");
|
return c.getStringValue("/storage_service/partitioner_name");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns the threshold for warning of queries with many tombstones */
|
/** Returns the threshold for warning of queries with many tombstones */
|
||||||
public int getTombstoneWarnThreshold() {
|
public int getTombstoneWarnThreshold() {
|
||||||
log(" getTombstoneWarnThreshold()");
|
log(" getTombstoneWarnThreshold()");
|
||||||
return c.getIntValue("");
|
return c.getIntValue("/storage_service/tombstone_warn_threshold");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Sets the threshold for warning queries with many tombstones */
|
/** Sets the threshold for warning queries with many tombstones */
|
||||||
public void setTombstoneWarnThreshold(int tombstoneDebugThreshold) {
|
public void setTombstoneWarnThreshold(int tombstoneDebugThreshold) {
|
||||||
log(" setTombstoneWarnThreshold(int tombstoneDebugThreshold)");
|
log(" setTombstoneWarnThreshold(int tombstoneDebugThreshold)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("debug_threshold",
|
||||||
|
Integer.toString(tombstoneDebugThreshold));
|
||||||
|
c.post("/storage_service/tombstone_warn_threshold", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns the threshold for abandoning queries with many tombstones */
|
/** Returns the threshold for abandoning queries with many tombstones */
|
||||||
public int getTombstoneFailureThreshold() {
|
public int getTombstoneFailureThreshold() {
|
||||||
log(" getTombstoneFailureThreshold()");
|
log(" getTombstoneFailureThreshold()");
|
||||||
return c.getIntValue("");
|
return c.getIntValue("/storage_service/tombstone_failure_threshold");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Sets the threshold for abandoning queries with many tombstones */
|
/** Sets the threshold for abandoning queries with many tombstones */
|
||||||
public void setTombstoneFailureThreshold(int tombstoneDebugThreshold) {
|
public void setTombstoneFailureThreshold(int tombstoneDebugThreshold) {
|
||||||
log(" setTombstoneFailureThreshold(int tombstoneDebugThreshold)");
|
log(" setTombstoneFailureThreshold(int tombstoneDebugThreshold)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("debug_threshold",
|
||||||
|
Integer.toString(tombstoneDebugThreshold));
|
||||||
|
c.post("/storage_service/tombstone_failure_threshold", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns the threshold for rejecting queries due to a large batch size */
|
/** Returns the threshold for rejecting queries due to a large batch size */
|
||||||
public int getBatchSizeFailureThreshold() {
|
public int getBatchSizeFailureThreshold() {
|
||||||
log(" getBatchSizeFailureThreshold()");
|
log(" getBatchSizeFailureThreshold()");
|
||||||
return c.getIntValue("");
|
return c.getIntValue("/storage_service/batch_size_failure_threshold");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Sets the threshold for rejecting queries due to a large batch size */
|
/** Sets the threshold for rejecting queries due to a large batch size */
|
||||||
public void setBatchSizeFailureThreshold(int batchSizeDebugThreshold) {
|
public void setBatchSizeFailureThreshold(int batchSizeDebugThreshold) {
|
||||||
log(" setBatchSizeFailureThreshold(int batchSizeDebugThreshold)");
|
log(" setBatchSizeFailureThreshold(int batchSizeDebugThreshold)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("threshold", Integer.toString(batchSizeDebugThreshold));
|
||||||
|
c.post("/storage_service/batch_size_failure_threshold", queryParams);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Sets the hinted handoff throttle in kb per second, per delivery thread. */
|
/** Sets the hinted handoff throttle in kb per second, per delivery thread. */
|
||||||
public void setHintedHandoffThrottleInKB(int throttleInKB) {
|
public void setHintedHandoffThrottleInKB(int throttleInKB) {
|
||||||
log(" setHintedHandoffThrottleInKB(int throttleInKB)");
|
log(" setHintedHandoffThrottleInKB(int throttleInKB)");
|
||||||
|
MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
|
||||||
|
queryParams.add("throttle", Integer.toString(throttleInKB));
|
||||||
|
c.post("/storage_service/hinted_handoff", queryParams);
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user