Merge branch 'cassandra-1.2' into cassandra-2.0 Conflicts: src/java/org/apache/cassandra/tools/NodeProbe.java
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ecee3955 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ecee3955 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ecee3955 Branch: refs/heads/trunk Commit: ecee39551c16e2edb203e586215cdf17400fbbac Parents: cce6863 fbe19b8 Author: Yuki Morishita <yu...@apache.org> Authored: Wed Oct 9 10:36:42 2013 -0500 Committer: Yuki Morishita <yu...@apache.org> Committed: Wed Oct 9 10:36:42 2013 -0500 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../org/apache/cassandra/tools/NodeProbe.java | 57 ++++++++++++++------ 2 files changed, 42 insertions(+), 16 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ecee3955/CHANGES.txt ---------------------------------------------------------------------- diff --cc CHANGES.txt index 53de29e,f0ee993..64febd8 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -44,43 -19,10 +44,44 @@@ Merged from 1.2 * Update sstablesPerReadHistogram to use biased sampling (CASSANDRA-6164) * Log UnknownColumnfamilyException when closing socket (CASSANDRA-5725) * Properly error out on CREATE INDEX for counters table (CASSANDRA-6160) + * Handle JMX notification failure for repair (CASSANDRA-6097) -1.2.10 +2.0.1 + * Fix bug that could allow reading deleted data temporarily (CASSANDRA-6025) + * Improve memory use defaults (CASSANDRA-5069) + * Make ThriftServer more easlly extensible (CASSANDRA-6058) + * Remove Hadoop dependency from ITransportFactory (CASSANDRA-6062) + * add file_cache_size_in_mb setting (CASSANDRA-5661) + * Improve error message when yaml contains invalid properties (CASSANDRA-5958) + * Improve leveled compaction's ability to find non-overlapping L0 compactions + to work on concurrently (CASSANDRA-5921) + * Notify indexer of columns shadowed by range tombstones (CASSANDRA-5614) + * Log Merkle tree stats (CASSANDRA-2698) + * Switch from crc32 to adler32 for compressed sstable checksums (CASSANDRA-5862) + * Improve offheap memcpy performance (CASSANDRA-5884) + * Use a range aware scanner for cleanup (CASSANDRA-2524) + * Cleanup doesn't need to inspect sstables that contain only local data + (CASSANDRA-5722) + * Add ability for CQL3 to list partition keys (CASSANDRA-4536) + * Improve native protocol serialization (CASSANDRA-5664) + * Upgrade Thrift to 0.9.1 (CASSANDRA-5923) + * Require superuser status for adding triggers (CASSANDRA-5963) + * Make standalone scrubber handle old and new style leveled manifest + (CASSANDRA-6005) + * Fix paxos bugs (CASSANDRA-6012, 6013, 6023) + * Fix paged ranges with multiple replicas (CASSANDRA-6004) + * Fix potential AssertionError during tracing (CASSANDRA-6041) + * Fix NPE in sstablesplit (CASSANDRA-6027) + * Migrate pre-2.0 key/value/column aliases to system.schema_columns + (CASSANDRA-6009) + * Paging filter empty rows too agressively (CASSANDRA-6040) + * Support variadic parameters for IN clauses (CASSANDRA-4210) + * cqlsh: return the result of CAS writes (CASSANDRA-5796) + * Fix validation of IN clauses with 2ndary indexes (CASSANDRA-6050) + * Support named bind variables in CQL (CASSANDRA-6033) +Merged from 1.2: + * Allow cache-keys-to-save to be set at runtime (CASSANDRA-5980) * Avoid second-guessing out-of-space state (CASSANDRA-5605) * Tuning knobs for dealing with large blobs and many CFs (CASSANDRA-5982) * (Hadoop) Fix CQLRW for thrift tables (CASSANDRA-6002) http://git-wip-us.apache.org/repos/asf/cassandra/blob/ecee3955/src/java/org/apache/cassandra/tools/NodeProbe.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java index f05cfb7,1557a6a..0d58979 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@@ -32,7 -32,7 +32,8 @@@ import java.util.concurrent.ExecutionEx import java.util.concurrent.TimeoutException; import java.util.concurrent.locks.Condition; import javax.management.*; +import javax.management.openmbean.CompositeData; + import javax.management.remote.JMXConnectionNotification; import javax.management.remote.JMXConnector; import javax.management.remote.JMXConnectorFactory; import javax.management.remote.JMXServiceURL; @@@ -184,41 -183,42 +185,42 @@@ public class NodeProb jmxc.close(); } - public void forceTableCleanup(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException + public void forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException { - ssProxy.forceTableCleanup(tableName, columnFamilies); + ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies); } - public void scrub(boolean disableSnapshot, String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException + public void scrub(boolean disableSnapshot, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException { - ssProxy.scrub(disableSnapshot, tableName, columnFamilies); + ssProxy.scrub(disableSnapshot, keyspaceName, columnFamilies); } - public void upgradeSSTables(String tableName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException + public void upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException { - ssProxy.upgradeSSTables(tableName, excludeCurrentVersion, columnFamilies); + ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies); } - public void forceTableCompaction(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException + public void forceKeyspaceCompaction(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException { - ssProxy.forceTableCompaction(tableName, columnFamilies); + ssProxy.forceKeyspaceCompaction(keyspaceName, columnFamilies); } - public void forceTableFlush(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException + public void forceKeyspaceFlush(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException { - ssProxy.forceTableFlush(tableName, columnFamilies); + ssProxy.forceKeyspaceFlush(keyspaceName, columnFamilies); } - public void forceTableRepair(String tableName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException + public void forceKeyspaceRepair(String keyspaceName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException { - ssProxy.forceTableRepair(tableName, isSequential, isLocal, columnFamilies); + ssProxy.forceKeyspaceRepair(keyspaceName, isSequential, isLocal, columnFamilies); } - public void forceRepairAsync(final PrintStream out, final String tableName, boolean isSequential, boolean isLocal, boolean primaryRange, String... columnFamilies) throws IOException + public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean isSequential, boolean isLocal, boolean primaryRange, String... columnFamilies) throws IOException { - RepairRunner runner = new RepairRunner(out, tableName, columnFamilies); + RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies); try { + jmxc.addConnectionNotificationListener(runner, null, null); ssProxy.addNotificationListener(runner, null, null); if (!runner.repairAndWait(ssProxy, isSequential, isLocal, primaryRange)) failed = true; @@@ -237,11 -238,12 +240,12 @@@ } } - public void forceRepairRangeAsync(final PrintStream out, final String tableName, boolean isSequential, boolean isLocal, final String startToken, final String endToken, String... columnFamilies) throws IOException + public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, boolean isSequential, boolean isLocal, final String startToken, final String endToken, String... columnFamilies) throws IOException { - RepairRunner runner = new RepairRunner(out, tableName, columnFamilies); + RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies); try { + jmxc.addConnectionNotificationListener(runner, null, null); ssProxy.addNotificationListener(runner, null, null); if (!runner.repairRangeAndWait(ssProxy, isSequential, isLocal, startToken, endToken)) failed = true;