Merge branch 'cassandra-2.0' into cassandra-2.1
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e8f2cb50 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e8f2cb50 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e8f2cb50 Branch: refs/heads/cassandra-3.0 Commit: e8f2cb50f76a530951efe97efe527bcf82df0a1b Parents: e9b975c c9587cd Author: Sam Tunnicliffe <[email protected]> Authored: Fri Aug 7 15:41:37 2015 +0100 Committer: Sam Tunnicliffe <[email protected]> Committed: Fri Aug 7 15:41:37 2015 +0100 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../JMXEnabledScheduledThreadPoolExecutor.java | 137 ------------------- ...EnabledScheduledThreadPoolExecutorMBean.java | 26 ---- .../cassandra/db/HintedHandOffManager.java | 29 +++- 4 files changed, 23 insertions(+), 170 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8f2cb50/CHANGES.txt ---------------------------------------------------------------------- diff --cc CHANGES.txt index 75bdcde,fe060af..781b60e --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,20 -1,5 +1,21 @@@ -2.0.17 +2.1.9 + * Release snapshot selfRef when doing snapshot repair (CASSANDRA-9998) + * Cannot replace token does not exist - DN node removed as Fat Client (CASSANDRA-9871) + * Fix handling of enable/disable autocompaction (CASSANDRA-9899) + * Commit log segment recycling is disabled by default (CASSANDRA-9896) + * Add consistency level to tracing ouput (CASSANDRA-9827) + * Fix MarshalException when upgrading superColumn family (CASSANDRA-9582) + * Fix broken logging for "empty" flushes in Memtable (CASSANDRA-9837) + * Handle corrupt files on startup (CASSANDRA-9686) + * Fix clientutil jar and tests (CASSANDRA-9760) + * (cqlsh) Allow the SSL protocol version to be specified through the + config file or environment variables (CASSANDRA-9544) + * Remove repair snapshot leftover on startup (CASSANDRA-7357) + * Use random nodes for batch log when only 2 racks (CASSANDRA-8735) + * Ensure atomicity inside thrift and stream session (CASSANDRA-7757) + * Fix nodetool info error when the node is not joined (CASSANDRA-9031) +Merged from 2.0: + * Remove erroneous pending HH tasks from tpstats/jmx (CASSANDRA-9129) * Don't cast expected bf size to an int (CASSANDRA-9959) * Log when messages are dropped due to cross_node_timeout (CASSANDRA-9793) * checkForEndpointCollision fails for legitimate collisions (CASSANDRA-9765) http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8f2cb50/src/java/org/apache/cassandra/db/HintedHandOffManager.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/HintedHandOffManager.java index 9821574,dab0f75..6f9d0b1 --- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java +++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java @@@ -37,16 -37,14 +37,19 @@@ import com.google.common.util.concurren import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.cassandra.concurrent.JMXEnabledScheduledThreadPoolExecutor; + import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor; + import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor; import org.apache.cassandra.concurrent.NamedThreadFactory; +import org.apache.cassandra.config.CFMetaData; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.Schema; import org.apache.cassandra.db.compaction.CompactionManager; - import org.apache.cassandra.db.composites.*; ++import org.apache.cassandra.db.composites.CellName; ++import org.apache.cassandra.db.composites.Composite; ++import org.apache.cassandra.db.composites.Composites; import org.apache.cassandra.db.filter.*; -import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.UUIDType; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; @@@ -59,8 -57,12 +62,10 @@@ import org.apache.cassandra.io.sstable. import org.apache.cassandra.metrics.HintedHandoffMetrics; import org.apache.cassandra.net.MessageOut; import org.apache.cassandra.net.MessagingService; - import org.apache.cassandra.service.*; + import org.apache.cassandra.service.StorageProxy; + import org.apache.cassandra.service.StorageService; + import org.apache.cassandra.service.WriteResponseHandler; -import org.apache.cassandra.utils.ByteBufferUtil; -import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.UUIDGen; +import org.apache.cassandra.utils.*; import org.cliffc.high_scale_lib.NonBlockingHashSet; /** @@@ -98,13 -100,25 +103,24 @@@ public class HintedHandOffManager imple private volatile boolean hintedHandOffPaused = false; - static final CompositeType comparator = CompositeType.getInstance(Arrays.<AbstractType<?>>asList(UUIDType.instance, Int32Type.instance)); static final int maxHintTTL = Integer.parseInt(System.getProperty("cassandra.maxHintTTL", String.valueOf(Integer.MAX_VALUE))); - private final NonBlockingHashSet<InetAddress> queuedDeliveries = new NonBlockingHashSet<InetAddress>(); + private final NonBlockingHashSet<InetAddress> queuedDeliveries = new NonBlockingHashSet<>(); - private final JMXEnabledScheduledThreadPoolExecutor executor = - new JMXEnabledScheduledThreadPoolExecutor( + // To keep metrics consistent with earlier versions, where periodic tasks were run on a shared executor, + // we run them on this executor and so keep counts separate from those for hint delivery tasks. See CASSANDRA-9129 + private final DebuggableScheduledThreadPoolExecutor executor = + new DebuggableScheduledThreadPoolExecutor(1, new NamedThreadFactory("HintedHandoffManager", Thread.MIN_PRIORITY)); + + // Non-scheduled executor to run the actual hint delivery tasks. + // Per CASSANDRA-9129, this is where the values displayed in nodetool tpstats + // and via the HintedHandoff mbean are obtained. + private final ThreadPoolExecutor hintDeliveryExecutor = + new JMXEnabledThreadPoolExecutor( DatabaseDescriptor.getMaxHintsThread(), + Integer.MAX_VALUE, + TimeUnit.SECONDS, + new LinkedBlockingQueue<Runnable>(), new NamedThreadFactory("HintedHandoff", Thread.MIN_PRIORITY), "internal");
