Merge branch 'cassandra-2.1' into cassandra-2.2
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/dee675f1 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/dee675f1 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/dee675f1 Branch: refs/heads/trunk Commit: dee675f1ef148b40351c365b6d42c39f081cb706 Parents: 8b021db 9966419 Author: Yuki Morishita <[email protected]> Authored: Wed Jun 17 20:48:40 2015 -0500 Committer: Yuki Morishita <[email protected]> Committed: Wed Jun 17 20:48:40 2015 -0500 ---------------------------------------------------------------------- CHANGES.txt | 3 +- .../cassandra/service/StorageService.java | 48 +++++++++++++------- 2 files changed, 34 insertions(+), 17 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/dee675f1/CHANGES.txt ---------------------------------------------------------------------- diff --cc CHANGES.txt index c32596c,1d72c9a..3b16b6f --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,41 -1,13 +1,42 @@@ -2.1.7 +2.2 + * Fix connection leak in CqlRecordWriter (CASSANDRA-9576) + * Mlockall before opening system sstables & remove boot_without_jna option (CASSANDRA-9573) + * Add functions to convert timeuuid to date or time, deprecate dateOf and unixTimestampOf (CASSANDRA-9229) + * Make sure we cancel non-compacting sstables from LifecycleTransaction (CASSANDRA-9566) + * Fix deprecated repair JMX API (CASSANDRA-9570) ++ * Add logback metrics (CASSANDRA-9378) +Merged from 2.1: * Fix memory leak in Ref due to ConcurrentLinkedQueue.remove() behaviour (CASSANDRA-9549) ++ * Make rebuild only run one at a time (CASSANDRA-9119) Merged from 2.0 * ArrivalWindow should use primitives (CASSANDRA-9496) * Periodically submit background compaction tasks (CASSANDRA-9592) * Set HAS_MORE_PAGES flag to false when PagingState is null (CASSANDRA-9571) - * Add logback metrics (CASSANDRA-9378) - * Make rebuild only run one at a time (CASSANDRA-9119) -2.1.6 +2.2.0-rc1 + * Compressed commit log should measure compressed space used (CASSANDRA-9095) + * Fix comparison bug in CassandraRoleManager#collectRoles (CASSANDRA-9551) + * Add tinyint,smallint,time,date support for UDFs (CASSANDRA-9400) + * Deprecates SSTableSimpleWriter and SSTableSimpleUnsortedWriter (CASSANDRA-9546) + * Empty INITCOND treated as null in aggregate (CASSANDRA-9457) + * Remove use of Cell in Thrift MapReduce classes (CASSANDRA-8609) + * Integrate pre-release Java Driver 2.2-rc1, custom build (CASSANDRA-9493) + * Clean up gossiper logic for old versions (CASSANDRA-9370) + * Fix custom payload coding/decoding to match the spec (CASSANDRA-9515) + * ant test-all results incomplete when parsed (CASSANDRA-9463) + * Disallow frozen<> types in function arguments and return types for + clarity (CASSANDRA-9411) + * Static Analysis to warn on unsafe use of Autocloseable instances (CASSANDRA-9431) + * Update commitlog archiving examples now that commitlog segments are + not recycled (CASSANDRA-9350) + * Extend Transactional API to sstable lifecycle management (CASSANDRA-8568) + * (cqlsh) Add support for native protocol 4 (CASSANDRA-9399) + * Ensure that UDF and UDAs are keyspace-isolated (CASSANDRA-9409) + * Revert CASSANDRA-7807 (tracing completion client notifications) (CASSANDRA-9429) + * Add ability to stop compaction by ID (CASSANDRA-7207) + * Let CassandraVersion handle SNAPSHOT version (CASSANDRA-9438) +Merged from 2.1: * (cqlsh) Fix using COPY through SOURCE or -f (CASSANDRA-9083) * Fix occasional lack of `system` keyspace in schema tables (CASSANDRA-8487) * Use ProtocolError code instead of ServerError code for native protocol http://git-wip-us.apache.org/repos/asf/cassandra/blob/dee675f1/src/java/org/apache/cassandra/service/StorageService.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/service/StorageService.java index 2dd56b5,e063c63..3edbe22 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@@ -50,7 -50,9 +50,8 @@@ import java.util.concurrent.Future import java.util.concurrent.FutureTask; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; + import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import javax.management.JMX; import javax.management.MBeanServer; @@@ -239,11 -235,15 +240,13 @@@ public class StorageService extends Not private InetAddress removingNode; /* Are we starting this node in bootstrap mode? */ - private boolean isBootstrapMode; + private volatile boolean isBootstrapMode; /* we bootstrap but do NOT join the ring unless told to do so */ - private boolean isSurveyMode= Boolean.parseBoolean(System.getProperty("cassandra.write_survey", "false")); + private boolean isSurveyMode = Boolean.parseBoolean(System.getProperty("cassandra.write_survey", "false")); + /* true if node is rebuilding and receiving data */ + private final AtomicBoolean isRebuilding = new AtomicBoolean(); - /* when intialized as a client, we shouldn't write to the system keyspace. */ - private boolean isClientMode; private boolean initialized; private volatile boolean joined = false; @@@ -1071,7 -1036,17 +1066,23 @@@ try { - streamer.fetchAsync().get(); - RangeStreamer streamer = new RangeStreamer(tokenMetadata, FBUtilities.getBroadcastAddress(), "Rebuild"); ++ RangeStreamer streamer = new RangeStreamer(tokenMetadata, ++ null, ++ FBUtilities.getBroadcastAddress(), ++ "Rebuild", ++ !replacing && useStrictConsistency, ++ DatabaseDescriptor.getEndpointSnitch(), ++ streamStateStore); + streamer.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(FailureDetector.instance)); + if (sourceDc != null) + streamer.addSourceFilter(new RangeStreamer.SingleDatacenterFilter(DatabaseDescriptor.getEndpointSnitch(), sourceDc)); + + for (String keyspaceName : Schema.instance.getNonSystemKeyspaces()) + streamer.addRanges(keyspaceName, getLocalRanges(keyspaceName)); + + StreamResultFuture resultFuture = streamer.fetchAsync(); + // wait for result + resultFuture.get(); } catch (InterruptedException e) {
