You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yu...@apache.org on 2016/12/14 00:45:40 UTC

[01/19] cassandra git commit: Temporarily fix bug that creates commit log when running offline tools

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 fb2940050 -> 66f1aaf88
  refs/heads/cassandra-3.0 e9b7a0f25 -> 0fe82be83
  refs/heads/cassandra-3.11 1cafc3ccd -> 66e214592
  refs/heads/cassandra-3.X ddefb1d89 -> c9ead4826
  refs/heads/trunk cd5aacc83 -> 402f8e006


Temporarily fix bug that creates commit log when running offline tools

patch by yukim; reviewed by thobbs for CASSANDRA-8616


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/66f1aaf8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/66f1aaf8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/66f1aaf8

Branch: refs/heads/cassandra-2.2
Commit: 66f1aaf88d3cde5c52b13d71d3326da5eda16fb1
Parents: fb29400
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Feb 11 19:06:27 2016 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:51:37 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/config/DatabaseDescriptor.java    | 12 +++++++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++++--
 .../cassandra/db/compaction/CompactionTask.java | 22 +++++++-----
 .../apache/cassandra/db/lifecycle/Tracker.java  | 36 ++++++++++++--------
 .../io/sstable/format/SSTableReader.java        |  8 +++--
 .../cassandra/service/CassandraDaemon.java      |  1 +
 .../service/EmbeddedCassandraService.java       |  2 ++
 .../io/sstable/CQLSSTableWriterLongTest.java    |  2 ++
 .../unit/org/apache/cassandra/SchemaLoader.java |  1 +
 .../config/DatabaseDescriptorTest.java          |  6 ++++
 .../apache/cassandra/db/CounterCellTest.java    |  3 ++
 .../org/apache/cassandra/db/NativeCellTest.java |  3 ++
 .../apache/cassandra/db/SystemKeyspaceTest.java |  2 ++
 .../db/context/CounterContextTest.java          |  8 +++++
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 ++-
 .../cassandra/db/lifecycle/TrackerTest.java     |  7 ++--
 .../cassandra/dht/StreamStateStoreTest.java     |  7 ++++
 .../cassandra/gms/FailureDetectorTest.java      |  2 ++
 .../org/apache/cassandra/gms/GossiperTest.java  |  5 +++
 .../io/sstable/CQLSSTableWriterTest.java        |  2 ++
 .../cassandra/locator/CloudstackSnitchTest.java |  2 ++
 .../apache/cassandra/locator/EC2SnitchTest.java |  2 ++
 .../locator/GoogleCloudSnitchTest.java          |  2 ++
 .../service/StorageServiceServerTest.java       |  1 +
 25 files changed, 118 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f02350d..8cff097 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.9
+ * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
  * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
  * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
  * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index a3fb79b..35debd0 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -102,6 +102,18 @@ public class DatabaseDescriptor
     private static Comparator<InetAddress> localComparator;
     private static boolean hasLoggedConfig;
 
+    private static boolean daemonInitialized;
+
+    public static boolean isDaemonInitialized()
+    {
+        return daemonInitialized;
+    }
+
+    public static void setDaemonInitialized()
+    {
+        daemonInitialized = true;
+    }
+
     public static void forceStaticInitialization() {}
     static
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c6b69dc..4bc46d0 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -388,9 +388,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         logger.info("Initializing {}.{}", keyspace.getName(), name);
 
-        // scan for sstables corresponding to this cf and load them
-        data = new Tracker(this, loadSSTables);
+        // Create Memtable only on online
+        Memtable initialMemtable = null;
+        if (DatabaseDescriptor.isDaemonInitialized())
+            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
+        data = new Tracker(initialMemtable, loadSSTables);
 
+        // scan for sstables corresponding to this cf and load them
         if (data.loadsstables)
         {
             Directories.SSTableLister sstableFiles = directories.sstableLister().skipTemporary(true);
@@ -2754,7 +2758,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             {
                 public Void call()
                 {
-                    cfs.data.reset();
+                    cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
                     cfs.getCompactionStrategy().shutdown();
                     cfs.getCompactionStrategy().startup();
                     return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 575c326..20d3dc0 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -127,7 +127,7 @@ public class CompactionTask extends AbstractCompactionTask
             }
         });
 
-        UUID taskId = SystemKeyspace.startCompaction(cfs, transaction.originals());
+        UUID taskId = offline ? null : SystemKeyspace.startCompaction(cfs, transaction.originals());
 
         // new sstables from flush can be added during a compaction, but only the compaction can remove them,
         // so in our single-threaded compaction world this is a valid way of determining if we're compacting
@@ -218,16 +218,20 @@ public class CompactionTask extends AbstractCompactionTask
             for (SSTableReader reader : newSStables)
                 newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
 
-            double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
-            long totalSourceRows = 0;
-            String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
-            logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                      taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
-            logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-            logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
-
             if (offline)
+            {
                 Refs.release(Refs.selfRefs(newSStables));
+            }
+            else
+            {
+                double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
+                long totalSourceRows = 0;
+                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
+                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
+                                           taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
+                logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
+                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten));
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index 5d5701f..e77ef78 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -31,7 +31,6 @@ import com.google.common.collect.*;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Memtable;
-import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,6 +51,7 @@ import static com.google.common.base.Predicates.not;
 import static com.google.common.collect.ImmutableSet.copyOf;
 import static com.google.common.collect.Iterables.filter;
 import static java.util.Collections.singleton;
+import static java.util.Collections.singletonList;
 import static org.apache.cassandra.db.lifecycle.Helpers.*;
 import static org.apache.cassandra.db.lifecycle.View.permitCompacting;
 import static org.apache.cassandra.db.lifecycle.View.updateCompacting;
@@ -61,21 +61,29 @@ import static org.apache.cassandra.utils.Throwables.merge;
 import static org.apache.cassandra.utils.concurrent.Refs.release;
 import static org.apache.cassandra.utils.concurrent.Refs.selfRefs;
 
+/**
+ * Tracker tracks live {@link View} of data store for a table.
+ */
 public class Tracker
 {
     private static final Logger logger = LoggerFactory.getLogger(Tracker.class);
 
-    public final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+    private final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+
     public final ColumnFamilyStore cfstore;
     final AtomicReference<View> view;
     public final boolean loadsstables;
 
-    public Tracker(ColumnFamilyStore cfstore, boolean loadsstables)
+    /**
+     * @param memtable Initial Memtable. Can be null.
+     * @param loadsstables true to indicate to load SSTables (TODO: remove as this is only accessed from 2i)
+     */
+    public Tracker(Memtable memtable, boolean loadsstables)
     {
-        this.cfstore = cfstore;
+        this.cfstore = memtable != null ? memtable.cfs : null;
         this.view = new AtomicReference<>();
         this.loadsstables = loadsstables;
-        this.reset();
+        this.reset(memtable);
     }
 
     public LifecycleTransaction tryModify(SSTableReader sstable, OperationType operationType)
@@ -196,16 +204,14 @@ public class Tracker
 
     /** (Re)initializes the tracker, purging all references. */
     @VisibleForTesting
-    public void reset()
+    public void reset(Memtable memtable)
     {
-        view.set(new View(
-                         !isDummy() ? ImmutableList.of(new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfstore))
-                                    : ImmutableList.<Memtable>of(),
-                         ImmutableList.<Memtable>of(),
-                         Collections.<SSTableReader, SSTableReader>emptyMap(),
-                         Collections.<SSTableReader>emptySet(),
-                         Collections.<SSTableReader>emptySet(),
-                         SSTableIntervalTree.empty()));
+        view.set(new View(memtable != null ? singletonList(memtable) : Collections.<Memtable>emptyList(),
+                          Collections.<Memtable>emptyList(),
+                          Collections.<SSTableReader, SSTableReader>emptyMap(),
+                          Collections.<SSTableReader>emptySet(),
+                          Collections.<SSTableReader>emptySet(),
+                          SSTableIntervalTree.empty()));
     }
 
     public Throwable dropSSTablesIfInvalid(Throwable accumulate)
@@ -473,7 +479,7 @@ public class Tracker
 
     public boolean isDummy()
     {
-        return cfstore == null;
+        return cfstore == null || !DatabaseDescriptor.isDaemonInitialized();
     }
 
     public void subscribe(INotificationConsumer consumer)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index fddf058..ba060d4 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -2246,7 +2246,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
             // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist
             // the read meter when in client mode.
-            if (SystemKeyspace.NAME.equals(desc.ksname))
+            if (SystemKeyspace.NAME.equals(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
             {
                 readMeter = null;
                 readMeterSyncFuture = null;
@@ -2272,9 +2272,11 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         {
             lookup.remove(desc);
             if (readMeterSyncFuture != null)
+            {
                 readMeterSyncFuture.cancel(true);
-            if (isCompacted.get())
-                SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+                if (isCompacted.get())
+                    SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+            }
             // don't ideally want to dropPageCache for the file until all instances have been released
             CLibrary.trySkipCache(desc.filenameFor(Component.DATA), 0, 0);
             CLibrary.trySkipCache(desc.filenameFor(Component.PRIMARY_INDEX), 0, 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index cf5f3c2..35e990f 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -505,6 +505,7 @@ public class CassandraDaemon
             try
             {
                 DatabaseDescriptor.forceStaticInitialization();
+                DatabaseDescriptor.setDaemonInitialized();
             }
             catch (ExceptionInInitializerError e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
index 659d851..6c154cd 100644
--- a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
+++ b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.service;
 
 import java.io.IOException;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.service.CassandraDaemon;
 
 /**
@@ -49,6 +50,7 @@ public class EmbeddedCassandraService
     public void start() throws IOException
     {
         cassandraDaemon = new CassandraDaemon();
+        DatabaseDescriptor.setDaemonInitialized();
         cassandraDaemon.init(null);
         cassandraDaemon.start();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
index fcec40d..ee719d1 100644
--- a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
+++ b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
@@ -30,6 +30,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.service.StorageService;
@@ -39,6 +40,7 @@ public class CQLSSTableWriterLongTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index a7cf7b4..2048f74 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -90,6 +90,7 @@ public class SchemaLoader
             }
         });
 
+        DatabaseDescriptor.setDaemonInitialized();
         Keyspace.setInitialized();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 5f231c3..7409535 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -47,6 +47,12 @@ import static org.junit.Assert.assertNull;
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class DatabaseDescriptorTest
 {
+    @BeforeClass
+    public static void setupDatabaseDescriptor()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testCFMetaDataSerialization() throws ConfigurationException, InvalidRequestException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/CounterCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
index 8d75b9a..5d4b8a8 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -30,6 +30,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
 import org.apache.cassandra.db.context.CounterContext;
@@ -57,6 +58,8 @@ public class CounterCellTest
         countLength   = 8; // size of long
 
         stepLength    = idLength + clockLength + countLength;
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/NativeCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NativeCellTest.java b/test/unit/org/apache/cassandra/db/NativeCellTest.java
index 70b7b87..4145a91 100644
--- a/test/unit/org/apache/cassandra/db/NativeCellTest.java
+++ b/test/unit/org/apache/cassandra/db/NativeCellTest.java
@@ -32,6 +32,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -128,6 +129,8 @@ public class NativeCellTest
         {
             throw new AssertionError();
         }
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index b8aa161..c3047b8 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -42,6 +42,8 @@ public class SystemKeyspaceTest
     @BeforeClass
     public static void prepSnapshotTracker()
     {
+        DatabaseDescriptor.setDaemonInitialized();
+
         if (FBUtilities.isWindows())
             WindowsFailedSnapshotTracker.deleteOldSnapshots();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
index a72d30d..4f587c6 100644
--- a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
+++ b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
@@ -22,9 +22,11 @@ package org.apache.cassandra.db.context;
 
 import java.nio.ByteBuffer;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ClockAndCount;
 import org.apache.cassandra.db.context.CounterContext.Relationship;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -48,6 +50,12 @@ public class CounterContextTest
     private static final int countLength = 8;
     private static final int stepLength = idLength + clockLength + countLength;
 
+    @BeforeClass
+    public static void setupDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testAllocate()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
index f13d1b7..737392e 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.lifecycle;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.junit.After;
 import org.junit.Before;
@@ -30,6 +31,8 @@ import junit.framework.Assert;
 import org.apache.cassandra.MockSchema;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
@@ -270,7 +273,7 @@ public class LifecycleTransactionTest extends AbstractTransactionalTest
 
         private static Tracker tracker(ColumnFamilyStore cfs, List<SSTableReader> readers)
         {
-            Tracker tracker = new Tracker(cfs, false);
+            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs), false);
             tracker.addInitialSSTables(readers);
             return tracker;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
index adeb778..04b4e4a 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -75,6 +75,7 @@ public class TrackerTest
     @BeforeClass
     public static void setUp()
     {
+        DatabaseDescriptor.setDaemonInitialized();
         MockSchema.cleanup();
     }
 
@@ -82,7 +83,7 @@ public class TrackerTest
     public void testTryModify()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = new Tracker(null, false);
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, true, cfs), MockSchema.sstable(1, cfs), MockSchema.sstable(2, cfs));
         tracker.addInitialSSTables(copyOf(readers));
         Assert.assertNull(tracker.tryModify(ImmutableList.of(MockSchema.sstable(0, cfs)), OperationType.COMPACTION));
@@ -146,7 +147,7 @@ public class TrackerTest
     public void testAddInitialSSTables()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = cfs.getTracker();
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),
                                                        MockSchema.sstable(1, 121, cfs),
                                                        MockSchema.sstable(2, 9, cfs));
@@ -163,7 +164,7 @@ public class TrackerTest
         boolean backups = DatabaseDescriptor.isIncrementalBackupsEnabled();
         DatabaseDescriptor.setIncrementalBackupsEnabled(false);
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = cfs.getTracker();
         MockListener listener = new MockListener(false);
         tracker.subscribe(listener);
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
index 86781d9..bdb654a 100644
--- a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
+++ b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
@@ -20,8 +20,10 @@ package org.apache.cassandra.dht;
 import java.net.InetAddress;
 import java.util.Collections;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.streaming.DefaultConnectionFactory;
 import org.apache.cassandra.streaming.StreamEvent;
 import org.apache.cassandra.streaming.StreamSession;
@@ -32,6 +34,11 @@ import static org.junit.Assert.assertTrue;
 
 public class StreamStateStoreTest
 {
+    @BeforeClass
+    public static void initDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
 
     @Test
     public void testUpdateAndQueryAvailableRanges()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
index 9325922..af099b0 100644
--- a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
+++ b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
@@ -45,6 +45,8 @@ public class FailureDetectorTest
     {
         // slow unit tests can cause problems with FailureDetector's GC pause handling
         System.setProperty("cassandra.max_local_pause_in_ms", "20000");
+
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/gms/GossiperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java
index ad07165..eb01305 100644
--- a/test/unit/org/apache/cassandra/gms/GossiperTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java
@@ -33,6 +33,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Token;
@@ -44,6 +45,10 @@ import static org.junit.Assert.assertEquals;
 
 public class GossiperTest
 {
+    static
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
     static final IPartitioner partitioner = new RandomPartitioner();
     StorageService ss = StorageService.instance;
     TokenMetadata tmd = StorageService.instance.getTokenMetadata();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 8a14428..5e2fffe 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -34,6 +34,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -52,6 +53,7 @@ public class CQLSSTableWriterTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 90e63e0..bc5be46 100644
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@ -27,6 +27,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
@@ -43,6 +44,7 @@ public class CloudstackSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
index 56bbb77..32383d9 100644
--- a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
@@ -33,6 +33,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -51,6 +52,7 @@ public class EC2SnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
index 1521454..f2450f4 100644
--- a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
@@ -31,6 +31,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -47,6 +48,7 @@ public class GoogleCloudSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index 4481501..a693a23 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -63,6 +63,7 @@ public class StorageServiceServerTest
     @BeforeClass
     public static void setUp() throws ConfigurationException
     {
+        DatabaseDescriptor.setDaemonInitialized();
         IEndpointSnitch snitch = new PropertyFileSnitch();
         DatabaseDescriptor.setEndpointSnitch(snitch);
         Keyspace.setInitialized();


[02/19] cassandra git commit: Temporarily fix bug that creates commit log when running offline tools

Posted by yu...@apache.org.
Temporarily fix bug that creates commit log when running offline tools

patch by yukim; reviewed by thobbs for CASSANDRA-8616


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/66f1aaf8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/66f1aaf8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/66f1aaf8

Branch: refs/heads/cassandra-3.0
Commit: 66f1aaf88d3cde5c52b13d71d3326da5eda16fb1
Parents: fb29400
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Feb 11 19:06:27 2016 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:51:37 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/config/DatabaseDescriptor.java    | 12 +++++++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++++--
 .../cassandra/db/compaction/CompactionTask.java | 22 +++++++-----
 .../apache/cassandra/db/lifecycle/Tracker.java  | 36 ++++++++++++--------
 .../io/sstable/format/SSTableReader.java        |  8 +++--
 .../cassandra/service/CassandraDaemon.java      |  1 +
 .../service/EmbeddedCassandraService.java       |  2 ++
 .../io/sstable/CQLSSTableWriterLongTest.java    |  2 ++
 .../unit/org/apache/cassandra/SchemaLoader.java |  1 +
 .../config/DatabaseDescriptorTest.java          |  6 ++++
 .../apache/cassandra/db/CounterCellTest.java    |  3 ++
 .../org/apache/cassandra/db/NativeCellTest.java |  3 ++
 .../apache/cassandra/db/SystemKeyspaceTest.java |  2 ++
 .../db/context/CounterContextTest.java          |  8 +++++
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 ++-
 .../cassandra/db/lifecycle/TrackerTest.java     |  7 ++--
 .../cassandra/dht/StreamStateStoreTest.java     |  7 ++++
 .../cassandra/gms/FailureDetectorTest.java      |  2 ++
 .../org/apache/cassandra/gms/GossiperTest.java  |  5 +++
 .../io/sstable/CQLSSTableWriterTest.java        |  2 ++
 .../cassandra/locator/CloudstackSnitchTest.java |  2 ++
 .../apache/cassandra/locator/EC2SnitchTest.java |  2 ++
 .../locator/GoogleCloudSnitchTest.java          |  2 ++
 .../service/StorageServiceServerTest.java       |  1 +
 25 files changed, 118 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f02350d..8cff097 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.9
+ * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
  * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
  * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
  * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index a3fb79b..35debd0 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -102,6 +102,18 @@ public class DatabaseDescriptor
     private static Comparator<InetAddress> localComparator;
     private static boolean hasLoggedConfig;
 
+    private static boolean daemonInitialized;
+
+    public static boolean isDaemonInitialized()
+    {
+        return daemonInitialized;
+    }
+
+    public static void setDaemonInitialized()
+    {
+        daemonInitialized = true;
+    }
+
     public static void forceStaticInitialization() {}
     static
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c6b69dc..4bc46d0 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -388,9 +388,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         logger.info("Initializing {}.{}", keyspace.getName(), name);
 
-        // scan for sstables corresponding to this cf and load them
-        data = new Tracker(this, loadSSTables);
+        // Create Memtable only on online
+        Memtable initialMemtable = null;
+        if (DatabaseDescriptor.isDaemonInitialized())
+            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
+        data = new Tracker(initialMemtable, loadSSTables);
 
+        // scan for sstables corresponding to this cf and load them
         if (data.loadsstables)
         {
             Directories.SSTableLister sstableFiles = directories.sstableLister().skipTemporary(true);
@@ -2754,7 +2758,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             {
                 public Void call()
                 {
-                    cfs.data.reset();
+                    cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
                     cfs.getCompactionStrategy().shutdown();
                     cfs.getCompactionStrategy().startup();
                     return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 575c326..20d3dc0 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -127,7 +127,7 @@ public class CompactionTask extends AbstractCompactionTask
             }
         });
 
-        UUID taskId = SystemKeyspace.startCompaction(cfs, transaction.originals());
+        UUID taskId = offline ? null : SystemKeyspace.startCompaction(cfs, transaction.originals());
 
         // new sstables from flush can be added during a compaction, but only the compaction can remove them,
         // so in our single-threaded compaction world this is a valid way of determining if we're compacting
@@ -218,16 +218,20 @@ public class CompactionTask extends AbstractCompactionTask
             for (SSTableReader reader : newSStables)
                 newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
 
-            double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
-            long totalSourceRows = 0;
-            String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
-            logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                      taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
-            logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-            logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
-
             if (offline)
+            {
                 Refs.release(Refs.selfRefs(newSStables));
+            }
+            else
+            {
+                double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
+                long totalSourceRows = 0;
+                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
+                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
+                                           taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
+                logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
+                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten));
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index 5d5701f..e77ef78 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -31,7 +31,6 @@ import com.google.common.collect.*;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Memtable;
-import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,6 +51,7 @@ import static com.google.common.base.Predicates.not;
 import static com.google.common.collect.ImmutableSet.copyOf;
 import static com.google.common.collect.Iterables.filter;
 import static java.util.Collections.singleton;
+import static java.util.Collections.singletonList;
 import static org.apache.cassandra.db.lifecycle.Helpers.*;
 import static org.apache.cassandra.db.lifecycle.View.permitCompacting;
 import static org.apache.cassandra.db.lifecycle.View.updateCompacting;
@@ -61,21 +61,29 @@ import static org.apache.cassandra.utils.Throwables.merge;
 import static org.apache.cassandra.utils.concurrent.Refs.release;
 import static org.apache.cassandra.utils.concurrent.Refs.selfRefs;
 
+/**
+ * Tracker tracks live {@link View} of data store for a table.
+ */
 public class Tracker
 {
     private static final Logger logger = LoggerFactory.getLogger(Tracker.class);
 
-    public final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+    private final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+
     public final ColumnFamilyStore cfstore;
     final AtomicReference<View> view;
     public final boolean loadsstables;
 
-    public Tracker(ColumnFamilyStore cfstore, boolean loadsstables)
+    /**
+     * @param memtable Initial Memtable. Can be null.
+     * @param loadsstables true to indicate to load SSTables (TODO: remove as this is only accessed from 2i)
+     */
+    public Tracker(Memtable memtable, boolean loadsstables)
     {
-        this.cfstore = cfstore;
+        this.cfstore = memtable != null ? memtable.cfs : null;
         this.view = new AtomicReference<>();
         this.loadsstables = loadsstables;
-        this.reset();
+        this.reset(memtable);
     }
 
     public LifecycleTransaction tryModify(SSTableReader sstable, OperationType operationType)
@@ -196,16 +204,14 @@ public class Tracker
 
     /** (Re)initializes the tracker, purging all references. */
     @VisibleForTesting
-    public void reset()
+    public void reset(Memtable memtable)
     {
-        view.set(new View(
-                         !isDummy() ? ImmutableList.of(new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfstore))
-                                    : ImmutableList.<Memtable>of(),
-                         ImmutableList.<Memtable>of(),
-                         Collections.<SSTableReader, SSTableReader>emptyMap(),
-                         Collections.<SSTableReader>emptySet(),
-                         Collections.<SSTableReader>emptySet(),
-                         SSTableIntervalTree.empty()));
+        view.set(new View(memtable != null ? singletonList(memtable) : Collections.<Memtable>emptyList(),
+                          Collections.<Memtable>emptyList(),
+                          Collections.<SSTableReader, SSTableReader>emptyMap(),
+                          Collections.<SSTableReader>emptySet(),
+                          Collections.<SSTableReader>emptySet(),
+                          SSTableIntervalTree.empty()));
     }
 
     public Throwable dropSSTablesIfInvalid(Throwable accumulate)
@@ -473,7 +479,7 @@ public class Tracker
 
     public boolean isDummy()
     {
-        return cfstore == null;
+        return cfstore == null || !DatabaseDescriptor.isDaemonInitialized();
     }
 
     public void subscribe(INotificationConsumer consumer)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index fddf058..ba060d4 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -2246,7 +2246,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
             // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist
             // the read meter when in client mode.
-            if (SystemKeyspace.NAME.equals(desc.ksname))
+            if (SystemKeyspace.NAME.equals(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
             {
                 readMeter = null;
                 readMeterSyncFuture = null;
@@ -2272,9 +2272,11 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         {
             lookup.remove(desc);
             if (readMeterSyncFuture != null)
+            {
                 readMeterSyncFuture.cancel(true);
-            if (isCompacted.get())
-                SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+                if (isCompacted.get())
+                    SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+            }
             // don't ideally want to dropPageCache for the file until all instances have been released
             CLibrary.trySkipCache(desc.filenameFor(Component.DATA), 0, 0);
             CLibrary.trySkipCache(desc.filenameFor(Component.PRIMARY_INDEX), 0, 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index cf5f3c2..35e990f 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -505,6 +505,7 @@ public class CassandraDaemon
             try
             {
                 DatabaseDescriptor.forceStaticInitialization();
+                DatabaseDescriptor.setDaemonInitialized();
             }
             catch (ExceptionInInitializerError e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
index 659d851..6c154cd 100644
--- a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
+++ b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.service;
 
 import java.io.IOException;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.service.CassandraDaemon;
 
 /**
@@ -49,6 +50,7 @@ public class EmbeddedCassandraService
     public void start() throws IOException
     {
         cassandraDaemon = new CassandraDaemon();
+        DatabaseDescriptor.setDaemonInitialized();
         cassandraDaemon.init(null);
         cassandraDaemon.start();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
index fcec40d..ee719d1 100644
--- a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
+++ b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
@@ -30,6 +30,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.service.StorageService;
@@ -39,6 +40,7 @@ public class CQLSSTableWriterLongTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index a7cf7b4..2048f74 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -90,6 +90,7 @@ public class SchemaLoader
             }
         });
 
+        DatabaseDescriptor.setDaemonInitialized();
         Keyspace.setInitialized();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 5f231c3..7409535 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -47,6 +47,12 @@ import static org.junit.Assert.assertNull;
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class DatabaseDescriptorTest
 {
+    @BeforeClass
+    public static void setupDatabaseDescriptor()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testCFMetaDataSerialization() throws ConfigurationException, InvalidRequestException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/CounterCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
index 8d75b9a..5d4b8a8 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -30,6 +30,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
 import org.apache.cassandra.db.context.CounterContext;
@@ -57,6 +58,8 @@ public class CounterCellTest
         countLength   = 8; // size of long
 
         stepLength    = idLength + clockLength + countLength;
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/NativeCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NativeCellTest.java b/test/unit/org/apache/cassandra/db/NativeCellTest.java
index 70b7b87..4145a91 100644
--- a/test/unit/org/apache/cassandra/db/NativeCellTest.java
+++ b/test/unit/org/apache/cassandra/db/NativeCellTest.java
@@ -32,6 +32,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -128,6 +129,8 @@ public class NativeCellTest
         {
             throw new AssertionError();
         }
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index b8aa161..c3047b8 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -42,6 +42,8 @@ public class SystemKeyspaceTest
     @BeforeClass
     public static void prepSnapshotTracker()
     {
+        DatabaseDescriptor.setDaemonInitialized();
+
         if (FBUtilities.isWindows())
             WindowsFailedSnapshotTracker.deleteOldSnapshots();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
index a72d30d..4f587c6 100644
--- a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
+++ b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
@@ -22,9 +22,11 @@ package org.apache.cassandra.db.context;
 
 import java.nio.ByteBuffer;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ClockAndCount;
 import org.apache.cassandra.db.context.CounterContext.Relationship;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -48,6 +50,12 @@ public class CounterContextTest
     private static final int countLength = 8;
     private static final int stepLength = idLength + clockLength + countLength;
 
+    @BeforeClass
+    public static void setupDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testAllocate()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
index f13d1b7..737392e 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.lifecycle;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.junit.After;
 import org.junit.Before;
@@ -30,6 +31,8 @@ import junit.framework.Assert;
 import org.apache.cassandra.MockSchema;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
@@ -270,7 +273,7 @@ public class LifecycleTransactionTest extends AbstractTransactionalTest
 
         private static Tracker tracker(ColumnFamilyStore cfs, List<SSTableReader> readers)
         {
-            Tracker tracker = new Tracker(cfs, false);
+            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs), false);
             tracker.addInitialSSTables(readers);
             return tracker;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
index adeb778..04b4e4a 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -75,6 +75,7 @@ public class TrackerTest
     @BeforeClass
     public static void setUp()
     {
+        DatabaseDescriptor.setDaemonInitialized();
         MockSchema.cleanup();
     }
 
@@ -82,7 +83,7 @@ public class TrackerTest
     public void testTryModify()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = new Tracker(null, false);
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, true, cfs), MockSchema.sstable(1, cfs), MockSchema.sstable(2, cfs));
         tracker.addInitialSSTables(copyOf(readers));
         Assert.assertNull(tracker.tryModify(ImmutableList.of(MockSchema.sstable(0, cfs)), OperationType.COMPACTION));
@@ -146,7 +147,7 @@ public class TrackerTest
     public void testAddInitialSSTables()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = cfs.getTracker();
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),
                                                        MockSchema.sstable(1, 121, cfs),
                                                        MockSchema.sstable(2, 9, cfs));
@@ -163,7 +164,7 @@ public class TrackerTest
         boolean backups = DatabaseDescriptor.isIncrementalBackupsEnabled();
         DatabaseDescriptor.setIncrementalBackupsEnabled(false);
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = cfs.getTracker();
         MockListener listener = new MockListener(false);
         tracker.subscribe(listener);
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
index 86781d9..bdb654a 100644
--- a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
+++ b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
@@ -20,8 +20,10 @@ package org.apache.cassandra.dht;
 import java.net.InetAddress;
 import java.util.Collections;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.streaming.DefaultConnectionFactory;
 import org.apache.cassandra.streaming.StreamEvent;
 import org.apache.cassandra.streaming.StreamSession;
@@ -32,6 +34,11 @@ import static org.junit.Assert.assertTrue;
 
 public class StreamStateStoreTest
 {
+    @BeforeClass
+    public static void initDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
 
     @Test
     public void testUpdateAndQueryAvailableRanges()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
index 9325922..af099b0 100644
--- a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
+++ b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
@@ -45,6 +45,8 @@ public class FailureDetectorTest
     {
         // slow unit tests can cause problems with FailureDetector's GC pause handling
         System.setProperty("cassandra.max_local_pause_in_ms", "20000");
+
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/gms/GossiperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java
index ad07165..eb01305 100644
--- a/test/unit/org/apache/cassandra/gms/GossiperTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java
@@ -33,6 +33,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Token;
@@ -44,6 +45,10 @@ import static org.junit.Assert.assertEquals;
 
 public class GossiperTest
 {
+    static
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
     static final IPartitioner partitioner = new RandomPartitioner();
     StorageService ss = StorageService.instance;
     TokenMetadata tmd = StorageService.instance.getTokenMetadata();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 8a14428..5e2fffe 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -34,6 +34,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -52,6 +53,7 @@ public class CQLSSTableWriterTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 90e63e0..bc5be46 100644
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@ -27,6 +27,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
@@ -43,6 +44,7 @@ public class CloudstackSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
index 56bbb77..32383d9 100644
--- a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
@@ -33,6 +33,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -51,6 +52,7 @@ public class EC2SnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
index 1521454..f2450f4 100644
--- a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
@@ -31,6 +31,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -47,6 +48,7 @@ public class GoogleCloudSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index 4481501..a693a23 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -63,6 +63,7 @@ public class StorageServiceServerTest
     @BeforeClass
     public static void setUp() throws ConfigurationException
     {
+        DatabaseDescriptor.setDaemonInitialized();
         IEndpointSnitch snitch = new PropertyFileSnitch();
         DatabaseDescriptor.setEndpointSnitch(snitch);
         Keyspace.setInitialized();


[06/19] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index 003b624,0000000..a6ed3ba
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@@ -1,1112 -1,0 +1,1114 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.index;
 +
 +import java.lang.reflect.Constructor;
 +import java.util.*;
 +import java.util.concurrent.*;
 +import java.util.function.Function;
 +import java.util.stream.Collectors;
 +import java.util.stream.Stream;
 +
 +import com.google.common.base.Joiner;
 +import com.google.common.base.Strings;
 +import com.google.common.collect.ImmutableSet;
 +import com.google.common.collect.Iterables;
 +import com.google.common.collect.Maps;
 +import com.google.common.collect.Sets;
 +import com.google.common.primitives.Longs;
 +import com.google.common.util.concurrent.Futures;
 +import com.google.common.util.concurrent.MoreExecutors;
 +import org.apache.commons.lang3.StringUtils;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 +import org.apache.cassandra.concurrent.NamedThreadFactory;
 +import org.apache.cassandra.concurrent.StageManager;
 +import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.statements.IndexTarget;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.filter.RowFilter;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.lifecycle.View;
 +import org.apache.cassandra.db.partitions.PartitionIterators;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.index.internal.CassandraIndex;
 +import org.apache.cassandra.index.transactions.*;
 +import org.apache.cassandra.io.sstable.ReducingKeyIterator;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.schema.Indexes;
 +import org.apache.cassandra.service.pager.SinglePartitionPager;
 +import org.apache.cassandra.tracing.Tracing;
 +import org.apache.cassandra.transport.Server;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.apache.cassandra.utils.concurrent.Refs;
 +
 +/**
 + * Handles the core maintenance functionality associated with indexes: adding/removing them to or from
 + * a table, (re)building during bootstrap or other streaming operations, flushing, reloading metadata
 + * and so on.
 + *
 + * The Index interface defines a number of methods which return Callable<?>. These are primarily the
 + * management tasks for an index implementation. Most of them are currently executed in a blocking
 + * fashion via submission to SIM's blockingExecutor. This provides the desired behaviour in pretty
 + * much all cases, as tasks like flushing an index needs to be executed synchronously to avoid potentially
 + * deadlocking on the FlushWriter or PostFlusher. Several of these Callable<?> returning methods on Index could
 + * then be defined with as void and called directly from SIM (rather than being run via the executor service).
 + * Separating the task defintion from execution gives us greater flexibility though, so that in future, for example,
 + * if the flush process allows it we leave open the possibility of executing more of these tasks asynchronously.
 + *
 + * The primary exception to the above is the Callable returned from Index#addIndexedColumn. This may
 + * involve a significant effort, building a new index over any existing data. We perform this task asynchronously;
 + * as it is called as part of a schema update, which we do not want to block for a long period. Building non-custom
 + * indexes is performed on the CompactionManager.
 + *
 + * This class also provides instances of processors which listen to updates to the base table and forward to
 + * registered Indexes the info required to keep those indexes up to date.
 + * There are two variants of these processors, each with a factory method provided by SIM:
 + *      IndexTransaction: deals with updates generated on the regular write path.
 + *      CleanupTransaction: used when partitions are modified during compaction or cleanup operations.
 + * Further details on their usage and lifecycles can be found in the interface definitions below.
 + *
 + * Finally, the bestIndexFor method is used at query time to identify the most selective index of those able
 + * to satisfy any search predicates defined by a ReadCommand's RowFilter. It returns a thin IndexAccessor object
 + * which enables the ReadCommand to access the appropriate functions of the Index at various stages in its lifecycle.
 + * e.g. the getEstimatedResultRows is required when StorageProxy calculates the initial concurrency factor for
 + * distributing requests to replicas, whereas a Searcher instance is needed when the ReadCommand is executed locally on
 + * a target replica.
 + */
 +public class SecondaryIndexManager implements IndexRegistry
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(SecondaryIndexManager.class);
 +
 +    // default page size (in rows) when rebuilding the index for a whole partition
 +    public static final int DEFAULT_PAGE_SIZE = 10000;
 +
 +    private Map<String, Index> indexes = Maps.newConcurrentMap();
 +
 +    /**
 +     * The indexes that are ready to server requests.
 +     */
 +    private Set<String> builtIndexes = Sets.newConcurrentHashSet();
 +
 +    // executes tasks returned by Indexer#addIndexColumn which may require index(es) to be (re)built
 +    private static final ExecutorService asyncExecutor =
 +        new JMXEnabledThreadPoolExecutor(1,
 +                                         StageManager.KEEPALIVE,
 +                                         TimeUnit.SECONDS,
 +                                         new LinkedBlockingQueue<>(),
 +                                         new NamedThreadFactory("SecondaryIndexManagement"),
 +                                         "internal");
 +
 +    // executes all blocking tasks produced by Indexers e.g. getFlushTask, getMetadataReloadTask etc
 +    private static final ExecutorService blockingExecutor = MoreExecutors.newDirectExecutorService();
 +
 +    /**
 +     * The underlying column family containing the source data for these indexes
 +     */
 +    public final ColumnFamilyStore baseCfs;
 +
 +    public SecondaryIndexManager(ColumnFamilyStore baseCfs)
 +    {
 +        this.baseCfs = baseCfs;
 +    }
 +
 +
 +    /**
 +     * Drops and adds new indexes associated with the underlying CF
 +     */
 +    public void reload()
 +    {
 +        // figure out what needs to be added and dropped.
 +        Indexes tableIndexes = baseCfs.metadata.getIndexes();
 +        indexes.keySet()
 +               .stream()
 +               .filter(indexName -> !tableIndexes.has(indexName))
 +               .forEach(this::removeIndex);
 +
 +        // we call add for every index definition in the collection as
 +        // some may not have been created here yet, only added to schema
 +        for (IndexMetadata tableIndex : tableIndexes)
 +            addIndex(tableIndex);
 +    }
 +
 +    private Future<?> reloadIndex(IndexMetadata indexDef)
 +    {
 +        Index index = indexes.get(indexDef.name);
 +        Callable<?> reloadTask = index.getMetadataReloadTask(indexDef);
 +        return reloadTask == null
 +               ? Futures.immediateFuture(null)
 +               : blockingExecutor.submit(reloadTask);
 +    }
 +
 +    private Future<?> createIndex(IndexMetadata indexDef)
 +    {
 +        Index index = createInstance(indexDef);
 +        index.register(this);
 +
 +        // if the index didn't register itself, we can probably assume that no initialization needs to happen
 +        final Callable<?> initialBuildTask = indexes.containsKey(indexDef.name)
 +                                           ? index.getInitializationTask()
 +                                           : null;
 +        if (initialBuildTask == null)
 +        {
 +            // We need to make sure that the index is marked as built in the case where the initialBuildTask
 +            // does not need to be run (if the index didn't register itself or if the base table was empty).
 +            markIndexBuilt(indexDef.name);
 +            return Futures.immediateFuture(null);
 +        }
 +        return asyncExecutor.submit(index.getInitializationTask());
 +    }
 +
 +    /**
 +     * Adds and builds a index
 +     * @param indexDef the IndexMetadata describing the index
 +     */
 +    public synchronized Future<?> addIndex(IndexMetadata indexDef)
 +    {
 +        if (indexes.containsKey(indexDef.name))
 +            return reloadIndex(indexDef);
 +        else
 +            return createIndex(indexDef);
 +    }
 +
 +    /**
 +     * Checks if the specified index is queryable.
 +     *
 +     * @param index the index
 +     * @return <code>true</code> if the specified index is queryable, <code>false</code> otherwise
 +     */
 +    public boolean isIndexQueryable(Index index)
 +    {
 +        return builtIndexes.contains(index.getIndexMetadata().name);
 +    }
 +
 +    public synchronized void removeIndex(String indexName)
 +    {
 +        Index index = unregisterIndex(indexName);
 +        if (null != index)
 +        {
 +            markIndexRemoved(indexName);
 +            executeBlocking(index.getInvalidateTask());
 +        }
 +    }
 +
 +
 +    public Set<IndexMetadata> getDependentIndexes(ColumnDefinition column)
 +    {
 +        if (indexes.isEmpty())
 +            return Collections.emptySet();
 +
 +        Set<IndexMetadata> dependentIndexes = new HashSet<>();
 +        for (Index index : indexes.values())
 +            if (index.dependsOn(column))
 +                dependentIndexes.add(index.getIndexMetadata());
 +
 +        return dependentIndexes;
 +    }
 +
 +    /**
 +     * Called when dropping a Table
 +     */
 +    public void markAllIndexesRemoved()
 +    {
 +       getBuiltIndexNames().forEach(this::markIndexRemoved);
 +    }
 +
 +    /**
 +    * Does a full, blocking rebuild of the indexes specified by columns from the sstables.
 +    * Caller must acquire and release references to the sstables used here.
 +    * Note also that only this method of (re)building indexes:
 +    *   a) takes a set of index *names* rather than Indexers
 +    *   b) marks exsiting indexes removed prior to rebuilding
 +    *
 +    * @param sstables the data to build from
 +    * @param indexNames the list of indexes to be rebuilt
 +    */
 +    public void rebuildIndexesBlocking(Collection<SSTableReader> sstables, Set<String> indexNames)
 +    {
 +        Set<Index> toRebuild = indexes.values().stream()
 +                                               .filter(index -> indexNames.contains(index.getIndexMetadata().name))
 +                                               .filter(Index::shouldBuildBlocking)
 +                                               .collect(Collectors.toSet());
 +        if (toRebuild.isEmpty())
 +        {
 +            logger.info("No defined indexes with the supplied names: {}", Joiner.on(',').join(indexNames));
 +            return;
 +        }
 +
 +        toRebuild.forEach(indexer -> markIndexRemoved(indexer.getIndexMetadata().name));
 +
 +        buildIndexesBlocking(sstables, toRebuild);
 +
 +        toRebuild.forEach(indexer -> markIndexBuilt(indexer.getIndexMetadata().name));
 +    }
 +
 +    public void buildAllIndexesBlocking(Collection<SSTableReader> sstables)
 +    {
 +        buildIndexesBlocking(sstables, indexes.values()
 +                                              .stream()
 +                                              .filter(Index::shouldBuildBlocking)
 +                                              .collect(Collectors.toSet()));
 +    }
 +
 +    // For convenience, may be called directly from Index impls
 +    public void buildIndexBlocking(Index index)
 +    {
 +        if (index.shouldBuildBlocking())
 +        {
 +            try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL));
 +                 Refs<SSTableReader> sstables = viewFragment.refs)
 +            {
 +                buildIndexesBlocking(sstables, Collections.singleton(index));
 +                markIndexBuilt(index.getIndexMetadata().name);
 +            }
 +        }
 +    }
 +
 +    /**
 +     * Checks if the specified {@link ColumnFamilyStore} is a secondary index.
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code> to check.
 +     * @return <code>true</code> if the specified <code>ColumnFamilyStore</code> is a secondary index,
 +     * <code>false</code> otherwise.
 +     */
 +    public static boolean isIndexColumnFamilyStore(ColumnFamilyStore cfs)
 +    {
 +        return isIndexColumnFamily(cfs.name);
 +    }
 +
 +    /**
 +     * Checks if the specified {@link ColumnFamilyStore} is the one secondary index.
 +     *
 +     * @param cfName the name of the <code>ColumnFamilyStore</code> to check.
 +     * @return <code>true</code> if the specified <code>ColumnFamilyStore</code> is a secondary index,
 +     * <code>false</code> otherwise.
 +     */
 +    public static boolean isIndexColumnFamily(String cfName)
 +    {
 +        return cfName.contains(Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    /**
 +     * Returns the parent of the specified {@link ColumnFamilyStore}.
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code>
 +     * @return the parent of the specified <code>ColumnFamilyStore</code>
 +     */
 +    public static ColumnFamilyStore getParentCfs(ColumnFamilyStore cfs)
 +    {
 +        String parentCfs = getParentCfsName(cfs.name);
 +        return cfs.keyspace.getColumnFamilyStore(parentCfs);
 +    }
 +
 +    /**
 +     * Returns the parent name of the specified {@link ColumnFamilyStore}.
 +     *
 +     * @param cfName the <code>ColumnFamilyStore</code> name
 +     * @return the parent name of the specified <code>ColumnFamilyStore</code>
 +     */
 +    public static String getParentCfsName(String cfName)
 +    {
 +        assert isIndexColumnFamily(cfName);
 +        return StringUtils.substringBefore(cfName, Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    /**
 +     * Returns the index name
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code>
 +     * @return the index name
 +     */
 +    public static String getIndexName(ColumnFamilyStore cfs)
 +    {
 +        return getIndexName(cfs.name);
 +    }
 +
 +    /**
 +     * Returns the index name
 +     *
 +     * @param cfName the <code>ColumnFamilyStore</code> name
 +     * @return the index name
 +     */
 +    public static String getIndexName(String cfName)
 +    {
 +        assert isIndexColumnFamily(cfName);
 +        return StringUtils.substringAfter(cfName, Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    private void buildIndexesBlocking(Collection<SSTableReader> sstables, Set<Index> indexes)
 +    {
 +        if (indexes.isEmpty())
 +            return;
 +
 +        logger.info("Submitting index build of {} for data in {}",
 +                    indexes.stream().map(i -> i.getIndexMetadata().name).collect(Collectors.joining(",")),
 +                    sstables.stream().map(SSTableReader::toString).collect(Collectors.joining(",")));
 +
 +        SecondaryIndexBuilder builder = new SecondaryIndexBuilder(baseCfs,
 +                                                                  indexes,
 +                                                                  new ReducingKeyIterator(sstables));
 +        Future<?> future = CompactionManager.instance.submitIndexBuild(builder);
 +        FBUtilities.waitOnFuture(future);
 +
 +        flushIndexesBlocking(indexes);
 +        logger.info("Index build of {} complete",
 +                    indexes.stream().map(i -> i.getIndexMetadata().name).collect(Collectors.joining(",")));
 +    }
 +
 +    /**
 +     * Marks the specified index as build.
 +     * <p>This method is public as it need to be accessible from the {@link Index} implementations</p>
 +     * @param indexName the index name
 +     */
 +    public void markIndexBuilt(String indexName)
 +    {
 +        builtIndexes.add(indexName);
-         SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), indexName);
++        if (DatabaseDescriptor.isDaemonInitialized())
++            SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), indexName);
 +    }
 +
 +    /**
 +     * Marks the specified index as removed.
 +     * <p>This method is public as it need to be accessible from the {@link Index} implementations</p>
 +     * @param indexName the index name
 +     */
 +    public void markIndexRemoved(String indexName)
 +    {
 +        SystemKeyspace.setIndexRemoved(baseCfs.keyspace.getName(), indexName);
 +    }
 +
 +    public Index getIndexByName(String indexName)
 +    {
 +        return indexes.get(indexName);
 +    }
 +
 +    private Index createInstance(IndexMetadata indexDef)
 +    {
 +        Index newIndex;
 +        if (indexDef.isCustom())
 +        {
 +            assert indexDef.options != null;
 +            String className = indexDef.options.get(IndexTarget.CUSTOM_INDEX_OPTION_NAME);
 +            assert ! Strings.isNullOrEmpty(className);
 +            try
 +            {
 +                Class<? extends Index> indexClass = FBUtilities.classForName(className, "Index");
 +                Constructor<? extends Index> ctor = indexClass.getConstructor(ColumnFamilyStore.class, IndexMetadata.class);
 +                newIndex = (Index)ctor.newInstance(baseCfs, indexDef);
 +            }
 +            catch (Exception e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +        }
 +        else
 +        {
 +            newIndex = CassandraIndex.newIndex(baseCfs, indexDef);
 +        }
 +        return newIndex;
 +    }
 +
 +    /**
 +     * Truncate all indexes
 +     */
 +    public void truncateAllIndexesBlocking(final long truncatedAt)
 +    {
 +        executeAllBlocking(indexes.values().stream(), (index) -> index.getTruncateTask(truncatedAt));
 +    }
 +
 +    /**
 +     * Remove all indexes
 +     */
 +    public void invalidateAllIndexesBlocking()
 +    {
 +        markAllIndexesRemoved();
 +        executeAllBlocking(indexes.values().stream(), Index::getInvalidateTask);
 +    }
 +
 +    /**
 +     * Perform a blocking flush all indexes
 +     */
 +    public void flushAllIndexesBlocking()
 +    {
 +       flushIndexesBlocking(ImmutableSet.copyOf(indexes.values()));
 +    }
 +
 +    /**
 +     * Perform a blocking flush of selected indexes
 +     */
 +    public void flushIndexesBlocking(Set<Index> indexes)
 +    {
 +        if (indexes.isEmpty())
 +            return;
 +
 +        List<Future<?>> wait = new ArrayList<>();
 +        List<Index> nonCfsIndexes = new ArrayList<>();
 +
 +        // for each CFS backed index, submit a flush task which we'll wait on for completion
 +        // for the non-CFS backed indexes, we'll flush those while we wait.
 +        synchronized (baseCfs.getTracker())
 +        {
 +            indexes.forEach(index ->
 +                index.getBackingTable()
 +                     .map(cfs -> wait.add(cfs.forceFlush()))
 +                     .orElseGet(() -> nonCfsIndexes.add(index)));
 +        }
 +
 +        executeAllBlocking(nonCfsIndexes.stream(), Index::getBlockingFlushTask);
 +        FBUtilities.waitOnFutures(wait);
 +    }
 +
 +    /**
 +     * Performs a blocking flush of all custom indexes
 +     */
 +    public void flushAllNonCFSBackedIndexesBlocking()
 +    {
 +        executeAllBlocking(indexes.values()
 +                                  .stream()
 +                                  .filter(index -> !index.getBackingTable().isPresent()),
 +                           Index::getBlockingFlushTask);
 +    }
 +
 +    /**
 +     * @return all indexes which are marked as built and ready to use
 +     */
 +    public List<String> getBuiltIndexNames()
 +    {
 +        Set<String> allIndexNames = new HashSet<>();
 +        indexes.values().stream()
 +                .map(i -> i.getIndexMetadata().name)
 +                .forEach(allIndexNames::add);
 +        return SystemKeyspace.getBuiltIndexes(baseCfs.keyspace.getName(), allIndexNames);
 +    }
 +
 +    /**
 +     * @return all backing Tables used by registered indexes
 +     */
 +    public Set<ColumnFamilyStore> getAllIndexColumnFamilyStores()
 +    {
 +        Set<ColumnFamilyStore> backingTables = new HashSet<>();
 +        indexes.values().forEach(index -> index.getBackingTable().ifPresent(backingTables::add));
 +        return backingTables;
 +    }
 +
 +    /**
 +     * @return if there are ANY indexes registered for this table
 +     */
 +    public boolean hasIndexes()
 +    {
 +        return !indexes.isEmpty();
 +    }
 +
 +    /**
 +     * When building an index against existing data in sstables, add the given partition to the index
 +     */
 +    public void indexPartition(DecoratedKey key, Set<Index> indexes, int pageSize)
 +    {
 +        if (logger.isTraceEnabled())
 +            logger.trace("Indexing partition {}", baseCfs.metadata.getKeyValidator().getString(key.getKey()));
 +
 +        if (!indexes.isEmpty())
 +        {
 +            SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(baseCfs.metadata,
 +                                                                                          FBUtilities.nowInSeconds(),
 +                                                                                          key);
 +            int nowInSec = cmd.nowInSec();
 +            boolean readStatic = false;
 +
 +            SinglePartitionPager pager = new SinglePartitionPager(cmd, null, Server.CURRENT_VERSION);
 +            while (!pager.isExhausted())
 +            {
 +                try (ReadOrderGroup readGroup = cmd.startOrderGroup();
 +                     OpOrder.Group writeGroup = Keyspace.writeOrder.start();
 +                     RowIterator partition =
 +                        PartitionIterators.getOnlyElement(pager.fetchPageInternal(pageSize,readGroup),
 +                                                          cmd))
 +                {
 +                    Set<Index.Indexer> indexers = indexes.stream()
 +                                                         .map(index -> index.indexerFor(key,
 +                                                                                        partition.columns(),
 +                                                                                        nowInSec,
 +                                                                                        writeGroup,
 +                                                                                        IndexTransaction.Type.UPDATE))
 +                                                         .filter(Objects::nonNull)
 +                                                         .collect(Collectors.toSet());
 +
 +                    indexers.forEach(Index.Indexer::begin);
 +
 +                    // only process the static row once per partition
 +                    if (!readStatic && !partition.staticRow().isEmpty())
 +                    {
 +                        indexers.forEach(indexer -> indexer.insertRow(partition.staticRow()));
 +                        readStatic = true;
 +                    }
 +
 +                    while (partition.hasNext())
 +                    {
 +                        Row row = partition.next();
 +                        indexers.forEach(indexer -> indexer.insertRow(row));
 +                    }
 +
 +                    indexers.forEach(Index.Indexer::finish);
 +                }
 +            }
 +        }
 +    }
 +
 +    /**
 +     * Return the page size used when indexing an entire partition
 +     */
 +    public int calculateIndexingPageSize()
 +    {
 +        if (Boolean.getBoolean("cassandra.force_default_indexing_page_size"))
 +            return DEFAULT_PAGE_SIZE;
 +
 +        double targetPageSizeInBytes = 32 * 1024 * 1024;
 +        double meanPartitionSize = baseCfs.getMeanPartitionSize();
 +        if (meanPartitionSize <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int meanCellsPerPartition = baseCfs.getMeanColumns();
 +        if (meanCellsPerPartition <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int columnsPerRow = baseCfs.metadata.partitionColumns().regulars.size();
 +        if (meanCellsPerPartition <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int meanRowsPerPartition = meanCellsPerPartition / columnsPerRow;
 +        double meanRowSize = meanPartitionSize / meanRowsPerPartition;
 +
 +        int pageSize = (int) Math.max(1, Math.min(DEFAULT_PAGE_SIZE, targetPageSizeInBytes / meanRowSize));
 +
 +        logger.trace("Calculated page size {} for indexing {}.{} ({}/{}/{}/{})",
 +                     pageSize,
 +                     baseCfs.metadata.ksName,
 +                     baseCfs.metadata.cfName,
 +                     meanPartitionSize,
 +                     meanCellsPerPartition,
 +                     meanRowsPerPartition,
 +                     meanRowSize);
 +
 +        return pageSize;
 +    }
 +
 +    /**
 +     * Delete all data from all indexes for this partition.
 +     * For when cleanup rips a partition out entirely.
 +     *
 +     * TODO : improve cleanup transaction to batch updates & perform them async
 +     */
 +    public void deletePartition(UnfilteredRowIterator partition, int nowInSec)
 +    {
 +        // we need to acquire memtable lock because secondary index deletion may
 +        // cause a race (see CASSANDRA-3712). This is done internally by the
 +        // index transaction when it commits
 +        CleanupTransaction indexTransaction = newCleanupTransaction(partition.partitionKey(),
 +                                                                    partition.columns(),
 +                                                                    nowInSec);
 +        indexTransaction.start();
 +        indexTransaction.onPartitionDeletion(new DeletionTime(FBUtilities.timestampMicros(), nowInSec));
 +        indexTransaction.commit();
 +
 +        while (partition.hasNext())
 +        {
 +            Unfiltered unfiltered = partition.next();
 +            if (unfiltered.kind() != Unfiltered.Kind.ROW)
 +                continue;
 +
 +            indexTransaction = newCleanupTransaction(partition.partitionKey(),
 +                                                     partition.columns(),
 +                                                     nowInSec);
 +            indexTransaction.start();
 +            indexTransaction.onRowDelete((Row)unfiltered);
 +            indexTransaction.commit();
 +        }
 +    }
 +
 +    /**
 +     * Called at query time to choose which (if any) of the registered index implementations to use for a given query.
 +     *
 +     * This is a two step processes, firstly compiling the set of searchable indexes then choosing the one which reduces
 +     * the search space the most.
 +     *
 +     * In the first phase, if the command's RowFilter contains any custom index expressions, the indexes that they
 +     * specify are automatically included. Following that, the registered indexes are filtered to include only those
 +     * which support the standard expressions in the RowFilter.
 +     *
 +     * The filtered set then sorted by selectivity, as reported by the Index implementations' getEstimatedResultRows
 +     * method.
 +     *
 +     * Implementation specific validation of the target expression, either custom or standard, by the selected
 +     * index should be performed in the searcherFor method to ensure that we pick the right index regardless of
 +     * the validity of the expression.
 +     *
 +     * This method is only called once during the lifecycle of a ReadCommand and the result is
 +     * cached for future use when obtaining a Searcher, getting the index's underlying CFS for
 +     * ReadOrderGroup, or an estimate of the result size from an average index query.
 +     *
 +     * @param command ReadCommand to be executed
 +     * @return an Index instance, ready to use during execution of the command, or null if none
 +     * of the registered indexes can support the command.
 +     */
 +    public Index getBestIndexFor(ReadCommand command)
 +    {
 +        if (indexes.isEmpty() || command.rowFilter().isEmpty())
 +            return null;
 +
 +        Set<Index> searchableIndexes = new HashSet<>();
 +        for (RowFilter.Expression expression : command.rowFilter())
 +        {
 +            if (expression.isCustom())
 +            {
 +                // Only a single custom expression is allowed per query and, if present,
 +                // we want to always favour the index specified in such an expression
 +                RowFilter.CustomExpression customExpression = (RowFilter.CustomExpression)expression;
 +                logger.trace("Command contains a custom index expression, using target index {}", customExpression.getTargetIndex().name);
 +                Tracing.trace("Command contains a custom index expression, using target index {}", customExpression.getTargetIndex().name);
 +                return indexes.get(customExpression.getTargetIndex().name);
 +            }
 +            else
 +            {
 +                indexes.values().stream()
 +                       .filter(index -> index.supportsExpression(expression.column(), expression.operator()))
 +                       .forEach(searchableIndexes::add);
 +            }
 +        }
 +
 +        if (searchableIndexes.isEmpty())
 +        {
 +            logger.trace("No applicable indexes found");
 +            Tracing.trace("No applicable indexes found");
 +            return null;
 +        }
 +
 +        Index selected = searchableIndexes.size() == 1
 +                         ? Iterables.getOnlyElement(searchableIndexes)
 +                         : searchableIndexes.stream()
 +                                            .min((a, b) -> Longs.compare(a.getEstimatedResultRows(),
 +                                                                         b.getEstimatedResultRows()))
 +                                            .orElseThrow(() -> new AssertionError("Could not select most selective index"));
 +
 +        // pay for an additional threadlocal get() rather than build the strings unnecessarily
 +        if (Tracing.isTracing())
 +        {
 +            Tracing.trace("Index mean cardinalities are {}. Scanning with {}.",
 +                          searchableIndexes.stream().map(i -> i.getIndexMetadata().name + ':' + i.getEstimatedResultRows())
 +                                           .collect(Collectors.joining(",")),
 +                          selected.getIndexMetadata().name);
 +        }
 +        return selected;
 +    }
 +
 +    /**
 +     * Called at write time to ensure that values present in the update
 +     * are valid according to the rules of all registered indexes which
 +     * will process it. The partition key as well as the clustering and
 +     * cell values for each row in the update may be checked by index
 +     * implementations
 +     * @param update PartitionUpdate containing the values to be validated by registered Index implementations
 +     * @throws InvalidRequestException
 +     */
 +    public void validate(PartitionUpdate update) throws InvalidRequestException
 +    {
 +        for (Index index : indexes.values())
 +            index.validate(update);
 +    }
 +
 +    /**
 +     * IndexRegistry methods
 +     */
 +    public void registerIndex(Index index)
 +    {
 +        String name = index.getIndexMetadata().name;
 +        indexes.put(name, index);
 +        logger.trace("Registered index {}", name);
 +    }
 +
 +    public void unregisterIndex(Index index)
 +    {
 +        unregisterIndex(index.getIndexMetadata().name);
 +    }
 +
 +    private Index unregisterIndex(String name)
 +    {
 +        Index removed = indexes.remove(name);
 +        builtIndexes.remove(name);
 +        logger.trace(removed == null ? "Index {} was not registered" : "Removed index {} from registry",
 +                     name);
 +        return removed;
 +    }
 +
 +    public Index getIndex(IndexMetadata metadata)
 +    {
 +        return indexes.get(metadata.name);
 +    }
 +
 +    public Collection<Index> listIndexes()
 +    {
 +        return ImmutableSet.copyOf(indexes.values());
 +    }
 +
 +    /**
 +     * Handling of index updates.
 +     * Implementations of the various IndexTransaction interfaces, for keeping indexes in sync with base data
 +     * during updates, compaction and cleanup. Plus factory methods for obtaining transaction instances.
 +     */
 +
 +    /**
 +     * Transaction for updates on the write path.
 +     */
 +    public UpdateTransaction newUpdateTransaction(PartitionUpdate update, OpOrder.Group opGroup, int nowInSec)
 +    {
 +        if (!hasIndexes())
 +            return UpdateTransaction.NO_OP;
 +
 +        Index.Indexer[] indexers = indexes.values().stream()
 +                                          .map(i -> i.indexerFor(update.partitionKey(),
 +                                                                 update.columns(),
 +                                                                 nowInSec,
 +                                                                 opGroup,
 +                                                                 IndexTransaction.Type.UPDATE))
 +                                          .filter(Objects::nonNull)
 +                                          .toArray(Index.Indexer[]::new);
 +
 +        return indexers.length == 0 ? UpdateTransaction.NO_OP : new WriteTimeTransaction(indexers);
 +    }
 +
 +    /**
 +     * Transaction for use when merging rows during compaction
 +     */
 +    public CompactionTransaction newCompactionTransaction(DecoratedKey key,
 +                                                          PartitionColumns partitionColumns,
 +                                                          int versions,
 +                                                          int nowInSec)
 +    {
 +        // the check for whether there are any registered indexes is already done in CompactionIterator
 +        return new IndexGCTransaction(key, partitionColumns, versions, nowInSec, listIndexes());
 +    }
 +
 +    /**
 +     * Transaction for use when removing partitions during cleanup
 +     */
 +    public CleanupTransaction newCleanupTransaction(DecoratedKey key,
 +                                                    PartitionColumns partitionColumns,
 +                                                    int nowInSec)
 +    {
 +        if (!hasIndexes())
 +            return CleanupTransaction.NO_OP;
 +
 +        return new CleanupGCTransaction(key, partitionColumns, nowInSec, listIndexes());
 +    }
 +
 +    /**
 +     * A single use transaction for processing a partition update on the regular write path
 +     */
 +    private static final class WriteTimeTransaction implements UpdateTransaction
 +    {
 +        private final Index.Indexer[] indexers;
 +
 +        private WriteTimeTransaction(Index.Indexer...indexers)
 +        {
 +            // don't allow null indexers, if we don't need any use a NullUpdater object
 +            for (Index.Indexer indexer : indexers) assert indexer != null;
 +            this.indexers = indexers;
 +        }
 +
 +        public void start()
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.begin();
 +        }
 +
 +        public void onPartitionDeletion(DeletionTime deletionTime)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.partitionDelete(deletionTime);
 +        }
 +
 +        public void onRangeTombstone(RangeTombstone tombstone)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.rangeTombstone(tombstone);
 +        }
 +
 +        public void onInserted(Row row)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.insertRow(row);
 +        }
 +
 +        public void onUpdated(Row existing, Row updated)
 +        {
 +            final Row.Builder toRemove = BTreeRow.sortedBuilder();
 +            toRemove.newRow(existing.clustering());
 +            toRemove.addPrimaryKeyLivenessInfo(existing.primaryKeyLivenessInfo());
 +            toRemove.addRowDeletion(existing.deletion());
 +            final Row.Builder toInsert = BTreeRow.sortedBuilder();
 +            toInsert.newRow(updated.clustering());
 +            toInsert.addPrimaryKeyLivenessInfo(updated.primaryKeyLivenessInfo());
 +            toInsert.addRowDeletion(updated.deletion());
 +            // diff listener collates the columns to be added & removed from the indexes
 +            RowDiffListener diffListener = new RowDiffListener()
 +            {
 +                public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
 +                {
 +                }
 +
 +                public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original)
 +                {
 +                }
 +
 +                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
 +                {
 +                }
 +
 +                public void onCell(int i, Clustering clustering, Cell merged, Cell original)
 +                {
 +                    if (merged != null && !merged.equals(original))
 +                        toInsert.addCell(merged);
 +
 +                    if (merged == null || (original != null && shouldCleanupOldValue(original, merged)))
 +                        toRemove.addCell(original);
 +
 +                }
 +            };
 +            Rows.diff(diffListener, updated, existing);
 +            Row oldRow = toRemove.build();
 +            Row newRow = toInsert.build();
 +            for (Index.Indexer indexer : indexers)
 +                indexer.updateRow(oldRow, newRow);
 +        }
 +
 +        public void commit()
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.finish();
 +        }
 +
 +        private boolean shouldCleanupOldValue(Cell oldCell, Cell newCell)
 +        {
 +            // If either the value or timestamp is different, then we
 +            // should delete from the index. If not, then we can infer that
 +            // at least one of the cells is an ExpiringColumn and that the
 +            // difference is in the expiry time. In this case, we don't want to
 +            // delete the old value from the index as the tombstone we insert
 +            // will just hide the inserted value.
 +            // Completely identical cells (including expiring columns with
 +            // identical ttl & localExpirationTime) will not get this far due
 +            // to the oldCell.equals(newCell) in StandardUpdater.update
 +            return !oldCell.value().equals(newCell.value()) || oldCell.timestamp() != newCell.timestamp();
 +        }
 +    }
 +
 +    /**
 +     * A single-use transaction for updating indexes for a single partition during compaction where the only
 +     * operation is to merge rows
 +     * TODO : make this smarter at batching updates so we can use a single transaction to process multiple rows in
 +     * a single partition
 +     */
 +    private static final class IndexGCTransaction implements CompactionTransaction
 +    {
 +        private final DecoratedKey key;
 +        private final PartitionColumns columns;
 +        private final int versions;
 +        private final int nowInSec;
 +        private final Collection<Index> indexes;
 +
 +        private Row[] rows;
 +
 +        private IndexGCTransaction(DecoratedKey key,
 +                                   PartitionColumns columns,
 +                                   int versions,
 +                                   int nowInSec,
 +                                   Collection<Index> indexes)
 +        {
 +            this.key = key;
 +            this.columns = columns;
 +            this.versions = versions;
 +            this.indexes = indexes;
 +            this.nowInSec = nowInSec;
 +        }
 +
 +        public void start()
 +        {
 +            if (versions > 0)
 +                rows = new Row[versions];
 +        }
 +
 +        public void onRowMerge(Row merged, Row...versions)
 +        {
 +            // Diff listener constructs rows representing deltas between the merged and original versions
 +            // These delta rows are then passed to registered indexes for removal processing
 +            final Row.Builder[] builders = new Row.Builder[versions.length];
 +            RowDiffListener diffListener = new RowDiffListener()
 +            {
 +                public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
 +                {
 +                    if (original != null && (merged == null || !merged.isLive(nowInSec)))
 +                        getBuilder(i, clustering).addPrimaryKeyLivenessInfo(original);
 +                }
 +
 +                public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original)
 +                {
 +                }
 +
 +                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
 +                {
 +                }
 +
 +                public void onCell(int i, Clustering clustering, Cell merged, Cell original)
 +                {
 +                    if (original != null && (merged == null || !merged.isLive(nowInSec)))
 +                        getBuilder(i, clustering).addCell(original);
 +                }
 +
 +                private Row.Builder getBuilder(int index, Clustering clustering)
 +                {
 +                    if (builders[index] == null)
 +                    {
 +                        builders[index] = BTreeRow.sortedBuilder();
 +                        builders[index].newRow(clustering);
 +                    }
 +                    return builders[index];
 +                }
 +            };
 +
 +            Rows.diff(diffListener, merged, versions);
 +
 +            for(int i = 0; i < builders.length; i++)
 +                if (builders[i] != null)
 +                    rows[i] = builders[i].build();
 +        }
 +
 +        public void commit()
 +        {
 +            if (rows == null)
 +                return;
 +
 +            try (OpOrder.Group opGroup = Keyspace.writeOrder.start())
 +            {
 +                for (Index index : indexes)
 +                {
 +                    Index.Indexer indexer = index.indexerFor(key, columns, nowInSec, opGroup, Type.COMPACTION);
 +                    if (indexer == null)
 +                        continue;
 +
 +                    indexer.begin();
 +                    for (Row row : rows)
 +                        if (row != null)
 +                            indexer.removeRow(row);
 +                    indexer.finish();
 +                }
 +            }
 +        }
 +    }
 +
 +    /**
 +     * A single-use transaction for updating indexes for a single partition during cleanup, where
 +     * partitions and rows are only removed
 +     * TODO : make this smarter at batching updates so we can use a single transaction to process multiple rows in
 +     * a single partition
 +     */
 +    private static final class CleanupGCTransaction implements CleanupTransaction
 +    {
 +        private final DecoratedKey key;
 +        private final PartitionColumns columns;
 +        private final int nowInSec;
 +        private final Collection<Index> indexes;
 +
 +        private Row row;
 +        private DeletionTime partitionDelete;
 +
 +        private CleanupGCTransaction(DecoratedKey key,
 +                                     PartitionColumns columns,
 +                                     int nowInSec,
 +                                     Collection<Index> indexes)
 +        {
 +            this.key = key;
 +            this.columns = columns;
 +            this.indexes = indexes;
 +            this.nowInSec = nowInSec;
 +        }
 +
 +        public void start()
 +        {
 +        }
 +
 +        public void onPartitionDeletion(DeletionTime deletionTime)
 +        {
 +            partitionDelete = deletionTime;
 +        }
 +
 +        public void onRowDelete(Row row)
 +        {
 +            this.row = row;
 +        }
 +
 +        public void commit()
 +        {
 +            if (row == null && partitionDelete == null)
 +                return;
 +
 +            try (OpOrder.Group opGroup = Keyspace.writeOrder.start())
 +            {
 +                for (Index index : indexes)
 +                {
 +                    Index.Indexer indexer = index.indexerFor(key, columns, nowInSec, opGroup, Type.CLEANUP);
 +                    if (indexer == null)
 +                        continue;
 +
 +                    indexer.begin();
 +
 +                    if (partitionDelete != null)
 +                        indexer.partitionDelete(partitionDelete);
 +
 +                    if (row != null)
 +                        indexer.removeRow(row);
 +
 +                    indexer.finish();
 +                }
 +            }
 +        }
 +    }
 +
 +    private static void executeBlocking(Callable<?> task)
 +    {
 +        if (null != task)
 +            FBUtilities.waitOnFuture(blockingExecutor.submit(task));
 +    }
 +
 +    private static void executeAllBlocking(Stream<Index> indexers, Function<Index, Callable<?>> function)
 +    {
 +        List<Future<?>> waitFor = new ArrayList<>();
 +        indexers.forEach(indexer -> {
 +            Callable<?> task = function.apply(indexer);
 +            if (null != task)
 +                waitFor.add(blockingExecutor.submit(task));
 +        });
 +        FBUtilities.waitOnFutures(waitFor);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index f0cdcf5,ba060d4..3283723
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@@ -2183,10 -2246,10 +2183,10 @@@ public abstract class SSTableReader ext
  
              // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist
              // the read meter when in client mode.
-             if (Schema.isSystemKeyspace(desc.ksname))
 -            if (SystemKeyspace.NAME.equals(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
++            if (Schema.isSystemKeyspace(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
              {
                  readMeter = null;
 -                readMeterSyncFuture = null;
 +                readMeterSyncFuture = NULL;
                  return;
              }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/CQLTester.java
index 3d45393,98b8e23..3d8d03b
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@@ -140,86 -137,7 +140,87 @@@ public abstract class CQLTeste
      // We don't use USE_PREPARED_VALUES in the code below so some test can foce value preparation (if the result
      // is not expected to be the same without preparation)
      private boolean usePrepared = USE_PREPARED_VALUES;
 -    private static final boolean reusePrepared = Boolean.valueOf(System.getProperty("cassandra.test.reuse_prepared", "true"));
 +    private static boolean reusePrepared = REUSE_PREPARED;
 +
 +    public static void prepareServer()
 +    {
 +        if (isServerPrepared)
 +            return;
 +
 +        // Cleanup first
 +        try
 +        {
 +            cleanupAndLeaveDirs();
 +        }
 +        catch (IOException e)
 +        {
 +            logger.error("Failed to cleanup and recreate directories.");
 +            throw new RuntimeException(e);
 +        }
 +
 +        Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()
 +        {
 +            public void uncaughtException(Thread t, Throwable e)
 +            {
 +                logger.error("Fatal exception in thread " + t, e);
 +            }
 +        });
 +
 +        ThreadAwareSecurityManager.install();
 +
++        DatabaseDescriptor.setDaemonInitialized();
 +        Keyspace.setInitialized();
 +        isServerPrepared = true;
 +    }
 +
 +    public static void cleanupAndLeaveDirs() throws IOException
 +    {
 +        // We need to stop and unmap all CLS instances prior to cleanup() or we'll get failures on Windows.
 +        CommitLog.instance.stopUnsafe(true);
 +        mkdirs();
 +        cleanup();
 +        mkdirs();
 +        CommitLog.instance.restartUnsafe();
 +    }
 +
 +    public static void cleanup()
 +    {
 +        // clean up commitlog
 +        String[] directoryNames = { DatabaseDescriptor.getCommitLogLocation(), };
 +        for (String dirName : directoryNames)
 +        {
 +            File dir = new File(dirName);
 +            if (!dir.exists())
 +                throw new RuntimeException("No such directory: " + dir.getAbsolutePath());
 +            FileUtils.deleteRecursive(dir);
 +        }
 +
 +        cleanupSavedCaches();
 +
 +        // clean up data directory which are stored as data directory/keyspace/data files
 +        for (String dirName : DatabaseDescriptor.getAllDataFileLocations())
 +        {
 +            File dir = new File(dirName);
 +            if (!dir.exists())
 +                throw new RuntimeException("No such directory: " + dir.getAbsolutePath());
 +            FileUtils.deleteRecursive(dir);
 +        }
 +    }
 +
 +    public static void mkdirs()
 +    {
 +        DatabaseDescriptor.createAllDirectories();
 +    }
 +
 +    public static void cleanupSavedCaches()
 +    {
 +        File cachesDir = new File(DatabaseDescriptor.getSavedCachesLocation());
 +
 +        if (!cachesDir.exists() || !cachesDir.isDirectory())
 +            return;
 +
 +        FileUtils.delete(cachesDir.listFiles());
 +    }
  
      @BeforeClass
      public static void setUpClass()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 557beba,5e2fffe..e6d18c4
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@@ -32,9 -32,9 +32,10 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.Util;
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.Config;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.config.Schema;
  import org.apache.cassandra.cql3.QueryProcessor;
  import org.apache.cassandra.cql3.UntypedResultSet;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 5ac1b31,bc5be46..37b5fa9
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@@ -28,7 -27,7 +28,8 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.exceptions.ConfigurationException;
  import org.apache.cassandra.gms.ApplicationState;
  import org.apache.cassandra.gms.Gossiper;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
index 6f76db4,0000000..a2c9cf9
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
@@@ -1,56 -1,0 +1,63 @@@
 +/*
 + * 
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + * 
 + *   http://www.apache.org/licenses/LICENSE-2.0
 + * 
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + * 
 + */
 +package org.apache.cassandra.metrics;
 +
 +import java.net.InetAddress;
 +import java.util.Map;
 +import java.util.UUID;
 +
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import com.google.common.collect.Iterators;
 +
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.UntypedResultSet;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.db.marshal.Int32Type;
 +import org.apache.cassandra.db.marshal.UUIDType;
 +import org.apache.cassandra.hints.HintsService;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 +
 +public class HintedHandOffMetricsTest
 +{
++    @BeforeClass
++    public static void initDD()
++    {
++        DatabaseDescriptor.setDaemonInitialized();
++    }
++
 +    @Test
 +    public void testHintsMetrics() throws Exception
 +    {
 +        DatabaseDescriptor.getHintsDirectory().mkdirs();
 +
 +        for (int i = 0; i < 99; i++)
 +            HintsService.instance.metrics.incrPastWindow(InetAddress.getLocalHost());
 +        HintsService.instance.metrics.log();
 +
 +        UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS);
 +        Map<UUID, Integer> returned = rows.one().getMap("hints_dropped", UUIDType.instance, Int32Type.instance);
 +        assertEquals(Iterators.getLast(returned.values().iterator()).intValue(), 99);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
index f0c850d,4e160c2..bb2b9b0
--- a/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
+++ b/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
@@@ -18,14 -18,14 +18,21 @@@
  */
  package org.apache.cassandra.utils.concurrent;
  
++import org.junit.BeforeClass;
  import org.junit.Ignore;
  import org.junit.Test;
  
  import junit.framework.Assert;
++import org.apache.cassandra.config.DatabaseDescriptor;
  
  @Ignore
  public abstract class AbstractTransactionalTest
  {
++    @BeforeClass
++    public static void setupDD()
++    {
++        DatabaseDescriptor.setDaemonInitialized();
++    }
  
      protected abstract TestableTransaction newTest() throws Exception;
  


[16/19] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by yu...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/66e21459
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/66e21459
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/66e21459

Branch: refs/heads/cassandra-3.11
Commit: 66e214592875e296bb540a966f1648f1106b2464
Parents: 1cafc3c 0fe82be
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Dec 13 15:59:14 2016 -0800
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:59:14 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++-
 .../cassandra/db/PartitionRangeReadCommand.java |  4 +-
 .../cassandra/db/compaction/CompactionTask.java | 81 ++++++++++----------
 .../cassandra/db/lifecycle/LogTransaction.java  |  3 +-
 .../apache/cassandra/db/lifecycle/Tracker.java  | 34 ++++----
 .../cassandra/index/SecondaryIndexManager.java  |  4 +-
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  6 +-
 .../org/apache/cassandra/tools/ToolsTester.java |  3 -
 10 files changed, 85 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 28ebf36,145afb9..f95dd81
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -168,12 -59,6 +168,13 @@@ Merged from 3.0
   * Correct log message for statistics of offheap memtable flush (CASSANDRA-12776)
   * Explicitly set locale for string validation (CASSANDRA-12541,CASSANDRA-12542,CASSANDRA-12543,CASSANDRA-12545)
  Merged from 2.2:
++ * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
 + * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
 + * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
 + * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
 + * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
 + * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
   * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
   * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
   * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 881fb00,39ed804..a5f76bd4
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -405,9 -388,13 +405,13 @@@ public class ColumnFamilyStore implemen
  
          logger.info("Initializing {}.{}", keyspace.getName(), name);
  
-         // scan for sstables corresponding to this cf and load them
-         data = new Tracker(this, loadSSTables);
+         // Create Memtable only on online
+         Memtable initialMemtable = null;
+         if (DatabaseDescriptor.isDaemonInitialized())
 -            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
++            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getCurrentPosition()), this);
+         data = new Tracker(initialMemtable, loadSSTables);
  
+         // scan for sstables corresponding to this cf and load them
          if (data.loadsstables)
          {
              Directories.SSTableLister sstableFiles = directories.sstableLister(Directories.OnTxnErr.IGNORE).skipTemporary(true);
@@@ -2118,7 -1957,7 +2122,7 @@@
              {
                  public Void call()
                  {
-                     cfs.data.reset();
 -                    cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
++                    cfs.data.reset(new Memtable(new AtomicReference<>(CommitLogPosition.NONE), cfs));
                      return null;
                  }
              }, true, false);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 50b568e,17adef0..045fc26
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -221,7 -199,8 +221,9 @@@ public class PartitionRangeReadCommand 
                  if (!sstable.isRepaired())
                      oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime());
              }
-             return checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
++            // iterators can be empty for offline tools
+             return iterators.isEmpty() ? EmptyIterators.unfilteredPartition(metadata(), isForThrift())
+                                        : checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
          }
          catch (RuntimeException | Error e)
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 2f90c7b,f0a1f47..a9d6c7c
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@@ -228,47 -213,29 +228,50 @@@ public class CompactionTask extends Abs
                  }
              }
  
--            // log a bunch of statistics about the result and save to system table compaction_history
- 
-             long durationInNano = System.nanoTime() - start;
-             long dTime = TimeUnit.NANOSECONDS.toMillis(durationInNano);
-             long startsize = inputSizeBytes;
 -            long dTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
 -            long startsize = SSTableReader.getTotalBytes(transaction.originals());
--            long endsize = SSTableReader.getTotalBytes(newSStables);
--            double ratio = (double) endsize / (double) startsize;
--
--            StringBuilder newSSTableNames = new StringBuilder();
--            for (SSTableReader reader : newSStables)
--                newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
- 
-             long totalSourceRows = 0;
-             for (int i = 0; i < mergedRowCounts.length; i++)
-                 totalSourceRows += mergedRowCounts[i] * (i + 1);
- 
-             String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
-             logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %s to %s (~%d%% of original) in %,dms.  Read Throughput = %s, Write Throughput = %s, Row Throughput = ~%,d/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                       taskId,
-                                       transaction.originals().size(),
-                                       newSSTableNames.toString(),
-                                       getLevel(),
-                                       FBUtilities.prettyPrintMemory(startsize),
-                                       FBUtilities.prettyPrintMemory(endsize),
-                                       (int) (ratio * 100),
-                                       dTime,
-                                       FBUtilities.prettyPrintMemoryPerSecond(startsize, durationInNano),
-                                       FBUtilities.prettyPrintMemoryPerSecond(endsize, durationInNano),
-                                       (int) totalSourceCQLRows / (TimeUnit.NANOSECONDS.toSeconds(durationInNano) + 1),
-                                       totalSourceRows,
-                                       totalKeysWritten,
-                                       mergeSummary));
-             logger.trace("CF Total Bytes Compacted: {}", FBUtilities.prettyPrintMemory(CompactionTask.addToTotalBytesCompacted(endsize)));
-             logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
-             cfs.getCompactionStrategyManager().compactionLogger.compaction(startTime, transaction.originals(), System.currentTimeMillis(), newSStables);
- 
-             // update the metrics
-             cfs.metric.compactionBytesWritten.inc(endsize);
--
 -            if (offline)
 +            if (transaction.isOffline())
+             {
                  Refs.release(Refs.selfRefs(newSStables));
+             }
+             else
+             {
 -                double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
 -                Summary mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
 -                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
 -                                           taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, mergeSummary.totalSourceRows, totalKeysWritten, mergeSummary.partitionMerge));
 -                logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
++                // log a bunch of statistics about the result and save to system table compaction_history
++
++                long durationInNano = System.nanoTime() - start;
++                long dTime = TimeUnit.NANOSECONDS.toMillis(durationInNano);
++                long startsize = inputSizeBytes;
++                long endsize = SSTableReader.getTotalBytes(newSStables);
++                double ratio = (double) endsize / (double) startsize;
++
++                StringBuilder newSSTableNames = new StringBuilder();
++                for (SSTableReader reader : newSStables)
++                    newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
++                long totalSourceRows = 0;
++                for (int i = 0; i < mergedRowCounts.length; i++)
++                    totalSourceRows += mergedRowCounts[i] * (i + 1);
++
++                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getTableName(), mergedRowCounts, startsize, endsize);
++                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %s to %s (~%d%% of original) in %,dms.  Read Throughput = %s, Write Throughput = %s, Row Throughput = ~%,d/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
++                                           taskId,
++                                           transaction.originals().size(),
++                                           newSSTableNames.toString(),
++                                           getLevel(),
++                                           FBUtilities.prettyPrintMemory(startsize),
++                                           FBUtilities.prettyPrintMemory(endsize),
++                                           (int) (ratio * 100),
++                                           dTime,
++                                           FBUtilities.prettyPrintMemoryPerSecond(startsize, durationInNano),
++                                           FBUtilities.prettyPrintMemoryPerSecond(endsize, durationInNano),
++                                           (int) totalSourceCQLRows / (TimeUnit.NANOSECONDS.toSeconds(durationInNano) + 1),
++                                           totalSourceRows,
++                                           totalKeysWritten,
++                                           mergeSummary));
++                logger.trace("CF Total Bytes Compacted: {}", FBUtilities.prettyPrintMemory(CompactionTask.addToTotalBytesCompacted(endsize)));
+                 logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
++                cfs.getCompactionStrategyManager().compactionLogger.compaction(startTime, transaction.originals(), System.currentTimeMillis(), newSStables);
++
++                // update the metrics
++                cfs.metric.compactionBytesWritten.inc(endsize);
+             }
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index f464e08,9feaa3e..e2fcb06
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@@ -31,8 -31,7 +31,7 @@@ import com.google.common.collect.*
  import org.apache.cassandra.db.ColumnFamilyStore;
  import org.apache.cassandra.db.Directories;
  import org.apache.cassandra.db.Memtable;
- import org.apache.cassandra.db.commitlog.CommitLog;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
index 6435e3e,0d87cc9..4514b72
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@@ -30,6 -31,8 +31,8 @@@ import junit.framework.Assert
  import org.apache.cassandra.MockSchema;
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.Memtable;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
++import org.apache.cassandra.db.commitlog.CommitLogPosition;
  import org.apache.cassandra.db.compaction.OperationType;
  import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
  import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
@@@ -268,7 -271,7 +271,7 @@@ public class LifecycleTransactionTest e
  
          private static Tracker tracker(ColumnFamilyStore cfs, List<SSTableReader> readers)
          {
-             Tracker tracker = new Tracker(cfs, false);
 -            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs), false);
++            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(CommitLogPosition.NONE), cfs), false);
              tracker.addInitialSSTables(readers);
              return tracker;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/test/unit/org/apache/cassandra/tools/ToolsTester.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/tools/ToolsTester.java
index 97b19c9,0000000..ead4e31
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/tools/ToolsTester.java
+++ b/test/unit/org/apache/cassandra/tools/ToolsTester.java
@@@ -1,296 -1,0 +1,293 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.cassandra.tools;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.lang.management.ManagementFactory;
 +import java.lang.management.ThreadInfo;
 +import java.lang.management.ThreadMXBean;
 +import java.lang.reflect.InvocationTargetException;
 +import java.lang.reflect.Method;
 +import java.security.Permission;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Set;
 +import java.util.regex.Pattern;
 +import java.util.stream.Collectors;
 +
 +import org.apache.commons.io.FileUtils;
 +import org.junit.BeforeClass;
 +
 +import org.slf4j.LoggerFactory;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.fail;
 +
 +/**
 + * Base unit test class for standalone tools
 + */
 +public abstract class ToolsTester
 +{
 +    private static List<ThreadInfo> initialThreads;
 +
 +    static final String[] EXPECTED_THREADS_WITH_SCHEMA = {
-     "(NativePool|SlabPool|HeapPool)Cleaner",
-     "COMMIT-LOG-ALLOCATOR",
-     "COMMIT-LOG-WRITER",
 +    "PerDiskMemtableFlushWriter_0:[1-9]",
 +    "MemtablePostFlush:[1-9]",
 +    "MemtableFlushWriter:[1-9]",
 +    "MemtableReclaimMemory:[1-9]",
 +    };
 +    static final String[] OPTIONAL_THREADS_WITH_SCHEMA = {
 +    "ScheduledTasks:[1-9]",
 +    "OptionalTasks:[1-9]",
 +    "Reference-Reaper:[1-9]",
 +    "LocalPool-Cleaner:[1-9]",
 +    "CacheCleanupExecutor:[1-9]",
 +    "CompactionExecutor:[1-9]",
 +    "ValidationExecutor:[1-9]",
 +    "NonPeriodicTasks:[1-9]",
 +    "Sampler:[1-9]",
 +    "SecondaryIndexManagement:[1-9]",
 +    "Strong-Reference-Leak-Detector:[1-9]",
 +    "Background_Reporter:[1-9]",
 +    "EXPIRING-MAP-REAPER:[1-9]",
 +    };
 +
 +    public void assertNoUnexpectedThreadsStarted(String[] expectedThreadNames, String[] optionalThreadNames)
 +    {
 +        ThreadMXBean threads = ManagementFactory.getThreadMXBean();
 +
 +        Set<String> initial = initialThreads
 +                              .stream()
 +                              .map(ThreadInfo::getThreadName)
 +                              .collect(Collectors.toSet());
 +
 +        Set<String> current = Arrays.stream(threads.getThreadInfo(threads.getAllThreadIds()))
 +                                    .map(ThreadInfo::getThreadName)
 +                                    .collect(Collectors.toSet());
 +
 +        List<Pattern> expected = expectedThreadNames != null
 +                                 ? Arrays.stream(expectedThreadNames).map(Pattern::compile).collect(Collectors.toList())
 +                                 : Collections.emptyList();
 +
 +        List<Pattern> optional = optionalThreadNames != null
 +                                 ? Arrays.stream(optionalThreadNames).map(Pattern::compile).collect(Collectors.toList())
 +                                 : Collections.emptyList();
 +
 +        current.removeAll(initial);
 +
 +        List<Pattern> notPresent = expected.stream()
 +                                           .filter(threadNamePattern -> !current.stream().anyMatch(threadName -> threadNamePattern.matcher(threadName).matches()))
 +                                           .collect(Collectors.toList());
 +
 +        Set<String> remain = current.stream()
 +                                    .filter(threadName -> expected.stream().anyMatch(pattern -> pattern.matcher(threadName).matches()))
 +                                    .filter(threadName -> optional.stream().anyMatch(pattern -> pattern.matcher(threadName).matches()))
 +                                    .collect(Collectors.toSet());
 +
 +        if (!current.isEmpty())
 +            System.err.println("Unexpected thread names: " + remain);
 +        if (!notPresent.isEmpty())
 +            System.err.println("Mandatory thread missing: " + notPresent);
 +
 +        assertTrue("Wrong thread status", remain.isEmpty() && notPresent.isEmpty());
 +    }
 +
 +    public void assertSchemaNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.config.Schema");
 +    }
 +
 +    public void assertSchemaLoaded()
 +    {
 +        assertClassLoaded("org.apache.cassandra.config.Schema");
 +    }
 +
 +    public void assertKeyspaceNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.db.Keyspace");
 +    }
 +
 +    public void assertKeyspaceLoaded()
 +    {
 +        assertClassLoaded("org.apache.cassandra.db.Keyspace");
 +    }
 +
 +    public void assertServerNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.transport.Server");
 +    }
 +
 +    public void assertSystemKSNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.db.SystemKeyspace");
 +    }
 +
 +    public void assertCLSMNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.db.commitlog.CommitLogSegmentManager");
 +    }
 +
 +    public void assertClassLoaded(String clazz)
 +    {
 +        assertClassLoadedStatus(clazz, true);
 +    }
 +
 +    public void assertClassNotLoaded(String clazz)
 +    {
 +        assertClassLoadedStatus(clazz, false);
 +    }
 +
 +    private void assertClassLoadedStatus(String clazz, boolean expected)
 +    {
 +        for (ClassLoader cl = Thread.currentThread().getContextClassLoader(); cl != null; cl = cl.getParent())
 +        {
 +            try
 +            {
 +                Method mFindLoadedClass = ClassLoader.class.getDeclaredMethod("findLoadedClass", String.class);
 +                mFindLoadedClass.setAccessible(true);
 +                boolean loaded = mFindLoadedClass.invoke(cl, clazz) != null;
 +
 +                if (expected)
 +                {
 +                    if (loaded)
 +                        return;
 +                }
 +                else
 +                    assertFalse(clazz + " has been loaded", loaded);
 +            }
 +            catch (Exception e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +        }
 +
 +        if (expected)
 +            fail(clazz + " has not been loaded");
 +    }
 +
 +    public void runTool(int expectedExitCode, String clazz, String... args)
 +    {
 +        try
 +        {
 +            // install security manager to get informed about the exit-code
 +            System.setSecurityManager(new SecurityManager()
 +            {
 +                public void checkExit(int status)
 +                {
 +                    throw new SystemExitException(status);
 +                }
 +
 +                public void checkPermission(Permission perm)
 +                {
 +                }
 +
 +                public void checkPermission(Permission perm, Object context)
 +                {
 +                }
 +            });
 +
 +            try
 +            {
 +                Class.forName(clazz).getDeclaredMethod("main", String[].class).invoke(null, (Object) args);
 +            }
 +            catch (InvocationTargetException e)
 +            {
 +                Throwable cause = e.getCause();
 +                if (cause instanceof Error)
 +                    throw (Error) cause;
 +                if (cause instanceof RuntimeException)
 +                    throw (RuntimeException) cause;
 +                throw e;
 +            }
 +
 +            assertEquals("Unexpected exit code", expectedExitCode, 0);
 +        }
 +        catch (SystemExitException e)
 +        {
 +            assertEquals("Unexpected exit code", expectedExitCode, e.status);
 +        }
 +        catch (InvocationTargetException e)
 +        {
 +            throw new RuntimeException(e.getTargetException());
 +        }
 +        catch (Exception e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +        finally
 +        {
 +            // uninstall security manager
 +            System.setSecurityManager(null);
 +        }
 +    }
 +
 +    @BeforeClass
 +    public static void setupTester()
 +    {
 +        System.setProperty("cassandra.partitioner", "org.apache.cassandra.dht.Murmur3Partitioner");
 +
 +        // may start an async appender
 +        LoggerFactory.getLogger(ToolsTester.class);
 +
 +        ThreadMXBean threads = ManagementFactory.getThreadMXBean();
 +        initialThreads = Arrays.asList(threads.getThreadInfo(threads.getAllThreadIds()));
 +    }
 +
 +    public static class SystemExitException extends Error
 +    {
 +        public final int status;
 +
 +        public SystemExitException(int status)
 +        {
 +            this.status = status;
 +        }
 +    }
 +
 +    public static String findOneSSTable(String ks, String cf) throws IOException
 +    {
 +        File cfDir = sstableDir(ks, cf);
 +        File[] sstableFiles = cfDir.listFiles((file) -> file.isFile() && file.getName().endsWith("-Data.db"));
 +        return sstableFiles[0].getAbsolutePath();
 +    }
 +
 +    public static String sstableDirName(String ks, String cf) throws IOException
 +    {
 +        return sstableDir(ks, cf).getAbsolutePath();
 +    }
 +
 +    public static File sstableDir(String ks, String cf) throws IOException
 +    {
 +        File dataDir = copySSTables();
 +        File ksDir = new File(dataDir, ks);
 +        File[] cfDirs = ksDir.listFiles((dir, name) -> cf.equals(name) || name.startsWith(cf + '-'));
 +        return cfDirs[0];
 +    }
 +
 +    public static File copySSTables() throws IOException
 +    {
 +        File dataDir = new File("build/test/cassandra/data");
 +        File srcDir = new File("test/data/legacy-sstables/ma");
 +        FileUtils.copyDirectory(new File(srcDir, "legacy_tables"), new File(dataDir, "legacy_sstables"));
 +        return dataDir;
 +    }
 +}


[19/19] cassandra git commit: Merge branch 'cassandra-3.X' into trunk

Posted by yu...@apache.org.
Merge branch 'cassandra-3.X' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/402f8e00
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/402f8e00
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/402f8e00

Branch: refs/heads/trunk
Commit: 402f8e00606a54dcac46c60463358501a6a2e686
Parents: cd5aacc c9ead48
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Dec 13 16:27:05 2016 -0800
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 16:27:05 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++-
 .../cassandra/db/PartitionRangeReadCommand.java |  4 +-
 .../cassandra/db/compaction/CompactionTask.java | 81 ++++++++++----------
 .../cassandra/db/lifecycle/LogTransaction.java  |  3 +-
 .../apache/cassandra/db/lifecycle/Tracker.java  | 34 ++++----
 .../cassandra/index/SecondaryIndexManager.java  |  4 +-
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  6 +-
 .../org/apache/cassandra/tools/ToolsTester.java |  3 -
 10 files changed, 85 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/402f8e00/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/402f8e00/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/402f8e00/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 9477aeb,045fc26..df4d63c
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -219,7 -221,9 +219,9 @@@ public class PartitionRangeReadCommand 
                  if (!sstable.isRepaired())
                      oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime());
              }
-             return checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
+             // iterators can be empty for offline tools
 -            return iterators.isEmpty() ? EmptyIterators.unfilteredPartition(metadata(), isForThrift())
++            return iterators.isEmpty() ? EmptyIterators.unfilteredPartition(metadata())
+                                        : checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
          }
          catch (RuntimeException | Error e)
          {


[09/19] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by yu...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0fe82be8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0fe82be8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0fe82be8

Branch: refs/heads/cassandra-3.0
Commit: 0fe82be83cceceb12172d63913388678253413bc
Parents: e9b7a0f 66f1aaf
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Dec 13 15:55:34 2016 -0800
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:55:34 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/config/DatabaseDescriptor.java    | 12 +++++++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++++--
 .../cassandra/db/PartitionRangeReadCommand.java |  3 +-
 .../cassandra/db/compaction/CompactionTask.java | 18 +++++++----
 .../cassandra/db/lifecycle/LogTransaction.java  |  3 +-
 .../apache/cassandra/db/lifecycle/Tracker.java  | 34 ++++++++++++--------
 .../cassandra/index/SecondaryIndexManager.java  |  4 ++-
 .../io/sstable/format/SSTableReader.java        |  2 +-
 .../cassandra/service/CassandraDaemon.java      |  1 +
 .../service/EmbeddedCassandraService.java       |  2 ++
 .../config/DatabaseDescriptorTest.java          |  6 ++++
 .../org/apache/cassandra/cql3/CQLTester.java    |  1 +
 .../apache/cassandra/db/SystemKeyspaceTest.java |  2 ++
 .../db/context/CounterContextTest.java          |  8 +++++
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 ++-
 .../cassandra/db/lifecycle/TrackerTest.java     |  7 ++--
 .../cassandra/dht/StreamStateStoreTest.java     |  7 ++++
 .../cassandra/gms/FailureDetectorTest.java      |  2 ++
 .../org/apache/cassandra/gms/GossiperTest.java  |  5 +++
 .../io/sstable/CQLSSTableWriterTest.java        |  2 ++
 .../cassandra/locator/CloudstackSnitchTest.java |  2 ++
 .../apache/cassandra/locator/EC2SnitchTest.java |  2 ++
 .../locator/GoogleCloudSnitchTest.java          |  2 ++
 .../metrics/HintedHandOffMetricsTest.java       |  7 ++++
 .../service/StorageServiceServerTest.java       |  1 +
 .../concurrent/AbstractTransactionalTest.java   |  7 ++++
 27 files changed, 124 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 5621c93,8cff097..145afb9
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,18 -1,5 +1,19 @@@
 -2.2.9
 +3.0.11
 + * Mark MVs as built after successful bootstrap (CASSANDRA-12984)
 + * Estimated TS drop-time histogram updated with Cell.NO_DELETION_TIME (CASSANDRA-13040)
 + * Nodetool compactionstats fails with NullPointerException (CASSANDRA-13021)
 + * Thread local pools never cleaned up (CASSANDRA-13033)
 + * Set RPC_READY to false when draining or if a node is marked as shutdown (CASSANDRA-12781)
 + * CQL often queries static columns unnecessarily (CASSANDRA-12768)
 + * Make sure sstables only get committed when it's safe to discard commit log records (CASSANDRA-12956)
 + * Reject default_time_to_live option when creating or altering MVs (CASSANDRA-12868)
 + * Nodetool should use a more sane max heap size (CASSANDRA-12739)
 + * LocalToken ensures token values are cloned on heap (CASSANDRA-12651)
 + * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
 + * Prevent reloading of logback.xml from UDF sandbox (CASSANDRA-12535)
 + * Reenable HeapPool (CASSANDRA-12900)
 +Merged from 2.2:
+  * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
   * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
   * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
   * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 71e1653,4bc46d0..39ed804
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -388,13 -388,17 +388,17 @@@ public class ColumnFamilyStore implemen
  
          logger.info("Initializing {}.{}", keyspace.getName(), name);
  
-         // scan for sstables corresponding to this cf and load them
-         data = new Tracker(this, loadSSTables);
+         // Create Memtable only on online
+         Memtable initialMemtable = null;
+         if (DatabaseDescriptor.isDaemonInitialized())
+             initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
+         data = new Tracker(initialMemtable, loadSSTables);
  
+         // scan for sstables corresponding to this cf and load them
          if (data.loadsstables)
          {
 -            Directories.SSTableLister sstableFiles = directories.sstableLister().skipTemporary(true);
 -            Collection<SSTableReader> sstables = SSTableReader.openAll(sstableFiles.list().entrySet(), metadata, this.partitioner);
 +            Directories.SSTableLister sstableFiles = directories.sstableLister(Directories.OnTxnErr.IGNORE).skipTemporary(true);
 +            Collection<SSTableReader> sstables = SSTableReader.openAll(sstableFiles.list().entrySet(), metadata);
              data.addInitialSSTables(sstables);
          }
  
@@@ -1953,10 -2758,12 +1957,10 @@@
              {
                  public Void call()
                  {
-                     cfs.data.reset();
+                     cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
 -                    cfs.getCompactionStrategy().shutdown();
 -                    cfs.getCompactionStrategy().startup();
                      return null;
                  }
 -            }, true);
 +            }, true, false);
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 99e24c8,0000000..17adef0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -1,322 -1,0 +1,323 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.db;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.List;
 +import java.util.Optional;
 +
 +import com.google.common.collect.Iterables;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.db.filter.*;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.lifecycle.View;
 +import org.apache.cassandra.db.partitions.*;
 +import org.apache.cassandra.db.rows.BaseRowIterator;
 +import org.apache.cassandra.db.transform.Transformation;
 +import org.apache.cassandra.dht.AbstractBounds;
 +import org.apache.cassandra.exceptions.RequestExecutionException;
 +import org.apache.cassandra.index.Index;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.metrics.TableMetrics;
 +import org.apache.cassandra.net.MessageOut;
 +import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.service.StorageProxy;
 +import org.apache.cassandra.service.pager.*;
 +import org.apache.cassandra.thrift.ThriftResultsMerger;
 +import org.apache.cassandra.tracing.Tracing;
 +import org.apache.cassandra.utils.FBUtilities;
 +
 +/**
 + * A read command that selects a (part of a) range of partitions.
 + */
 +public class PartitionRangeReadCommand extends ReadCommand
 +{
 +    protected static final SelectionDeserializer selectionDeserializer = new Deserializer();
 +
 +    private final DataRange dataRange;
 +    private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
 +
 +    public PartitionRangeReadCommand(boolean isDigest,
 +                                     int digestVersion,
 +                                     boolean isForThrift,
 +                                     CFMetaData metadata,
 +                                     int nowInSec,
 +                                     ColumnFilter columnFilter,
 +                                     RowFilter rowFilter,
 +                                     DataLimits limits,
 +                                     DataRange dataRange,
 +                                     Optional<IndexMetadata> index)
 +    {
 +        super(Kind.PARTITION_RANGE, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits);
 +        this.dataRange = dataRange;
 +        this.index = index;
 +    }
 +
 +    public PartitionRangeReadCommand(CFMetaData metadata,
 +                                     int nowInSec,
 +                                     ColumnFilter columnFilter,
 +                                     RowFilter rowFilter,
 +                                     DataLimits limits,
 +                                     DataRange dataRange,
 +                                     Optional<IndexMetadata> index)
 +    {
 +        this(false, 0, false, metadata, nowInSec, columnFilter, rowFilter, limits, dataRange, index);
 +    }
 +
 +    /**
 +     * Creates a new read command that query all the data in the table.
 +     *
 +     * @param metadata the table to query.
 +     * @param nowInSec the time in seconds to use are "now" for this query.
 +     *
 +     * @return a newly created read command that queries everything in the table.
 +     */
 +    public static PartitionRangeReadCommand allDataRead(CFMetaData metadata, int nowInSec)
 +    {
 +        return new PartitionRangeReadCommand(metadata,
 +                                             nowInSec,
 +                                             ColumnFilter.all(metadata),
 +                                             RowFilter.NONE,
 +                                             DataLimits.NONE,
 +                                             DataRange.allData(metadata.partitioner),
 +                                             Optional.empty());
 +    }
 +
 +    public DataRange dataRange()
 +    {
 +        return dataRange;
 +    }
 +
 +    public ClusteringIndexFilter clusteringIndexFilter(DecoratedKey key)
 +    {
 +        return dataRange.clusteringIndexFilter(key);
 +    }
 +
 +    public boolean isNamesQuery()
 +    {
 +        return dataRange.isNamesQuery();
 +    }
 +
 +    public PartitionRangeReadCommand forSubRange(AbstractBounds<PartitionPosition> range)
 +    {
 +        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange().forSubRange(range), index);
 +    }
 +
 +    public PartitionRangeReadCommand copy()
 +    {
 +        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange(), index);
 +    }
 +
 +    public PartitionRangeReadCommand withUpdatedLimit(DataLimits newLimits)
 +    {
 +        return new PartitionRangeReadCommand(metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, dataRange(), index);
 +    }
 +
 +    public long getTimeout()
 +    {
 +        return DatabaseDescriptor.getRangeRpcTimeout();
 +    }
 +
 +    public boolean selectsKey(DecoratedKey key)
 +    {
 +        if (!dataRange().contains(key))
 +            return false;
 +
 +        return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().getKeyValidator());
 +    }
 +
 +    public boolean selectsClustering(DecoratedKey key, Clustering clustering)
 +    {
 +        if (clustering == Clustering.STATIC_CLUSTERING)
 +            return !columnFilter().fetchedColumns().statics.isEmpty();
 +
 +        if (!dataRange().clusteringIndexFilter(key).selects(clustering))
 +            return false;
 +        return rowFilter().clusteringKeyRestrictionsAreSatisfiedBy(clustering);
 +    }
 +
 +    public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState) throws RequestExecutionException
 +    {
 +        return StorageProxy.getRangeSlice(this, consistency);
 +    }
 +
 +    public QueryPager getPager(PagingState pagingState, int protocolVersion)
 +    {
 +            return new PartitionRangeQueryPager(this, pagingState, protocolVersion);
 +    }
 +
 +    protected void recordLatency(TableMetrics metric, long latencyNanos)
 +    {
 +        metric.rangeLatency.addNano(latencyNanos);
 +    }
 +
 +    protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
 +    {
 +        ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
 +        Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));
 +
 +        // fetch data from current memtable, historical memtables, and SSTables in the correct order.
 +        final List<UnfilteredPartitionIterator> iterators = new ArrayList<>(Iterables.size(view.memtables) + view.sstables.size());
 +
 +        try
 +        {
 +            for (Memtable memtable : view.memtables)
 +            {
 +                @SuppressWarnings("resource") // We close on exception and on closing the result returned by this method
 +                Memtable.MemtableUnfilteredPartitionIterator iter = memtable.makePartitionIterator(columnFilter(), dataRange(), isForThrift());
 +                oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, iter.getMinLocalDeletionTime());
 +                iterators.add(isForThrift() ? ThriftResultsMerger.maybeWrap(iter, metadata(), nowInSec()) : iter);
 +            }
 +
 +            for (SSTableReader sstable : view.sstables)
 +            {
 +                @SuppressWarnings("resource") // We close on exception and on closing the result returned by this method
 +                UnfilteredPartitionIterator iter = sstable.getScanner(columnFilter(), dataRange(), isForThrift());
 +                iterators.add(isForThrift() ? ThriftResultsMerger.maybeWrap(iter, metadata(), nowInSec()) : iter);
 +                if (!sstable.isRepaired())
 +                    oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime());
 +            }
-             return checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
++            return iterators.isEmpty() ? EmptyIterators.unfilteredPartition(metadata(), isForThrift())
++                                       : checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
 +        }
 +        catch (RuntimeException | Error e)
 +        {
 +            try
 +            {
 +                FBUtilities.closeAll(iterators);
 +            }
 +            catch (Exception suppressed)
 +            {
 +                e.addSuppressed(suppressed);
 +            }
 +
 +            throw e;
 +        }
 +    }
 +
 +    @Override
 +    protected int oldestUnrepairedTombstone()
 +    {
 +        return oldestUnrepairedTombstone;
 +    }
 +
 +    private UnfilteredPartitionIterator checkCacheFilter(UnfilteredPartitionIterator iter, final ColumnFamilyStore cfs)
 +    {
 +        class CacheFilter extends Transformation
 +        {
 +            @Override
 +            public BaseRowIterator applyToPartition(BaseRowIterator iter)
 +            {
 +                // Note that we rely on the fact that until we actually advance 'iter', no really costly operation is actually done
 +                // (except for reading the partition key from the index file) due to the call to mergeLazily in queryStorage.
 +                DecoratedKey dk = iter.partitionKey();
 +
 +                // Check if this partition is in the rowCache and if it is, if  it covers our filter
 +                CachedPartition cached = cfs.getRawCachedPartition(dk);
 +                ClusteringIndexFilter filter = dataRange().clusteringIndexFilter(dk);
 +
 +                if (cached != null && cfs.isFilterFullyCoveredBy(filter, limits(), cached, nowInSec()))
 +                {
 +                    // We won't use 'iter' so close it now.
 +                    iter.close();
 +
 +                    return filter.getUnfilteredRowIterator(columnFilter(), cached);
 +                }
 +
 +                return iter;
 +            }
 +        }
 +        return Transformation.apply(iter, new CacheFilter());
 +    }
 +
 +    public MessageOut<ReadCommand> createMessage(int version)
 +    {
 +        return dataRange().isPaging()
 +             ? new MessageOut<>(MessagingService.Verb.PAGED_RANGE, this, pagedRangeSerializer)
 +             : new MessageOut<>(MessagingService.Verb.RANGE_SLICE, this, rangeSliceSerializer);
 +    }
 +
 +    protected void appendCQLWhereClause(StringBuilder sb)
 +    {
 +        if (dataRange.isUnrestricted() && rowFilter().isEmpty())
 +            return;
 +
 +        sb.append(" WHERE ");
 +        // We put the row filter first because the data range can end by "ORDER BY"
 +        if (!rowFilter().isEmpty())
 +        {
 +            sb.append(rowFilter());
 +            if (!dataRange.isUnrestricted())
 +                sb.append(" AND ");
 +        }
 +        if (!dataRange.isUnrestricted())
 +            sb.append(dataRange.toCQLString(metadata()));
 +    }
 +
 +    /**
 +     * Allow to post-process the result of the query after it has been reconciled on the coordinator
 +     * but before it is passed to the CQL layer to return the ResultSet.
 +     *
 +     * See CASSANDRA-8717 for why this exists.
 +     */
 +    public PartitionIterator postReconciliationProcessing(PartitionIterator result)
 +    {
 +        ColumnFamilyStore cfs = Keyspace.open(metadata().ksName).getColumnFamilyStore(metadata().cfName);
 +        Index index = getIndex(cfs);
 +        return index == null ? result : index.postProcessorFor(this).apply(result, this);
 +    }
 +
 +    @Override
 +    public String toString()
 +    {
 +        return String.format("Read(%s.%s columns=%s rowfilter=%s limits=%s %s)",
 +                             metadata().ksName,
 +                             metadata().cfName,
 +                             columnFilter(),
 +                             rowFilter(),
 +                             limits(),
 +                             dataRange().toString(metadata()));
 +    }
 +
 +    protected void serializeSelection(DataOutputPlus out, int version) throws IOException
 +    {
 +        DataRange.serializer.serialize(dataRange(), out, version, metadata());
 +    }
 +
 +    protected long selectionSerializedSize(int version)
 +    {
 +        return DataRange.serializer.serializedSize(dataRange(), version, metadata());
 +    }
 +
 +    private static class Deserializer extends SelectionDeserializer
 +    {
 +        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
 +        throws IOException
 +        {
 +            DataRange range = DataRange.serializer.deserialize(in, version, metadata);
 +            return new PartitionRangeReadCommand(isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, range, index);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 0c4e144,20d3dc0..f0a1f47
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@@ -223,15 -218,20 +223,19 @@@ public class CompactionTask extends Abs
              for (SSTableReader reader : newSStables)
                  newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
  
-             double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
-             Summary mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
-             logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                       taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, mergeSummary.totalSourceRows, totalKeysWritten, mergeSummary.partitionMerge));
-             logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-             logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
- 
              if (offline)
+             {
                  Refs.release(Refs.selfRefs(newSStables));
+             }
+             else
+             {
+                 double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
 -                long totalSourceRows = 0;
 -                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
++                Summary mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
+                 logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
 -                                           taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
++                                           taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, mergeSummary.totalSourceRows, totalKeysWritten, mergeSummary.partitionMerge));
+                 logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
 -                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten));
++                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
+             }
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
index ca644eb,0000000..350477c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
@@@ -1,444 -1,0 +1,445 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.db.lifecycle;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.nio.file.Files;
 +import java.nio.file.NoSuchFileException;
 +import java.util.*;
 +import java.util.concurrent.ConcurrentLinkedQueue;
 +import java.util.concurrent.TimeUnit;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.util.concurrent.Runnables;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.ScheduledExecutors;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.db.ColumnFamilyStore;
 +import org.apache.cassandra.db.Directories;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.db.compaction.OperationType;
 +import org.apache.cassandra.db.lifecycle.LogRecord.Type;
 +import org.apache.cassandra.io.sstable.Component;
 +import org.apache.cassandra.io.sstable.Descriptor;
 +import org.apache.cassandra.io.sstable.SSTable;
 +import org.apache.cassandra.io.sstable.SnapshotDeletingTask;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.utils.*;
 +import org.apache.cassandra.utils.concurrent.Ref;
 +import org.apache.cassandra.utils.concurrent.RefCounted;
 +import org.apache.cassandra.utils.concurrent.Transactional;
 +
 +/**
 + * IMPORTANT: When this object is involved in a transactional graph, and is not encapsulated in a LifecycleTransaction,
 + * for correct behaviour its commit MUST occur before any others, since it may legitimately fail. This is consistent
 + * with the Transactional API, which permits one failing action to occur at the beginning of the commit phase, but also
 + * *requires* that the prepareToCommit() phase only take actions that can be rolled back.
 + *
 + * IMPORTANT: The transaction must complete (commit or abort) before any temporary files are deleted, even though the
 + * txn log file itself will not be deleted until all tracked files are deleted. This is required by FileLister to ensure
 + * a consistent disk state. LifecycleTransaction ensures this requirement, so this class should really never be used
 + * outside of LT. @see FileLister.classifyFiles(TransactionData txn)
 + *
 + * A class that tracks sstable files involved in a transaction across sstables:
 + * if the transaction succeeds the old files should be deleted and the new ones kept; vice-versa if it fails.
 + *
 + * The transaction log file contains new and old sstables as follows:
 + *
 + * add:[sstable-2][CRC]
 + * remove:[sstable-1,max_update_time,num files][CRC]
 + *
 + * where sstable-2 is a new sstable to be retained if the transaction succeeds and sstable-1 is an old sstable to be
 + * removed. CRC is an incremental CRC of the file content up to this point. For old sstable files we also log the
 + * last update time of all files for the sstable descriptor and a checksum of vital properties such as update times
 + * and file sizes.
 + *
 + * Upon commit we add a final line to the log file:
 + *
 + * commit:[commit_time][CRC]
 + *
 + * When the transaction log is cleaned-up by the TransactionTidier, which happens only after any old sstables have been
 + * osoleted, then any sstable files for old sstables are removed before deleting the transaction log if the transaction
 + * was committed, vice-versa if the transaction was aborted.
 + *
 + * On start-up we look for any transaction log files and repeat the cleanup process described above.
 + *
 + * See CASSANDRA-7066 for full details.
 + */
 +class LogTransaction extends Transactional.AbstractTransactional implements Transactional
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(LogTransaction.class);
 +
 +    /**
 +     * If the format of the lines in the transaction log is wrong or the checksum
 +     * does not match, then we throw this exception.
 +     */
 +    public static final class CorruptTransactionLogException extends RuntimeException
 +    {
 +        public final LogFile txnFile;
 +
 +        public CorruptTransactionLogException(String message, LogFile txnFile)
 +        {
 +            super(message);
 +            this.txnFile = txnFile;
 +        }
 +    }
 +
 +    private final Tracker tracker;
 +    private final LogFile txnFile;
 +    private final Ref<LogTransaction> selfRef;
 +    // Deleting sstables is tricky because the mmapping might not have been finalized yet,
 +    // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
 +    // Additionally, we need to make sure to delete the data file first, so on restart the others
 +    // will be recognized as GCable.
 +    private static final Queue<Runnable> failedDeletions = new ConcurrentLinkedQueue<>();
 +
 +    LogTransaction(OperationType opType)
 +    {
 +        this(opType, null);
 +    }
 +
 +    LogTransaction(OperationType opType, Tracker tracker)
 +    {
 +        this.tracker = tracker;
 +        this.txnFile = new LogFile(opType, UUIDGen.getTimeUUID());
 +        this.selfRef = new Ref<>(this, new TransactionTidier(txnFile));
 +
 +        if (logger.isTraceEnabled())
 +            logger.trace("Created transaction logs with id {}", txnFile.id());
 +    }
 +
 +    /**
 +     * Track a reader as new.
 +     **/
 +    void trackNew(SSTable table)
 +    {
 +        txnFile.add(Type.ADD, table);
 +    }
 +
 +    /**
 +     * Stop tracking a reader as new.
 +     */
 +    void untrackNew(SSTable table)
 +    {
 +        txnFile.remove(Type.ADD, table);
 +    }
 +
 +    /**
 +     * Schedule a reader for deletion as soon as it is fully unreferenced.
 +     */
 +    SSTableTidier obsoleted(SSTableReader reader)
 +    {
 +        if (txnFile.contains(Type.ADD, reader))
 +        {
 +            if (txnFile.contains(Type.REMOVE, reader))
 +                throw new IllegalArgumentException();
 +
 +            return new SSTableTidier(reader, true, this);
 +        }
 +
 +        txnFile.add(Type.REMOVE, reader);
 +
 +        if (tracker != null)
 +            tracker.notifyDeleting(reader);
 +
 +        return new SSTableTidier(reader, false, this);
 +    }
 +
 +    OperationType type()
 +    {
 +        return txnFile.type();
 +    }
 +
 +    UUID id()
 +    {
 +        return txnFile.id();
 +    }
 +
 +    @VisibleForTesting
 +    LogFile txnFile()
 +    {
 +        return txnFile;
 +    }
 +
 +    @VisibleForTesting
 +    List<File> logFiles()
 +    {
 +        return txnFile.getFiles();
 +    }
 +
 +    @VisibleForTesting
 +    List<String> logFilePaths()
 +    {
 +        return txnFile.getFilePaths();
 +    }
 +
 +    static void delete(File file)
 +    {
 +        try
 +        {
 +            if (logger.isTraceEnabled())
 +                logger.trace("Deleting {}", file);
 +
 +            Files.delete(file.toPath());
 +        }
 +        catch (NoSuchFileException e)
 +        {
 +            logger.error("Unable to delete {} as it does not exist", file);
 +        }
 +        catch (IOException e)
 +        {
 +            logger.error("Unable to delete {}", file, e);
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    /**
 +     * The transaction tidier.
 +     *
 +     * When the transaction reference is fully released we try to delete all the obsolete files
 +     * depending on the transaction result, as well as the transaction log file.
 +     */
 +    private static class TransactionTidier implements RefCounted.Tidy, Runnable
 +    {
 +        private final LogFile data;
 +
 +        TransactionTidier(LogFile data)
 +        {
 +            this.data = data;
 +        }
 +
 +        public void tidy() throws Exception
 +        {
 +            run();
 +        }
 +
 +        public String name()
 +        {
 +            return data.toString();
 +        }
 +
 +        public void run()
 +        {
 +            if (logger.isTraceEnabled())
 +                logger.trace("Removing files for transaction {}", name());
 +
 +            if (!data.completed())
 +            { // this happens if we forget to close a txn and the garbage collector closes it for us
 +                logger.error("{} was not completed, trying to abort it now", data);
 +                Throwable err = Throwables.perform((Throwable)null, data::abort);
 +                if (err != null)
 +                    logger.error("Failed to abort {}", data, err);
 +            }
 +
 +            Throwable err = data.removeUnfinishedLeftovers(null);
 +
 +            if (err != null)
 +            {
 +                logger.info("Failed deleting files for transaction {}, we'll retry after GC and on on server restart", name(), err);
 +                failedDeletions.add(this);
 +            }
 +            else
 +            {
 +                if (logger.isTraceEnabled())
 +                    logger.trace("Closing file transaction {}", name());
 +
 +                data.close();
 +            }
 +        }
 +    }
 +
 +    static class Obsoletion
 +    {
 +        final SSTableReader reader;
 +        final SSTableTidier tidier;
 +
 +        Obsoletion(SSTableReader reader, SSTableTidier tidier)
 +        {
 +            this.reader = reader;
 +            this.tidier = tidier;
 +        }
 +    }
 +
 +    /**
 +     * The SSTableReader tidier. When a reader is fully released and no longer referenced
 +     * by any one, we run this. It keeps a reference to the parent transaction and releases
 +     * it when done, so that the final transaction cleanup can run when all obsolete readers
 +     * are released.
 +     */
 +    public static class SSTableTidier implements Runnable
 +    {
 +        // must not retain a reference to the SSTableReader, else leak detection cannot kick in
 +        private final Descriptor desc;
 +        private final long sizeOnDisk;
 +        private final Tracker tracker;
 +        private final boolean wasNew;
 +        private final Ref<LogTransaction> parentRef;
 +
 +        public SSTableTidier(SSTableReader referent, boolean wasNew, LogTransaction parent)
 +        {
 +            this.desc = referent.descriptor;
 +            this.sizeOnDisk = referent.bytesOnDisk();
 +            this.tracker = parent.tracker;
 +            this.wasNew = wasNew;
 +            this.parentRef = parent.selfRef.tryRef();
 +        }
 +
 +        public void run()
 +        {
-             SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
++            if (tracker != null && !tracker.isDummy())
++                SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
 +
 +            try
 +            {
 +                // If we can't successfully delete the DATA component, set the task to be retried later: see TransactionTidier
 +                File datafile = new File(desc.filenameFor(Component.DATA));
 +
 +                delete(datafile);
 +                // let the remainder be cleaned up by delete
 +                SSTable.delete(desc, SSTable.discoverComponentsFor(desc));
 +            }
 +            catch (Throwable t)
 +            {
 +                logger.error("Failed deletion for {}, we'll retry after GC and on server restart", desc);
 +                failedDeletions.add(this);
 +                return;
 +            }
 +
 +            if (tracker != null && tracker.cfstore != null && !wasNew)
 +                tracker.cfstore.metric.totalDiskSpaceUsed.dec(sizeOnDisk);
 +
 +            // release the referent to the parent so that the all transaction files can be released
 +            parentRef.release();
 +        }
 +
 +        public void abort()
 +        {
 +            parentRef.release();
 +        }
 +    }
 +
 +
 +    static void rescheduleFailedDeletions()
 +    {
 +        Runnable task;
 +        while ( null != (task = failedDeletions.poll()))
 +            ScheduledExecutors.nonPeriodicTasks.submit(task);
 +
 +        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
 +        SnapshotDeletingTask.rescheduleFailedTasks();
 +    }
 +
 +    static void waitForDeletions()
 +    {
 +        FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(Runnables.doNothing(), 0, TimeUnit.MILLISECONDS));
 +    }
 +
 +    @VisibleForTesting
 +    Throwable complete(Throwable accumulate)
 +    {
 +        try
 +        {
 +            accumulate = selfRef.ensureReleased(accumulate);
 +            return accumulate;
 +        }
 +        catch (Throwable t)
 +        {
 +            logger.error("Failed to complete file transaction {}", id(), t);
 +            return Throwables.merge(accumulate, t);
 +        }
 +    }
 +
 +    protected Throwable doCommit(Throwable accumulate)
 +    {
 +        return complete(Throwables.perform(accumulate, txnFile::commit));
 +    }
 +
 +    protected Throwable doAbort(Throwable accumulate)
 +    {
 +        return complete(Throwables.perform(accumulate, txnFile::abort));
 +    }
 +
 +    protected void doPrepare() { }
 +
 +    /**
 +     * Called on startup to scan existing folders for any unfinished leftovers of
 +     * operations that were ongoing when the process exited. Also called by the standalone
 +     * sstableutil tool when the cleanup option is specified, @see StandaloneSSTableUtil.
 +     *
 +     */
 +    static void removeUnfinishedLeftovers(CFMetaData metadata)
 +    {
 +        removeUnfinishedLeftovers(new Directories(metadata, ColumnFamilyStore.getInitialDirectories()).getCFDirectories());
 +    }
 +
 +    @VisibleForTesting
 +    static void removeUnfinishedLeftovers(List<File> folders)
 +    {
 +        LogFilesByName logFiles = new LogFilesByName();
 +        folders.forEach(logFiles::list);
 +        logFiles.removeUnfinishedLeftovers();
 +    }
 +
 +    private static final class LogFilesByName
 +    {
 +        Map<String, List<File>> files = new HashMap<>();
 +
 +        void list(File folder)
 +        {
 +            Arrays.stream(folder.listFiles(LogFile::isLogFile)).forEach(this::add);
 +        }
 +
 +        void add(File file)
 +        {
 +            List<File> filesByName = files.get(file.getName());
 +            if (filesByName == null)
 +            {
 +                filesByName = new ArrayList<>();
 +                files.put(file.getName(), filesByName);
 +            }
 +
 +            filesByName.add(file);
 +        }
 +
 +        void removeUnfinishedLeftovers()
 +        {
 +            files.forEach(LogFilesByName::removeUnfinishedLeftovers);
 +        }
 +
 +        static void removeUnfinishedLeftovers(String name, List<File> logFiles)
 +        {
 +
 +            try(LogFile txn = LogFile.make(name, logFiles))
 +            {
 +                if (txn.verify())
 +                {
 +                    Throwable failure = txn.removeUnfinishedLeftovers(null);
 +                    if (failure != null)
 +                        logger.error("Failed to remove unfinished transaction leftovers for txn {}", txn, failure);
 +                }
 +                else
 +                {
 +                    logger.error("Unexpected disk state: failed to read transaction txn {}", txn);
 +                }
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index 5a3d524,e77ef78..9feaa3e
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@@ -193,15 -204,14 +201,13 @@@ public class Tracke
  
      /** (Re)initializes the tracker, purging all references. */
      @VisibleForTesting
-     public void reset()
+     public void reset(Memtable memtable)
      {
-         view.set(new View(
-                          !isDummy() ? ImmutableList.of(new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfstore))
-                                     : ImmutableList.<Memtable>of(),
-                          ImmutableList.<Memtable>of(),
-                          Collections.<SSTableReader, SSTableReader>emptyMap(),
-                          Collections.<SSTableReader, SSTableReader>emptyMap(),
-                          SSTableIntervalTree.empty()));
 -        view.set(new View(memtable != null ? singletonList(memtable) : Collections.<Memtable>emptyList(),
 -                          Collections.<Memtable>emptyList(),
 -                          Collections.<SSTableReader, SSTableReader>emptyMap(),
 -                          Collections.<SSTableReader>emptySet(),
 -                          Collections.<SSTableReader>emptySet(),
++        view.set(new View(memtable != null ? singletonList(memtable) : Collections.emptyList(),
++                          Collections.emptyList(),
++                          Collections.emptyMap(),
++                          Collections.emptyMap(),
+                           SSTableIntervalTree.empty()));
      }
  
      public Throwable dropSSTablesIfInvalid(Throwable accumulate)


[17/19] cassandra git commit: Merge branch 'cassandra-3.11' into cassandra-3.X

Posted by yu...@apache.org.
Merge branch 'cassandra-3.11' into cassandra-3.X


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c9ead482
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c9ead482
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c9ead482

Branch: refs/heads/cassandra-3.X
Commit: c9ead4826a1dfb246b4ef6c916106bb13fcdbdc5
Parents: ddefb1d 66e2145
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Dec 13 15:59:23 2016 -0800
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:59:23 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++-
 .../cassandra/db/PartitionRangeReadCommand.java |  4 +-
 .../cassandra/db/compaction/CompactionTask.java | 81 ++++++++++----------
 .../cassandra/db/lifecycle/LogTransaction.java  |  3 +-
 .../apache/cassandra/db/lifecycle/Tracker.java  | 34 ++++----
 .../cassandra/index/SecondaryIndexManager.java  |  4 +-
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  6 +-
 .../org/apache/cassandra/tools/ToolsTester.java |  3 -
 10 files changed, 85 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9ead482/CHANGES.txt
----------------------------------------------------------------------


[14/19] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by yu...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/66e21459
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/66e21459
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/66e21459

Branch: refs/heads/trunk
Commit: 66e214592875e296bb540a966f1648f1106b2464
Parents: 1cafc3c 0fe82be
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Dec 13 15:59:14 2016 -0800
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:59:14 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++-
 .../cassandra/db/PartitionRangeReadCommand.java |  4 +-
 .../cassandra/db/compaction/CompactionTask.java | 81 ++++++++++----------
 .../cassandra/db/lifecycle/LogTransaction.java  |  3 +-
 .../apache/cassandra/db/lifecycle/Tracker.java  | 34 ++++----
 .../cassandra/index/SecondaryIndexManager.java  |  4 +-
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  6 +-
 .../org/apache/cassandra/tools/ToolsTester.java |  3 -
 10 files changed, 85 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 28ebf36,145afb9..f95dd81
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -168,12 -59,6 +168,13 @@@ Merged from 3.0
   * Correct log message for statistics of offheap memtable flush (CASSANDRA-12776)
   * Explicitly set locale for string validation (CASSANDRA-12541,CASSANDRA-12542,CASSANDRA-12543,CASSANDRA-12545)
  Merged from 2.2:
++ * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
 + * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
 + * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
 + * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
 + * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
 + * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
   * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
   * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
   * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 881fb00,39ed804..a5f76bd4
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -405,9 -388,13 +405,13 @@@ public class ColumnFamilyStore implemen
  
          logger.info("Initializing {}.{}", keyspace.getName(), name);
  
-         // scan for sstables corresponding to this cf and load them
-         data = new Tracker(this, loadSSTables);
+         // Create Memtable only on online
+         Memtable initialMemtable = null;
+         if (DatabaseDescriptor.isDaemonInitialized())
 -            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
++            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getCurrentPosition()), this);
+         data = new Tracker(initialMemtable, loadSSTables);
  
+         // scan for sstables corresponding to this cf and load them
          if (data.loadsstables)
          {
              Directories.SSTableLister sstableFiles = directories.sstableLister(Directories.OnTxnErr.IGNORE).skipTemporary(true);
@@@ -2118,7 -1957,7 +2122,7 @@@
              {
                  public Void call()
                  {
-                     cfs.data.reset();
 -                    cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
++                    cfs.data.reset(new Memtable(new AtomicReference<>(CommitLogPosition.NONE), cfs));
                      return null;
                  }
              }, true, false);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 50b568e,17adef0..045fc26
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -221,7 -199,8 +221,9 @@@ public class PartitionRangeReadCommand 
                  if (!sstable.isRepaired())
                      oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime());
              }
-             return checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
++            // iterators can be empty for offline tools
+             return iterators.isEmpty() ? EmptyIterators.unfilteredPartition(metadata(), isForThrift())
+                                        : checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
          }
          catch (RuntimeException | Error e)
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 2f90c7b,f0a1f47..a9d6c7c
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@@ -228,47 -213,29 +228,50 @@@ public class CompactionTask extends Abs
                  }
              }
  
--            // log a bunch of statistics about the result and save to system table compaction_history
- 
-             long durationInNano = System.nanoTime() - start;
-             long dTime = TimeUnit.NANOSECONDS.toMillis(durationInNano);
-             long startsize = inputSizeBytes;
 -            long dTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
 -            long startsize = SSTableReader.getTotalBytes(transaction.originals());
--            long endsize = SSTableReader.getTotalBytes(newSStables);
--            double ratio = (double) endsize / (double) startsize;
--
--            StringBuilder newSSTableNames = new StringBuilder();
--            for (SSTableReader reader : newSStables)
--                newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
- 
-             long totalSourceRows = 0;
-             for (int i = 0; i < mergedRowCounts.length; i++)
-                 totalSourceRows += mergedRowCounts[i] * (i + 1);
- 
-             String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
-             logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %s to %s (~%d%% of original) in %,dms.  Read Throughput = %s, Write Throughput = %s, Row Throughput = ~%,d/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                       taskId,
-                                       transaction.originals().size(),
-                                       newSSTableNames.toString(),
-                                       getLevel(),
-                                       FBUtilities.prettyPrintMemory(startsize),
-                                       FBUtilities.prettyPrintMemory(endsize),
-                                       (int) (ratio * 100),
-                                       dTime,
-                                       FBUtilities.prettyPrintMemoryPerSecond(startsize, durationInNano),
-                                       FBUtilities.prettyPrintMemoryPerSecond(endsize, durationInNano),
-                                       (int) totalSourceCQLRows / (TimeUnit.NANOSECONDS.toSeconds(durationInNano) + 1),
-                                       totalSourceRows,
-                                       totalKeysWritten,
-                                       mergeSummary));
-             logger.trace("CF Total Bytes Compacted: {}", FBUtilities.prettyPrintMemory(CompactionTask.addToTotalBytesCompacted(endsize)));
-             logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
-             cfs.getCompactionStrategyManager().compactionLogger.compaction(startTime, transaction.originals(), System.currentTimeMillis(), newSStables);
- 
-             // update the metrics
-             cfs.metric.compactionBytesWritten.inc(endsize);
--
 -            if (offline)
 +            if (transaction.isOffline())
+             {
                  Refs.release(Refs.selfRefs(newSStables));
+             }
+             else
+             {
 -                double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
 -                Summary mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
 -                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
 -                                           taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, mergeSummary.totalSourceRows, totalKeysWritten, mergeSummary.partitionMerge));
 -                logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
++                // log a bunch of statistics about the result and save to system table compaction_history
++
++                long durationInNano = System.nanoTime() - start;
++                long dTime = TimeUnit.NANOSECONDS.toMillis(durationInNano);
++                long startsize = inputSizeBytes;
++                long endsize = SSTableReader.getTotalBytes(newSStables);
++                double ratio = (double) endsize / (double) startsize;
++
++                StringBuilder newSSTableNames = new StringBuilder();
++                for (SSTableReader reader : newSStables)
++                    newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
++                long totalSourceRows = 0;
++                for (int i = 0; i < mergedRowCounts.length; i++)
++                    totalSourceRows += mergedRowCounts[i] * (i + 1);
++
++                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getTableName(), mergedRowCounts, startsize, endsize);
++                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %s to %s (~%d%% of original) in %,dms.  Read Throughput = %s, Write Throughput = %s, Row Throughput = ~%,d/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
++                                           taskId,
++                                           transaction.originals().size(),
++                                           newSSTableNames.toString(),
++                                           getLevel(),
++                                           FBUtilities.prettyPrintMemory(startsize),
++                                           FBUtilities.prettyPrintMemory(endsize),
++                                           (int) (ratio * 100),
++                                           dTime,
++                                           FBUtilities.prettyPrintMemoryPerSecond(startsize, durationInNano),
++                                           FBUtilities.prettyPrintMemoryPerSecond(endsize, durationInNano),
++                                           (int) totalSourceCQLRows / (TimeUnit.NANOSECONDS.toSeconds(durationInNano) + 1),
++                                           totalSourceRows,
++                                           totalKeysWritten,
++                                           mergeSummary));
++                logger.trace("CF Total Bytes Compacted: {}", FBUtilities.prettyPrintMemory(CompactionTask.addToTotalBytesCompacted(endsize)));
+                 logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
++                cfs.getCompactionStrategyManager().compactionLogger.compaction(startTime, transaction.originals(), System.currentTimeMillis(), newSStables);
++
++                // update the metrics
++                cfs.metric.compactionBytesWritten.inc(endsize);
+             }
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index f464e08,9feaa3e..e2fcb06
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@@ -31,8 -31,7 +31,7 @@@ import com.google.common.collect.*
  import org.apache.cassandra.db.ColumnFamilyStore;
  import org.apache.cassandra.db.Directories;
  import org.apache.cassandra.db.Memtable;
- import org.apache.cassandra.db.commitlog.CommitLog;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
index 6435e3e,0d87cc9..4514b72
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@@ -30,6 -31,8 +31,8 @@@ import junit.framework.Assert
  import org.apache.cassandra.MockSchema;
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.Memtable;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
++import org.apache.cassandra.db.commitlog.CommitLogPosition;
  import org.apache.cassandra.db.compaction.OperationType;
  import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
  import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
@@@ -268,7 -271,7 +271,7 @@@ public class LifecycleTransactionTest e
  
          private static Tracker tracker(ColumnFamilyStore cfs, List<SSTableReader> readers)
          {
-             Tracker tracker = new Tracker(cfs, false);
 -            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs), false);
++            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(CommitLogPosition.NONE), cfs), false);
              tracker.addInitialSSTables(readers);
              return tracker;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/test/unit/org/apache/cassandra/tools/ToolsTester.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/tools/ToolsTester.java
index 97b19c9,0000000..ead4e31
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/tools/ToolsTester.java
+++ b/test/unit/org/apache/cassandra/tools/ToolsTester.java
@@@ -1,296 -1,0 +1,293 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.cassandra.tools;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.lang.management.ManagementFactory;
 +import java.lang.management.ThreadInfo;
 +import java.lang.management.ThreadMXBean;
 +import java.lang.reflect.InvocationTargetException;
 +import java.lang.reflect.Method;
 +import java.security.Permission;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Set;
 +import java.util.regex.Pattern;
 +import java.util.stream.Collectors;
 +
 +import org.apache.commons.io.FileUtils;
 +import org.junit.BeforeClass;
 +
 +import org.slf4j.LoggerFactory;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.fail;
 +
 +/**
 + * Base unit test class for standalone tools
 + */
 +public abstract class ToolsTester
 +{
 +    private static List<ThreadInfo> initialThreads;
 +
 +    static final String[] EXPECTED_THREADS_WITH_SCHEMA = {
-     "(NativePool|SlabPool|HeapPool)Cleaner",
-     "COMMIT-LOG-ALLOCATOR",
-     "COMMIT-LOG-WRITER",
 +    "PerDiskMemtableFlushWriter_0:[1-9]",
 +    "MemtablePostFlush:[1-9]",
 +    "MemtableFlushWriter:[1-9]",
 +    "MemtableReclaimMemory:[1-9]",
 +    };
 +    static final String[] OPTIONAL_THREADS_WITH_SCHEMA = {
 +    "ScheduledTasks:[1-9]",
 +    "OptionalTasks:[1-9]",
 +    "Reference-Reaper:[1-9]",
 +    "LocalPool-Cleaner:[1-9]",
 +    "CacheCleanupExecutor:[1-9]",
 +    "CompactionExecutor:[1-9]",
 +    "ValidationExecutor:[1-9]",
 +    "NonPeriodicTasks:[1-9]",
 +    "Sampler:[1-9]",
 +    "SecondaryIndexManagement:[1-9]",
 +    "Strong-Reference-Leak-Detector:[1-9]",
 +    "Background_Reporter:[1-9]",
 +    "EXPIRING-MAP-REAPER:[1-9]",
 +    };
 +
 +    public void assertNoUnexpectedThreadsStarted(String[] expectedThreadNames, String[] optionalThreadNames)
 +    {
 +        ThreadMXBean threads = ManagementFactory.getThreadMXBean();
 +
 +        Set<String> initial = initialThreads
 +                              .stream()
 +                              .map(ThreadInfo::getThreadName)
 +                              .collect(Collectors.toSet());
 +
 +        Set<String> current = Arrays.stream(threads.getThreadInfo(threads.getAllThreadIds()))
 +                                    .map(ThreadInfo::getThreadName)
 +                                    .collect(Collectors.toSet());
 +
 +        List<Pattern> expected = expectedThreadNames != null
 +                                 ? Arrays.stream(expectedThreadNames).map(Pattern::compile).collect(Collectors.toList())
 +                                 : Collections.emptyList();
 +
 +        List<Pattern> optional = optionalThreadNames != null
 +                                 ? Arrays.stream(optionalThreadNames).map(Pattern::compile).collect(Collectors.toList())
 +                                 : Collections.emptyList();
 +
 +        current.removeAll(initial);
 +
 +        List<Pattern> notPresent = expected.stream()
 +                                           .filter(threadNamePattern -> !current.stream().anyMatch(threadName -> threadNamePattern.matcher(threadName).matches()))
 +                                           .collect(Collectors.toList());
 +
 +        Set<String> remain = current.stream()
 +                                    .filter(threadName -> expected.stream().anyMatch(pattern -> pattern.matcher(threadName).matches()))
 +                                    .filter(threadName -> optional.stream().anyMatch(pattern -> pattern.matcher(threadName).matches()))
 +                                    .collect(Collectors.toSet());
 +
 +        if (!current.isEmpty())
 +            System.err.println("Unexpected thread names: " + remain);
 +        if (!notPresent.isEmpty())
 +            System.err.println("Mandatory thread missing: " + notPresent);
 +
 +        assertTrue("Wrong thread status", remain.isEmpty() && notPresent.isEmpty());
 +    }
 +
 +    public void assertSchemaNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.config.Schema");
 +    }
 +
 +    public void assertSchemaLoaded()
 +    {
 +        assertClassLoaded("org.apache.cassandra.config.Schema");
 +    }
 +
 +    public void assertKeyspaceNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.db.Keyspace");
 +    }
 +
 +    public void assertKeyspaceLoaded()
 +    {
 +        assertClassLoaded("org.apache.cassandra.db.Keyspace");
 +    }
 +
 +    public void assertServerNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.transport.Server");
 +    }
 +
 +    public void assertSystemKSNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.db.SystemKeyspace");
 +    }
 +
 +    public void assertCLSMNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.db.commitlog.CommitLogSegmentManager");
 +    }
 +
 +    public void assertClassLoaded(String clazz)
 +    {
 +        assertClassLoadedStatus(clazz, true);
 +    }
 +
 +    public void assertClassNotLoaded(String clazz)
 +    {
 +        assertClassLoadedStatus(clazz, false);
 +    }
 +
 +    private void assertClassLoadedStatus(String clazz, boolean expected)
 +    {
 +        for (ClassLoader cl = Thread.currentThread().getContextClassLoader(); cl != null; cl = cl.getParent())
 +        {
 +            try
 +            {
 +                Method mFindLoadedClass = ClassLoader.class.getDeclaredMethod("findLoadedClass", String.class);
 +                mFindLoadedClass.setAccessible(true);
 +                boolean loaded = mFindLoadedClass.invoke(cl, clazz) != null;
 +
 +                if (expected)
 +                {
 +                    if (loaded)
 +                        return;
 +                }
 +                else
 +                    assertFalse(clazz + " has been loaded", loaded);
 +            }
 +            catch (Exception e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +        }
 +
 +        if (expected)
 +            fail(clazz + " has not been loaded");
 +    }
 +
 +    public void runTool(int expectedExitCode, String clazz, String... args)
 +    {
 +        try
 +        {
 +            // install security manager to get informed about the exit-code
 +            System.setSecurityManager(new SecurityManager()
 +            {
 +                public void checkExit(int status)
 +                {
 +                    throw new SystemExitException(status);
 +                }
 +
 +                public void checkPermission(Permission perm)
 +                {
 +                }
 +
 +                public void checkPermission(Permission perm, Object context)
 +                {
 +                }
 +            });
 +
 +            try
 +            {
 +                Class.forName(clazz).getDeclaredMethod("main", String[].class).invoke(null, (Object) args);
 +            }
 +            catch (InvocationTargetException e)
 +            {
 +                Throwable cause = e.getCause();
 +                if (cause instanceof Error)
 +                    throw (Error) cause;
 +                if (cause instanceof RuntimeException)
 +                    throw (RuntimeException) cause;
 +                throw e;
 +            }
 +
 +            assertEquals("Unexpected exit code", expectedExitCode, 0);
 +        }
 +        catch (SystemExitException e)
 +        {
 +            assertEquals("Unexpected exit code", expectedExitCode, e.status);
 +        }
 +        catch (InvocationTargetException e)
 +        {
 +            throw new RuntimeException(e.getTargetException());
 +        }
 +        catch (Exception e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +        finally
 +        {
 +            // uninstall security manager
 +            System.setSecurityManager(null);
 +        }
 +    }
 +
 +    @BeforeClass
 +    public static void setupTester()
 +    {
 +        System.setProperty("cassandra.partitioner", "org.apache.cassandra.dht.Murmur3Partitioner");
 +
 +        // may start an async appender
 +        LoggerFactory.getLogger(ToolsTester.class);
 +
 +        ThreadMXBean threads = ManagementFactory.getThreadMXBean();
 +        initialThreads = Arrays.asList(threads.getThreadInfo(threads.getAllThreadIds()));
 +    }
 +
 +    public static class SystemExitException extends Error
 +    {
 +        public final int status;
 +
 +        public SystemExitException(int status)
 +        {
 +            this.status = status;
 +        }
 +    }
 +
 +    public static String findOneSSTable(String ks, String cf) throws IOException
 +    {
 +        File cfDir = sstableDir(ks, cf);
 +        File[] sstableFiles = cfDir.listFiles((file) -> file.isFile() && file.getName().endsWith("-Data.db"));
 +        return sstableFiles[0].getAbsolutePath();
 +    }
 +
 +    public static String sstableDirName(String ks, String cf) throws IOException
 +    {
 +        return sstableDir(ks, cf).getAbsolutePath();
 +    }
 +
 +    public static File sstableDir(String ks, String cf) throws IOException
 +    {
 +        File dataDir = copySSTables();
 +        File ksDir = new File(dataDir, ks);
 +        File[] cfDirs = ksDir.listFiles((dir, name) -> cf.equals(name) || name.startsWith(cf + '-'));
 +        return cfDirs[0];
 +    }
 +
 +    public static File copySSTables() throws IOException
 +    {
 +        File dataDir = new File("build/test/cassandra/data");
 +        File srcDir = new File("test/data/legacy-sstables/ma");
 +        FileUtils.copyDirectory(new File(srcDir, "legacy_tables"), new File(dataDir, "legacy_sstables"));
 +        return dataDir;
 +    }
 +}


[10/19] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index 003b624,0000000..a6ed3ba
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@@ -1,1112 -1,0 +1,1114 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.index;
 +
 +import java.lang.reflect.Constructor;
 +import java.util.*;
 +import java.util.concurrent.*;
 +import java.util.function.Function;
 +import java.util.stream.Collectors;
 +import java.util.stream.Stream;
 +
 +import com.google.common.base.Joiner;
 +import com.google.common.base.Strings;
 +import com.google.common.collect.ImmutableSet;
 +import com.google.common.collect.Iterables;
 +import com.google.common.collect.Maps;
 +import com.google.common.collect.Sets;
 +import com.google.common.primitives.Longs;
 +import com.google.common.util.concurrent.Futures;
 +import com.google.common.util.concurrent.MoreExecutors;
 +import org.apache.commons.lang3.StringUtils;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 +import org.apache.cassandra.concurrent.NamedThreadFactory;
 +import org.apache.cassandra.concurrent.StageManager;
 +import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.statements.IndexTarget;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.filter.RowFilter;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.lifecycle.View;
 +import org.apache.cassandra.db.partitions.PartitionIterators;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.index.internal.CassandraIndex;
 +import org.apache.cassandra.index.transactions.*;
 +import org.apache.cassandra.io.sstable.ReducingKeyIterator;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.schema.Indexes;
 +import org.apache.cassandra.service.pager.SinglePartitionPager;
 +import org.apache.cassandra.tracing.Tracing;
 +import org.apache.cassandra.transport.Server;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.apache.cassandra.utils.concurrent.Refs;
 +
 +/**
 + * Handles the core maintenance functionality associated with indexes: adding/removing them to or from
 + * a table, (re)building during bootstrap or other streaming operations, flushing, reloading metadata
 + * and so on.
 + *
 + * The Index interface defines a number of methods which return Callable<?>. These are primarily the
 + * management tasks for an index implementation. Most of them are currently executed in a blocking
 + * fashion via submission to SIM's blockingExecutor. This provides the desired behaviour in pretty
 + * much all cases, as tasks like flushing an index needs to be executed synchronously to avoid potentially
 + * deadlocking on the FlushWriter or PostFlusher. Several of these Callable<?> returning methods on Index could
 + * then be defined with as void and called directly from SIM (rather than being run via the executor service).
 + * Separating the task defintion from execution gives us greater flexibility though, so that in future, for example,
 + * if the flush process allows it we leave open the possibility of executing more of these tasks asynchronously.
 + *
 + * The primary exception to the above is the Callable returned from Index#addIndexedColumn. This may
 + * involve a significant effort, building a new index over any existing data. We perform this task asynchronously;
 + * as it is called as part of a schema update, which we do not want to block for a long period. Building non-custom
 + * indexes is performed on the CompactionManager.
 + *
 + * This class also provides instances of processors which listen to updates to the base table and forward to
 + * registered Indexes the info required to keep those indexes up to date.
 + * There are two variants of these processors, each with a factory method provided by SIM:
 + *      IndexTransaction: deals with updates generated on the regular write path.
 + *      CleanupTransaction: used when partitions are modified during compaction or cleanup operations.
 + * Further details on their usage and lifecycles can be found in the interface definitions below.
 + *
 + * Finally, the bestIndexFor method is used at query time to identify the most selective index of those able
 + * to satisfy any search predicates defined by a ReadCommand's RowFilter. It returns a thin IndexAccessor object
 + * which enables the ReadCommand to access the appropriate functions of the Index at various stages in its lifecycle.
 + * e.g. the getEstimatedResultRows is required when StorageProxy calculates the initial concurrency factor for
 + * distributing requests to replicas, whereas a Searcher instance is needed when the ReadCommand is executed locally on
 + * a target replica.
 + */
 +public class SecondaryIndexManager implements IndexRegistry
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(SecondaryIndexManager.class);
 +
 +    // default page size (in rows) when rebuilding the index for a whole partition
 +    public static final int DEFAULT_PAGE_SIZE = 10000;
 +
 +    private Map<String, Index> indexes = Maps.newConcurrentMap();
 +
 +    /**
 +     * The indexes that are ready to server requests.
 +     */
 +    private Set<String> builtIndexes = Sets.newConcurrentHashSet();
 +
 +    // executes tasks returned by Indexer#addIndexColumn which may require index(es) to be (re)built
 +    private static final ExecutorService asyncExecutor =
 +        new JMXEnabledThreadPoolExecutor(1,
 +                                         StageManager.KEEPALIVE,
 +                                         TimeUnit.SECONDS,
 +                                         new LinkedBlockingQueue<>(),
 +                                         new NamedThreadFactory("SecondaryIndexManagement"),
 +                                         "internal");
 +
 +    // executes all blocking tasks produced by Indexers e.g. getFlushTask, getMetadataReloadTask etc
 +    private static final ExecutorService blockingExecutor = MoreExecutors.newDirectExecutorService();
 +
 +    /**
 +     * The underlying column family containing the source data for these indexes
 +     */
 +    public final ColumnFamilyStore baseCfs;
 +
 +    public SecondaryIndexManager(ColumnFamilyStore baseCfs)
 +    {
 +        this.baseCfs = baseCfs;
 +    }
 +
 +
 +    /**
 +     * Drops and adds new indexes associated with the underlying CF
 +     */
 +    public void reload()
 +    {
 +        // figure out what needs to be added and dropped.
 +        Indexes tableIndexes = baseCfs.metadata.getIndexes();
 +        indexes.keySet()
 +               .stream()
 +               .filter(indexName -> !tableIndexes.has(indexName))
 +               .forEach(this::removeIndex);
 +
 +        // we call add for every index definition in the collection as
 +        // some may not have been created here yet, only added to schema
 +        for (IndexMetadata tableIndex : tableIndexes)
 +            addIndex(tableIndex);
 +    }
 +
 +    private Future<?> reloadIndex(IndexMetadata indexDef)
 +    {
 +        Index index = indexes.get(indexDef.name);
 +        Callable<?> reloadTask = index.getMetadataReloadTask(indexDef);
 +        return reloadTask == null
 +               ? Futures.immediateFuture(null)
 +               : blockingExecutor.submit(reloadTask);
 +    }
 +
 +    private Future<?> createIndex(IndexMetadata indexDef)
 +    {
 +        Index index = createInstance(indexDef);
 +        index.register(this);
 +
 +        // if the index didn't register itself, we can probably assume that no initialization needs to happen
 +        final Callable<?> initialBuildTask = indexes.containsKey(indexDef.name)
 +                                           ? index.getInitializationTask()
 +                                           : null;
 +        if (initialBuildTask == null)
 +        {
 +            // We need to make sure that the index is marked as built in the case where the initialBuildTask
 +            // does not need to be run (if the index didn't register itself or if the base table was empty).
 +            markIndexBuilt(indexDef.name);
 +            return Futures.immediateFuture(null);
 +        }
 +        return asyncExecutor.submit(index.getInitializationTask());
 +    }
 +
 +    /**
 +     * Adds and builds a index
 +     * @param indexDef the IndexMetadata describing the index
 +     */
 +    public synchronized Future<?> addIndex(IndexMetadata indexDef)
 +    {
 +        if (indexes.containsKey(indexDef.name))
 +            return reloadIndex(indexDef);
 +        else
 +            return createIndex(indexDef);
 +    }
 +
 +    /**
 +     * Checks if the specified index is queryable.
 +     *
 +     * @param index the index
 +     * @return <code>true</code> if the specified index is queryable, <code>false</code> otherwise
 +     */
 +    public boolean isIndexQueryable(Index index)
 +    {
 +        return builtIndexes.contains(index.getIndexMetadata().name);
 +    }
 +
 +    public synchronized void removeIndex(String indexName)
 +    {
 +        Index index = unregisterIndex(indexName);
 +        if (null != index)
 +        {
 +            markIndexRemoved(indexName);
 +            executeBlocking(index.getInvalidateTask());
 +        }
 +    }
 +
 +
 +    public Set<IndexMetadata> getDependentIndexes(ColumnDefinition column)
 +    {
 +        if (indexes.isEmpty())
 +            return Collections.emptySet();
 +
 +        Set<IndexMetadata> dependentIndexes = new HashSet<>();
 +        for (Index index : indexes.values())
 +            if (index.dependsOn(column))
 +                dependentIndexes.add(index.getIndexMetadata());
 +
 +        return dependentIndexes;
 +    }
 +
 +    /**
 +     * Called when dropping a Table
 +     */
 +    public void markAllIndexesRemoved()
 +    {
 +       getBuiltIndexNames().forEach(this::markIndexRemoved);
 +    }
 +
 +    /**
 +    * Does a full, blocking rebuild of the indexes specified by columns from the sstables.
 +    * Caller must acquire and release references to the sstables used here.
 +    * Note also that only this method of (re)building indexes:
 +    *   a) takes a set of index *names* rather than Indexers
 +    *   b) marks exsiting indexes removed prior to rebuilding
 +    *
 +    * @param sstables the data to build from
 +    * @param indexNames the list of indexes to be rebuilt
 +    */
 +    public void rebuildIndexesBlocking(Collection<SSTableReader> sstables, Set<String> indexNames)
 +    {
 +        Set<Index> toRebuild = indexes.values().stream()
 +                                               .filter(index -> indexNames.contains(index.getIndexMetadata().name))
 +                                               .filter(Index::shouldBuildBlocking)
 +                                               .collect(Collectors.toSet());
 +        if (toRebuild.isEmpty())
 +        {
 +            logger.info("No defined indexes with the supplied names: {}", Joiner.on(',').join(indexNames));
 +            return;
 +        }
 +
 +        toRebuild.forEach(indexer -> markIndexRemoved(indexer.getIndexMetadata().name));
 +
 +        buildIndexesBlocking(sstables, toRebuild);
 +
 +        toRebuild.forEach(indexer -> markIndexBuilt(indexer.getIndexMetadata().name));
 +    }
 +
 +    public void buildAllIndexesBlocking(Collection<SSTableReader> sstables)
 +    {
 +        buildIndexesBlocking(sstables, indexes.values()
 +                                              .stream()
 +                                              .filter(Index::shouldBuildBlocking)
 +                                              .collect(Collectors.toSet()));
 +    }
 +
 +    // For convenience, may be called directly from Index impls
 +    public void buildIndexBlocking(Index index)
 +    {
 +        if (index.shouldBuildBlocking())
 +        {
 +            try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL));
 +                 Refs<SSTableReader> sstables = viewFragment.refs)
 +            {
 +                buildIndexesBlocking(sstables, Collections.singleton(index));
 +                markIndexBuilt(index.getIndexMetadata().name);
 +            }
 +        }
 +    }
 +
 +    /**
 +     * Checks if the specified {@link ColumnFamilyStore} is a secondary index.
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code> to check.
 +     * @return <code>true</code> if the specified <code>ColumnFamilyStore</code> is a secondary index,
 +     * <code>false</code> otherwise.
 +     */
 +    public static boolean isIndexColumnFamilyStore(ColumnFamilyStore cfs)
 +    {
 +        return isIndexColumnFamily(cfs.name);
 +    }
 +
 +    /**
 +     * Checks if the specified {@link ColumnFamilyStore} is the one secondary index.
 +     *
 +     * @param cfName the name of the <code>ColumnFamilyStore</code> to check.
 +     * @return <code>true</code> if the specified <code>ColumnFamilyStore</code> is a secondary index,
 +     * <code>false</code> otherwise.
 +     */
 +    public static boolean isIndexColumnFamily(String cfName)
 +    {
 +        return cfName.contains(Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    /**
 +     * Returns the parent of the specified {@link ColumnFamilyStore}.
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code>
 +     * @return the parent of the specified <code>ColumnFamilyStore</code>
 +     */
 +    public static ColumnFamilyStore getParentCfs(ColumnFamilyStore cfs)
 +    {
 +        String parentCfs = getParentCfsName(cfs.name);
 +        return cfs.keyspace.getColumnFamilyStore(parentCfs);
 +    }
 +
 +    /**
 +     * Returns the parent name of the specified {@link ColumnFamilyStore}.
 +     *
 +     * @param cfName the <code>ColumnFamilyStore</code> name
 +     * @return the parent name of the specified <code>ColumnFamilyStore</code>
 +     */
 +    public static String getParentCfsName(String cfName)
 +    {
 +        assert isIndexColumnFamily(cfName);
 +        return StringUtils.substringBefore(cfName, Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    /**
 +     * Returns the index name
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code>
 +     * @return the index name
 +     */
 +    public static String getIndexName(ColumnFamilyStore cfs)
 +    {
 +        return getIndexName(cfs.name);
 +    }
 +
 +    /**
 +     * Returns the index name
 +     *
 +     * @param cfName the <code>ColumnFamilyStore</code> name
 +     * @return the index name
 +     */
 +    public static String getIndexName(String cfName)
 +    {
 +        assert isIndexColumnFamily(cfName);
 +        return StringUtils.substringAfter(cfName, Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    private void buildIndexesBlocking(Collection<SSTableReader> sstables, Set<Index> indexes)
 +    {
 +        if (indexes.isEmpty())
 +            return;
 +
 +        logger.info("Submitting index build of {} for data in {}",
 +                    indexes.stream().map(i -> i.getIndexMetadata().name).collect(Collectors.joining(",")),
 +                    sstables.stream().map(SSTableReader::toString).collect(Collectors.joining(",")));
 +
 +        SecondaryIndexBuilder builder = new SecondaryIndexBuilder(baseCfs,
 +                                                                  indexes,
 +                                                                  new ReducingKeyIterator(sstables));
 +        Future<?> future = CompactionManager.instance.submitIndexBuild(builder);
 +        FBUtilities.waitOnFuture(future);
 +
 +        flushIndexesBlocking(indexes);
 +        logger.info("Index build of {} complete",
 +                    indexes.stream().map(i -> i.getIndexMetadata().name).collect(Collectors.joining(",")));
 +    }
 +
 +    /**
 +     * Marks the specified index as build.
 +     * <p>This method is public as it need to be accessible from the {@link Index} implementations</p>
 +     * @param indexName the index name
 +     */
 +    public void markIndexBuilt(String indexName)
 +    {
 +        builtIndexes.add(indexName);
-         SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), indexName);
++        if (DatabaseDescriptor.isDaemonInitialized())
++            SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), indexName);
 +    }
 +
 +    /**
 +     * Marks the specified index as removed.
 +     * <p>This method is public as it need to be accessible from the {@link Index} implementations</p>
 +     * @param indexName the index name
 +     */
 +    public void markIndexRemoved(String indexName)
 +    {
 +        SystemKeyspace.setIndexRemoved(baseCfs.keyspace.getName(), indexName);
 +    }
 +
 +    public Index getIndexByName(String indexName)
 +    {
 +        return indexes.get(indexName);
 +    }
 +
 +    private Index createInstance(IndexMetadata indexDef)
 +    {
 +        Index newIndex;
 +        if (indexDef.isCustom())
 +        {
 +            assert indexDef.options != null;
 +            String className = indexDef.options.get(IndexTarget.CUSTOM_INDEX_OPTION_NAME);
 +            assert ! Strings.isNullOrEmpty(className);
 +            try
 +            {
 +                Class<? extends Index> indexClass = FBUtilities.classForName(className, "Index");
 +                Constructor<? extends Index> ctor = indexClass.getConstructor(ColumnFamilyStore.class, IndexMetadata.class);
 +                newIndex = (Index)ctor.newInstance(baseCfs, indexDef);
 +            }
 +            catch (Exception e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +        }
 +        else
 +        {
 +            newIndex = CassandraIndex.newIndex(baseCfs, indexDef);
 +        }
 +        return newIndex;
 +    }
 +
 +    /**
 +     * Truncate all indexes
 +     */
 +    public void truncateAllIndexesBlocking(final long truncatedAt)
 +    {
 +        executeAllBlocking(indexes.values().stream(), (index) -> index.getTruncateTask(truncatedAt));
 +    }
 +
 +    /**
 +     * Remove all indexes
 +     */
 +    public void invalidateAllIndexesBlocking()
 +    {
 +        markAllIndexesRemoved();
 +        executeAllBlocking(indexes.values().stream(), Index::getInvalidateTask);
 +    }
 +
 +    /**
 +     * Perform a blocking flush all indexes
 +     */
 +    public void flushAllIndexesBlocking()
 +    {
 +       flushIndexesBlocking(ImmutableSet.copyOf(indexes.values()));
 +    }
 +
 +    /**
 +     * Perform a blocking flush of selected indexes
 +     */
 +    public void flushIndexesBlocking(Set<Index> indexes)
 +    {
 +        if (indexes.isEmpty())
 +            return;
 +
 +        List<Future<?>> wait = new ArrayList<>();
 +        List<Index> nonCfsIndexes = new ArrayList<>();
 +
 +        // for each CFS backed index, submit a flush task which we'll wait on for completion
 +        // for the non-CFS backed indexes, we'll flush those while we wait.
 +        synchronized (baseCfs.getTracker())
 +        {
 +            indexes.forEach(index ->
 +                index.getBackingTable()
 +                     .map(cfs -> wait.add(cfs.forceFlush()))
 +                     .orElseGet(() -> nonCfsIndexes.add(index)));
 +        }
 +
 +        executeAllBlocking(nonCfsIndexes.stream(), Index::getBlockingFlushTask);
 +        FBUtilities.waitOnFutures(wait);
 +    }
 +
 +    /**
 +     * Performs a blocking flush of all custom indexes
 +     */
 +    public void flushAllNonCFSBackedIndexesBlocking()
 +    {
 +        executeAllBlocking(indexes.values()
 +                                  .stream()
 +                                  .filter(index -> !index.getBackingTable().isPresent()),
 +                           Index::getBlockingFlushTask);
 +    }
 +
 +    /**
 +     * @return all indexes which are marked as built and ready to use
 +     */
 +    public List<String> getBuiltIndexNames()
 +    {
 +        Set<String> allIndexNames = new HashSet<>();
 +        indexes.values().stream()
 +                .map(i -> i.getIndexMetadata().name)
 +                .forEach(allIndexNames::add);
 +        return SystemKeyspace.getBuiltIndexes(baseCfs.keyspace.getName(), allIndexNames);
 +    }
 +
 +    /**
 +     * @return all backing Tables used by registered indexes
 +     */
 +    public Set<ColumnFamilyStore> getAllIndexColumnFamilyStores()
 +    {
 +        Set<ColumnFamilyStore> backingTables = new HashSet<>();
 +        indexes.values().forEach(index -> index.getBackingTable().ifPresent(backingTables::add));
 +        return backingTables;
 +    }
 +
 +    /**
 +     * @return if there are ANY indexes registered for this table
 +     */
 +    public boolean hasIndexes()
 +    {
 +        return !indexes.isEmpty();
 +    }
 +
 +    /**
 +     * When building an index against existing data in sstables, add the given partition to the index
 +     */
 +    public void indexPartition(DecoratedKey key, Set<Index> indexes, int pageSize)
 +    {
 +        if (logger.isTraceEnabled())
 +            logger.trace("Indexing partition {}", baseCfs.metadata.getKeyValidator().getString(key.getKey()));
 +
 +        if (!indexes.isEmpty())
 +        {
 +            SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(baseCfs.metadata,
 +                                                                                          FBUtilities.nowInSeconds(),
 +                                                                                          key);
 +            int nowInSec = cmd.nowInSec();
 +            boolean readStatic = false;
 +
 +            SinglePartitionPager pager = new SinglePartitionPager(cmd, null, Server.CURRENT_VERSION);
 +            while (!pager.isExhausted())
 +            {
 +                try (ReadOrderGroup readGroup = cmd.startOrderGroup();
 +                     OpOrder.Group writeGroup = Keyspace.writeOrder.start();
 +                     RowIterator partition =
 +                        PartitionIterators.getOnlyElement(pager.fetchPageInternal(pageSize,readGroup),
 +                                                          cmd))
 +                {
 +                    Set<Index.Indexer> indexers = indexes.stream()
 +                                                         .map(index -> index.indexerFor(key,
 +                                                                                        partition.columns(),
 +                                                                                        nowInSec,
 +                                                                                        writeGroup,
 +                                                                                        IndexTransaction.Type.UPDATE))
 +                                                         .filter(Objects::nonNull)
 +                                                         .collect(Collectors.toSet());
 +
 +                    indexers.forEach(Index.Indexer::begin);
 +
 +                    // only process the static row once per partition
 +                    if (!readStatic && !partition.staticRow().isEmpty())
 +                    {
 +                        indexers.forEach(indexer -> indexer.insertRow(partition.staticRow()));
 +                        readStatic = true;
 +                    }
 +
 +                    while (partition.hasNext())
 +                    {
 +                        Row row = partition.next();
 +                        indexers.forEach(indexer -> indexer.insertRow(row));
 +                    }
 +
 +                    indexers.forEach(Index.Indexer::finish);
 +                }
 +            }
 +        }
 +    }
 +
 +    /**
 +     * Return the page size used when indexing an entire partition
 +     */
 +    public int calculateIndexingPageSize()
 +    {
 +        if (Boolean.getBoolean("cassandra.force_default_indexing_page_size"))
 +            return DEFAULT_PAGE_SIZE;
 +
 +        double targetPageSizeInBytes = 32 * 1024 * 1024;
 +        double meanPartitionSize = baseCfs.getMeanPartitionSize();
 +        if (meanPartitionSize <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int meanCellsPerPartition = baseCfs.getMeanColumns();
 +        if (meanCellsPerPartition <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int columnsPerRow = baseCfs.metadata.partitionColumns().regulars.size();
 +        if (meanCellsPerPartition <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int meanRowsPerPartition = meanCellsPerPartition / columnsPerRow;
 +        double meanRowSize = meanPartitionSize / meanRowsPerPartition;
 +
 +        int pageSize = (int) Math.max(1, Math.min(DEFAULT_PAGE_SIZE, targetPageSizeInBytes / meanRowSize));
 +
 +        logger.trace("Calculated page size {} for indexing {}.{} ({}/{}/{}/{})",
 +                     pageSize,
 +                     baseCfs.metadata.ksName,
 +                     baseCfs.metadata.cfName,
 +                     meanPartitionSize,
 +                     meanCellsPerPartition,
 +                     meanRowsPerPartition,
 +                     meanRowSize);
 +
 +        return pageSize;
 +    }
 +
 +    /**
 +     * Delete all data from all indexes for this partition.
 +     * For when cleanup rips a partition out entirely.
 +     *
 +     * TODO : improve cleanup transaction to batch updates & perform them async
 +     */
 +    public void deletePartition(UnfilteredRowIterator partition, int nowInSec)
 +    {
 +        // we need to acquire memtable lock because secondary index deletion may
 +        // cause a race (see CASSANDRA-3712). This is done internally by the
 +        // index transaction when it commits
 +        CleanupTransaction indexTransaction = newCleanupTransaction(partition.partitionKey(),
 +                                                                    partition.columns(),
 +                                                                    nowInSec);
 +        indexTransaction.start();
 +        indexTransaction.onPartitionDeletion(new DeletionTime(FBUtilities.timestampMicros(), nowInSec));
 +        indexTransaction.commit();
 +
 +        while (partition.hasNext())
 +        {
 +            Unfiltered unfiltered = partition.next();
 +            if (unfiltered.kind() != Unfiltered.Kind.ROW)
 +                continue;
 +
 +            indexTransaction = newCleanupTransaction(partition.partitionKey(),
 +                                                     partition.columns(),
 +                                                     nowInSec);
 +            indexTransaction.start();
 +            indexTransaction.onRowDelete((Row)unfiltered);
 +            indexTransaction.commit();
 +        }
 +    }
 +
 +    /**
 +     * Called at query time to choose which (if any) of the registered index implementations to use for a given query.
 +     *
 +     * This is a two step processes, firstly compiling the set of searchable indexes then choosing the one which reduces
 +     * the search space the most.
 +     *
 +     * In the first phase, if the command's RowFilter contains any custom index expressions, the indexes that they
 +     * specify are automatically included. Following that, the registered indexes are filtered to include only those
 +     * which support the standard expressions in the RowFilter.
 +     *
 +     * The filtered set then sorted by selectivity, as reported by the Index implementations' getEstimatedResultRows
 +     * method.
 +     *
 +     * Implementation specific validation of the target expression, either custom or standard, by the selected
 +     * index should be performed in the searcherFor method to ensure that we pick the right index regardless of
 +     * the validity of the expression.
 +     *
 +     * This method is only called once during the lifecycle of a ReadCommand and the result is
 +     * cached for future use when obtaining a Searcher, getting the index's underlying CFS for
 +     * ReadOrderGroup, or an estimate of the result size from an average index query.
 +     *
 +     * @param command ReadCommand to be executed
 +     * @return an Index instance, ready to use during execution of the command, or null if none
 +     * of the registered indexes can support the command.
 +     */
 +    public Index getBestIndexFor(ReadCommand command)
 +    {
 +        if (indexes.isEmpty() || command.rowFilter().isEmpty())
 +            return null;
 +
 +        Set<Index> searchableIndexes = new HashSet<>();
 +        for (RowFilter.Expression expression : command.rowFilter())
 +        {
 +            if (expression.isCustom())
 +            {
 +                // Only a single custom expression is allowed per query and, if present,
 +                // we want to always favour the index specified in such an expression
 +                RowFilter.CustomExpression customExpression = (RowFilter.CustomExpression)expression;
 +                logger.trace("Command contains a custom index expression, using target index {}", customExpression.getTargetIndex().name);
 +                Tracing.trace("Command contains a custom index expression, using target index {}", customExpression.getTargetIndex().name);
 +                return indexes.get(customExpression.getTargetIndex().name);
 +            }
 +            else
 +            {
 +                indexes.values().stream()
 +                       .filter(index -> index.supportsExpression(expression.column(), expression.operator()))
 +                       .forEach(searchableIndexes::add);
 +            }
 +        }
 +
 +        if (searchableIndexes.isEmpty())
 +        {
 +            logger.trace("No applicable indexes found");
 +            Tracing.trace("No applicable indexes found");
 +            return null;
 +        }
 +
 +        Index selected = searchableIndexes.size() == 1
 +                         ? Iterables.getOnlyElement(searchableIndexes)
 +                         : searchableIndexes.stream()
 +                                            .min((a, b) -> Longs.compare(a.getEstimatedResultRows(),
 +                                                                         b.getEstimatedResultRows()))
 +                                            .orElseThrow(() -> new AssertionError("Could not select most selective index"));
 +
 +        // pay for an additional threadlocal get() rather than build the strings unnecessarily
 +        if (Tracing.isTracing())
 +        {
 +            Tracing.trace("Index mean cardinalities are {}. Scanning with {}.",
 +                          searchableIndexes.stream().map(i -> i.getIndexMetadata().name + ':' + i.getEstimatedResultRows())
 +                                           .collect(Collectors.joining(",")),
 +                          selected.getIndexMetadata().name);
 +        }
 +        return selected;
 +    }
 +
 +    /**
 +     * Called at write time to ensure that values present in the update
 +     * are valid according to the rules of all registered indexes which
 +     * will process it. The partition key as well as the clustering and
 +     * cell values for each row in the update may be checked by index
 +     * implementations
 +     * @param update PartitionUpdate containing the values to be validated by registered Index implementations
 +     * @throws InvalidRequestException
 +     */
 +    public void validate(PartitionUpdate update) throws InvalidRequestException
 +    {
 +        for (Index index : indexes.values())
 +            index.validate(update);
 +    }
 +
 +    /**
 +     * IndexRegistry methods
 +     */
 +    public void registerIndex(Index index)
 +    {
 +        String name = index.getIndexMetadata().name;
 +        indexes.put(name, index);
 +        logger.trace("Registered index {}", name);
 +    }
 +
 +    public void unregisterIndex(Index index)
 +    {
 +        unregisterIndex(index.getIndexMetadata().name);
 +    }
 +
 +    private Index unregisterIndex(String name)
 +    {
 +        Index removed = indexes.remove(name);
 +        builtIndexes.remove(name);
 +        logger.trace(removed == null ? "Index {} was not registered" : "Removed index {} from registry",
 +                     name);
 +        return removed;
 +    }
 +
 +    public Index getIndex(IndexMetadata metadata)
 +    {
 +        return indexes.get(metadata.name);
 +    }
 +
 +    public Collection<Index> listIndexes()
 +    {
 +        return ImmutableSet.copyOf(indexes.values());
 +    }
 +
 +    /**
 +     * Handling of index updates.
 +     * Implementations of the various IndexTransaction interfaces, for keeping indexes in sync with base data
 +     * during updates, compaction and cleanup. Plus factory methods for obtaining transaction instances.
 +     */
 +
 +    /**
 +     * Transaction for updates on the write path.
 +     */
 +    public UpdateTransaction newUpdateTransaction(PartitionUpdate update, OpOrder.Group opGroup, int nowInSec)
 +    {
 +        if (!hasIndexes())
 +            return UpdateTransaction.NO_OP;
 +
 +        Index.Indexer[] indexers = indexes.values().stream()
 +                                          .map(i -> i.indexerFor(update.partitionKey(),
 +                                                                 update.columns(),
 +                                                                 nowInSec,
 +                                                                 opGroup,
 +                                                                 IndexTransaction.Type.UPDATE))
 +                                          .filter(Objects::nonNull)
 +                                          .toArray(Index.Indexer[]::new);
 +
 +        return indexers.length == 0 ? UpdateTransaction.NO_OP : new WriteTimeTransaction(indexers);
 +    }
 +
 +    /**
 +     * Transaction for use when merging rows during compaction
 +     */
 +    public CompactionTransaction newCompactionTransaction(DecoratedKey key,
 +                                                          PartitionColumns partitionColumns,
 +                                                          int versions,
 +                                                          int nowInSec)
 +    {
 +        // the check for whether there are any registered indexes is already done in CompactionIterator
 +        return new IndexGCTransaction(key, partitionColumns, versions, nowInSec, listIndexes());
 +    }
 +
 +    /**
 +     * Transaction for use when removing partitions during cleanup
 +     */
 +    public CleanupTransaction newCleanupTransaction(DecoratedKey key,
 +                                                    PartitionColumns partitionColumns,
 +                                                    int nowInSec)
 +    {
 +        if (!hasIndexes())
 +            return CleanupTransaction.NO_OP;
 +
 +        return new CleanupGCTransaction(key, partitionColumns, nowInSec, listIndexes());
 +    }
 +
 +    /**
 +     * A single use transaction for processing a partition update on the regular write path
 +     */
 +    private static final class WriteTimeTransaction implements UpdateTransaction
 +    {
 +        private final Index.Indexer[] indexers;
 +
 +        private WriteTimeTransaction(Index.Indexer...indexers)
 +        {
 +            // don't allow null indexers, if we don't need any use a NullUpdater object
 +            for (Index.Indexer indexer : indexers) assert indexer != null;
 +            this.indexers = indexers;
 +        }
 +
 +        public void start()
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.begin();
 +        }
 +
 +        public void onPartitionDeletion(DeletionTime deletionTime)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.partitionDelete(deletionTime);
 +        }
 +
 +        public void onRangeTombstone(RangeTombstone tombstone)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.rangeTombstone(tombstone);
 +        }
 +
 +        public void onInserted(Row row)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.insertRow(row);
 +        }
 +
 +        public void onUpdated(Row existing, Row updated)
 +        {
 +            final Row.Builder toRemove = BTreeRow.sortedBuilder();
 +            toRemove.newRow(existing.clustering());
 +            toRemove.addPrimaryKeyLivenessInfo(existing.primaryKeyLivenessInfo());
 +            toRemove.addRowDeletion(existing.deletion());
 +            final Row.Builder toInsert = BTreeRow.sortedBuilder();
 +            toInsert.newRow(updated.clustering());
 +            toInsert.addPrimaryKeyLivenessInfo(updated.primaryKeyLivenessInfo());
 +            toInsert.addRowDeletion(updated.deletion());
 +            // diff listener collates the columns to be added & removed from the indexes
 +            RowDiffListener diffListener = new RowDiffListener()
 +            {
 +                public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
 +                {
 +                }
 +
 +                public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original)
 +                {
 +                }
 +
 +                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
 +                {
 +                }
 +
 +                public void onCell(int i, Clustering clustering, Cell merged, Cell original)
 +                {
 +                    if (merged != null && !merged.equals(original))
 +                        toInsert.addCell(merged);
 +
 +                    if (merged == null || (original != null && shouldCleanupOldValue(original, merged)))
 +                        toRemove.addCell(original);
 +
 +                }
 +            };
 +            Rows.diff(diffListener, updated, existing);
 +            Row oldRow = toRemove.build();
 +            Row newRow = toInsert.build();
 +            for (Index.Indexer indexer : indexers)
 +                indexer.updateRow(oldRow, newRow);
 +        }
 +
 +        public void commit()
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.finish();
 +        }
 +
 +        private boolean shouldCleanupOldValue(Cell oldCell, Cell newCell)
 +        {
 +            // If either the value or timestamp is different, then we
 +            // should delete from the index. If not, then we can infer that
 +            // at least one of the cells is an ExpiringColumn and that the
 +            // difference is in the expiry time. In this case, we don't want to
 +            // delete the old value from the index as the tombstone we insert
 +            // will just hide the inserted value.
 +            // Completely identical cells (including expiring columns with
 +            // identical ttl & localExpirationTime) will not get this far due
 +            // to the oldCell.equals(newCell) in StandardUpdater.update
 +            return !oldCell.value().equals(newCell.value()) || oldCell.timestamp() != newCell.timestamp();
 +        }
 +    }
 +
 +    /**
 +     * A single-use transaction for updating indexes for a single partition during compaction where the only
 +     * operation is to merge rows
 +     * TODO : make this smarter at batching updates so we can use a single transaction to process multiple rows in
 +     * a single partition
 +     */
 +    private static final class IndexGCTransaction implements CompactionTransaction
 +    {
 +        private final DecoratedKey key;
 +        private final PartitionColumns columns;
 +        private final int versions;
 +        private final int nowInSec;
 +        private final Collection<Index> indexes;
 +
 +        private Row[] rows;
 +
 +        private IndexGCTransaction(DecoratedKey key,
 +                                   PartitionColumns columns,
 +                                   int versions,
 +                                   int nowInSec,
 +                                   Collection<Index> indexes)
 +        {
 +            this.key = key;
 +            this.columns = columns;
 +            this.versions = versions;
 +            this.indexes = indexes;
 +            this.nowInSec = nowInSec;
 +        }
 +
 +        public void start()
 +        {
 +            if (versions > 0)
 +                rows = new Row[versions];
 +        }
 +
 +        public void onRowMerge(Row merged, Row...versions)
 +        {
 +            // Diff listener constructs rows representing deltas between the merged and original versions
 +            // These delta rows are then passed to registered indexes for removal processing
 +            final Row.Builder[] builders = new Row.Builder[versions.length];
 +            RowDiffListener diffListener = new RowDiffListener()
 +            {
 +                public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
 +                {
 +                    if (original != null && (merged == null || !merged.isLive(nowInSec)))
 +                        getBuilder(i, clustering).addPrimaryKeyLivenessInfo(original);
 +                }
 +
 +                public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original)
 +                {
 +                }
 +
 +                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
 +                {
 +                }
 +
 +                public void onCell(int i, Clustering clustering, Cell merged, Cell original)
 +                {
 +                    if (original != null && (merged == null || !merged.isLive(nowInSec)))
 +                        getBuilder(i, clustering).addCell(original);
 +                }
 +
 +                private Row.Builder getBuilder(int index, Clustering clustering)
 +                {
 +                    if (builders[index] == null)
 +                    {
 +                        builders[index] = BTreeRow.sortedBuilder();
 +                        builders[index].newRow(clustering);
 +                    }
 +                    return builders[index];
 +                }
 +            };
 +
 +            Rows.diff(diffListener, merged, versions);
 +
 +            for(int i = 0; i < builders.length; i++)
 +                if (builders[i] != null)
 +                    rows[i] = builders[i].build();
 +        }
 +
 +        public void commit()
 +        {
 +            if (rows == null)
 +                return;
 +
 +            try (OpOrder.Group opGroup = Keyspace.writeOrder.start())
 +            {
 +                for (Index index : indexes)
 +                {
 +                    Index.Indexer indexer = index.indexerFor(key, columns, nowInSec, opGroup, Type.COMPACTION);
 +                    if (indexer == null)
 +                        continue;
 +
 +                    indexer.begin();
 +                    for (Row row : rows)
 +                        if (row != null)
 +                            indexer.removeRow(row);
 +                    indexer.finish();
 +                }
 +            }
 +        }
 +    }
 +
 +    /**
 +     * A single-use transaction for updating indexes for a single partition during cleanup, where
 +     * partitions and rows are only removed
 +     * TODO : make this smarter at batching updates so we can use a single transaction to process multiple rows in
 +     * a single partition
 +     */
 +    private static final class CleanupGCTransaction implements CleanupTransaction
 +    {
 +        private final DecoratedKey key;
 +        private final PartitionColumns columns;
 +        private final int nowInSec;
 +        private final Collection<Index> indexes;
 +
 +        private Row row;
 +        private DeletionTime partitionDelete;
 +
 +        private CleanupGCTransaction(DecoratedKey key,
 +                                     PartitionColumns columns,
 +                                     int nowInSec,
 +                                     Collection<Index> indexes)
 +        {
 +            this.key = key;
 +            this.columns = columns;
 +            this.indexes = indexes;
 +            this.nowInSec = nowInSec;
 +        }
 +
 +        public void start()
 +        {
 +        }
 +
 +        public void onPartitionDeletion(DeletionTime deletionTime)
 +        {
 +            partitionDelete = deletionTime;
 +        }
 +
 +        public void onRowDelete(Row row)
 +        {
 +            this.row = row;
 +        }
 +
 +        public void commit()
 +        {
 +            if (row == null && partitionDelete == null)
 +                return;
 +
 +            try (OpOrder.Group opGroup = Keyspace.writeOrder.start())
 +            {
 +                for (Index index : indexes)
 +                {
 +                    Index.Indexer indexer = index.indexerFor(key, columns, nowInSec, opGroup, Type.CLEANUP);
 +                    if (indexer == null)
 +                        continue;
 +
 +                    indexer.begin();
 +
 +                    if (partitionDelete != null)
 +                        indexer.partitionDelete(partitionDelete);
 +
 +                    if (row != null)
 +                        indexer.removeRow(row);
 +
 +                    indexer.finish();
 +                }
 +            }
 +        }
 +    }
 +
 +    private static void executeBlocking(Callable<?> task)
 +    {
 +        if (null != task)
 +            FBUtilities.waitOnFuture(blockingExecutor.submit(task));
 +    }
 +
 +    private static void executeAllBlocking(Stream<Index> indexers, Function<Index, Callable<?>> function)
 +    {
 +        List<Future<?>> waitFor = new ArrayList<>();
 +        indexers.forEach(indexer -> {
 +            Callable<?> task = function.apply(indexer);
 +            if (null != task)
 +                waitFor.add(blockingExecutor.submit(task));
 +        });
 +        FBUtilities.waitOnFutures(waitFor);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index f0cdcf5,ba060d4..3283723
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@@ -2183,10 -2246,10 +2183,10 @@@ public abstract class SSTableReader ext
  
              // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist
              // the read meter when in client mode.
-             if (Schema.isSystemKeyspace(desc.ksname))
 -            if (SystemKeyspace.NAME.equals(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
++            if (Schema.isSystemKeyspace(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
              {
                  readMeter = null;
 -                readMeterSyncFuture = null;
 +                readMeterSyncFuture = NULL;
                  return;
              }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/CQLTester.java
index 3d45393,98b8e23..3d8d03b
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@@ -140,86 -137,7 +140,87 @@@ public abstract class CQLTeste
      // We don't use USE_PREPARED_VALUES in the code below so some test can foce value preparation (if the result
      // is not expected to be the same without preparation)
      private boolean usePrepared = USE_PREPARED_VALUES;
 -    private static final boolean reusePrepared = Boolean.valueOf(System.getProperty("cassandra.test.reuse_prepared", "true"));
 +    private static boolean reusePrepared = REUSE_PREPARED;
 +
 +    public static void prepareServer()
 +    {
 +        if (isServerPrepared)
 +            return;
 +
 +        // Cleanup first
 +        try
 +        {
 +            cleanupAndLeaveDirs();
 +        }
 +        catch (IOException e)
 +        {
 +            logger.error("Failed to cleanup and recreate directories.");
 +            throw new RuntimeException(e);
 +        }
 +
 +        Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()
 +        {
 +            public void uncaughtException(Thread t, Throwable e)
 +            {
 +                logger.error("Fatal exception in thread " + t, e);
 +            }
 +        });
 +
 +        ThreadAwareSecurityManager.install();
 +
++        DatabaseDescriptor.setDaemonInitialized();
 +        Keyspace.setInitialized();
 +        isServerPrepared = true;
 +    }
 +
 +    public static void cleanupAndLeaveDirs() throws IOException
 +    {
 +        // We need to stop and unmap all CLS instances prior to cleanup() or we'll get failures on Windows.
 +        CommitLog.instance.stopUnsafe(true);
 +        mkdirs();
 +        cleanup();
 +        mkdirs();
 +        CommitLog.instance.restartUnsafe();
 +    }
 +
 +    public static void cleanup()
 +    {
 +        // clean up commitlog
 +        String[] directoryNames = { DatabaseDescriptor.getCommitLogLocation(), };
 +        for (String dirName : directoryNames)
 +        {
 +            File dir = new File(dirName);
 +            if (!dir.exists())
 +                throw new RuntimeException("No such directory: " + dir.getAbsolutePath());
 +            FileUtils.deleteRecursive(dir);
 +        }
 +
 +        cleanupSavedCaches();
 +
 +        // clean up data directory which are stored as data directory/keyspace/data files
 +        for (String dirName : DatabaseDescriptor.getAllDataFileLocations())
 +        {
 +            File dir = new File(dirName);
 +            if (!dir.exists())
 +                throw new RuntimeException("No such directory: " + dir.getAbsolutePath());
 +            FileUtils.deleteRecursive(dir);
 +        }
 +    }
 +
 +    public static void mkdirs()
 +    {
 +        DatabaseDescriptor.createAllDirectories();
 +    }
 +
 +    public static void cleanupSavedCaches()
 +    {
 +        File cachesDir = new File(DatabaseDescriptor.getSavedCachesLocation());
 +
 +        if (!cachesDir.exists() || !cachesDir.isDirectory())
 +            return;
 +
 +        FileUtils.delete(cachesDir.listFiles());
 +    }
  
      @BeforeClass
      public static void setUpClass()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 557beba,5e2fffe..e6d18c4
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@@ -32,9 -32,9 +32,10 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.Util;
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.Config;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.config.Schema;
  import org.apache.cassandra.cql3.QueryProcessor;
  import org.apache.cassandra.cql3.UntypedResultSet;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 5ac1b31,bc5be46..37b5fa9
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@@ -28,7 -27,7 +28,8 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.exceptions.ConfigurationException;
  import org.apache.cassandra.gms.ApplicationState;
  import org.apache.cassandra.gms.Gossiper;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
index 6f76db4,0000000..a2c9cf9
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
@@@ -1,56 -1,0 +1,63 @@@
 +/*
 + * 
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + * 
 + *   http://www.apache.org/licenses/LICENSE-2.0
 + * 
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + * 
 + */
 +package org.apache.cassandra.metrics;
 +
 +import java.net.InetAddress;
 +import java.util.Map;
 +import java.util.UUID;
 +
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import com.google.common.collect.Iterators;
 +
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.UntypedResultSet;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.db.marshal.Int32Type;
 +import org.apache.cassandra.db.marshal.UUIDType;
 +import org.apache.cassandra.hints.HintsService;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 +
 +public class HintedHandOffMetricsTest
 +{
++    @BeforeClass
++    public static void initDD()
++    {
++        DatabaseDescriptor.setDaemonInitialized();
++    }
++
 +    @Test
 +    public void testHintsMetrics() throws Exception
 +    {
 +        DatabaseDescriptor.getHintsDirectory().mkdirs();
 +
 +        for (int i = 0; i < 99; i++)
 +            HintsService.instance.metrics.incrPastWindow(InetAddress.getLocalHost());
 +        HintsService.instance.metrics.log();
 +
 +        UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS);
 +        Map<UUID, Integer> returned = rows.one().getMap("hints_dropped", UUIDType.instance, Int32Type.instance);
 +        assertEquals(Iterators.getLast(returned.values().iterator()).intValue(), 99);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
index f0c850d,4e160c2..bb2b9b0
--- a/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
+++ b/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
@@@ -18,14 -18,14 +18,21 @@@
  */
  package org.apache.cassandra.utils.concurrent;
  
++import org.junit.BeforeClass;
  import org.junit.Ignore;
  import org.junit.Test;
  
  import junit.framework.Assert;
++import org.apache.cassandra.config.DatabaseDescriptor;
  
  @Ignore
  public abstract class AbstractTransactionalTest
  {
++    @BeforeClass
++    public static void setupDD()
++    {
++        DatabaseDescriptor.setDaemonInitialized();
++    }
  
      protected abstract TestableTransaction newTest() throws Exception;
  


[11/19] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by yu...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0fe82be8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0fe82be8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0fe82be8

Branch: refs/heads/cassandra-3.11
Commit: 0fe82be83cceceb12172d63913388678253413bc
Parents: e9b7a0f 66f1aaf
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Dec 13 15:55:34 2016 -0800
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:55:34 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/config/DatabaseDescriptor.java    | 12 +++++++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++++--
 .../cassandra/db/PartitionRangeReadCommand.java |  3 +-
 .../cassandra/db/compaction/CompactionTask.java | 18 +++++++----
 .../cassandra/db/lifecycle/LogTransaction.java  |  3 +-
 .../apache/cassandra/db/lifecycle/Tracker.java  | 34 ++++++++++++--------
 .../cassandra/index/SecondaryIndexManager.java  |  4 ++-
 .../io/sstable/format/SSTableReader.java        |  2 +-
 .../cassandra/service/CassandraDaemon.java      |  1 +
 .../service/EmbeddedCassandraService.java       |  2 ++
 .../config/DatabaseDescriptorTest.java          |  6 ++++
 .../org/apache/cassandra/cql3/CQLTester.java    |  1 +
 .../apache/cassandra/db/SystemKeyspaceTest.java |  2 ++
 .../db/context/CounterContextTest.java          |  8 +++++
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 ++-
 .../cassandra/db/lifecycle/TrackerTest.java     |  7 ++--
 .../cassandra/dht/StreamStateStoreTest.java     |  7 ++++
 .../cassandra/gms/FailureDetectorTest.java      |  2 ++
 .../org/apache/cassandra/gms/GossiperTest.java  |  5 +++
 .../io/sstable/CQLSSTableWriterTest.java        |  2 ++
 .../cassandra/locator/CloudstackSnitchTest.java |  2 ++
 .../apache/cassandra/locator/EC2SnitchTest.java |  2 ++
 .../locator/GoogleCloudSnitchTest.java          |  2 ++
 .../metrics/HintedHandOffMetricsTest.java       |  7 ++++
 .../service/StorageServiceServerTest.java       |  1 +
 .../concurrent/AbstractTransactionalTest.java   |  7 ++++
 27 files changed, 124 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 5621c93,8cff097..145afb9
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,18 -1,5 +1,19 @@@
 -2.2.9
 +3.0.11
 + * Mark MVs as built after successful bootstrap (CASSANDRA-12984)
 + * Estimated TS drop-time histogram updated with Cell.NO_DELETION_TIME (CASSANDRA-13040)
 + * Nodetool compactionstats fails with NullPointerException (CASSANDRA-13021)
 + * Thread local pools never cleaned up (CASSANDRA-13033)
 + * Set RPC_READY to false when draining or if a node is marked as shutdown (CASSANDRA-12781)
 + * CQL often queries static columns unnecessarily (CASSANDRA-12768)
 + * Make sure sstables only get committed when it's safe to discard commit log records (CASSANDRA-12956)
 + * Reject default_time_to_live option when creating or altering MVs (CASSANDRA-12868)
 + * Nodetool should use a more sane max heap size (CASSANDRA-12739)
 + * LocalToken ensures token values are cloned on heap (CASSANDRA-12651)
 + * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
 + * Prevent reloading of logback.xml from UDF sandbox (CASSANDRA-12535)
 + * Reenable HeapPool (CASSANDRA-12900)
 +Merged from 2.2:
+  * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
   * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
   * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
   * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 71e1653,4bc46d0..39ed804
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -388,13 -388,17 +388,17 @@@ public class ColumnFamilyStore implemen
  
          logger.info("Initializing {}.{}", keyspace.getName(), name);
  
-         // scan for sstables corresponding to this cf and load them
-         data = new Tracker(this, loadSSTables);
+         // Create Memtable only on online
+         Memtable initialMemtable = null;
+         if (DatabaseDescriptor.isDaemonInitialized())
+             initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
+         data = new Tracker(initialMemtable, loadSSTables);
  
+         // scan for sstables corresponding to this cf and load them
          if (data.loadsstables)
          {
 -            Directories.SSTableLister sstableFiles = directories.sstableLister().skipTemporary(true);
 -            Collection<SSTableReader> sstables = SSTableReader.openAll(sstableFiles.list().entrySet(), metadata, this.partitioner);
 +            Directories.SSTableLister sstableFiles = directories.sstableLister(Directories.OnTxnErr.IGNORE).skipTemporary(true);
 +            Collection<SSTableReader> sstables = SSTableReader.openAll(sstableFiles.list().entrySet(), metadata);
              data.addInitialSSTables(sstables);
          }
  
@@@ -1953,10 -2758,12 +1957,10 @@@
              {
                  public Void call()
                  {
-                     cfs.data.reset();
+                     cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
 -                    cfs.getCompactionStrategy().shutdown();
 -                    cfs.getCompactionStrategy().startup();
                      return null;
                  }
 -            }, true);
 +            }, true, false);
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 99e24c8,0000000..17adef0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -1,322 -1,0 +1,323 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.db;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.List;
 +import java.util.Optional;
 +
 +import com.google.common.collect.Iterables;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.db.filter.*;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.lifecycle.View;
 +import org.apache.cassandra.db.partitions.*;
 +import org.apache.cassandra.db.rows.BaseRowIterator;
 +import org.apache.cassandra.db.transform.Transformation;
 +import org.apache.cassandra.dht.AbstractBounds;
 +import org.apache.cassandra.exceptions.RequestExecutionException;
 +import org.apache.cassandra.index.Index;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.metrics.TableMetrics;
 +import org.apache.cassandra.net.MessageOut;
 +import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.service.StorageProxy;
 +import org.apache.cassandra.service.pager.*;
 +import org.apache.cassandra.thrift.ThriftResultsMerger;
 +import org.apache.cassandra.tracing.Tracing;
 +import org.apache.cassandra.utils.FBUtilities;
 +
 +/**
 + * A read command that selects a (part of a) range of partitions.
 + */
 +public class PartitionRangeReadCommand extends ReadCommand
 +{
 +    protected static final SelectionDeserializer selectionDeserializer = new Deserializer();
 +
 +    private final DataRange dataRange;
 +    private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
 +
 +    public PartitionRangeReadCommand(boolean isDigest,
 +                                     int digestVersion,
 +                                     boolean isForThrift,
 +                                     CFMetaData metadata,
 +                                     int nowInSec,
 +                                     ColumnFilter columnFilter,
 +                                     RowFilter rowFilter,
 +                                     DataLimits limits,
 +                                     DataRange dataRange,
 +                                     Optional<IndexMetadata> index)
 +    {
 +        super(Kind.PARTITION_RANGE, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits);
 +        this.dataRange = dataRange;
 +        this.index = index;
 +    }
 +
 +    public PartitionRangeReadCommand(CFMetaData metadata,
 +                                     int nowInSec,
 +                                     ColumnFilter columnFilter,
 +                                     RowFilter rowFilter,
 +                                     DataLimits limits,
 +                                     DataRange dataRange,
 +                                     Optional<IndexMetadata> index)
 +    {
 +        this(false, 0, false, metadata, nowInSec, columnFilter, rowFilter, limits, dataRange, index);
 +    }
 +
 +    /**
 +     * Creates a new read command that query all the data in the table.
 +     *
 +     * @param metadata the table to query.
 +     * @param nowInSec the time in seconds to use are "now" for this query.
 +     *
 +     * @return a newly created read command that queries everything in the table.
 +     */
 +    public static PartitionRangeReadCommand allDataRead(CFMetaData metadata, int nowInSec)
 +    {
 +        return new PartitionRangeReadCommand(metadata,
 +                                             nowInSec,
 +                                             ColumnFilter.all(metadata),
 +                                             RowFilter.NONE,
 +                                             DataLimits.NONE,
 +                                             DataRange.allData(metadata.partitioner),
 +                                             Optional.empty());
 +    }
 +
 +    public DataRange dataRange()
 +    {
 +        return dataRange;
 +    }
 +
 +    public ClusteringIndexFilter clusteringIndexFilter(DecoratedKey key)
 +    {
 +        return dataRange.clusteringIndexFilter(key);
 +    }
 +
 +    public boolean isNamesQuery()
 +    {
 +        return dataRange.isNamesQuery();
 +    }
 +
 +    public PartitionRangeReadCommand forSubRange(AbstractBounds<PartitionPosition> range)
 +    {
 +        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange().forSubRange(range), index);
 +    }
 +
 +    public PartitionRangeReadCommand copy()
 +    {
 +        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange(), index);
 +    }
 +
 +    public PartitionRangeReadCommand withUpdatedLimit(DataLimits newLimits)
 +    {
 +        return new PartitionRangeReadCommand(metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, dataRange(), index);
 +    }
 +
 +    public long getTimeout()
 +    {
 +        return DatabaseDescriptor.getRangeRpcTimeout();
 +    }
 +
 +    public boolean selectsKey(DecoratedKey key)
 +    {
 +        if (!dataRange().contains(key))
 +            return false;
 +
 +        return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().getKeyValidator());
 +    }
 +
 +    public boolean selectsClustering(DecoratedKey key, Clustering clustering)
 +    {
 +        if (clustering == Clustering.STATIC_CLUSTERING)
 +            return !columnFilter().fetchedColumns().statics.isEmpty();
 +
 +        if (!dataRange().clusteringIndexFilter(key).selects(clustering))
 +            return false;
 +        return rowFilter().clusteringKeyRestrictionsAreSatisfiedBy(clustering);
 +    }
 +
 +    public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState) throws RequestExecutionException
 +    {
 +        return StorageProxy.getRangeSlice(this, consistency);
 +    }
 +
 +    public QueryPager getPager(PagingState pagingState, int protocolVersion)
 +    {
 +            return new PartitionRangeQueryPager(this, pagingState, protocolVersion);
 +    }
 +
 +    protected void recordLatency(TableMetrics metric, long latencyNanos)
 +    {
 +        metric.rangeLatency.addNano(latencyNanos);
 +    }
 +
 +    protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
 +    {
 +        ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
 +        Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));
 +
 +        // fetch data from current memtable, historical memtables, and SSTables in the correct order.
 +        final List<UnfilteredPartitionIterator> iterators = new ArrayList<>(Iterables.size(view.memtables) + view.sstables.size());
 +
 +        try
 +        {
 +            for (Memtable memtable : view.memtables)
 +            {
 +                @SuppressWarnings("resource") // We close on exception and on closing the result returned by this method
 +                Memtable.MemtableUnfilteredPartitionIterator iter = memtable.makePartitionIterator(columnFilter(), dataRange(), isForThrift());
 +                oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, iter.getMinLocalDeletionTime());
 +                iterators.add(isForThrift() ? ThriftResultsMerger.maybeWrap(iter, metadata(), nowInSec()) : iter);
 +            }
 +
 +            for (SSTableReader sstable : view.sstables)
 +            {
 +                @SuppressWarnings("resource") // We close on exception and on closing the result returned by this method
 +                UnfilteredPartitionIterator iter = sstable.getScanner(columnFilter(), dataRange(), isForThrift());
 +                iterators.add(isForThrift() ? ThriftResultsMerger.maybeWrap(iter, metadata(), nowInSec()) : iter);
 +                if (!sstable.isRepaired())
 +                    oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime());
 +            }
-             return checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
++            return iterators.isEmpty() ? EmptyIterators.unfilteredPartition(metadata(), isForThrift())
++                                       : checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
 +        }
 +        catch (RuntimeException | Error e)
 +        {
 +            try
 +            {
 +                FBUtilities.closeAll(iterators);
 +            }
 +            catch (Exception suppressed)
 +            {
 +                e.addSuppressed(suppressed);
 +            }
 +
 +            throw e;
 +        }
 +    }
 +
 +    @Override
 +    protected int oldestUnrepairedTombstone()
 +    {
 +        return oldestUnrepairedTombstone;
 +    }
 +
 +    private UnfilteredPartitionIterator checkCacheFilter(UnfilteredPartitionIterator iter, final ColumnFamilyStore cfs)
 +    {
 +        class CacheFilter extends Transformation
 +        {
 +            @Override
 +            public BaseRowIterator applyToPartition(BaseRowIterator iter)
 +            {
 +                // Note that we rely on the fact that until we actually advance 'iter', no really costly operation is actually done
 +                // (except for reading the partition key from the index file) due to the call to mergeLazily in queryStorage.
 +                DecoratedKey dk = iter.partitionKey();
 +
 +                // Check if this partition is in the rowCache and if it is, if  it covers our filter
 +                CachedPartition cached = cfs.getRawCachedPartition(dk);
 +                ClusteringIndexFilter filter = dataRange().clusteringIndexFilter(dk);
 +
 +                if (cached != null && cfs.isFilterFullyCoveredBy(filter, limits(), cached, nowInSec()))
 +                {
 +                    // We won't use 'iter' so close it now.
 +                    iter.close();
 +
 +                    return filter.getUnfilteredRowIterator(columnFilter(), cached);
 +                }
 +
 +                return iter;
 +            }
 +        }
 +        return Transformation.apply(iter, new CacheFilter());
 +    }
 +
 +    public MessageOut<ReadCommand> createMessage(int version)
 +    {
 +        return dataRange().isPaging()
 +             ? new MessageOut<>(MessagingService.Verb.PAGED_RANGE, this, pagedRangeSerializer)
 +             : new MessageOut<>(MessagingService.Verb.RANGE_SLICE, this, rangeSliceSerializer);
 +    }
 +
 +    protected void appendCQLWhereClause(StringBuilder sb)
 +    {
 +        if (dataRange.isUnrestricted() && rowFilter().isEmpty())
 +            return;
 +
 +        sb.append(" WHERE ");
 +        // We put the row filter first because the data range can end by "ORDER BY"
 +        if (!rowFilter().isEmpty())
 +        {
 +            sb.append(rowFilter());
 +            if (!dataRange.isUnrestricted())
 +                sb.append(" AND ");
 +        }
 +        if (!dataRange.isUnrestricted())
 +            sb.append(dataRange.toCQLString(metadata()));
 +    }
 +
 +    /**
 +     * Allow to post-process the result of the query after it has been reconciled on the coordinator
 +     * but before it is passed to the CQL layer to return the ResultSet.
 +     *
 +     * See CASSANDRA-8717 for why this exists.
 +     */
 +    public PartitionIterator postReconciliationProcessing(PartitionIterator result)
 +    {
 +        ColumnFamilyStore cfs = Keyspace.open(metadata().ksName).getColumnFamilyStore(metadata().cfName);
 +        Index index = getIndex(cfs);
 +        return index == null ? result : index.postProcessorFor(this).apply(result, this);
 +    }
 +
 +    @Override
 +    public String toString()
 +    {
 +        return String.format("Read(%s.%s columns=%s rowfilter=%s limits=%s %s)",
 +                             metadata().ksName,
 +                             metadata().cfName,
 +                             columnFilter(),
 +                             rowFilter(),
 +                             limits(),
 +                             dataRange().toString(metadata()));
 +    }
 +
 +    protected void serializeSelection(DataOutputPlus out, int version) throws IOException
 +    {
 +        DataRange.serializer.serialize(dataRange(), out, version, metadata());
 +    }
 +
 +    protected long selectionSerializedSize(int version)
 +    {
 +        return DataRange.serializer.serializedSize(dataRange(), version, metadata());
 +    }
 +
 +    private static class Deserializer extends SelectionDeserializer
 +    {
 +        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
 +        throws IOException
 +        {
 +            DataRange range = DataRange.serializer.deserialize(in, version, metadata);
 +            return new PartitionRangeReadCommand(isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, range, index);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 0c4e144,20d3dc0..f0a1f47
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@@ -223,15 -218,20 +223,19 @@@ public class CompactionTask extends Abs
              for (SSTableReader reader : newSStables)
                  newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
  
-             double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
-             Summary mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
-             logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                       taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, mergeSummary.totalSourceRows, totalKeysWritten, mergeSummary.partitionMerge));
-             logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-             logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
- 
              if (offline)
+             {
                  Refs.release(Refs.selfRefs(newSStables));
+             }
+             else
+             {
+                 double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
 -                long totalSourceRows = 0;
 -                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
++                Summary mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
+                 logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
 -                                           taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
++                                           taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, mergeSummary.totalSourceRows, totalKeysWritten, mergeSummary.partitionMerge));
+                 logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
 -                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten));
++                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
+             }
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
index ca644eb,0000000..350477c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
@@@ -1,444 -1,0 +1,445 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.db.lifecycle;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.nio.file.Files;
 +import java.nio.file.NoSuchFileException;
 +import java.util.*;
 +import java.util.concurrent.ConcurrentLinkedQueue;
 +import java.util.concurrent.TimeUnit;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.util.concurrent.Runnables;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.ScheduledExecutors;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.db.ColumnFamilyStore;
 +import org.apache.cassandra.db.Directories;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.db.compaction.OperationType;
 +import org.apache.cassandra.db.lifecycle.LogRecord.Type;
 +import org.apache.cassandra.io.sstable.Component;
 +import org.apache.cassandra.io.sstable.Descriptor;
 +import org.apache.cassandra.io.sstable.SSTable;
 +import org.apache.cassandra.io.sstable.SnapshotDeletingTask;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.utils.*;
 +import org.apache.cassandra.utils.concurrent.Ref;
 +import org.apache.cassandra.utils.concurrent.RefCounted;
 +import org.apache.cassandra.utils.concurrent.Transactional;
 +
 +/**
 + * IMPORTANT: When this object is involved in a transactional graph, and is not encapsulated in a LifecycleTransaction,
 + * for correct behaviour its commit MUST occur before any others, since it may legitimately fail. This is consistent
 + * with the Transactional API, which permits one failing action to occur at the beginning of the commit phase, but also
 + * *requires* that the prepareToCommit() phase only take actions that can be rolled back.
 + *
 + * IMPORTANT: The transaction must complete (commit or abort) before any temporary files are deleted, even though the
 + * txn log file itself will not be deleted until all tracked files are deleted. This is required by FileLister to ensure
 + * a consistent disk state. LifecycleTransaction ensures this requirement, so this class should really never be used
 + * outside of LT. @see FileLister.classifyFiles(TransactionData txn)
 + *
 + * A class that tracks sstable files involved in a transaction across sstables:
 + * if the transaction succeeds the old files should be deleted and the new ones kept; vice-versa if it fails.
 + *
 + * The transaction log file contains new and old sstables as follows:
 + *
 + * add:[sstable-2][CRC]
 + * remove:[sstable-1,max_update_time,num files][CRC]
 + *
 + * where sstable-2 is a new sstable to be retained if the transaction succeeds and sstable-1 is an old sstable to be
 + * removed. CRC is an incremental CRC of the file content up to this point. For old sstable files we also log the
 + * last update time of all files for the sstable descriptor and a checksum of vital properties such as update times
 + * and file sizes.
 + *
 + * Upon commit we add a final line to the log file:
 + *
 + * commit:[commit_time][CRC]
 + *
 + * When the transaction log is cleaned-up by the TransactionTidier, which happens only after any old sstables have been
 + * osoleted, then any sstable files for old sstables are removed before deleting the transaction log if the transaction
 + * was committed, vice-versa if the transaction was aborted.
 + *
 + * On start-up we look for any transaction log files and repeat the cleanup process described above.
 + *
 + * See CASSANDRA-7066 for full details.
 + */
 +class LogTransaction extends Transactional.AbstractTransactional implements Transactional
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(LogTransaction.class);
 +
 +    /**
 +     * If the format of the lines in the transaction log is wrong or the checksum
 +     * does not match, then we throw this exception.
 +     */
 +    public static final class CorruptTransactionLogException extends RuntimeException
 +    {
 +        public final LogFile txnFile;
 +
 +        public CorruptTransactionLogException(String message, LogFile txnFile)
 +        {
 +            super(message);
 +            this.txnFile = txnFile;
 +        }
 +    }
 +
 +    private final Tracker tracker;
 +    private final LogFile txnFile;
 +    private final Ref<LogTransaction> selfRef;
 +    // Deleting sstables is tricky because the mmapping might not have been finalized yet,
 +    // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
 +    // Additionally, we need to make sure to delete the data file first, so on restart the others
 +    // will be recognized as GCable.
 +    private static final Queue<Runnable> failedDeletions = new ConcurrentLinkedQueue<>();
 +
 +    LogTransaction(OperationType opType)
 +    {
 +        this(opType, null);
 +    }
 +
 +    LogTransaction(OperationType opType, Tracker tracker)
 +    {
 +        this.tracker = tracker;
 +        this.txnFile = new LogFile(opType, UUIDGen.getTimeUUID());
 +        this.selfRef = new Ref<>(this, new TransactionTidier(txnFile));
 +
 +        if (logger.isTraceEnabled())
 +            logger.trace("Created transaction logs with id {}", txnFile.id());
 +    }
 +
 +    /**
 +     * Track a reader as new.
 +     **/
 +    void trackNew(SSTable table)
 +    {
 +        txnFile.add(Type.ADD, table);
 +    }
 +
 +    /**
 +     * Stop tracking a reader as new.
 +     */
 +    void untrackNew(SSTable table)
 +    {
 +        txnFile.remove(Type.ADD, table);
 +    }
 +
 +    /**
 +     * Schedule a reader for deletion as soon as it is fully unreferenced.
 +     */
 +    SSTableTidier obsoleted(SSTableReader reader)
 +    {
 +        if (txnFile.contains(Type.ADD, reader))
 +        {
 +            if (txnFile.contains(Type.REMOVE, reader))
 +                throw new IllegalArgumentException();
 +
 +            return new SSTableTidier(reader, true, this);
 +        }
 +
 +        txnFile.add(Type.REMOVE, reader);
 +
 +        if (tracker != null)
 +            tracker.notifyDeleting(reader);
 +
 +        return new SSTableTidier(reader, false, this);
 +    }
 +
 +    OperationType type()
 +    {
 +        return txnFile.type();
 +    }
 +
 +    UUID id()
 +    {
 +        return txnFile.id();
 +    }
 +
 +    @VisibleForTesting
 +    LogFile txnFile()
 +    {
 +        return txnFile;
 +    }
 +
 +    @VisibleForTesting
 +    List<File> logFiles()
 +    {
 +        return txnFile.getFiles();
 +    }
 +
 +    @VisibleForTesting
 +    List<String> logFilePaths()
 +    {
 +        return txnFile.getFilePaths();
 +    }
 +
 +    static void delete(File file)
 +    {
 +        try
 +        {
 +            if (logger.isTraceEnabled())
 +                logger.trace("Deleting {}", file);
 +
 +            Files.delete(file.toPath());
 +        }
 +        catch (NoSuchFileException e)
 +        {
 +            logger.error("Unable to delete {} as it does not exist", file);
 +        }
 +        catch (IOException e)
 +        {
 +            logger.error("Unable to delete {}", file, e);
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    /**
 +     * The transaction tidier.
 +     *
 +     * When the transaction reference is fully released we try to delete all the obsolete files
 +     * depending on the transaction result, as well as the transaction log file.
 +     */
 +    private static class TransactionTidier implements RefCounted.Tidy, Runnable
 +    {
 +        private final LogFile data;
 +
 +        TransactionTidier(LogFile data)
 +        {
 +            this.data = data;
 +        }
 +
 +        public void tidy() throws Exception
 +        {
 +            run();
 +        }
 +
 +        public String name()
 +        {
 +            return data.toString();
 +        }
 +
 +        public void run()
 +        {
 +            if (logger.isTraceEnabled())
 +                logger.trace("Removing files for transaction {}", name());
 +
 +            if (!data.completed())
 +            { // this happens if we forget to close a txn and the garbage collector closes it for us
 +                logger.error("{} was not completed, trying to abort it now", data);
 +                Throwable err = Throwables.perform((Throwable)null, data::abort);
 +                if (err != null)
 +                    logger.error("Failed to abort {}", data, err);
 +            }
 +
 +            Throwable err = data.removeUnfinishedLeftovers(null);
 +
 +            if (err != null)
 +            {
 +                logger.info("Failed deleting files for transaction {}, we'll retry after GC and on on server restart", name(), err);
 +                failedDeletions.add(this);
 +            }
 +            else
 +            {
 +                if (logger.isTraceEnabled())
 +                    logger.trace("Closing file transaction {}", name());
 +
 +                data.close();
 +            }
 +        }
 +    }
 +
 +    static class Obsoletion
 +    {
 +        final SSTableReader reader;
 +        final SSTableTidier tidier;
 +
 +        Obsoletion(SSTableReader reader, SSTableTidier tidier)
 +        {
 +            this.reader = reader;
 +            this.tidier = tidier;
 +        }
 +    }
 +
 +    /**
 +     * The SSTableReader tidier. When a reader is fully released and no longer referenced
 +     * by any one, we run this. It keeps a reference to the parent transaction and releases
 +     * it when done, so that the final transaction cleanup can run when all obsolete readers
 +     * are released.
 +     */
 +    public static class SSTableTidier implements Runnable
 +    {
 +        // must not retain a reference to the SSTableReader, else leak detection cannot kick in
 +        private final Descriptor desc;
 +        private final long sizeOnDisk;
 +        private final Tracker tracker;
 +        private final boolean wasNew;
 +        private final Ref<LogTransaction> parentRef;
 +
 +        public SSTableTidier(SSTableReader referent, boolean wasNew, LogTransaction parent)
 +        {
 +            this.desc = referent.descriptor;
 +            this.sizeOnDisk = referent.bytesOnDisk();
 +            this.tracker = parent.tracker;
 +            this.wasNew = wasNew;
 +            this.parentRef = parent.selfRef.tryRef();
 +        }
 +
 +        public void run()
 +        {
-             SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
++            if (tracker != null && !tracker.isDummy())
++                SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
 +
 +            try
 +            {
 +                // If we can't successfully delete the DATA component, set the task to be retried later: see TransactionTidier
 +                File datafile = new File(desc.filenameFor(Component.DATA));
 +
 +                delete(datafile);
 +                // let the remainder be cleaned up by delete
 +                SSTable.delete(desc, SSTable.discoverComponentsFor(desc));
 +            }
 +            catch (Throwable t)
 +            {
 +                logger.error("Failed deletion for {}, we'll retry after GC and on server restart", desc);
 +                failedDeletions.add(this);
 +                return;
 +            }
 +
 +            if (tracker != null && tracker.cfstore != null && !wasNew)
 +                tracker.cfstore.metric.totalDiskSpaceUsed.dec(sizeOnDisk);
 +
 +            // release the referent to the parent so that the all transaction files can be released
 +            parentRef.release();
 +        }
 +
 +        public void abort()
 +        {
 +            parentRef.release();
 +        }
 +    }
 +
 +
 +    static void rescheduleFailedDeletions()
 +    {
 +        Runnable task;
 +        while ( null != (task = failedDeletions.poll()))
 +            ScheduledExecutors.nonPeriodicTasks.submit(task);
 +
 +        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
 +        SnapshotDeletingTask.rescheduleFailedTasks();
 +    }
 +
 +    static void waitForDeletions()
 +    {
 +        FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(Runnables.doNothing(), 0, TimeUnit.MILLISECONDS));
 +    }
 +
 +    @VisibleForTesting
 +    Throwable complete(Throwable accumulate)
 +    {
 +        try
 +        {
 +            accumulate = selfRef.ensureReleased(accumulate);
 +            return accumulate;
 +        }
 +        catch (Throwable t)
 +        {
 +            logger.error("Failed to complete file transaction {}", id(), t);
 +            return Throwables.merge(accumulate, t);
 +        }
 +    }
 +
 +    protected Throwable doCommit(Throwable accumulate)
 +    {
 +        return complete(Throwables.perform(accumulate, txnFile::commit));
 +    }
 +
 +    protected Throwable doAbort(Throwable accumulate)
 +    {
 +        return complete(Throwables.perform(accumulate, txnFile::abort));
 +    }
 +
 +    protected void doPrepare() { }
 +
 +    /**
 +     * Called on startup to scan existing folders for any unfinished leftovers of
 +     * operations that were ongoing when the process exited. Also called by the standalone
 +     * sstableutil tool when the cleanup option is specified, @see StandaloneSSTableUtil.
 +     *
 +     */
 +    static void removeUnfinishedLeftovers(CFMetaData metadata)
 +    {
 +        removeUnfinishedLeftovers(new Directories(metadata, ColumnFamilyStore.getInitialDirectories()).getCFDirectories());
 +    }
 +
 +    @VisibleForTesting
 +    static void removeUnfinishedLeftovers(List<File> folders)
 +    {
 +        LogFilesByName logFiles = new LogFilesByName();
 +        folders.forEach(logFiles::list);
 +        logFiles.removeUnfinishedLeftovers();
 +    }
 +
 +    private static final class LogFilesByName
 +    {
 +        Map<String, List<File>> files = new HashMap<>();
 +
 +        void list(File folder)
 +        {
 +            Arrays.stream(folder.listFiles(LogFile::isLogFile)).forEach(this::add);
 +        }
 +
 +        void add(File file)
 +        {
 +            List<File> filesByName = files.get(file.getName());
 +            if (filesByName == null)
 +            {
 +                filesByName = new ArrayList<>();
 +                files.put(file.getName(), filesByName);
 +            }
 +
 +            filesByName.add(file);
 +        }
 +
 +        void removeUnfinishedLeftovers()
 +        {
 +            files.forEach(LogFilesByName::removeUnfinishedLeftovers);
 +        }
 +
 +        static void removeUnfinishedLeftovers(String name, List<File> logFiles)
 +        {
 +
 +            try(LogFile txn = LogFile.make(name, logFiles))
 +            {
 +                if (txn.verify())
 +                {
 +                    Throwable failure = txn.removeUnfinishedLeftovers(null);
 +                    if (failure != null)
 +                        logger.error("Failed to remove unfinished transaction leftovers for txn {}", txn, failure);
 +                }
 +                else
 +                {
 +                    logger.error("Unexpected disk state: failed to read transaction txn {}", txn);
 +                }
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index 5a3d524,e77ef78..9feaa3e
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@@ -193,15 -204,14 +201,13 @@@ public class Tracke
  
      /** (Re)initializes the tracker, purging all references. */
      @VisibleForTesting
-     public void reset()
+     public void reset(Memtable memtable)
      {
-         view.set(new View(
-                          !isDummy() ? ImmutableList.of(new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfstore))
-                                     : ImmutableList.<Memtable>of(),
-                          ImmutableList.<Memtable>of(),
-                          Collections.<SSTableReader, SSTableReader>emptyMap(),
-                          Collections.<SSTableReader, SSTableReader>emptyMap(),
-                          SSTableIntervalTree.empty()));
 -        view.set(new View(memtable != null ? singletonList(memtable) : Collections.<Memtable>emptyList(),
 -                          Collections.<Memtable>emptyList(),
 -                          Collections.<SSTableReader, SSTableReader>emptyMap(),
 -                          Collections.<SSTableReader>emptySet(),
 -                          Collections.<SSTableReader>emptySet(),
++        view.set(new View(memtable != null ? singletonList(memtable) : Collections.emptyList(),
++                          Collections.emptyList(),
++                          Collections.emptyMap(),
++                          Collections.emptyMap(),
+                           SSTableIntervalTree.empty()));
      }
  
      public Throwable dropSSTablesIfInvalid(Throwable accumulate)


[03/19] cassandra git commit: Temporarily fix bug that creates commit log when running offline tools

Posted by yu...@apache.org.
Temporarily fix bug that creates commit log when running offline tools

patch by yukim; reviewed by thobbs for CASSANDRA-8616


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/66f1aaf8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/66f1aaf8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/66f1aaf8

Branch: refs/heads/trunk
Commit: 66f1aaf88d3cde5c52b13d71d3326da5eda16fb1
Parents: fb29400
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Feb 11 19:06:27 2016 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:51:37 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/config/DatabaseDescriptor.java    | 12 +++++++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++++--
 .../cassandra/db/compaction/CompactionTask.java | 22 +++++++-----
 .../apache/cassandra/db/lifecycle/Tracker.java  | 36 ++++++++++++--------
 .../io/sstable/format/SSTableReader.java        |  8 +++--
 .../cassandra/service/CassandraDaemon.java      |  1 +
 .../service/EmbeddedCassandraService.java       |  2 ++
 .../io/sstable/CQLSSTableWriterLongTest.java    |  2 ++
 .../unit/org/apache/cassandra/SchemaLoader.java |  1 +
 .../config/DatabaseDescriptorTest.java          |  6 ++++
 .../apache/cassandra/db/CounterCellTest.java    |  3 ++
 .../org/apache/cassandra/db/NativeCellTest.java |  3 ++
 .../apache/cassandra/db/SystemKeyspaceTest.java |  2 ++
 .../db/context/CounterContextTest.java          |  8 +++++
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 ++-
 .../cassandra/db/lifecycle/TrackerTest.java     |  7 ++--
 .../cassandra/dht/StreamStateStoreTest.java     |  7 ++++
 .../cassandra/gms/FailureDetectorTest.java      |  2 ++
 .../org/apache/cassandra/gms/GossiperTest.java  |  5 +++
 .../io/sstable/CQLSSTableWriterTest.java        |  2 ++
 .../cassandra/locator/CloudstackSnitchTest.java |  2 ++
 .../apache/cassandra/locator/EC2SnitchTest.java |  2 ++
 .../locator/GoogleCloudSnitchTest.java          |  2 ++
 .../service/StorageServiceServerTest.java       |  1 +
 25 files changed, 118 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f02350d..8cff097 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.9
+ * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
  * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
  * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
  * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index a3fb79b..35debd0 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -102,6 +102,18 @@ public class DatabaseDescriptor
     private static Comparator<InetAddress> localComparator;
     private static boolean hasLoggedConfig;
 
+    private static boolean daemonInitialized;
+
+    public static boolean isDaemonInitialized()
+    {
+        return daemonInitialized;
+    }
+
+    public static void setDaemonInitialized()
+    {
+        daemonInitialized = true;
+    }
+
     public static void forceStaticInitialization() {}
     static
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c6b69dc..4bc46d0 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -388,9 +388,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         logger.info("Initializing {}.{}", keyspace.getName(), name);
 
-        // scan for sstables corresponding to this cf and load them
-        data = new Tracker(this, loadSSTables);
+        // Create Memtable only on online
+        Memtable initialMemtable = null;
+        if (DatabaseDescriptor.isDaemonInitialized())
+            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
+        data = new Tracker(initialMemtable, loadSSTables);
 
+        // scan for sstables corresponding to this cf and load them
         if (data.loadsstables)
         {
             Directories.SSTableLister sstableFiles = directories.sstableLister().skipTemporary(true);
@@ -2754,7 +2758,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             {
                 public Void call()
                 {
-                    cfs.data.reset();
+                    cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
                     cfs.getCompactionStrategy().shutdown();
                     cfs.getCompactionStrategy().startup();
                     return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 575c326..20d3dc0 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -127,7 +127,7 @@ public class CompactionTask extends AbstractCompactionTask
             }
         });
 
-        UUID taskId = SystemKeyspace.startCompaction(cfs, transaction.originals());
+        UUID taskId = offline ? null : SystemKeyspace.startCompaction(cfs, transaction.originals());
 
         // new sstables from flush can be added during a compaction, but only the compaction can remove them,
         // so in our single-threaded compaction world this is a valid way of determining if we're compacting
@@ -218,16 +218,20 @@ public class CompactionTask extends AbstractCompactionTask
             for (SSTableReader reader : newSStables)
                 newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
 
-            double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
-            long totalSourceRows = 0;
-            String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
-            logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                      taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
-            logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-            logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
-
             if (offline)
+            {
                 Refs.release(Refs.selfRefs(newSStables));
+            }
+            else
+            {
+                double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
+                long totalSourceRows = 0;
+                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
+                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
+                                           taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
+                logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
+                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten));
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index 5d5701f..e77ef78 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -31,7 +31,6 @@ import com.google.common.collect.*;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Memtable;
-import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,6 +51,7 @@ import static com.google.common.base.Predicates.not;
 import static com.google.common.collect.ImmutableSet.copyOf;
 import static com.google.common.collect.Iterables.filter;
 import static java.util.Collections.singleton;
+import static java.util.Collections.singletonList;
 import static org.apache.cassandra.db.lifecycle.Helpers.*;
 import static org.apache.cassandra.db.lifecycle.View.permitCompacting;
 import static org.apache.cassandra.db.lifecycle.View.updateCompacting;
@@ -61,21 +61,29 @@ import static org.apache.cassandra.utils.Throwables.merge;
 import static org.apache.cassandra.utils.concurrent.Refs.release;
 import static org.apache.cassandra.utils.concurrent.Refs.selfRefs;
 
+/**
+ * Tracker tracks live {@link View} of data store for a table.
+ */
 public class Tracker
 {
     private static final Logger logger = LoggerFactory.getLogger(Tracker.class);
 
-    public final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+    private final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+
     public final ColumnFamilyStore cfstore;
     final AtomicReference<View> view;
     public final boolean loadsstables;
 
-    public Tracker(ColumnFamilyStore cfstore, boolean loadsstables)
+    /**
+     * @param memtable Initial Memtable. Can be null.
+     * @param loadsstables true to indicate to load SSTables (TODO: remove as this is only accessed from 2i)
+     */
+    public Tracker(Memtable memtable, boolean loadsstables)
     {
-        this.cfstore = cfstore;
+        this.cfstore = memtable != null ? memtable.cfs : null;
         this.view = new AtomicReference<>();
         this.loadsstables = loadsstables;
-        this.reset();
+        this.reset(memtable);
     }
 
     public LifecycleTransaction tryModify(SSTableReader sstable, OperationType operationType)
@@ -196,16 +204,14 @@ public class Tracker
 
     /** (Re)initializes the tracker, purging all references. */
     @VisibleForTesting
-    public void reset()
+    public void reset(Memtable memtable)
     {
-        view.set(new View(
-                         !isDummy() ? ImmutableList.of(new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfstore))
-                                    : ImmutableList.<Memtable>of(),
-                         ImmutableList.<Memtable>of(),
-                         Collections.<SSTableReader, SSTableReader>emptyMap(),
-                         Collections.<SSTableReader>emptySet(),
-                         Collections.<SSTableReader>emptySet(),
-                         SSTableIntervalTree.empty()));
+        view.set(new View(memtable != null ? singletonList(memtable) : Collections.<Memtable>emptyList(),
+                          Collections.<Memtable>emptyList(),
+                          Collections.<SSTableReader, SSTableReader>emptyMap(),
+                          Collections.<SSTableReader>emptySet(),
+                          Collections.<SSTableReader>emptySet(),
+                          SSTableIntervalTree.empty()));
     }
 
     public Throwable dropSSTablesIfInvalid(Throwable accumulate)
@@ -473,7 +479,7 @@ public class Tracker
 
     public boolean isDummy()
     {
-        return cfstore == null;
+        return cfstore == null || !DatabaseDescriptor.isDaemonInitialized();
     }
 
     public void subscribe(INotificationConsumer consumer)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index fddf058..ba060d4 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -2246,7 +2246,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
             // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist
             // the read meter when in client mode.
-            if (SystemKeyspace.NAME.equals(desc.ksname))
+            if (SystemKeyspace.NAME.equals(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
             {
                 readMeter = null;
                 readMeterSyncFuture = null;
@@ -2272,9 +2272,11 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         {
             lookup.remove(desc);
             if (readMeterSyncFuture != null)
+            {
                 readMeterSyncFuture.cancel(true);
-            if (isCompacted.get())
-                SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+                if (isCompacted.get())
+                    SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+            }
             // don't ideally want to dropPageCache for the file until all instances have been released
             CLibrary.trySkipCache(desc.filenameFor(Component.DATA), 0, 0);
             CLibrary.trySkipCache(desc.filenameFor(Component.PRIMARY_INDEX), 0, 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index cf5f3c2..35e990f 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -505,6 +505,7 @@ public class CassandraDaemon
             try
             {
                 DatabaseDescriptor.forceStaticInitialization();
+                DatabaseDescriptor.setDaemonInitialized();
             }
             catch (ExceptionInInitializerError e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
index 659d851..6c154cd 100644
--- a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
+++ b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.service;
 
 import java.io.IOException;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.service.CassandraDaemon;
 
 /**
@@ -49,6 +50,7 @@ public class EmbeddedCassandraService
     public void start() throws IOException
     {
         cassandraDaemon = new CassandraDaemon();
+        DatabaseDescriptor.setDaemonInitialized();
         cassandraDaemon.init(null);
         cassandraDaemon.start();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
index fcec40d..ee719d1 100644
--- a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
+++ b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
@@ -30,6 +30,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.service.StorageService;
@@ -39,6 +40,7 @@ public class CQLSSTableWriterLongTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index a7cf7b4..2048f74 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -90,6 +90,7 @@ public class SchemaLoader
             }
         });
 
+        DatabaseDescriptor.setDaemonInitialized();
         Keyspace.setInitialized();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 5f231c3..7409535 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -47,6 +47,12 @@ import static org.junit.Assert.assertNull;
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class DatabaseDescriptorTest
 {
+    @BeforeClass
+    public static void setupDatabaseDescriptor()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testCFMetaDataSerialization() throws ConfigurationException, InvalidRequestException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/CounterCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
index 8d75b9a..5d4b8a8 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -30,6 +30,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
 import org.apache.cassandra.db.context.CounterContext;
@@ -57,6 +58,8 @@ public class CounterCellTest
         countLength   = 8; // size of long
 
         stepLength    = idLength + clockLength + countLength;
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/NativeCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NativeCellTest.java b/test/unit/org/apache/cassandra/db/NativeCellTest.java
index 70b7b87..4145a91 100644
--- a/test/unit/org/apache/cassandra/db/NativeCellTest.java
+++ b/test/unit/org/apache/cassandra/db/NativeCellTest.java
@@ -32,6 +32,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -128,6 +129,8 @@ public class NativeCellTest
         {
             throw new AssertionError();
         }
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index b8aa161..c3047b8 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -42,6 +42,8 @@ public class SystemKeyspaceTest
     @BeforeClass
     public static void prepSnapshotTracker()
     {
+        DatabaseDescriptor.setDaemonInitialized();
+
         if (FBUtilities.isWindows())
             WindowsFailedSnapshotTracker.deleteOldSnapshots();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
index a72d30d..4f587c6 100644
--- a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
+++ b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
@@ -22,9 +22,11 @@ package org.apache.cassandra.db.context;
 
 import java.nio.ByteBuffer;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ClockAndCount;
 import org.apache.cassandra.db.context.CounterContext.Relationship;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -48,6 +50,12 @@ public class CounterContextTest
     private static final int countLength = 8;
     private static final int stepLength = idLength + clockLength + countLength;
 
+    @BeforeClass
+    public static void setupDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testAllocate()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
index f13d1b7..737392e 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.lifecycle;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.junit.After;
 import org.junit.Before;
@@ -30,6 +31,8 @@ import junit.framework.Assert;
 import org.apache.cassandra.MockSchema;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
@@ -270,7 +273,7 @@ public class LifecycleTransactionTest extends AbstractTransactionalTest
 
         private static Tracker tracker(ColumnFamilyStore cfs, List<SSTableReader> readers)
         {
-            Tracker tracker = new Tracker(cfs, false);
+            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs), false);
             tracker.addInitialSSTables(readers);
             return tracker;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
index adeb778..04b4e4a 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -75,6 +75,7 @@ public class TrackerTest
     @BeforeClass
     public static void setUp()
     {
+        DatabaseDescriptor.setDaemonInitialized();
         MockSchema.cleanup();
     }
 
@@ -82,7 +83,7 @@ public class TrackerTest
     public void testTryModify()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = new Tracker(null, false);
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, true, cfs), MockSchema.sstable(1, cfs), MockSchema.sstable(2, cfs));
         tracker.addInitialSSTables(copyOf(readers));
         Assert.assertNull(tracker.tryModify(ImmutableList.of(MockSchema.sstable(0, cfs)), OperationType.COMPACTION));
@@ -146,7 +147,7 @@ public class TrackerTest
     public void testAddInitialSSTables()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = cfs.getTracker();
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),
                                                        MockSchema.sstable(1, 121, cfs),
                                                        MockSchema.sstable(2, 9, cfs));
@@ -163,7 +164,7 @@ public class TrackerTest
         boolean backups = DatabaseDescriptor.isIncrementalBackupsEnabled();
         DatabaseDescriptor.setIncrementalBackupsEnabled(false);
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = cfs.getTracker();
         MockListener listener = new MockListener(false);
         tracker.subscribe(listener);
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
index 86781d9..bdb654a 100644
--- a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
+++ b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
@@ -20,8 +20,10 @@ package org.apache.cassandra.dht;
 import java.net.InetAddress;
 import java.util.Collections;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.streaming.DefaultConnectionFactory;
 import org.apache.cassandra.streaming.StreamEvent;
 import org.apache.cassandra.streaming.StreamSession;
@@ -32,6 +34,11 @@ import static org.junit.Assert.assertTrue;
 
 public class StreamStateStoreTest
 {
+    @BeforeClass
+    public static void initDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
 
     @Test
     public void testUpdateAndQueryAvailableRanges()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
index 9325922..af099b0 100644
--- a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
+++ b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
@@ -45,6 +45,8 @@ public class FailureDetectorTest
     {
         // slow unit tests can cause problems with FailureDetector's GC pause handling
         System.setProperty("cassandra.max_local_pause_in_ms", "20000");
+
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/gms/GossiperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java
index ad07165..eb01305 100644
--- a/test/unit/org/apache/cassandra/gms/GossiperTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java
@@ -33,6 +33,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Token;
@@ -44,6 +45,10 @@ import static org.junit.Assert.assertEquals;
 
 public class GossiperTest
 {
+    static
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
     static final IPartitioner partitioner = new RandomPartitioner();
     StorageService ss = StorageService.instance;
     TokenMetadata tmd = StorageService.instance.getTokenMetadata();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 8a14428..5e2fffe 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -34,6 +34,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -52,6 +53,7 @@ public class CQLSSTableWriterTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 90e63e0..bc5be46 100644
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@ -27,6 +27,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
@@ -43,6 +44,7 @@ public class CloudstackSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
index 56bbb77..32383d9 100644
--- a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
@@ -33,6 +33,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -51,6 +52,7 @@ public class EC2SnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
index 1521454..f2450f4 100644
--- a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
@@ -31,6 +31,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -47,6 +48,7 @@ public class GoogleCloudSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index 4481501..a693a23 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -63,6 +63,7 @@ public class StorageServiceServerTest
     @BeforeClass
     public static void setUp() throws ConfigurationException
     {
+        DatabaseDescriptor.setDaemonInitialized();
         IEndpointSnitch snitch = new PropertyFileSnitch();
         DatabaseDescriptor.setEndpointSnitch(snitch);
         Keyspace.setInitialized();


[07/19] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by yu...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0fe82be8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0fe82be8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0fe82be8

Branch: refs/heads/trunk
Commit: 0fe82be83cceceb12172d63913388678253413bc
Parents: e9b7a0f 66f1aaf
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Dec 13 15:55:34 2016 -0800
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:55:34 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/config/DatabaseDescriptor.java    | 12 +++++++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++++--
 .../cassandra/db/PartitionRangeReadCommand.java |  3 +-
 .../cassandra/db/compaction/CompactionTask.java | 18 +++++++----
 .../cassandra/db/lifecycle/LogTransaction.java  |  3 +-
 .../apache/cassandra/db/lifecycle/Tracker.java  | 34 ++++++++++++--------
 .../cassandra/index/SecondaryIndexManager.java  |  4 ++-
 .../io/sstable/format/SSTableReader.java        |  2 +-
 .../cassandra/service/CassandraDaemon.java      |  1 +
 .../service/EmbeddedCassandraService.java       |  2 ++
 .../config/DatabaseDescriptorTest.java          |  6 ++++
 .../org/apache/cassandra/cql3/CQLTester.java    |  1 +
 .../apache/cassandra/db/SystemKeyspaceTest.java |  2 ++
 .../db/context/CounterContextTest.java          |  8 +++++
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 ++-
 .../cassandra/db/lifecycle/TrackerTest.java     |  7 ++--
 .../cassandra/dht/StreamStateStoreTest.java     |  7 ++++
 .../cassandra/gms/FailureDetectorTest.java      |  2 ++
 .../org/apache/cassandra/gms/GossiperTest.java  |  5 +++
 .../io/sstable/CQLSSTableWriterTest.java        |  2 ++
 .../cassandra/locator/CloudstackSnitchTest.java |  2 ++
 .../apache/cassandra/locator/EC2SnitchTest.java |  2 ++
 .../locator/GoogleCloudSnitchTest.java          |  2 ++
 .../metrics/HintedHandOffMetricsTest.java       |  7 ++++
 .../service/StorageServiceServerTest.java       |  1 +
 .../concurrent/AbstractTransactionalTest.java   |  7 ++++
 27 files changed, 124 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 5621c93,8cff097..145afb9
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,18 -1,5 +1,19 @@@
 -2.2.9
 +3.0.11
 + * Mark MVs as built after successful bootstrap (CASSANDRA-12984)
 + * Estimated TS drop-time histogram updated with Cell.NO_DELETION_TIME (CASSANDRA-13040)
 + * Nodetool compactionstats fails with NullPointerException (CASSANDRA-13021)
 + * Thread local pools never cleaned up (CASSANDRA-13033)
 + * Set RPC_READY to false when draining or if a node is marked as shutdown (CASSANDRA-12781)
 + * CQL often queries static columns unnecessarily (CASSANDRA-12768)
 + * Make sure sstables only get committed when it's safe to discard commit log records (CASSANDRA-12956)
 + * Reject default_time_to_live option when creating or altering MVs (CASSANDRA-12868)
 + * Nodetool should use a more sane max heap size (CASSANDRA-12739)
 + * LocalToken ensures token values are cloned on heap (CASSANDRA-12651)
 + * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
 + * Prevent reloading of logback.xml from UDF sandbox (CASSANDRA-12535)
 + * Reenable HeapPool (CASSANDRA-12900)
 +Merged from 2.2:
+  * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
   * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
   * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
   * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 71e1653,4bc46d0..39ed804
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -388,13 -388,17 +388,17 @@@ public class ColumnFamilyStore implemen
  
          logger.info("Initializing {}.{}", keyspace.getName(), name);
  
-         // scan for sstables corresponding to this cf and load them
-         data = new Tracker(this, loadSSTables);
+         // Create Memtable only on online
+         Memtable initialMemtable = null;
+         if (DatabaseDescriptor.isDaemonInitialized())
+             initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
+         data = new Tracker(initialMemtable, loadSSTables);
  
+         // scan for sstables corresponding to this cf and load them
          if (data.loadsstables)
          {
 -            Directories.SSTableLister sstableFiles = directories.sstableLister().skipTemporary(true);
 -            Collection<SSTableReader> sstables = SSTableReader.openAll(sstableFiles.list().entrySet(), metadata, this.partitioner);
 +            Directories.SSTableLister sstableFiles = directories.sstableLister(Directories.OnTxnErr.IGNORE).skipTemporary(true);
 +            Collection<SSTableReader> sstables = SSTableReader.openAll(sstableFiles.list().entrySet(), metadata);
              data.addInitialSSTables(sstables);
          }
  
@@@ -1953,10 -2758,12 +1957,10 @@@
              {
                  public Void call()
                  {
-                     cfs.data.reset();
+                     cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
 -                    cfs.getCompactionStrategy().shutdown();
 -                    cfs.getCompactionStrategy().startup();
                      return null;
                  }
 -            }, true);
 +            }, true, false);
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 99e24c8,0000000..17adef0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -1,322 -1,0 +1,323 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.db;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.List;
 +import java.util.Optional;
 +
 +import com.google.common.collect.Iterables;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.db.filter.*;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.lifecycle.View;
 +import org.apache.cassandra.db.partitions.*;
 +import org.apache.cassandra.db.rows.BaseRowIterator;
 +import org.apache.cassandra.db.transform.Transformation;
 +import org.apache.cassandra.dht.AbstractBounds;
 +import org.apache.cassandra.exceptions.RequestExecutionException;
 +import org.apache.cassandra.index.Index;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.metrics.TableMetrics;
 +import org.apache.cassandra.net.MessageOut;
 +import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.service.StorageProxy;
 +import org.apache.cassandra.service.pager.*;
 +import org.apache.cassandra.thrift.ThriftResultsMerger;
 +import org.apache.cassandra.tracing.Tracing;
 +import org.apache.cassandra.utils.FBUtilities;
 +
 +/**
 + * A read command that selects a (part of a) range of partitions.
 + */
 +public class PartitionRangeReadCommand extends ReadCommand
 +{
 +    protected static final SelectionDeserializer selectionDeserializer = new Deserializer();
 +
 +    private final DataRange dataRange;
 +    private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
 +
 +    public PartitionRangeReadCommand(boolean isDigest,
 +                                     int digestVersion,
 +                                     boolean isForThrift,
 +                                     CFMetaData metadata,
 +                                     int nowInSec,
 +                                     ColumnFilter columnFilter,
 +                                     RowFilter rowFilter,
 +                                     DataLimits limits,
 +                                     DataRange dataRange,
 +                                     Optional<IndexMetadata> index)
 +    {
 +        super(Kind.PARTITION_RANGE, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits);
 +        this.dataRange = dataRange;
 +        this.index = index;
 +    }
 +
 +    public PartitionRangeReadCommand(CFMetaData metadata,
 +                                     int nowInSec,
 +                                     ColumnFilter columnFilter,
 +                                     RowFilter rowFilter,
 +                                     DataLimits limits,
 +                                     DataRange dataRange,
 +                                     Optional<IndexMetadata> index)
 +    {
 +        this(false, 0, false, metadata, nowInSec, columnFilter, rowFilter, limits, dataRange, index);
 +    }
 +
 +    /**
 +     * Creates a new read command that query all the data in the table.
 +     *
 +     * @param metadata the table to query.
 +     * @param nowInSec the time in seconds to use are "now" for this query.
 +     *
 +     * @return a newly created read command that queries everything in the table.
 +     */
 +    public static PartitionRangeReadCommand allDataRead(CFMetaData metadata, int nowInSec)
 +    {
 +        return new PartitionRangeReadCommand(metadata,
 +                                             nowInSec,
 +                                             ColumnFilter.all(metadata),
 +                                             RowFilter.NONE,
 +                                             DataLimits.NONE,
 +                                             DataRange.allData(metadata.partitioner),
 +                                             Optional.empty());
 +    }
 +
 +    public DataRange dataRange()
 +    {
 +        return dataRange;
 +    }
 +
 +    public ClusteringIndexFilter clusteringIndexFilter(DecoratedKey key)
 +    {
 +        return dataRange.clusteringIndexFilter(key);
 +    }
 +
 +    public boolean isNamesQuery()
 +    {
 +        return dataRange.isNamesQuery();
 +    }
 +
 +    public PartitionRangeReadCommand forSubRange(AbstractBounds<PartitionPosition> range)
 +    {
 +        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange().forSubRange(range), index);
 +    }
 +
 +    public PartitionRangeReadCommand copy()
 +    {
 +        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange(), index);
 +    }
 +
 +    public PartitionRangeReadCommand withUpdatedLimit(DataLimits newLimits)
 +    {
 +        return new PartitionRangeReadCommand(metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, dataRange(), index);
 +    }
 +
 +    public long getTimeout()
 +    {
 +        return DatabaseDescriptor.getRangeRpcTimeout();
 +    }
 +
 +    public boolean selectsKey(DecoratedKey key)
 +    {
 +        if (!dataRange().contains(key))
 +            return false;
 +
 +        return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().getKeyValidator());
 +    }
 +
 +    public boolean selectsClustering(DecoratedKey key, Clustering clustering)
 +    {
 +        if (clustering == Clustering.STATIC_CLUSTERING)
 +            return !columnFilter().fetchedColumns().statics.isEmpty();
 +
 +        if (!dataRange().clusteringIndexFilter(key).selects(clustering))
 +            return false;
 +        return rowFilter().clusteringKeyRestrictionsAreSatisfiedBy(clustering);
 +    }
 +
 +    public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState) throws RequestExecutionException
 +    {
 +        return StorageProxy.getRangeSlice(this, consistency);
 +    }
 +
 +    public QueryPager getPager(PagingState pagingState, int protocolVersion)
 +    {
 +            return new PartitionRangeQueryPager(this, pagingState, protocolVersion);
 +    }
 +
 +    protected void recordLatency(TableMetrics metric, long latencyNanos)
 +    {
 +        metric.rangeLatency.addNano(latencyNanos);
 +    }
 +
 +    protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
 +    {
 +        ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
 +        Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));
 +
 +        // fetch data from current memtable, historical memtables, and SSTables in the correct order.
 +        final List<UnfilteredPartitionIterator> iterators = new ArrayList<>(Iterables.size(view.memtables) + view.sstables.size());
 +
 +        try
 +        {
 +            for (Memtable memtable : view.memtables)
 +            {
 +                @SuppressWarnings("resource") // We close on exception and on closing the result returned by this method
 +                Memtable.MemtableUnfilteredPartitionIterator iter = memtable.makePartitionIterator(columnFilter(), dataRange(), isForThrift());
 +                oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, iter.getMinLocalDeletionTime());
 +                iterators.add(isForThrift() ? ThriftResultsMerger.maybeWrap(iter, metadata(), nowInSec()) : iter);
 +            }
 +
 +            for (SSTableReader sstable : view.sstables)
 +            {
 +                @SuppressWarnings("resource") // We close on exception and on closing the result returned by this method
 +                UnfilteredPartitionIterator iter = sstable.getScanner(columnFilter(), dataRange(), isForThrift());
 +                iterators.add(isForThrift() ? ThriftResultsMerger.maybeWrap(iter, metadata(), nowInSec()) : iter);
 +                if (!sstable.isRepaired())
 +                    oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime());
 +            }
-             return checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
++            return iterators.isEmpty() ? EmptyIterators.unfilteredPartition(metadata(), isForThrift())
++                                       : checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
 +        }
 +        catch (RuntimeException | Error e)
 +        {
 +            try
 +            {
 +                FBUtilities.closeAll(iterators);
 +            }
 +            catch (Exception suppressed)
 +            {
 +                e.addSuppressed(suppressed);
 +            }
 +
 +            throw e;
 +        }
 +    }
 +
 +    @Override
 +    protected int oldestUnrepairedTombstone()
 +    {
 +        return oldestUnrepairedTombstone;
 +    }
 +
 +    private UnfilteredPartitionIterator checkCacheFilter(UnfilteredPartitionIterator iter, final ColumnFamilyStore cfs)
 +    {
 +        class CacheFilter extends Transformation
 +        {
 +            @Override
 +            public BaseRowIterator applyToPartition(BaseRowIterator iter)
 +            {
 +                // Note that we rely on the fact that until we actually advance 'iter', no really costly operation is actually done
 +                // (except for reading the partition key from the index file) due to the call to mergeLazily in queryStorage.
 +                DecoratedKey dk = iter.partitionKey();
 +
 +                // Check if this partition is in the rowCache and if it is, if  it covers our filter
 +                CachedPartition cached = cfs.getRawCachedPartition(dk);
 +                ClusteringIndexFilter filter = dataRange().clusteringIndexFilter(dk);
 +
 +                if (cached != null && cfs.isFilterFullyCoveredBy(filter, limits(), cached, nowInSec()))
 +                {
 +                    // We won't use 'iter' so close it now.
 +                    iter.close();
 +
 +                    return filter.getUnfilteredRowIterator(columnFilter(), cached);
 +                }
 +
 +                return iter;
 +            }
 +        }
 +        return Transformation.apply(iter, new CacheFilter());
 +    }
 +
 +    public MessageOut<ReadCommand> createMessage(int version)
 +    {
 +        return dataRange().isPaging()
 +             ? new MessageOut<>(MessagingService.Verb.PAGED_RANGE, this, pagedRangeSerializer)
 +             : new MessageOut<>(MessagingService.Verb.RANGE_SLICE, this, rangeSliceSerializer);
 +    }
 +
 +    protected void appendCQLWhereClause(StringBuilder sb)
 +    {
 +        if (dataRange.isUnrestricted() && rowFilter().isEmpty())
 +            return;
 +
 +        sb.append(" WHERE ");
 +        // We put the row filter first because the data range can end by "ORDER BY"
 +        if (!rowFilter().isEmpty())
 +        {
 +            sb.append(rowFilter());
 +            if (!dataRange.isUnrestricted())
 +                sb.append(" AND ");
 +        }
 +        if (!dataRange.isUnrestricted())
 +            sb.append(dataRange.toCQLString(metadata()));
 +    }
 +
 +    /**
 +     * Allow to post-process the result of the query after it has been reconciled on the coordinator
 +     * but before it is passed to the CQL layer to return the ResultSet.
 +     *
 +     * See CASSANDRA-8717 for why this exists.
 +     */
 +    public PartitionIterator postReconciliationProcessing(PartitionIterator result)
 +    {
 +        ColumnFamilyStore cfs = Keyspace.open(metadata().ksName).getColumnFamilyStore(metadata().cfName);
 +        Index index = getIndex(cfs);
 +        return index == null ? result : index.postProcessorFor(this).apply(result, this);
 +    }
 +
 +    @Override
 +    public String toString()
 +    {
 +        return String.format("Read(%s.%s columns=%s rowfilter=%s limits=%s %s)",
 +                             metadata().ksName,
 +                             metadata().cfName,
 +                             columnFilter(),
 +                             rowFilter(),
 +                             limits(),
 +                             dataRange().toString(metadata()));
 +    }
 +
 +    protected void serializeSelection(DataOutputPlus out, int version) throws IOException
 +    {
 +        DataRange.serializer.serialize(dataRange(), out, version, metadata());
 +    }
 +
 +    protected long selectionSerializedSize(int version)
 +    {
 +        return DataRange.serializer.serializedSize(dataRange(), version, metadata());
 +    }
 +
 +    private static class Deserializer extends SelectionDeserializer
 +    {
 +        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
 +        throws IOException
 +        {
 +            DataRange range = DataRange.serializer.deserialize(in, version, metadata);
 +            return new PartitionRangeReadCommand(isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, range, index);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 0c4e144,20d3dc0..f0a1f47
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@@ -223,15 -218,20 +223,19 @@@ public class CompactionTask extends Abs
              for (SSTableReader reader : newSStables)
                  newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
  
-             double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
-             Summary mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
-             logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                       taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, mergeSummary.totalSourceRows, totalKeysWritten, mergeSummary.partitionMerge));
-             logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-             logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
- 
              if (offline)
+             {
                  Refs.release(Refs.selfRefs(newSStables));
+             }
+             else
+             {
+                 double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
 -                long totalSourceRows = 0;
 -                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
++                Summary mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
+                 logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
 -                                           taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
++                                           taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, mergeSummary.totalSourceRows, totalKeysWritten, mergeSummary.partitionMerge));
+                 logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
 -                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten));
++                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
+             }
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
index ca644eb,0000000..350477c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
@@@ -1,444 -1,0 +1,445 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.db.lifecycle;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.nio.file.Files;
 +import java.nio.file.NoSuchFileException;
 +import java.util.*;
 +import java.util.concurrent.ConcurrentLinkedQueue;
 +import java.util.concurrent.TimeUnit;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.util.concurrent.Runnables;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.ScheduledExecutors;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.db.ColumnFamilyStore;
 +import org.apache.cassandra.db.Directories;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.db.compaction.OperationType;
 +import org.apache.cassandra.db.lifecycle.LogRecord.Type;
 +import org.apache.cassandra.io.sstable.Component;
 +import org.apache.cassandra.io.sstable.Descriptor;
 +import org.apache.cassandra.io.sstable.SSTable;
 +import org.apache.cassandra.io.sstable.SnapshotDeletingTask;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.utils.*;
 +import org.apache.cassandra.utils.concurrent.Ref;
 +import org.apache.cassandra.utils.concurrent.RefCounted;
 +import org.apache.cassandra.utils.concurrent.Transactional;
 +
 +/**
 + * IMPORTANT: When this object is involved in a transactional graph, and is not encapsulated in a LifecycleTransaction,
 + * for correct behaviour its commit MUST occur before any others, since it may legitimately fail. This is consistent
 + * with the Transactional API, which permits one failing action to occur at the beginning of the commit phase, but also
 + * *requires* that the prepareToCommit() phase only take actions that can be rolled back.
 + *
 + * IMPORTANT: The transaction must complete (commit or abort) before any temporary files are deleted, even though the
 + * txn log file itself will not be deleted until all tracked files are deleted. This is required by FileLister to ensure
 + * a consistent disk state. LifecycleTransaction ensures this requirement, so this class should really never be used
 + * outside of LT. @see FileLister.classifyFiles(TransactionData txn)
 + *
 + * A class that tracks sstable files involved in a transaction across sstables:
 + * if the transaction succeeds the old files should be deleted and the new ones kept; vice-versa if it fails.
 + *
 + * The transaction log file contains new and old sstables as follows:
 + *
 + * add:[sstable-2][CRC]
 + * remove:[sstable-1,max_update_time,num files][CRC]
 + *
 + * where sstable-2 is a new sstable to be retained if the transaction succeeds and sstable-1 is an old sstable to be
 + * removed. CRC is an incremental CRC of the file content up to this point. For old sstable files we also log the
 + * last update time of all files for the sstable descriptor and a checksum of vital properties such as update times
 + * and file sizes.
 + *
 + * Upon commit we add a final line to the log file:
 + *
 + * commit:[commit_time][CRC]
 + *
 + * When the transaction log is cleaned-up by the TransactionTidier, which happens only after any old sstables have been
 + * osoleted, then any sstable files for old sstables are removed before deleting the transaction log if the transaction
 + * was committed, vice-versa if the transaction was aborted.
 + *
 + * On start-up we look for any transaction log files and repeat the cleanup process described above.
 + *
 + * See CASSANDRA-7066 for full details.
 + */
 +class LogTransaction extends Transactional.AbstractTransactional implements Transactional
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(LogTransaction.class);
 +
 +    /**
 +     * If the format of the lines in the transaction log is wrong or the checksum
 +     * does not match, then we throw this exception.
 +     */
 +    public static final class CorruptTransactionLogException extends RuntimeException
 +    {
 +        public final LogFile txnFile;
 +
 +        public CorruptTransactionLogException(String message, LogFile txnFile)
 +        {
 +            super(message);
 +            this.txnFile = txnFile;
 +        }
 +    }
 +
 +    private final Tracker tracker;
 +    private final LogFile txnFile;
 +    private final Ref<LogTransaction> selfRef;
 +    // Deleting sstables is tricky because the mmapping might not have been finalized yet,
 +    // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
 +    // Additionally, we need to make sure to delete the data file first, so on restart the others
 +    // will be recognized as GCable.
 +    private static final Queue<Runnable> failedDeletions = new ConcurrentLinkedQueue<>();
 +
 +    LogTransaction(OperationType opType)
 +    {
 +        this(opType, null);
 +    }
 +
 +    LogTransaction(OperationType opType, Tracker tracker)
 +    {
 +        this.tracker = tracker;
 +        this.txnFile = new LogFile(opType, UUIDGen.getTimeUUID());
 +        this.selfRef = new Ref<>(this, new TransactionTidier(txnFile));
 +
 +        if (logger.isTraceEnabled())
 +            logger.trace("Created transaction logs with id {}", txnFile.id());
 +    }
 +
 +    /**
 +     * Track a reader as new.
 +     **/
 +    void trackNew(SSTable table)
 +    {
 +        txnFile.add(Type.ADD, table);
 +    }
 +
 +    /**
 +     * Stop tracking a reader as new.
 +     */
 +    void untrackNew(SSTable table)
 +    {
 +        txnFile.remove(Type.ADD, table);
 +    }
 +
 +    /**
 +     * Schedule a reader for deletion as soon as it is fully unreferenced.
 +     */
 +    SSTableTidier obsoleted(SSTableReader reader)
 +    {
 +        if (txnFile.contains(Type.ADD, reader))
 +        {
 +            if (txnFile.contains(Type.REMOVE, reader))
 +                throw new IllegalArgumentException();
 +
 +            return new SSTableTidier(reader, true, this);
 +        }
 +
 +        txnFile.add(Type.REMOVE, reader);
 +
 +        if (tracker != null)
 +            tracker.notifyDeleting(reader);
 +
 +        return new SSTableTidier(reader, false, this);
 +    }
 +
 +    OperationType type()
 +    {
 +        return txnFile.type();
 +    }
 +
 +    UUID id()
 +    {
 +        return txnFile.id();
 +    }
 +
 +    @VisibleForTesting
 +    LogFile txnFile()
 +    {
 +        return txnFile;
 +    }
 +
 +    @VisibleForTesting
 +    List<File> logFiles()
 +    {
 +        return txnFile.getFiles();
 +    }
 +
 +    @VisibleForTesting
 +    List<String> logFilePaths()
 +    {
 +        return txnFile.getFilePaths();
 +    }
 +
 +    static void delete(File file)
 +    {
 +        try
 +        {
 +            if (logger.isTraceEnabled())
 +                logger.trace("Deleting {}", file);
 +
 +            Files.delete(file.toPath());
 +        }
 +        catch (NoSuchFileException e)
 +        {
 +            logger.error("Unable to delete {} as it does not exist", file);
 +        }
 +        catch (IOException e)
 +        {
 +            logger.error("Unable to delete {}", file, e);
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    /**
 +     * The transaction tidier.
 +     *
 +     * When the transaction reference is fully released we try to delete all the obsolete files
 +     * depending on the transaction result, as well as the transaction log file.
 +     */
 +    private static class TransactionTidier implements RefCounted.Tidy, Runnable
 +    {
 +        private final LogFile data;
 +
 +        TransactionTidier(LogFile data)
 +        {
 +            this.data = data;
 +        }
 +
 +        public void tidy() throws Exception
 +        {
 +            run();
 +        }
 +
 +        public String name()
 +        {
 +            return data.toString();
 +        }
 +
 +        public void run()
 +        {
 +            if (logger.isTraceEnabled())
 +                logger.trace("Removing files for transaction {}", name());
 +
 +            if (!data.completed())
 +            { // this happens if we forget to close a txn and the garbage collector closes it for us
 +                logger.error("{} was not completed, trying to abort it now", data);
 +                Throwable err = Throwables.perform((Throwable)null, data::abort);
 +                if (err != null)
 +                    logger.error("Failed to abort {}", data, err);
 +            }
 +
 +            Throwable err = data.removeUnfinishedLeftovers(null);
 +
 +            if (err != null)
 +            {
 +                logger.info("Failed deleting files for transaction {}, we'll retry after GC and on on server restart", name(), err);
 +                failedDeletions.add(this);
 +            }
 +            else
 +            {
 +                if (logger.isTraceEnabled())
 +                    logger.trace("Closing file transaction {}", name());
 +
 +                data.close();
 +            }
 +        }
 +    }
 +
 +    static class Obsoletion
 +    {
 +        final SSTableReader reader;
 +        final SSTableTidier tidier;
 +
 +        Obsoletion(SSTableReader reader, SSTableTidier tidier)
 +        {
 +            this.reader = reader;
 +            this.tidier = tidier;
 +        }
 +    }
 +
 +    /**
 +     * The SSTableReader tidier. When a reader is fully released and no longer referenced
 +     * by any one, we run this. It keeps a reference to the parent transaction and releases
 +     * it when done, so that the final transaction cleanup can run when all obsolete readers
 +     * are released.
 +     */
 +    public static class SSTableTidier implements Runnable
 +    {
 +        // must not retain a reference to the SSTableReader, else leak detection cannot kick in
 +        private final Descriptor desc;
 +        private final long sizeOnDisk;
 +        private final Tracker tracker;
 +        private final boolean wasNew;
 +        private final Ref<LogTransaction> parentRef;
 +
 +        public SSTableTidier(SSTableReader referent, boolean wasNew, LogTransaction parent)
 +        {
 +            this.desc = referent.descriptor;
 +            this.sizeOnDisk = referent.bytesOnDisk();
 +            this.tracker = parent.tracker;
 +            this.wasNew = wasNew;
 +            this.parentRef = parent.selfRef.tryRef();
 +        }
 +
 +        public void run()
 +        {
-             SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
++            if (tracker != null && !tracker.isDummy())
++                SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
 +
 +            try
 +            {
 +                // If we can't successfully delete the DATA component, set the task to be retried later: see TransactionTidier
 +                File datafile = new File(desc.filenameFor(Component.DATA));
 +
 +                delete(datafile);
 +                // let the remainder be cleaned up by delete
 +                SSTable.delete(desc, SSTable.discoverComponentsFor(desc));
 +            }
 +            catch (Throwable t)
 +            {
 +                logger.error("Failed deletion for {}, we'll retry after GC and on server restart", desc);
 +                failedDeletions.add(this);
 +                return;
 +            }
 +
 +            if (tracker != null && tracker.cfstore != null && !wasNew)
 +                tracker.cfstore.metric.totalDiskSpaceUsed.dec(sizeOnDisk);
 +
 +            // release the referent to the parent so that the all transaction files can be released
 +            parentRef.release();
 +        }
 +
 +        public void abort()
 +        {
 +            parentRef.release();
 +        }
 +    }
 +
 +
 +    static void rescheduleFailedDeletions()
 +    {
 +        Runnable task;
 +        while ( null != (task = failedDeletions.poll()))
 +            ScheduledExecutors.nonPeriodicTasks.submit(task);
 +
 +        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
 +        SnapshotDeletingTask.rescheduleFailedTasks();
 +    }
 +
 +    static void waitForDeletions()
 +    {
 +        FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(Runnables.doNothing(), 0, TimeUnit.MILLISECONDS));
 +    }
 +
 +    @VisibleForTesting
 +    Throwable complete(Throwable accumulate)
 +    {
 +        try
 +        {
 +            accumulate = selfRef.ensureReleased(accumulate);
 +            return accumulate;
 +        }
 +        catch (Throwable t)
 +        {
 +            logger.error("Failed to complete file transaction {}", id(), t);
 +            return Throwables.merge(accumulate, t);
 +        }
 +    }
 +
 +    protected Throwable doCommit(Throwable accumulate)
 +    {
 +        return complete(Throwables.perform(accumulate, txnFile::commit));
 +    }
 +
 +    protected Throwable doAbort(Throwable accumulate)
 +    {
 +        return complete(Throwables.perform(accumulate, txnFile::abort));
 +    }
 +
 +    protected void doPrepare() { }
 +
 +    /**
 +     * Called on startup to scan existing folders for any unfinished leftovers of
 +     * operations that were ongoing when the process exited. Also called by the standalone
 +     * sstableutil tool when the cleanup option is specified, @see StandaloneSSTableUtil.
 +     *
 +     */
 +    static void removeUnfinishedLeftovers(CFMetaData metadata)
 +    {
 +        removeUnfinishedLeftovers(new Directories(metadata, ColumnFamilyStore.getInitialDirectories()).getCFDirectories());
 +    }
 +
 +    @VisibleForTesting
 +    static void removeUnfinishedLeftovers(List<File> folders)
 +    {
 +        LogFilesByName logFiles = new LogFilesByName();
 +        folders.forEach(logFiles::list);
 +        logFiles.removeUnfinishedLeftovers();
 +    }
 +
 +    private static final class LogFilesByName
 +    {
 +        Map<String, List<File>> files = new HashMap<>();
 +
 +        void list(File folder)
 +        {
 +            Arrays.stream(folder.listFiles(LogFile::isLogFile)).forEach(this::add);
 +        }
 +
 +        void add(File file)
 +        {
 +            List<File> filesByName = files.get(file.getName());
 +            if (filesByName == null)
 +            {
 +                filesByName = new ArrayList<>();
 +                files.put(file.getName(), filesByName);
 +            }
 +
 +            filesByName.add(file);
 +        }
 +
 +        void removeUnfinishedLeftovers()
 +        {
 +            files.forEach(LogFilesByName::removeUnfinishedLeftovers);
 +        }
 +
 +        static void removeUnfinishedLeftovers(String name, List<File> logFiles)
 +        {
 +
 +            try(LogFile txn = LogFile.make(name, logFiles))
 +            {
 +                if (txn.verify())
 +                {
 +                    Throwable failure = txn.removeUnfinishedLeftovers(null);
 +                    if (failure != null)
 +                        logger.error("Failed to remove unfinished transaction leftovers for txn {}", txn, failure);
 +                }
 +                else
 +                {
 +                    logger.error("Unexpected disk state: failed to read transaction txn {}", txn);
 +                }
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index 5a3d524,e77ef78..9feaa3e
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@@ -193,15 -204,14 +201,13 @@@ public class Tracke
  
      /** (Re)initializes the tracker, purging all references. */
      @VisibleForTesting
-     public void reset()
+     public void reset(Memtable memtable)
      {
-         view.set(new View(
-                          !isDummy() ? ImmutableList.of(new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfstore))
-                                     : ImmutableList.<Memtable>of(),
-                          ImmutableList.<Memtable>of(),
-                          Collections.<SSTableReader, SSTableReader>emptyMap(),
-                          Collections.<SSTableReader, SSTableReader>emptyMap(),
-                          SSTableIntervalTree.empty()));
 -        view.set(new View(memtable != null ? singletonList(memtable) : Collections.<Memtable>emptyList(),
 -                          Collections.<Memtable>emptyList(),
 -                          Collections.<SSTableReader, SSTableReader>emptyMap(),
 -                          Collections.<SSTableReader>emptySet(),
 -                          Collections.<SSTableReader>emptySet(),
++        view.set(new View(memtable != null ? singletonList(memtable) : Collections.emptyList(),
++                          Collections.emptyList(),
++                          Collections.emptyMap(),
++                          Collections.emptyMap(),
+                           SSTableIntervalTree.empty()));
      }
  
      public Throwable dropSSTablesIfInvalid(Throwable accumulate)


[05/19] cassandra git commit: Temporarily fix bug that creates commit log when running offline tools

Posted by yu...@apache.org.
Temporarily fix bug that creates commit log when running offline tools

patch by yukim; reviewed by thobbs for CASSANDRA-8616


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/66f1aaf8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/66f1aaf8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/66f1aaf8

Branch: refs/heads/cassandra-3.X
Commit: 66f1aaf88d3cde5c52b13d71d3326da5eda16fb1
Parents: fb29400
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Feb 11 19:06:27 2016 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:51:37 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/config/DatabaseDescriptor.java    | 12 +++++++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++++--
 .../cassandra/db/compaction/CompactionTask.java | 22 +++++++-----
 .../apache/cassandra/db/lifecycle/Tracker.java  | 36 ++++++++++++--------
 .../io/sstable/format/SSTableReader.java        |  8 +++--
 .../cassandra/service/CassandraDaemon.java      |  1 +
 .../service/EmbeddedCassandraService.java       |  2 ++
 .../io/sstable/CQLSSTableWriterLongTest.java    |  2 ++
 .../unit/org/apache/cassandra/SchemaLoader.java |  1 +
 .../config/DatabaseDescriptorTest.java          |  6 ++++
 .../apache/cassandra/db/CounterCellTest.java    |  3 ++
 .../org/apache/cassandra/db/NativeCellTest.java |  3 ++
 .../apache/cassandra/db/SystemKeyspaceTest.java |  2 ++
 .../db/context/CounterContextTest.java          |  8 +++++
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 ++-
 .../cassandra/db/lifecycle/TrackerTest.java     |  7 ++--
 .../cassandra/dht/StreamStateStoreTest.java     |  7 ++++
 .../cassandra/gms/FailureDetectorTest.java      |  2 ++
 .../org/apache/cassandra/gms/GossiperTest.java  |  5 +++
 .../io/sstable/CQLSSTableWriterTest.java        |  2 ++
 .../cassandra/locator/CloudstackSnitchTest.java |  2 ++
 .../apache/cassandra/locator/EC2SnitchTest.java |  2 ++
 .../locator/GoogleCloudSnitchTest.java          |  2 ++
 .../service/StorageServiceServerTest.java       |  1 +
 25 files changed, 118 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f02350d..8cff097 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.9
+ * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
  * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
  * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
  * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index a3fb79b..35debd0 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -102,6 +102,18 @@ public class DatabaseDescriptor
     private static Comparator<InetAddress> localComparator;
     private static boolean hasLoggedConfig;
 
+    private static boolean daemonInitialized;
+
+    public static boolean isDaemonInitialized()
+    {
+        return daemonInitialized;
+    }
+
+    public static void setDaemonInitialized()
+    {
+        daemonInitialized = true;
+    }
+
     public static void forceStaticInitialization() {}
     static
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c6b69dc..4bc46d0 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -388,9 +388,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         logger.info("Initializing {}.{}", keyspace.getName(), name);
 
-        // scan for sstables corresponding to this cf and load them
-        data = new Tracker(this, loadSSTables);
+        // Create Memtable only on online
+        Memtable initialMemtable = null;
+        if (DatabaseDescriptor.isDaemonInitialized())
+            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
+        data = new Tracker(initialMemtable, loadSSTables);
 
+        // scan for sstables corresponding to this cf and load them
         if (data.loadsstables)
         {
             Directories.SSTableLister sstableFiles = directories.sstableLister().skipTemporary(true);
@@ -2754,7 +2758,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             {
                 public Void call()
                 {
-                    cfs.data.reset();
+                    cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
                     cfs.getCompactionStrategy().shutdown();
                     cfs.getCompactionStrategy().startup();
                     return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 575c326..20d3dc0 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -127,7 +127,7 @@ public class CompactionTask extends AbstractCompactionTask
             }
         });
 
-        UUID taskId = SystemKeyspace.startCompaction(cfs, transaction.originals());
+        UUID taskId = offline ? null : SystemKeyspace.startCompaction(cfs, transaction.originals());
 
         // new sstables from flush can be added during a compaction, but only the compaction can remove them,
         // so in our single-threaded compaction world this is a valid way of determining if we're compacting
@@ -218,16 +218,20 @@ public class CompactionTask extends AbstractCompactionTask
             for (SSTableReader reader : newSStables)
                 newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
 
-            double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
-            long totalSourceRows = 0;
-            String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
-            logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                      taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
-            logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-            logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
-
             if (offline)
+            {
                 Refs.release(Refs.selfRefs(newSStables));
+            }
+            else
+            {
+                double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
+                long totalSourceRows = 0;
+                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
+                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
+                                           taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
+                logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
+                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten));
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index 5d5701f..e77ef78 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -31,7 +31,6 @@ import com.google.common.collect.*;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Memtable;
-import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,6 +51,7 @@ import static com.google.common.base.Predicates.not;
 import static com.google.common.collect.ImmutableSet.copyOf;
 import static com.google.common.collect.Iterables.filter;
 import static java.util.Collections.singleton;
+import static java.util.Collections.singletonList;
 import static org.apache.cassandra.db.lifecycle.Helpers.*;
 import static org.apache.cassandra.db.lifecycle.View.permitCompacting;
 import static org.apache.cassandra.db.lifecycle.View.updateCompacting;
@@ -61,21 +61,29 @@ import static org.apache.cassandra.utils.Throwables.merge;
 import static org.apache.cassandra.utils.concurrent.Refs.release;
 import static org.apache.cassandra.utils.concurrent.Refs.selfRefs;
 
+/**
+ * Tracker tracks live {@link View} of data store for a table.
+ */
 public class Tracker
 {
     private static final Logger logger = LoggerFactory.getLogger(Tracker.class);
 
-    public final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+    private final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+
     public final ColumnFamilyStore cfstore;
     final AtomicReference<View> view;
     public final boolean loadsstables;
 
-    public Tracker(ColumnFamilyStore cfstore, boolean loadsstables)
+    /**
+     * @param memtable Initial Memtable. Can be null.
+     * @param loadsstables true to indicate to load SSTables (TODO: remove as this is only accessed from 2i)
+     */
+    public Tracker(Memtable memtable, boolean loadsstables)
     {
-        this.cfstore = cfstore;
+        this.cfstore = memtable != null ? memtable.cfs : null;
         this.view = new AtomicReference<>();
         this.loadsstables = loadsstables;
-        this.reset();
+        this.reset(memtable);
     }
 
     public LifecycleTransaction tryModify(SSTableReader sstable, OperationType operationType)
@@ -196,16 +204,14 @@ public class Tracker
 
     /** (Re)initializes the tracker, purging all references. */
     @VisibleForTesting
-    public void reset()
+    public void reset(Memtable memtable)
     {
-        view.set(new View(
-                         !isDummy() ? ImmutableList.of(new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfstore))
-                                    : ImmutableList.<Memtable>of(),
-                         ImmutableList.<Memtable>of(),
-                         Collections.<SSTableReader, SSTableReader>emptyMap(),
-                         Collections.<SSTableReader>emptySet(),
-                         Collections.<SSTableReader>emptySet(),
-                         SSTableIntervalTree.empty()));
+        view.set(new View(memtable != null ? singletonList(memtable) : Collections.<Memtable>emptyList(),
+                          Collections.<Memtable>emptyList(),
+                          Collections.<SSTableReader, SSTableReader>emptyMap(),
+                          Collections.<SSTableReader>emptySet(),
+                          Collections.<SSTableReader>emptySet(),
+                          SSTableIntervalTree.empty()));
     }
 
     public Throwable dropSSTablesIfInvalid(Throwable accumulate)
@@ -473,7 +479,7 @@ public class Tracker
 
     public boolean isDummy()
     {
-        return cfstore == null;
+        return cfstore == null || !DatabaseDescriptor.isDaemonInitialized();
     }
 
     public void subscribe(INotificationConsumer consumer)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index fddf058..ba060d4 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -2246,7 +2246,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
             // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist
             // the read meter when in client mode.
-            if (SystemKeyspace.NAME.equals(desc.ksname))
+            if (SystemKeyspace.NAME.equals(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
             {
                 readMeter = null;
                 readMeterSyncFuture = null;
@@ -2272,9 +2272,11 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         {
             lookup.remove(desc);
             if (readMeterSyncFuture != null)
+            {
                 readMeterSyncFuture.cancel(true);
-            if (isCompacted.get())
-                SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+                if (isCompacted.get())
+                    SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+            }
             // don't ideally want to dropPageCache for the file until all instances have been released
             CLibrary.trySkipCache(desc.filenameFor(Component.DATA), 0, 0);
             CLibrary.trySkipCache(desc.filenameFor(Component.PRIMARY_INDEX), 0, 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index cf5f3c2..35e990f 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -505,6 +505,7 @@ public class CassandraDaemon
             try
             {
                 DatabaseDescriptor.forceStaticInitialization();
+                DatabaseDescriptor.setDaemonInitialized();
             }
             catch (ExceptionInInitializerError e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
index 659d851..6c154cd 100644
--- a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
+++ b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.service;
 
 import java.io.IOException;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.service.CassandraDaemon;
 
 /**
@@ -49,6 +50,7 @@ public class EmbeddedCassandraService
     public void start() throws IOException
     {
         cassandraDaemon = new CassandraDaemon();
+        DatabaseDescriptor.setDaemonInitialized();
         cassandraDaemon.init(null);
         cassandraDaemon.start();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
index fcec40d..ee719d1 100644
--- a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
+++ b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
@@ -30,6 +30,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.service.StorageService;
@@ -39,6 +40,7 @@ public class CQLSSTableWriterLongTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index a7cf7b4..2048f74 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -90,6 +90,7 @@ public class SchemaLoader
             }
         });
 
+        DatabaseDescriptor.setDaemonInitialized();
         Keyspace.setInitialized();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 5f231c3..7409535 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -47,6 +47,12 @@ import static org.junit.Assert.assertNull;
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class DatabaseDescriptorTest
 {
+    @BeforeClass
+    public static void setupDatabaseDescriptor()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testCFMetaDataSerialization() throws ConfigurationException, InvalidRequestException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/CounterCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
index 8d75b9a..5d4b8a8 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -30,6 +30,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
 import org.apache.cassandra.db.context.CounterContext;
@@ -57,6 +58,8 @@ public class CounterCellTest
         countLength   = 8; // size of long
 
         stepLength    = idLength + clockLength + countLength;
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/NativeCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NativeCellTest.java b/test/unit/org/apache/cassandra/db/NativeCellTest.java
index 70b7b87..4145a91 100644
--- a/test/unit/org/apache/cassandra/db/NativeCellTest.java
+++ b/test/unit/org/apache/cassandra/db/NativeCellTest.java
@@ -32,6 +32,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -128,6 +129,8 @@ public class NativeCellTest
         {
             throw new AssertionError();
         }
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index b8aa161..c3047b8 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -42,6 +42,8 @@ public class SystemKeyspaceTest
     @BeforeClass
     public static void prepSnapshotTracker()
     {
+        DatabaseDescriptor.setDaemonInitialized();
+
         if (FBUtilities.isWindows())
             WindowsFailedSnapshotTracker.deleteOldSnapshots();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
index a72d30d..4f587c6 100644
--- a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
+++ b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
@@ -22,9 +22,11 @@ package org.apache.cassandra.db.context;
 
 import java.nio.ByteBuffer;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ClockAndCount;
 import org.apache.cassandra.db.context.CounterContext.Relationship;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -48,6 +50,12 @@ public class CounterContextTest
     private static final int countLength = 8;
     private static final int stepLength = idLength + clockLength + countLength;
 
+    @BeforeClass
+    public static void setupDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testAllocate()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
index f13d1b7..737392e 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.lifecycle;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.junit.After;
 import org.junit.Before;
@@ -30,6 +31,8 @@ import junit.framework.Assert;
 import org.apache.cassandra.MockSchema;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
@@ -270,7 +273,7 @@ public class LifecycleTransactionTest extends AbstractTransactionalTest
 
         private static Tracker tracker(ColumnFamilyStore cfs, List<SSTableReader> readers)
         {
-            Tracker tracker = new Tracker(cfs, false);
+            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs), false);
             tracker.addInitialSSTables(readers);
             return tracker;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
index adeb778..04b4e4a 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -75,6 +75,7 @@ public class TrackerTest
     @BeforeClass
     public static void setUp()
     {
+        DatabaseDescriptor.setDaemonInitialized();
         MockSchema.cleanup();
     }
 
@@ -82,7 +83,7 @@ public class TrackerTest
     public void testTryModify()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = new Tracker(null, false);
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, true, cfs), MockSchema.sstable(1, cfs), MockSchema.sstable(2, cfs));
         tracker.addInitialSSTables(copyOf(readers));
         Assert.assertNull(tracker.tryModify(ImmutableList.of(MockSchema.sstable(0, cfs)), OperationType.COMPACTION));
@@ -146,7 +147,7 @@ public class TrackerTest
     public void testAddInitialSSTables()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = cfs.getTracker();
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),
                                                        MockSchema.sstable(1, 121, cfs),
                                                        MockSchema.sstable(2, 9, cfs));
@@ -163,7 +164,7 @@ public class TrackerTest
         boolean backups = DatabaseDescriptor.isIncrementalBackupsEnabled();
         DatabaseDescriptor.setIncrementalBackupsEnabled(false);
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = cfs.getTracker();
         MockListener listener = new MockListener(false);
         tracker.subscribe(listener);
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
index 86781d9..bdb654a 100644
--- a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
+++ b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
@@ -20,8 +20,10 @@ package org.apache.cassandra.dht;
 import java.net.InetAddress;
 import java.util.Collections;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.streaming.DefaultConnectionFactory;
 import org.apache.cassandra.streaming.StreamEvent;
 import org.apache.cassandra.streaming.StreamSession;
@@ -32,6 +34,11 @@ import static org.junit.Assert.assertTrue;
 
 public class StreamStateStoreTest
 {
+    @BeforeClass
+    public static void initDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
 
     @Test
     public void testUpdateAndQueryAvailableRanges()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
index 9325922..af099b0 100644
--- a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
+++ b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
@@ -45,6 +45,8 @@ public class FailureDetectorTest
     {
         // slow unit tests can cause problems with FailureDetector's GC pause handling
         System.setProperty("cassandra.max_local_pause_in_ms", "20000");
+
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/gms/GossiperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java
index ad07165..eb01305 100644
--- a/test/unit/org/apache/cassandra/gms/GossiperTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java
@@ -33,6 +33,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Token;
@@ -44,6 +45,10 @@ import static org.junit.Assert.assertEquals;
 
 public class GossiperTest
 {
+    static
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
     static final IPartitioner partitioner = new RandomPartitioner();
     StorageService ss = StorageService.instance;
     TokenMetadata tmd = StorageService.instance.getTokenMetadata();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 8a14428..5e2fffe 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -34,6 +34,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -52,6 +53,7 @@ public class CQLSSTableWriterTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 90e63e0..bc5be46 100644
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@ -27,6 +27,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
@@ -43,6 +44,7 @@ public class CloudstackSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
index 56bbb77..32383d9 100644
--- a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
@@ -33,6 +33,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -51,6 +52,7 @@ public class EC2SnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
index 1521454..f2450f4 100644
--- a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
@@ -31,6 +31,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -47,6 +48,7 @@ public class GoogleCloudSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index 4481501..a693a23 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -63,6 +63,7 @@ public class StorageServiceServerTest
     @BeforeClass
     public static void setUp() throws ConfigurationException
     {
+        DatabaseDescriptor.setDaemonInitialized();
         IEndpointSnitch snitch = new PropertyFileSnitch();
         DatabaseDescriptor.setEndpointSnitch(snitch);
         Keyspace.setInitialized();


[08/19] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index 003b624,0000000..a6ed3ba
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@@ -1,1112 -1,0 +1,1114 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.index;
 +
 +import java.lang.reflect.Constructor;
 +import java.util.*;
 +import java.util.concurrent.*;
 +import java.util.function.Function;
 +import java.util.stream.Collectors;
 +import java.util.stream.Stream;
 +
 +import com.google.common.base.Joiner;
 +import com.google.common.base.Strings;
 +import com.google.common.collect.ImmutableSet;
 +import com.google.common.collect.Iterables;
 +import com.google.common.collect.Maps;
 +import com.google.common.collect.Sets;
 +import com.google.common.primitives.Longs;
 +import com.google.common.util.concurrent.Futures;
 +import com.google.common.util.concurrent.MoreExecutors;
 +import org.apache.commons.lang3.StringUtils;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 +import org.apache.cassandra.concurrent.NamedThreadFactory;
 +import org.apache.cassandra.concurrent.StageManager;
 +import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.statements.IndexTarget;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.filter.RowFilter;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.lifecycle.View;
 +import org.apache.cassandra.db.partitions.PartitionIterators;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.index.internal.CassandraIndex;
 +import org.apache.cassandra.index.transactions.*;
 +import org.apache.cassandra.io.sstable.ReducingKeyIterator;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.schema.Indexes;
 +import org.apache.cassandra.service.pager.SinglePartitionPager;
 +import org.apache.cassandra.tracing.Tracing;
 +import org.apache.cassandra.transport.Server;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.apache.cassandra.utils.concurrent.Refs;
 +
 +/**
 + * Handles the core maintenance functionality associated with indexes: adding/removing them to or from
 + * a table, (re)building during bootstrap or other streaming operations, flushing, reloading metadata
 + * and so on.
 + *
 + * The Index interface defines a number of methods which return Callable<?>. These are primarily the
 + * management tasks for an index implementation. Most of them are currently executed in a blocking
 + * fashion via submission to SIM's blockingExecutor. This provides the desired behaviour in pretty
 + * much all cases, as tasks like flushing an index needs to be executed synchronously to avoid potentially
 + * deadlocking on the FlushWriter or PostFlusher. Several of these Callable<?> returning methods on Index could
 + * then be defined with as void and called directly from SIM (rather than being run via the executor service).
 + * Separating the task defintion from execution gives us greater flexibility though, so that in future, for example,
 + * if the flush process allows it we leave open the possibility of executing more of these tasks asynchronously.
 + *
 + * The primary exception to the above is the Callable returned from Index#addIndexedColumn. This may
 + * involve a significant effort, building a new index over any existing data. We perform this task asynchronously;
 + * as it is called as part of a schema update, which we do not want to block for a long period. Building non-custom
 + * indexes is performed on the CompactionManager.
 + *
 + * This class also provides instances of processors which listen to updates to the base table and forward to
 + * registered Indexes the info required to keep those indexes up to date.
 + * There are two variants of these processors, each with a factory method provided by SIM:
 + *      IndexTransaction: deals with updates generated on the regular write path.
 + *      CleanupTransaction: used when partitions are modified during compaction or cleanup operations.
 + * Further details on their usage and lifecycles can be found in the interface definitions below.
 + *
 + * Finally, the bestIndexFor method is used at query time to identify the most selective index of those able
 + * to satisfy any search predicates defined by a ReadCommand's RowFilter. It returns a thin IndexAccessor object
 + * which enables the ReadCommand to access the appropriate functions of the Index at various stages in its lifecycle.
 + * e.g. the getEstimatedResultRows is required when StorageProxy calculates the initial concurrency factor for
 + * distributing requests to replicas, whereas a Searcher instance is needed when the ReadCommand is executed locally on
 + * a target replica.
 + */
 +public class SecondaryIndexManager implements IndexRegistry
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(SecondaryIndexManager.class);
 +
 +    // default page size (in rows) when rebuilding the index for a whole partition
 +    public static final int DEFAULT_PAGE_SIZE = 10000;
 +
 +    private Map<String, Index> indexes = Maps.newConcurrentMap();
 +
 +    /**
 +     * The indexes that are ready to server requests.
 +     */
 +    private Set<String> builtIndexes = Sets.newConcurrentHashSet();
 +
 +    // executes tasks returned by Indexer#addIndexColumn which may require index(es) to be (re)built
 +    private static final ExecutorService asyncExecutor =
 +        new JMXEnabledThreadPoolExecutor(1,
 +                                         StageManager.KEEPALIVE,
 +                                         TimeUnit.SECONDS,
 +                                         new LinkedBlockingQueue<>(),
 +                                         new NamedThreadFactory("SecondaryIndexManagement"),
 +                                         "internal");
 +
 +    // executes all blocking tasks produced by Indexers e.g. getFlushTask, getMetadataReloadTask etc
 +    private static final ExecutorService blockingExecutor = MoreExecutors.newDirectExecutorService();
 +
 +    /**
 +     * The underlying column family containing the source data for these indexes
 +     */
 +    public final ColumnFamilyStore baseCfs;
 +
 +    public SecondaryIndexManager(ColumnFamilyStore baseCfs)
 +    {
 +        this.baseCfs = baseCfs;
 +    }
 +
 +
 +    /**
 +     * Drops and adds new indexes associated with the underlying CF
 +     */
 +    public void reload()
 +    {
 +        // figure out what needs to be added and dropped.
 +        Indexes tableIndexes = baseCfs.metadata.getIndexes();
 +        indexes.keySet()
 +               .stream()
 +               .filter(indexName -> !tableIndexes.has(indexName))
 +               .forEach(this::removeIndex);
 +
 +        // we call add for every index definition in the collection as
 +        // some may not have been created here yet, only added to schema
 +        for (IndexMetadata tableIndex : tableIndexes)
 +            addIndex(tableIndex);
 +    }
 +
 +    private Future<?> reloadIndex(IndexMetadata indexDef)
 +    {
 +        Index index = indexes.get(indexDef.name);
 +        Callable<?> reloadTask = index.getMetadataReloadTask(indexDef);
 +        return reloadTask == null
 +               ? Futures.immediateFuture(null)
 +               : blockingExecutor.submit(reloadTask);
 +    }
 +
 +    private Future<?> createIndex(IndexMetadata indexDef)
 +    {
 +        Index index = createInstance(indexDef);
 +        index.register(this);
 +
 +        // if the index didn't register itself, we can probably assume that no initialization needs to happen
 +        final Callable<?> initialBuildTask = indexes.containsKey(indexDef.name)
 +                                           ? index.getInitializationTask()
 +                                           : null;
 +        if (initialBuildTask == null)
 +        {
 +            // We need to make sure that the index is marked as built in the case where the initialBuildTask
 +            // does not need to be run (if the index didn't register itself or if the base table was empty).
 +            markIndexBuilt(indexDef.name);
 +            return Futures.immediateFuture(null);
 +        }
 +        return asyncExecutor.submit(index.getInitializationTask());
 +    }
 +
 +    /**
 +     * Adds and builds a index
 +     * @param indexDef the IndexMetadata describing the index
 +     */
 +    public synchronized Future<?> addIndex(IndexMetadata indexDef)
 +    {
 +        if (indexes.containsKey(indexDef.name))
 +            return reloadIndex(indexDef);
 +        else
 +            return createIndex(indexDef);
 +    }
 +
 +    /**
 +     * Checks if the specified index is queryable.
 +     *
 +     * @param index the index
 +     * @return <code>true</code> if the specified index is queryable, <code>false</code> otherwise
 +     */
 +    public boolean isIndexQueryable(Index index)
 +    {
 +        return builtIndexes.contains(index.getIndexMetadata().name);
 +    }
 +
 +    public synchronized void removeIndex(String indexName)
 +    {
 +        Index index = unregisterIndex(indexName);
 +        if (null != index)
 +        {
 +            markIndexRemoved(indexName);
 +            executeBlocking(index.getInvalidateTask());
 +        }
 +    }
 +
 +
 +    public Set<IndexMetadata> getDependentIndexes(ColumnDefinition column)
 +    {
 +        if (indexes.isEmpty())
 +            return Collections.emptySet();
 +
 +        Set<IndexMetadata> dependentIndexes = new HashSet<>();
 +        for (Index index : indexes.values())
 +            if (index.dependsOn(column))
 +                dependentIndexes.add(index.getIndexMetadata());
 +
 +        return dependentIndexes;
 +    }
 +
 +    /**
 +     * Called when dropping a Table
 +     */
 +    public void markAllIndexesRemoved()
 +    {
 +       getBuiltIndexNames().forEach(this::markIndexRemoved);
 +    }
 +
 +    /**
 +    * Does a full, blocking rebuild of the indexes specified by columns from the sstables.
 +    * Caller must acquire and release references to the sstables used here.
 +    * Note also that only this method of (re)building indexes:
 +    *   a) takes a set of index *names* rather than Indexers
 +    *   b) marks exsiting indexes removed prior to rebuilding
 +    *
 +    * @param sstables the data to build from
 +    * @param indexNames the list of indexes to be rebuilt
 +    */
 +    public void rebuildIndexesBlocking(Collection<SSTableReader> sstables, Set<String> indexNames)
 +    {
 +        Set<Index> toRebuild = indexes.values().stream()
 +                                               .filter(index -> indexNames.contains(index.getIndexMetadata().name))
 +                                               .filter(Index::shouldBuildBlocking)
 +                                               .collect(Collectors.toSet());
 +        if (toRebuild.isEmpty())
 +        {
 +            logger.info("No defined indexes with the supplied names: {}", Joiner.on(',').join(indexNames));
 +            return;
 +        }
 +
 +        toRebuild.forEach(indexer -> markIndexRemoved(indexer.getIndexMetadata().name));
 +
 +        buildIndexesBlocking(sstables, toRebuild);
 +
 +        toRebuild.forEach(indexer -> markIndexBuilt(indexer.getIndexMetadata().name));
 +    }
 +
 +    public void buildAllIndexesBlocking(Collection<SSTableReader> sstables)
 +    {
 +        buildIndexesBlocking(sstables, indexes.values()
 +                                              .stream()
 +                                              .filter(Index::shouldBuildBlocking)
 +                                              .collect(Collectors.toSet()));
 +    }
 +
 +    // For convenience, may be called directly from Index impls
 +    public void buildIndexBlocking(Index index)
 +    {
 +        if (index.shouldBuildBlocking())
 +        {
 +            try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL));
 +                 Refs<SSTableReader> sstables = viewFragment.refs)
 +            {
 +                buildIndexesBlocking(sstables, Collections.singleton(index));
 +                markIndexBuilt(index.getIndexMetadata().name);
 +            }
 +        }
 +    }
 +
 +    /**
 +     * Checks if the specified {@link ColumnFamilyStore} is a secondary index.
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code> to check.
 +     * @return <code>true</code> if the specified <code>ColumnFamilyStore</code> is a secondary index,
 +     * <code>false</code> otherwise.
 +     */
 +    public static boolean isIndexColumnFamilyStore(ColumnFamilyStore cfs)
 +    {
 +        return isIndexColumnFamily(cfs.name);
 +    }
 +
 +    /**
 +     * Checks if the specified {@link ColumnFamilyStore} is the one secondary index.
 +     *
 +     * @param cfName the name of the <code>ColumnFamilyStore</code> to check.
 +     * @return <code>true</code> if the specified <code>ColumnFamilyStore</code> is a secondary index,
 +     * <code>false</code> otherwise.
 +     */
 +    public static boolean isIndexColumnFamily(String cfName)
 +    {
 +        return cfName.contains(Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    /**
 +     * Returns the parent of the specified {@link ColumnFamilyStore}.
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code>
 +     * @return the parent of the specified <code>ColumnFamilyStore</code>
 +     */
 +    public static ColumnFamilyStore getParentCfs(ColumnFamilyStore cfs)
 +    {
 +        String parentCfs = getParentCfsName(cfs.name);
 +        return cfs.keyspace.getColumnFamilyStore(parentCfs);
 +    }
 +
 +    /**
 +     * Returns the parent name of the specified {@link ColumnFamilyStore}.
 +     *
 +     * @param cfName the <code>ColumnFamilyStore</code> name
 +     * @return the parent name of the specified <code>ColumnFamilyStore</code>
 +     */
 +    public static String getParentCfsName(String cfName)
 +    {
 +        assert isIndexColumnFamily(cfName);
 +        return StringUtils.substringBefore(cfName, Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    /**
 +     * Returns the index name
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code>
 +     * @return the index name
 +     */
 +    public static String getIndexName(ColumnFamilyStore cfs)
 +    {
 +        return getIndexName(cfs.name);
 +    }
 +
 +    /**
 +     * Returns the index name
 +     *
 +     * @param cfName the <code>ColumnFamilyStore</code> name
 +     * @return the index name
 +     */
 +    public static String getIndexName(String cfName)
 +    {
 +        assert isIndexColumnFamily(cfName);
 +        return StringUtils.substringAfter(cfName, Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    private void buildIndexesBlocking(Collection<SSTableReader> sstables, Set<Index> indexes)
 +    {
 +        if (indexes.isEmpty())
 +            return;
 +
 +        logger.info("Submitting index build of {} for data in {}",
 +                    indexes.stream().map(i -> i.getIndexMetadata().name).collect(Collectors.joining(",")),
 +                    sstables.stream().map(SSTableReader::toString).collect(Collectors.joining(",")));
 +
 +        SecondaryIndexBuilder builder = new SecondaryIndexBuilder(baseCfs,
 +                                                                  indexes,
 +                                                                  new ReducingKeyIterator(sstables));
 +        Future<?> future = CompactionManager.instance.submitIndexBuild(builder);
 +        FBUtilities.waitOnFuture(future);
 +
 +        flushIndexesBlocking(indexes);
 +        logger.info("Index build of {} complete",
 +                    indexes.stream().map(i -> i.getIndexMetadata().name).collect(Collectors.joining(",")));
 +    }
 +
 +    /**
 +     * Marks the specified index as build.
 +     * <p>This method is public as it need to be accessible from the {@link Index} implementations</p>
 +     * @param indexName the index name
 +     */
 +    public void markIndexBuilt(String indexName)
 +    {
 +        builtIndexes.add(indexName);
-         SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), indexName);
++        if (DatabaseDescriptor.isDaemonInitialized())
++            SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), indexName);
 +    }
 +
 +    /**
 +     * Marks the specified index as removed.
 +     * <p>This method is public as it need to be accessible from the {@link Index} implementations</p>
 +     * @param indexName the index name
 +     */
 +    public void markIndexRemoved(String indexName)
 +    {
 +        SystemKeyspace.setIndexRemoved(baseCfs.keyspace.getName(), indexName);
 +    }
 +
 +    public Index getIndexByName(String indexName)
 +    {
 +        return indexes.get(indexName);
 +    }
 +
 +    private Index createInstance(IndexMetadata indexDef)
 +    {
 +        Index newIndex;
 +        if (indexDef.isCustom())
 +        {
 +            assert indexDef.options != null;
 +            String className = indexDef.options.get(IndexTarget.CUSTOM_INDEX_OPTION_NAME);
 +            assert ! Strings.isNullOrEmpty(className);
 +            try
 +            {
 +                Class<? extends Index> indexClass = FBUtilities.classForName(className, "Index");
 +                Constructor<? extends Index> ctor = indexClass.getConstructor(ColumnFamilyStore.class, IndexMetadata.class);
 +                newIndex = (Index)ctor.newInstance(baseCfs, indexDef);
 +            }
 +            catch (Exception e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +        }
 +        else
 +        {
 +            newIndex = CassandraIndex.newIndex(baseCfs, indexDef);
 +        }
 +        return newIndex;
 +    }
 +
 +    /**
 +     * Truncate all indexes
 +     */
 +    public void truncateAllIndexesBlocking(final long truncatedAt)
 +    {
 +        executeAllBlocking(indexes.values().stream(), (index) -> index.getTruncateTask(truncatedAt));
 +    }
 +
 +    /**
 +     * Remove all indexes
 +     */
 +    public void invalidateAllIndexesBlocking()
 +    {
 +        markAllIndexesRemoved();
 +        executeAllBlocking(indexes.values().stream(), Index::getInvalidateTask);
 +    }
 +
 +    /**
 +     * Perform a blocking flush all indexes
 +     */
 +    public void flushAllIndexesBlocking()
 +    {
 +       flushIndexesBlocking(ImmutableSet.copyOf(indexes.values()));
 +    }
 +
 +    /**
 +     * Perform a blocking flush of selected indexes
 +     */
 +    public void flushIndexesBlocking(Set<Index> indexes)
 +    {
 +        if (indexes.isEmpty())
 +            return;
 +
 +        List<Future<?>> wait = new ArrayList<>();
 +        List<Index> nonCfsIndexes = new ArrayList<>();
 +
 +        // for each CFS backed index, submit a flush task which we'll wait on for completion
 +        // for the non-CFS backed indexes, we'll flush those while we wait.
 +        synchronized (baseCfs.getTracker())
 +        {
 +            indexes.forEach(index ->
 +                index.getBackingTable()
 +                     .map(cfs -> wait.add(cfs.forceFlush()))
 +                     .orElseGet(() -> nonCfsIndexes.add(index)));
 +        }
 +
 +        executeAllBlocking(nonCfsIndexes.stream(), Index::getBlockingFlushTask);
 +        FBUtilities.waitOnFutures(wait);
 +    }
 +
 +    /**
 +     * Performs a blocking flush of all custom indexes
 +     */
 +    public void flushAllNonCFSBackedIndexesBlocking()
 +    {
 +        executeAllBlocking(indexes.values()
 +                                  .stream()
 +                                  .filter(index -> !index.getBackingTable().isPresent()),
 +                           Index::getBlockingFlushTask);
 +    }
 +
 +    /**
 +     * @return all indexes which are marked as built and ready to use
 +     */
 +    public List<String> getBuiltIndexNames()
 +    {
 +        Set<String> allIndexNames = new HashSet<>();
 +        indexes.values().stream()
 +                .map(i -> i.getIndexMetadata().name)
 +                .forEach(allIndexNames::add);
 +        return SystemKeyspace.getBuiltIndexes(baseCfs.keyspace.getName(), allIndexNames);
 +    }
 +
 +    /**
 +     * @return all backing Tables used by registered indexes
 +     */
 +    public Set<ColumnFamilyStore> getAllIndexColumnFamilyStores()
 +    {
 +        Set<ColumnFamilyStore> backingTables = new HashSet<>();
 +        indexes.values().forEach(index -> index.getBackingTable().ifPresent(backingTables::add));
 +        return backingTables;
 +    }
 +
 +    /**
 +     * @return if there are ANY indexes registered for this table
 +     */
 +    public boolean hasIndexes()
 +    {
 +        return !indexes.isEmpty();
 +    }
 +
 +    /**
 +     * When building an index against existing data in sstables, add the given partition to the index
 +     */
 +    public void indexPartition(DecoratedKey key, Set<Index> indexes, int pageSize)
 +    {
 +        if (logger.isTraceEnabled())
 +            logger.trace("Indexing partition {}", baseCfs.metadata.getKeyValidator().getString(key.getKey()));
 +
 +        if (!indexes.isEmpty())
 +        {
 +            SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(baseCfs.metadata,
 +                                                                                          FBUtilities.nowInSeconds(),
 +                                                                                          key);
 +            int nowInSec = cmd.nowInSec();
 +            boolean readStatic = false;
 +
 +            SinglePartitionPager pager = new SinglePartitionPager(cmd, null, Server.CURRENT_VERSION);
 +            while (!pager.isExhausted())
 +            {
 +                try (ReadOrderGroup readGroup = cmd.startOrderGroup();
 +                     OpOrder.Group writeGroup = Keyspace.writeOrder.start();
 +                     RowIterator partition =
 +                        PartitionIterators.getOnlyElement(pager.fetchPageInternal(pageSize,readGroup),
 +                                                          cmd))
 +                {
 +                    Set<Index.Indexer> indexers = indexes.stream()
 +                                                         .map(index -> index.indexerFor(key,
 +                                                                                        partition.columns(),
 +                                                                                        nowInSec,
 +                                                                                        writeGroup,
 +                                                                                        IndexTransaction.Type.UPDATE))
 +                                                         .filter(Objects::nonNull)
 +                                                         .collect(Collectors.toSet());
 +
 +                    indexers.forEach(Index.Indexer::begin);
 +
 +                    // only process the static row once per partition
 +                    if (!readStatic && !partition.staticRow().isEmpty())
 +                    {
 +                        indexers.forEach(indexer -> indexer.insertRow(partition.staticRow()));
 +                        readStatic = true;
 +                    }
 +
 +                    while (partition.hasNext())
 +                    {
 +                        Row row = partition.next();
 +                        indexers.forEach(indexer -> indexer.insertRow(row));
 +                    }
 +
 +                    indexers.forEach(Index.Indexer::finish);
 +                }
 +            }
 +        }
 +    }
 +
 +    /**
 +     * Return the page size used when indexing an entire partition
 +     */
 +    public int calculateIndexingPageSize()
 +    {
 +        if (Boolean.getBoolean("cassandra.force_default_indexing_page_size"))
 +            return DEFAULT_PAGE_SIZE;
 +
 +        double targetPageSizeInBytes = 32 * 1024 * 1024;
 +        double meanPartitionSize = baseCfs.getMeanPartitionSize();
 +        if (meanPartitionSize <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int meanCellsPerPartition = baseCfs.getMeanColumns();
 +        if (meanCellsPerPartition <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int columnsPerRow = baseCfs.metadata.partitionColumns().regulars.size();
 +        if (meanCellsPerPartition <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int meanRowsPerPartition = meanCellsPerPartition / columnsPerRow;
 +        double meanRowSize = meanPartitionSize / meanRowsPerPartition;
 +
 +        int pageSize = (int) Math.max(1, Math.min(DEFAULT_PAGE_SIZE, targetPageSizeInBytes / meanRowSize));
 +
 +        logger.trace("Calculated page size {} for indexing {}.{} ({}/{}/{}/{})",
 +                     pageSize,
 +                     baseCfs.metadata.ksName,
 +                     baseCfs.metadata.cfName,
 +                     meanPartitionSize,
 +                     meanCellsPerPartition,
 +                     meanRowsPerPartition,
 +                     meanRowSize);
 +
 +        return pageSize;
 +    }
 +
 +    /**
 +     * Delete all data from all indexes for this partition.
 +     * For when cleanup rips a partition out entirely.
 +     *
 +     * TODO : improve cleanup transaction to batch updates & perform them async
 +     */
 +    public void deletePartition(UnfilteredRowIterator partition, int nowInSec)
 +    {
 +        // we need to acquire memtable lock because secondary index deletion may
 +        // cause a race (see CASSANDRA-3712). This is done internally by the
 +        // index transaction when it commits
 +        CleanupTransaction indexTransaction = newCleanupTransaction(partition.partitionKey(),
 +                                                                    partition.columns(),
 +                                                                    nowInSec);
 +        indexTransaction.start();
 +        indexTransaction.onPartitionDeletion(new DeletionTime(FBUtilities.timestampMicros(), nowInSec));
 +        indexTransaction.commit();
 +
 +        while (partition.hasNext())
 +        {
 +            Unfiltered unfiltered = partition.next();
 +            if (unfiltered.kind() != Unfiltered.Kind.ROW)
 +                continue;
 +
 +            indexTransaction = newCleanupTransaction(partition.partitionKey(),
 +                                                     partition.columns(),
 +                                                     nowInSec);
 +            indexTransaction.start();
 +            indexTransaction.onRowDelete((Row)unfiltered);
 +            indexTransaction.commit();
 +        }
 +    }
 +
 +    /**
 +     * Called at query time to choose which (if any) of the registered index implementations to use for a given query.
 +     *
 +     * This is a two step processes, firstly compiling the set of searchable indexes then choosing the one which reduces
 +     * the search space the most.
 +     *
 +     * In the first phase, if the command's RowFilter contains any custom index expressions, the indexes that they
 +     * specify are automatically included. Following that, the registered indexes are filtered to include only those
 +     * which support the standard expressions in the RowFilter.
 +     *
 +     * The filtered set then sorted by selectivity, as reported by the Index implementations' getEstimatedResultRows
 +     * method.
 +     *
 +     * Implementation specific validation of the target expression, either custom or standard, by the selected
 +     * index should be performed in the searcherFor method to ensure that we pick the right index regardless of
 +     * the validity of the expression.
 +     *
 +     * This method is only called once during the lifecycle of a ReadCommand and the result is
 +     * cached for future use when obtaining a Searcher, getting the index's underlying CFS for
 +     * ReadOrderGroup, or an estimate of the result size from an average index query.
 +     *
 +     * @param command ReadCommand to be executed
 +     * @return an Index instance, ready to use during execution of the command, or null if none
 +     * of the registered indexes can support the command.
 +     */
 +    public Index getBestIndexFor(ReadCommand command)
 +    {
 +        if (indexes.isEmpty() || command.rowFilter().isEmpty())
 +            return null;
 +
 +        Set<Index> searchableIndexes = new HashSet<>();
 +        for (RowFilter.Expression expression : command.rowFilter())
 +        {
 +            if (expression.isCustom())
 +            {
 +                // Only a single custom expression is allowed per query and, if present,
 +                // we want to always favour the index specified in such an expression
 +                RowFilter.CustomExpression customExpression = (RowFilter.CustomExpression)expression;
 +                logger.trace("Command contains a custom index expression, using target index {}", customExpression.getTargetIndex().name);
 +                Tracing.trace("Command contains a custom index expression, using target index {}", customExpression.getTargetIndex().name);
 +                return indexes.get(customExpression.getTargetIndex().name);
 +            }
 +            else
 +            {
 +                indexes.values().stream()
 +                       .filter(index -> index.supportsExpression(expression.column(), expression.operator()))
 +                       .forEach(searchableIndexes::add);
 +            }
 +        }
 +
 +        if (searchableIndexes.isEmpty())
 +        {
 +            logger.trace("No applicable indexes found");
 +            Tracing.trace("No applicable indexes found");
 +            return null;
 +        }
 +
 +        Index selected = searchableIndexes.size() == 1
 +                         ? Iterables.getOnlyElement(searchableIndexes)
 +                         : searchableIndexes.stream()
 +                                            .min((a, b) -> Longs.compare(a.getEstimatedResultRows(),
 +                                                                         b.getEstimatedResultRows()))
 +                                            .orElseThrow(() -> new AssertionError("Could not select most selective index"));
 +
 +        // pay for an additional threadlocal get() rather than build the strings unnecessarily
 +        if (Tracing.isTracing())
 +        {
 +            Tracing.trace("Index mean cardinalities are {}. Scanning with {}.",
 +                          searchableIndexes.stream().map(i -> i.getIndexMetadata().name + ':' + i.getEstimatedResultRows())
 +                                           .collect(Collectors.joining(",")),
 +                          selected.getIndexMetadata().name);
 +        }
 +        return selected;
 +    }
 +
 +    /**
 +     * Called at write time to ensure that values present in the update
 +     * are valid according to the rules of all registered indexes which
 +     * will process it. The partition key as well as the clustering and
 +     * cell values for each row in the update may be checked by index
 +     * implementations
 +     * @param update PartitionUpdate containing the values to be validated by registered Index implementations
 +     * @throws InvalidRequestException
 +     */
 +    public void validate(PartitionUpdate update) throws InvalidRequestException
 +    {
 +        for (Index index : indexes.values())
 +            index.validate(update);
 +    }
 +
 +    /**
 +     * IndexRegistry methods
 +     */
 +    public void registerIndex(Index index)
 +    {
 +        String name = index.getIndexMetadata().name;
 +        indexes.put(name, index);
 +        logger.trace("Registered index {}", name);
 +    }
 +
 +    public void unregisterIndex(Index index)
 +    {
 +        unregisterIndex(index.getIndexMetadata().name);
 +    }
 +
 +    private Index unregisterIndex(String name)
 +    {
 +        Index removed = indexes.remove(name);
 +        builtIndexes.remove(name);
 +        logger.trace(removed == null ? "Index {} was not registered" : "Removed index {} from registry",
 +                     name);
 +        return removed;
 +    }
 +
 +    public Index getIndex(IndexMetadata metadata)
 +    {
 +        return indexes.get(metadata.name);
 +    }
 +
 +    public Collection<Index> listIndexes()
 +    {
 +        return ImmutableSet.copyOf(indexes.values());
 +    }
 +
 +    /**
 +     * Handling of index updates.
 +     * Implementations of the various IndexTransaction interfaces, for keeping indexes in sync with base data
 +     * during updates, compaction and cleanup. Plus factory methods for obtaining transaction instances.
 +     */
 +
 +    /**
 +     * Transaction for updates on the write path.
 +     */
 +    public UpdateTransaction newUpdateTransaction(PartitionUpdate update, OpOrder.Group opGroup, int nowInSec)
 +    {
 +        if (!hasIndexes())
 +            return UpdateTransaction.NO_OP;
 +
 +        Index.Indexer[] indexers = indexes.values().stream()
 +                                          .map(i -> i.indexerFor(update.partitionKey(),
 +                                                                 update.columns(),
 +                                                                 nowInSec,
 +                                                                 opGroup,
 +                                                                 IndexTransaction.Type.UPDATE))
 +                                          .filter(Objects::nonNull)
 +                                          .toArray(Index.Indexer[]::new);
 +
 +        return indexers.length == 0 ? UpdateTransaction.NO_OP : new WriteTimeTransaction(indexers);
 +    }
 +
 +    /**
 +     * Transaction for use when merging rows during compaction
 +     */
 +    public CompactionTransaction newCompactionTransaction(DecoratedKey key,
 +                                                          PartitionColumns partitionColumns,
 +                                                          int versions,
 +                                                          int nowInSec)
 +    {
 +        // the check for whether there are any registered indexes is already done in CompactionIterator
 +        return new IndexGCTransaction(key, partitionColumns, versions, nowInSec, listIndexes());
 +    }
 +
 +    /**
 +     * Transaction for use when removing partitions during cleanup
 +     */
 +    public CleanupTransaction newCleanupTransaction(DecoratedKey key,
 +                                                    PartitionColumns partitionColumns,
 +                                                    int nowInSec)
 +    {
 +        if (!hasIndexes())
 +            return CleanupTransaction.NO_OP;
 +
 +        return new CleanupGCTransaction(key, partitionColumns, nowInSec, listIndexes());
 +    }
 +
 +    /**
 +     * A single use transaction for processing a partition update on the regular write path
 +     */
 +    private static final class WriteTimeTransaction implements UpdateTransaction
 +    {
 +        private final Index.Indexer[] indexers;
 +
 +        private WriteTimeTransaction(Index.Indexer...indexers)
 +        {
 +            // don't allow null indexers, if we don't need any use a NullUpdater object
 +            for (Index.Indexer indexer : indexers) assert indexer != null;
 +            this.indexers = indexers;
 +        }
 +
 +        public void start()
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.begin();
 +        }
 +
 +        public void onPartitionDeletion(DeletionTime deletionTime)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.partitionDelete(deletionTime);
 +        }
 +
 +        public void onRangeTombstone(RangeTombstone tombstone)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.rangeTombstone(tombstone);
 +        }
 +
 +        public void onInserted(Row row)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.insertRow(row);
 +        }
 +
 +        public void onUpdated(Row existing, Row updated)
 +        {
 +            final Row.Builder toRemove = BTreeRow.sortedBuilder();
 +            toRemove.newRow(existing.clustering());
 +            toRemove.addPrimaryKeyLivenessInfo(existing.primaryKeyLivenessInfo());
 +            toRemove.addRowDeletion(existing.deletion());
 +            final Row.Builder toInsert = BTreeRow.sortedBuilder();
 +            toInsert.newRow(updated.clustering());
 +            toInsert.addPrimaryKeyLivenessInfo(updated.primaryKeyLivenessInfo());
 +            toInsert.addRowDeletion(updated.deletion());
 +            // diff listener collates the columns to be added & removed from the indexes
 +            RowDiffListener diffListener = new RowDiffListener()
 +            {
 +                public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
 +                {
 +                }
 +
 +                public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original)
 +                {
 +                }
 +
 +                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
 +                {
 +                }
 +
 +                public void onCell(int i, Clustering clustering, Cell merged, Cell original)
 +                {
 +                    if (merged != null && !merged.equals(original))
 +                        toInsert.addCell(merged);
 +
 +                    if (merged == null || (original != null && shouldCleanupOldValue(original, merged)))
 +                        toRemove.addCell(original);
 +
 +                }
 +            };
 +            Rows.diff(diffListener, updated, existing);
 +            Row oldRow = toRemove.build();
 +            Row newRow = toInsert.build();
 +            for (Index.Indexer indexer : indexers)
 +                indexer.updateRow(oldRow, newRow);
 +        }
 +
 +        public void commit()
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.finish();
 +        }
 +
 +        private boolean shouldCleanupOldValue(Cell oldCell, Cell newCell)
 +        {
 +            // If either the value or timestamp is different, then we
 +            // should delete from the index. If not, then we can infer that
 +            // at least one of the cells is an ExpiringColumn and that the
 +            // difference is in the expiry time. In this case, we don't want to
 +            // delete the old value from the index as the tombstone we insert
 +            // will just hide the inserted value.
 +            // Completely identical cells (including expiring columns with
 +            // identical ttl & localExpirationTime) will not get this far due
 +            // to the oldCell.equals(newCell) in StandardUpdater.update
 +            return !oldCell.value().equals(newCell.value()) || oldCell.timestamp() != newCell.timestamp();
 +        }
 +    }
 +
 +    /**
 +     * A single-use transaction for updating indexes for a single partition during compaction where the only
 +     * operation is to merge rows
 +     * TODO : make this smarter at batching updates so we can use a single transaction to process multiple rows in
 +     * a single partition
 +     */
 +    private static final class IndexGCTransaction implements CompactionTransaction
 +    {
 +        private final DecoratedKey key;
 +        private final PartitionColumns columns;
 +        private final int versions;
 +        private final int nowInSec;
 +        private final Collection<Index> indexes;
 +
 +        private Row[] rows;
 +
 +        private IndexGCTransaction(DecoratedKey key,
 +                                   PartitionColumns columns,
 +                                   int versions,
 +                                   int nowInSec,
 +                                   Collection<Index> indexes)
 +        {
 +            this.key = key;
 +            this.columns = columns;
 +            this.versions = versions;
 +            this.indexes = indexes;
 +            this.nowInSec = nowInSec;
 +        }
 +
 +        public void start()
 +        {
 +            if (versions > 0)
 +                rows = new Row[versions];
 +        }
 +
 +        public void onRowMerge(Row merged, Row...versions)
 +        {
 +            // Diff listener constructs rows representing deltas between the merged and original versions
 +            // These delta rows are then passed to registered indexes for removal processing
 +            final Row.Builder[] builders = new Row.Builder[versions.length];
 +            RowDiffListener diffListener = new RowDiffListener()
 +            {
 +                public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
 +                {
 +                    if (original != null && (merged == null || !merged.isLive(nowInSec)))
 +                        getBuilder(i, clustering).addPrimaryKeyLivenessInfo(original);
 +                }
 +
 +                public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original)
 +                {
 +                }
 +
 +                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
 +                {
 +                }
 +
 +                public void onCell(int i, Clustering clustering, Cell merged, Cell original)
 +                {
 +                    if (original != null && (merged == null || !merged.isLive(nowInSec)))
 +                        getBuilder(i, clustering).addCell(original);
 +                }
 +
 +                private Row.Builder getBuilder(int index, Clustering clustering)
 +                {
 +                    if (builders[index] == null)
 +                    {
 +                        builders[index] = BTreeRow.sortedBuilder();
 +                        builders[index].newRow(clustering);
 +                    }
 +                    return builders[index];
 +                }
 +            };
 +
 +            Rows.diff(diffListener, merged, versions);
 +
 +            for(int i = 0; i < builders.length; i++)
 +                if (builders[i] != null)
 +                    rows[i] = builders[i].build();
 +        }
 +
 +        public void commit()
 +        {
 +            if (rows == null)
 +                return;
 +
 +            try (OpOrder.Group opGroup = Keyspace.writeOrder.start())
 +            {
 +                for (Index index : indexes)
 +                {
 +                    Index.Indexer indexer = index.indexerFor(key, columns, nowInSec, opGroup, Type.COMPACTION);
 +                    if (indexer == null)
 +                        continue;
 +
 +                    indexer.begin();
 +                    for (Row row : rows)
 +                        if (row != null)
 +                            indexer.removeRow(row);
 +                    indexer.finish();
 +                }
 +            }
 +        }
 +    }
 +
 +    /**
 +     * A single-use transaction for updating indexes for a single partition during cleanup, where
 +     * partitions and rows are only removed
 +     * TODO : make this smarter at batching updates so we can use a single transaction to process multiple rows in
 +     * a single partition
 +     */
 +    private static final class CleanupGCTransaction implements CleanupTransaction
 +    {
 +        private final DecoratedKey key;
 +        private final PartitionColumns columns;
 +        private final int nowInSec;
 +        private final Collection<Index> indexes;
 +
 +        private Row row;
 +        private DeletionTime partitionDelete;
 +
 +        private CleanupGCTransaction(DecoratedKey key,
 +                                     PartitionColumns columns,
 +                                     int nowInSec,
 +                                     Collection<Index> indexes)
 +        {
 +            this.key = key;
 +            this.columns = columns;
 +            this.indexes = indexes;
 +            this.nowInSec = nowInSec;
 +        }
 +
 +        public void start()
 +        {
 +        }
 +
 +        public void onPartitionDeletion(DeletionTime deletionTime)
 +        {
 +            partitionDelete = deletionTime;
 +        }
 +
 +        public void onRowDelete(Row row)
 +        {
 +            this.row = row;
 +        }
 +
 +        public void commit()
 +        {
 +            if (row == null && partitionDelete == null)
 +                return;
 +
 +            try (OpOrder.Group opGroup = Keyspace.writeOrder.start())
 +            {
 +                for (Index index : indexes)
 +                {
 +                    Index.Indexer indexer = index.indexerFor(key, columns, nowInSec, opGroup, Type.CLEANUP);
 +                    if (indexer == null)
 +                        continue;
 +
 +                    indexer.begin();
 +
 +                    if (partitionDelete != null)
 +                        indexer.partitionDelete(partitionDelete);
 +
 +                    if (row != null)
 +                        indexer.removeRow(row);
 +
 +                    indexer.finish();
 +                }
 +            }
 +        }
 +    }
 +
 +    private static void executeBlocking(Callable<?> task)
 +    {
 +        if (null != task)
 +            FBUtilities.waitOnFuture(blockingExecutor.submit(task));
 +    }
 +
 +    private static void executeAllBlocking(Stream<Index> indexers, Function<Index, Callable<?>> function)
 +    {
 +        List<Future<?>> waitFor = new ArrayList<>();
 +        indexers.forEach(indexer -> {
 +            Callable<?> task = function.apply(indexer);
 +            if (null != task)
 +                waitFor.add(blockingExecutor.submit(task));
 +        });
 +        FBUtilities.waitOnFutures(waitFor);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index f0cdcf5,ba060d4..3283723
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@@ -2183,10 -2246,10 +2183,10 @@@ public abstract class SSTableReader ext
  
              // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist
              // the read meter when in client mode.
-             if (Schema.isSystemKeyspace(desc.ksname))
 -            if (SystemKeyspace.NAME.equals(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
++            if (Schema.isSystemKeyspace(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
              {
                  readMeter = null;
 -                readMeterSyncFuture = null;
 +                readMeterSyncFuture = NULL;
                  return;
              }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/CQLTester.java
index 3d45393,98b8e23..3d8d03b
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@@ -140,86 -137,7 +140,87 @@@ public abstract class CQLTeste
      // We don't use USE_PREPARED_VALUES in the code below so some test can foce value preparation (if the result
      // is not expected to be the same without preparation)
      private boolean usePrepared = USE_PREPARED_VALUES;
 -    private static final boolean reusePrepared = Boolean.valueOf(System.getProperty("cassandra.test.reuse_prepared", "true"));
 +    private static boolean reusePrepared = REUSE_PREPARED;
 +
 +    public static void prepareServer()
 +    {
 +        if (isServerPrepared)
 +            return;
 +
 +        // Cleanup first
 +        try
 +        {
 +            cleanupAndLeaveDirs();
 +        }
 +        catch (IOException e)
 +        {
 +            logger.error("Failed to cleanup and recreate directories.");
 +            throw new RuntimeException(e);
 +        }
 +
 +        Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()
 +        {
 +            public void uncaughtException(Thread t, Throwable e)
 +            {
 +                logger.error("Fatal exception in thread " + t, e);
 +            }
 +        });
 +
 +        ThreadAwareSecurityManager.install();
 +
++        DatabaseDescriptor.setDaemonInitialized();
 +        Keyspace.setInitialized();
 +        isServerPrepared = true;
 +    }
 +
 +    public static void cleanupAndLeaveDirs() throws IOException
 +    {
 +        // We need to stop and unmap all CLS instances prior to cleanup() or we'll get failures on Windows.
 +        CommitLog.instance.stopUnsafe(true);
 +        mkdirs();
 +        cleanup();
 +        mkdirs();
 +        CommitLog.instance.restartUnsafe();
 +    }
 +
 +    public static void cleanup()
 +    {
 +        // clean up commitlog
 +        String[] directoryNames = { DatabaseDescriptor.getCommitLogLocation(), };
 +        for (String dirName : directoryNames)
 +        {
 +            File dir = new File(dirName);
 +            if (!dir.exists())
 +                throw new RuntimeException("No such directory: " + dir.getAbsolutePath());
 +            FileUtils.deleteRecursive(dir);
 +        }
 +
 +        cleanupSavedCaches();
 +
 +        // clean up data directory which are stored as data directory/keyspace/data files
 +        for (String dirName : DatabaseDescriptor.getAllDataFileLocations())
 +        {
 +            File dir = new File(dirName);
 +            if (!dir.exists())
 +                throw new RuntimeException("No such directory: " + dir.getAbsolutePath());
 +            FileUtils.deleteRecursive(dir);
 +        }
 +    }
 +
 +    public static void mkdirs()
 +    {
 +        DatabaseDescriptor.createAllDirectories();
 +    }
 +
 +    public static void cleanupSavedCaches()
 +    {
 +        File cachesDir = new File(DatabaseDescriptor.getSavedCachesLocation());
 +
 +        if (!cachesDir.exists() || !cachesDir.isDirectory())
 +            return;
 +
 +        FileUtils.delete(cachesDir.listFiles());
 +    }
  
      @BeforeClass
      public static void setUpClass()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 557beba,5e2fffe..e6d18c4
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@@ -32,9 -32,9 +32,10 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.Util;
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.Config;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.config.Schema;
  import org.apache.cassandra.cql3.QueryProcessor;
  import org.apache.cassandra.cql3.UntypedResultSet;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 5ac1b31,bc5be46..37b5fa9
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@@ -28,7 -27,7 +28,8 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.exceptions.ConfigurationException;
  import org.apache.cassandra.gms.ApplicationState;
  import org.apache.cassandra.gms.Gossiper;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
index 6f76db4,0000000..a2c9cf9
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
@@@ -1,56 -1,0 +1,63 @@@
 +/*
 + * 
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + * 
 + *   http://www.apache.org/licenses/LICENSE-2.0
 + * 
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + * 
 + */
 +package org.apache.cassandra.metrics;
 +
 +import java.net.InetAddress;
 +import java.util.Map;
 +import java.util.UUID;
 +
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import com.google.common.collect.Iterators;
 +
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.UntypedResultSet;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.db.marshal.Int32Type;
 +import org.apache.cassandra.db.marshal.UUIDType;
 +import org.apache.cassandra.hints.HintsService;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 +
 +public class HintedHandOffMetricsTest
 +{
++    @BeforeClass
++    public static void initDD()
++    {
++        DatabaseDescriptor.setDaemonInitialized();
++    }
++
 +    @Test
 +    public void testHintsMetrics() throws Exception
 +    {
 +        DatabaseDescriptor.getHintsDirectory().mkdirs();
 +
 +        for (int i = 0; i < 99; i++)
 +            HintsService.instance.metrics.incrPastWindow(InetAddress.getLocalHost());
 +        HintsService.instance.metrics.log();
 +
 +        UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS);
 +        Map<UUID, Integer> returned = rows.one().getMap("hints_dropped", UUIDType.instance, Int32Type.instance);
 +        assertEquals(Iterators.getLast(returned.values().iterator()).intValue(), 99);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
index f0c850d,4e160c2..bb2b9b0
--- a/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
+++ b/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
@@@ -18,14 -18,14 +18,21 @@@
  */
  package org.apache.cassandra.utils.concurrent;
  
++import org.junit.BeforeClass;
  import org.junit.Ignore;
  import org.junit.Test;
  
  import junit.framework.Assert;
++import org.apache.cassandra.config.DatabaseDescriptor;
  
  @Ignore
  public abstract class AbstractTransactionalTest
  {
++    @BeforeClass
++    public static void setupDD()
++    {
++        DatabaseDescriptor.setDaemonInitialized();
++    }
  
      protected abstract TestableTransaction newTest() throws Exception;
  


[18/19] cassandra git commit: Merge branch 'cassandra-3.11' into cassandra-3.X

Posted by yu...@apache.org.
Merge branch 'cassandra-3.11' into cassandra-3.X


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c9ead482
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c9ead482
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c9ead482

Branch: refs/heads/trunk
Commit: c9ead4826a1dfb246b4ef6c916106bb13fcdbdc5
Parents: ddefb1d 66e2145
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Dec 13 15:59:23 2016 -0800
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:59:23 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++-
 .../cassandra/db/PartitionRangeReadCommand.java |  4 +-
 .../cassandra/db/compaction/CompactionTask.java | 81 ++++++++++----------
 .../cassandra/db/lifecycle/LogTransaction.java  |  3 +-
 .../apache/cassandra/db/lifecycle/Tracker.java  | 34 ++++----
 .../cassandra/index/SecondaryIndexManager.java  |  4 +-
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  6 +-
 .../org/apache/cassandra/tools/ToolsTester.java |  3 -
 10 files changed, 85 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9ead482/CHANGES.txt
----------------------------------------------------------------------


[15/19] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by yu...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/66e21459
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/66e21459
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/66e21459

Branch: refs/heads/cassandra-3.X
Commit: 66e214592875e296bb540a966f1648f1106b2464
Parents: 1cafc3c 0fe82be
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Dec 13 15:59:14 2016 -0800
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:59:14 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++-
 .../cassandra/db/PartitionRangeReadCommand.java |  4 +-
 .../cassandra/db/compaction/CompactionTask.java | 81 ++++++++++----------
 .../cassandra/db/lifecycle/LogTransaction.java  |  3 +-
 .../apache/cassandra/db/lifecycle/Tracker.java  | 34 ++++----
 .../cassandra/index/SecondaryIndexManager.java  |  4 +-
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  6 +-
 .../org/apache/cassandra/tools/ToolsTester.java |  3 -
 10 files changed, 85 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 28ebf36,145afb9..f95dd81
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -168,12 -59,6 +168,13 @@@ Merged from 3.0
   * Correct log message for statistics of offheap memtable flush (CASSANDRA-12776)
   * Explicitly set locale for string validation (CASSANDRA-12541,CASSANDRA-12542,CASSANDRA-12543,CASSANDRA-12545)
  Merged from 2.2:
++ * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
 + * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
 + * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
 + * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
 + * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
 + * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
   * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
   * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
   * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 881fb00,39ed804..a5f76bd4
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -405,9 -388,13 +405,13 @@@ public class ColumnFamilyStore implemen
  
          logger.info("Initializing {}.{}", keyspace.getName(), name);
  
-         // scan for sstables corresponding to this cf and load them
-         data = new Tracker(this, loadSSTables);
+         // Create Memtable only on online
+         Memtable initialMemtable = null;
+         if (DatabaseDescriptor.isDaemonInitialized())
 -            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
++            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getCurrentPosition()), this);
+         data = new Tracker(initialMemtable, loadSSTables);
  
+         // scan for sstables corresponding to this cf and load them
          if (data.loadsstables)
          {
              Directories.SSTableLister sstableFiles = directories.sstableLister(Directories.OnTxnErr.IGNORE).skipTemporary(true);
@@@ -2118,7 -1957,7 +2122,7 @@@
              {
                  public Void call()
                  {
-                     cfs.data.reset();
 -                    cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
++                    cfs.data.reset(new Memtable(new AtomicReference<>(CommitLogPosition.NONE), cfs));
                      return null;
                  }
              }, true, false);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 50b568e,17adef0..045fc26
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -221,7 -199,8 +221,9 @@@ public class PartitionRangeReadCommand 
                  if (!sstable.isRepaired())
                      oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime());
              }
-             return checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
++            // iterators can be empty for offline tools
+             return iterators.isEmpty() ? EmptyIterators.unfilteredPartition(metadata(), isForThrift())
+                                        : checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
          }
          catch (RuntimeException | Error e)
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 2f90c7b,f0a1f47..a9d6c7c
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@@ -228,47 -213,29 +228,50 @@@ public class CompactionTask extends Abs
                  }
              }
  
--            // log a bunch of statistics about the result and save to system table compaction_history
- 
-             long durationInNano = System.nanoTime() - start;
-             long dTime = TimeUnit.NANOSECONDS.toMillis(durationInNano);
-             long startsize = inputSizeBytes;
 -            long dTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
 -            long startsize = SSTableReader.getTotalBytes(transaction.originals());
--            long endsize = SSTableReader.getTotalBytes(newSStables);
--            double ratio = (double) endsize / (double) startsize;
--
--            StringBuilder newSSTableNames = new StringBuilder();
--            for (SSTableReader reader : newSStables)
--                newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
- 
-             long totalSourceRows = 0;
-             for (int i = 0; i < mergedRowCounts.length; i++)
-                 totalSourceRows += mergedRowCounts[i] * (i + 1);
- 
-             String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
-             logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %s to %s (~%d%% of original) in %,dms.  Read Throughput = %s, Write Throughput = %s, Row Throughput = ~%,d/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                       taskId,
-                                       transaction.originals().size(),
-                                       newSSTableNames.toString(),
-                                       getLevel(),
-                                       FBUtilities.prettyPrintMemory(startsize),
-                                       FBUtilities.prettyPrintMemory(endsize),
-                                       (int) (ratio * 100),
-                                       dTime,
-                                       FBUtilities.prettyPrintMemoryPerSecond(startsize, durationInNano),
-                                       FBUtilities.prettyPrintMemoryPerSecond(endsize, durationInNano),
-                                       (int) totalSourceCQLRows / (TimeUnit.NANOSECONDS.toSeconds(durationInNano) + 1),
-                                       totalSourceRows,
-                                       totalKeysWritten,
-                                       mergeSummary));
-             logger.trace("CF Total Bytes Compacted: {}", FBUtilities.prettyPrintMemory(CompactionTask.addToTotalBytesCompacted(endsize)));
-             logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
-             cfs.getCompactionStrategyManager().compactionLogger.compaction(startTime, transaction.originals(), System.currentTimeMillis(), newSStables);
- 
-             // update the metrics
-             cfs.metric.compactionBytesWritten.inc(endsize);
--
 -            if (offline)
 +            if (transaction.isOffline())
+             {
                  Refs.release(Refs.selfRefs(newSStables));
+             }
+             else
+             {
 -                double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
 -                Summary mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
 -                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
 -                                           taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, mergeSummary.totalSourceRows, totalKeysWritten, mergeSummary.partitionMerge));
 -                logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
++                // log a bunch of statistics about the result and save to system table compaction_history
++
++                long durationInNano = System.nanoTime() - start;
++                long dTime = TimeUnit.NANOSECONDS.toMillis(durationInNano);
++                long startsize = inputSizeBytes;
++                long endsize = SSTableReader.getTotalBytes(newSStables);
++                double ratio = (double) endsize / (double) startsize;
++
++                StringBuilder newSSTableNames = new StringBuilder();
++                for (SSTableReader reader : newSStables)
++                    newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
++                long totalSourceRows = 0;
++                for (int i = 0; i < mergedRowCounts.length; i++)
++                    totalSourceRows += mergedRowCounts[i] * (i + 1);
++
++                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getTableName(), mergedRowCounts, startsize, endsize);
++                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %s to %s (~%d%% of original) in %,dms.  Read Throughput = %s, Write Throughput = %s, Row Throughput = ~%,d/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
++                                           taskId,
++                                           transaction.originals().size(),
++                                           newSSTableNames.toString(),
++                                           getLevel(),
++                                           FBUtilities.prettyPrintMemory(startsize),
++                                           FBUtilities.prettyPrintMemory(endsize),
++                                           (int) (ratio * 100),
++                                           dTime,
++                                           FBUtilities.prettyPrintMemoryPerSecond(startsize, durationInNano),
++                                           FBUtilities.prettyPrintMemoryPerSecond(endsize, durationInNano),
++                                           (int) totalSourceCQLRows / (TimeUnit.NANOSECONDS.toSeconds(durationInNano) + 1),
++                                           totalSourceRows,
++                                           totalKeysWritten,
++                                           mergeSummary));
++                logger.trace("CF Total Bytes Compacted: {}", FBUtilities.prettyPrintMemory(CompactionTask.addToTotalBytesCompacted(endsize)));
+                 logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
++                cfs.getCompactionStrategyManager().compactionLogger.compaction(startTime, transaction.originals(), System.currentTimeMillis(), newSStables);
++
++                // update the metrics
++                cfs.metric.compactionBytesWritten.inc(endsize);
+             }
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index f464e08,9feaa3e..e2fcb06
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@@ -31,8 -31,7 +31,7 @@@ import com.google.common.collect.*
  import org.apache.cassandra.db.ColumnFamilyStore;
  import org.apache.cassandra.db.Directories;
  import org.apache.cassandra.db.Memtable;
- import org.apache.cassandra.db.commitlog.CommitLog;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
index 6435e3e,0d87cc9..4514b72
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@@ -30,6 -31,8 +31,8 @@@ import junit.framework.Assert
  import org.apache.cassandra.MockSchema;
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.Memtable;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
++import org.apache.cassandra.db.commitlog.CommitLogPosition;
  import org.apache.cassandra.db.compaction.OperationType;
  import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
  import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
@@@ -268,7 -271,7 +271,7 @@@ public class LifecycleTransactionTest e
  
          private static Tracker tracker(ColumnFamilyStore cfs, List<SSTableReader> readers)
          {
-             Tracker tracker = new Tracker(cfs, false);
 -            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs), false);
++            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(CommitLogPosition.NONE), cfs), false);
              tracker.addInitialSSTables(readers);
              return tracker;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66e21459/test/unit/org/apache/cassandra/tools/ToolsTester.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/tools/ToolsTester.java
index 97b19c9,0000000..ead4e31
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/tools/ToolsTester.java
+++ b/test/unit/org/apache/cassandra/tools/ToolsTester.java
@@@ -1,296 -1,0 +1,293 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.cassandra.tools;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.lang.management.ManagementFactory;
 +import java.lang.management.ThreadInfo;
 +import java.lang.management.ThreadMXBean;
 +import java.lang.reflect.InvocationTargetException;
 +import java.lang.reflect.Method;
 +import java.security.Permission;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Set;
 +import java.util.regex.Pattern;
 +import java.util.stream.Collectors;
 +
 +import org.apache.commons.io.FileUtils;
 +import org.junit.BeforeClass;
 +
 +import org.slf4j.LoggerFactory;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.fail;
 +
 +/**
 + * Base unit test class for standalone tools
 + */
 +public abstract class ToolsTester
 +{
 +    private static List<ThreadInfo> initialThreads;
 +
 +    static final String[] EXPECTED_THREADS_WITH_SCHEMA = {
-     "(NativePool|SlabPool|HeapPool)Cleaner",
-     "COMMIT-LOG-ALLOCATOR",
-     "COMMIT-LOG-WRITER",
 +    "PerDiskMemtableFlushWriter_0:[1-9]",
 +    "MemtablePostFlush:[1-9]",
 +    "MemtableFlushWriter:[1-9]",
 +    "MemtableReclaimMemory:[1-9]",
 +    };
 +    static final String[] OPTIONAL_THREADS_WITH_SCHEMA = {
 +    "ScheduledTasks:[1-9]",
 +    "OptionalTasks:[1-9]",
 +    "Reference-Reaper:[1-9]",
 +    "LocalPool-Cleaner:[1-9]",
 +    "CacheCleanupExecutor:[1-9]",
 +    "CompactionExecutor:[1-9]",
 +    "ValidationExecutor:[1-9]",
 +    "NonPeriodicTasks:[1-9]",
 +    "Sampler:[1-9]",
 +    "SecondaryIndexManagement:[1-9]",
 +    "Strong-Reference-Leak-Detector:[1-9]",
 +    "Background_Reporter:[1-9]",
 +    "EXPIRING-MAP-REAPER:[1-9]",
 +    };
 +
 +    public void assertNoUnexpectedThreadsStarted(String[] expectedThreadNames, String[] optionalThreadNames)
 +    {
 +        ThreadMXBean threads = ManagementFactory.getThreadMXBean();
 +
 +        Set<String> initial = initialThreads
 +                              .stream()
 +                              .map(ThreadInfo::getThreadName)
 +                              .collect(Collectors.toSet());
 +
 +        Set<String> current = Arrays.stream(threads.getThreadInfo(threads.getAllThreadIds()))
 +                                    .map(ThreadInfo::getThreadName)
 +                                    .collect(Collectors.toSet());
 +
 +        List<Pattern> expected = expectedThreadNames != null
 +                                 ? Arrays.stream(expectedThreadNames).map(Pattern::compile).collect(Collectors.toList())
 +                                 : Collections.emptyList();
 +
 +        List<Pattern> optional = optionalThreadNames != null
 +                                 ? Arrays.stream(optionalThreadNames).map(Pattern::compile).collect(Collectors.toList())
 +                                 : Collections.emptyList();
 +
 +        current.removeAll(initial);
 +
 +        List<Pattern> notPresent = expected.stream()
 +                                           .filter(threadNamePattern -> !current.stream().anyMatch(threadName -> threadNamePattern.matcher(threadName).matches()))
 +                                           .collect(Collectors.toList());
 +
 +        Set<String> remain = current.stream()
 +                                    .filter(threadName -> expected.stream().anyMatch(pattern -> pattern.matcher(threadName).matches()))
 +                                    .filter(threadName -> optional.stream().anyMatch(pattern -> pattern.matcher(threadName).matches()))
 +                                    .collect(Collectors.toSet());
 +
 +        if (!current.isEmpty())
 +            System.err.println("Unexpected thread names: " + remain);
 +        if (!notPresent.isEmpty())
 +            System.err.println("Mandatory thread missing: " + notPresent);
 +
 +        assertTrue("Wrong thread status", remain.isEmpty() && notPresent.isEmpty());
 +    }
 +
 +    public void assertSchemaNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.config.Schema");
 +    }
 +
 +    public void assertSchemaLoaded()
 +    {
 +        assertClassLoaded("org.apache.cassandra.config.Schema");
 +    }
 +
 +    public void assertKeyspaceNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.db.Keyspace");
 +    }
 +
 +    public void assertKeyspaceLoaded()
 +    {
 +        assertClassLoaded("org.apache.cassandra.db.Keyspace");
 +    }
 +
 +    public void assertServerNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.transport.Server");
 +    }
 +
 +    public void assertSystemKSNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.db.SystemKeyspace");
 +    }
 +
 +    public void assertCLSMNotLoaded()
 +    {
 +        assertClassNotLoaded("org.apache.cassandra.db.commitlog.CommitLogSegmentManager");
 +    }
 +
 +    public void assertClassLoaded(String clazz)
 +    {
 +        assertClassLoadedStatus(clazz, true);
 +    }
 +
 +    public void assertClassNotLoaded(String clazz)
 +    {
 +        assertClassLoadedStatus(clazz, false);
 +    }
 +
 +    private void assertClassLoadedStatus(String clazz, boolean expected)
 +    {
 +        for (ClassLoader cl = Thread.currentThread().getContextClassLoader(); cl != null; cl = cl.getParent())
 +        {
 +            try
 +            {
 +                Method mFindLoadedClass = ClassLoader.class.getDeclaredMethod("findLoadedClass", String.class);
 +                mFindLoadedClass.setAccessible(true);
 +                boolean loaded = mFindLoadedClass.invoke(cl, clazz) != null;
 +
 +                if (expected)
 +                {
 +                    if (loaded)
 +                        return;
 +                }
 +                else
 +                    assertFalse(clazz + " has been loaded", loaded);
 +            }
 +            catch (Exception e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +        }
 +
 +        if (expected)
 +            fail(clazz + " has not been loaded");
 +    }
 +
 +    public void runTool(int expectedExitCode, String clazz, String... args)
 +    {
 +        try
 +        {
 +            // install security manager to get informed about the exit-code
 +            System.setSecurityManager(new SecurityManager()
 +            {
 +                public void checkExit(int status)
 +                {
 +                    throw new SystemExitException(status);
 +                }
 +
 +                public void checkPermission(Permission perm)
 +                {
 +                }
 +
 +                public void checkPermission(Permission perm, Object context)
 +                {
 +                }
 +            });
 +
 +            try
 +            {
 +                Class.forName(clazz).getDeclaredMethod("main", String[].class).invoke(null, (Object) args);
 +            }
 +            catch (InvocationTargetException e)
 +            {
 +                Throwable cause = e.getCause();
 +                if (cause instanceof Error)
 +                    throw (Error) cause;
 +                if (cause instanceof RuntimeException)
 +                    throw (RuntimeException) cause;
 +                throw e;
 +            }
 +
 +            assertEquals("Unexpected exit code", expectedExitCode, 0);
 +        }
 +        catch (SystemExitException e)
 +        {
 +            assertEquals("Unexpected exit code", expectedExitCode, e.status);
 +        }
 +        catch (InvocationTargetException e)
 +        {
 +            throw new RuntimeException(e.getTargetException());
 +        }
 +        catch (Exception e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +        finally
 +        {
 +            // uninstall security manager
 +            System.setSecurityManager(null);
 +        }
 +    }
 +
 +    @BeforeClass
 +    public static void setupTester()
 +    {
 +        System.setProperty("cassandra.partitioner", "org.apache.cassandra.dht.Murmur3Partitioner");
 +
 +        // may start an async appender
 +        LoggerFactory.getLogger(ToolsTester.class);
 +
 +        ThreadMXBean threads = ManagementFactory.getThreadMXBean();
 +        initialThreads = Arrays.asList(threads.getThreadInfo(threads.getAllThreadIds()));
 +    }
 +
 +    public static class SystemExitException extends Error
 +    {
 +        public final int status;
 +
 +        public SystemExitException(int status)
 +        {
 +            this.status = status;
 +        }
 +    }
 +
 +    public static String findOneSSTable(String ks, String cf) throws IOException
 +    {
 +        File cfDir = sstableDir(ks, cf);
 +        File[] sstableFiles = cfDir.listFiles((file) -> file.isFile() && file.getName().endsWith("-Data.db"));
 +        return sstableFiles[0].getAbsolutePath();
 +    }
 +
 +    public static String sstableDirName(String ks, String cf) throws IOException
 +    {
 +        return sstableDir(ks, cf).getAbsolutePath();
 +    }
 +
 +    public static File sstableDir(String ks, String cf) throws IOException
 +    {
 +        File dataDir = copySSTables();
 +        File ksDir = new File(dataDir, ks);
 +        File[] cfDirs = ksDir.listFiles((dir, name) -> cf.equals(name) || name.startsWith(cf + '-'));
 +        return cfDirs[0];
 +    }
 +
 +    public static File copySSTables() throws IOException
 +    {
 +        File dataDir = new File("build/test/cassandra/data");
 +        File srcDir = new File("test/data/legacy-sstables/ma");
 +        FileUtils.copyDirectory(new File(srcDir, "legacy_tables"), new File(dataDir, "legacy_sstables"));
 +        return dataDir;
 +    }
 +}


[12/19] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index 003b624,0000000..a6ed3ba
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@@ -1,1112 -1,0 +1,1114 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.index;
 +
 +import java.lang.reflect.Constructor;
 +import java.util.*;
 +import java.util.concurrent.*;
 +import java.util.function.Function;
 +import java.util.stream.Collectors;
 +import java.util.stream.Stream;
 +
 +import com.google.common.base.Joiner;
 +import com.google.common.base.Strings;
 +import com.google.common.collect.ImmutableSet;
 +import com.google.common.collect.Iterables;
 +import com.google.common.collect.Maps;
 +import com.google.common.collect.Sets;
 +import com.google.common.primitives.Longs;
 +import com.google.common.util.concurrent.Futures;
 +import com.google.common.util.concurrent.MoreExecutors;
 +import org.apache.commons.lang3.StringUtils;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 +import org.apache.cassandra.concurrent.NamedThreadFactory;
 +import org.apache.cassandra.concurrent.StageManager;
 +import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.statements.IndexTarget;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.filter.RowFilter;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.lifecycle.View;
 +import org.apache.cassandra.db.partitions.PartitionIterators;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.index.internal.CassandraIndex;
 +import org.apache.cassandra.index.transactions.*;
 +import org.apache.cassandra.io.sstable.ReducingKeyIterator;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.schema.Indexes;
 +import org.apache.cassandra.service.pager.SinglePartitionPager;
 +import org.apache.cassandra.tracing.Tracing;
 +import org.apache.cassandra.transport.Server;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.apache.cassandra.utils.concurrent.Refs;
 +
 +/**
 + * Handles the core maintenance functionality associated with indexes: adding/removing them to or from
 + * a table, (re)building during bootstrap or other streaming operations, flushing, reloading metadata
 + * and so on.
 + *
 + * The Index interface defines a number of methods which return Callable<?>. These are primarily the
 + * management tasks for an index implementation. Most of them are currently executed in a blocking
 + * fashion via submission to SIM's blockingExecutor. This provides the desired behaviour in pretty
 + * much all cases, as tasks like flushing an index needs to be executed synchronously to avoid potentially
 + * deadlocking on the FlushWriter or PostFlusher. Several of these Callable<?> returning methods on Index could
 + * then be defined with as void and called directly from SIM (rather than being run via the executor service).
 + * Separating the task defintion from execution gives us greater flexibility though, so that in future, for example,
 + * if the flush process allows it we leave open the possibility of executing more of these tasks asynchronously.
 + *
 + * The primary exception to the above is the Callable returned from Index#addIndexedColumn. This may
 + * involve a significant effort, building a new index over any existing data. We perform this task asynchronously;
 + * as it is called as part of a schema update, which we do not want to block for a long period. Building non-custom
 + * indexes is performed on the CompactionManager.
 + *
 + * This class also provides instances of processors which listen to updates to the base table and forward to
 + * registered Indexes the info required to keep those indexes up to date.
 + * There are two variants of these processors, each with a factory method provided by SIM:
 + *      IndexTransaction: deals with updates generated on the regular write path.
 + *      CleanupTransaction: used when partitions are modified during compaction or cleanup operations.
 + * Further details on their usage and lifecycles can be found in the interface definitions below.
 + *
 + * Finally, the bestIndexFor method is used at query time to identify the most selective index of those able
 + * to satisfy any search predicates defined by a ReadCommand's RowFilter. It returns a thin IndexAccessor object
 + * which enables the ReadCommand to access the appropriate functions of the Index at various stages in its lifecycle.
 + * e.g. the getEstimatedResultRows is required when StorageProxy calculates the initial concurrency factor for
 + * distributing requests to replicas, whereas a Searcher instance is needed when the ReadCommand is executed locally on
 + * a target replica.
 + */
 +public class SecondaryIndexManager implements IndexRegistry
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(SecondaryIndexManager.class);
 +
 +    // default page size (in rows) when rebuilding the index for a whole partition
 +    public static final int DEFAULT_PAGE_SIZE = 10000;
 +
 +    private Map<String, Index> indexes = Maps.newConcurrentMap();
 +
 +    /**
 +     * The indexes that are ready to server requests.
 +     */
 +    private Set<String> builtIndexes = Sets.newConcurrentHashSet();
 +
 +    // executes tasks returned by Indexer#addIndexColumn which may require index(es) to be (re)built
 +    private static final ExecutorService asyncExecutor =
 +        new JMXEnabledThreadPoolExecutor(1,
 +                                         StageManager.KEEPALIVE,
 +                                         TimeUnit.SECONDS,
 +                                         new LinkedBlockingQueue<>(),
 +                                         new NamedThreadFactory("SecondaryIndexManagement"),
 +                                         "internal");
 +
 +    // executes all blocking tasks produced by Indexers e.g. getFlushTask, getMetadataReloadTask etc
 +    private static final ExecutorService blockingExecutor = MoreExecutors.newDirectExecutorService();
 +
 +    /**
 +     * The underlying column family containing the source data for these indexes
 +     */
 +    public final ColumnFamilyStore baseCfs;
 +
 +    public SecondaryIndexManager(ColumnFamilyStore baseCfs)
 +    {
 +        this.baseCfs = baseCfs;
 +    }
 +
 +
 +    /**
 +     * Drops and adds new indexes associated with the underlying CF
 +     */
 +    public void reload()
 +    {
 +        // figure out what needs to be added and dropped.
 +        Indexes tableIndexes = baseCfs.metadata.getIndexes();
 +        indexes.keySet()
 +               .stream()
 +               .filter(indexName -> !tableIndexes.has(indexName))
 +               .forEach(this::removeIndex);
 +
 +        // we call add for every index definition in the collection as
 +        // some may not have been created here yet, only added to schema
 +        for (IndexMetadata tableIndex : tableIndexes)
 +            addIndex(tableIndex);
 +    }
 +
 +    private Future<?> reloadIndex(IndexMetadata indexDef)
 +    {
 +        Index index = indexes.get(indexDef.name);
 +        Callable<?> reloadTask = index.getMetadataReloadTask(indexDef);
 +        return reloadTask == null
 +               ? Futures.immediateFuture(null)
 +               : blockingExecutor.submit(reloadTask);
 +    }
 +
 +    private Future<?> createIndex(IndexMetadata indexDef)
 +    {
 +        Index index = createInstance(indexDef);
 +        index.register(this);
 +
 +        // if the index didn't register itself, we can probably assume that no initialization needs to happen
 +        final Callable<?> initialBuildTask = indexes.containsKey(indexDef.name)
 +                                           ? index.getInitializationTask()
 +                                           : null;
 +        if (initialBuildTask == null)
 +        {
 +            // We need to make sure that the index is marked as built in the case where the initialBuildTask
 +            // does not need to be run (if the index didn't register itself or if the base table was empty).
 +            markIndexBuilt(indexDef.name);
 +            return Futures.immediateFuture(null);
 +        }
 +        return asyncExecutor.submit(index.getInitializationTask());
 +    }
 +
 +    /**
 +     * Adds and builds a index
 +     * @param indexDef the IndexMetadata describing the index
 +     */
 +    public synchronized Future<?> addIndex(IndexMetadata indexDef)
 +    {
 +        if (indexes.containsKey(indexDef.name))
 +            return reloadIndex(indexDef);
 +        else
 +            return createIndex(indexDef);
 +    }
 +
 +    /**
 +     * Checks if the specified index is queryable.
 +     *
 +     * @param index the index
 +     * @return <code>true</code> if the specified index is queryable, <code>false</code> otherwise
 +     */
 +    public boolean isIndexQueryable(Index index)
 +    {
 +        return builtIndexes.contains(index.getIndexMetadata().name);
 +    }
 +
 +    public synchronized void removeIndex(String indexName)
 +    {
 +        Index index = unregisterIndex(indexName);
 +        if (null != index)
 +        {
 +            markIndexRemoved(indexName);
 +            executeBlocking(index.getInvalidateTask());
 +        }
 +    }
 +
 +
 +    public Set<IndexMetadata> getDependentIndexes(ColumnDefinition column)
 +    {
 +        if (indexes.isEmpty())
 +            return Collections.emptySet();
 +
 +        Set<IndexMetadata> dependentIndexes = new HashSet<>();
 +        for (Index index : indexes.values())
 +            if (index.dependsOn(column))
 +                dependentIndexes.add(index.getIndexMetadata());
 +
 +        return dependentIndexes;
 +    }
 +
 +    /**
 +     * Called when dropping a Table
 +     */
 +    public void markAllIndexesRemoved()
 +    {
 +       getBuiltIndexNames().forEach(this::markIndexRemoved);
 +    }
 +
 +    /**
 +    * Does a full, blocking rebuild of the indexes specified by columns from the sstables.
 +    * Caller must acquire and release references to the sstables used here.
 +    * Note also that only this method of (re)building indexes:
 +    *   a) takes a set of index *names* rather than Indexers
 +    *   b) marks exsiting indexes removed prior to rebuilding
 +    *
 +    * @param sstables the data to build from
 +    * @param indexNames the list of indexes to be rebuilt
 +    */
 +    public void rebuildIndexesBlocking(Collection<SSTableReader> sstables, Set<String> indexNames)
 +    {
 +        Set<Index> toRebuild = indexes.values().stream()
 +                                               .filter(index -> indexNames.contains(index.getIndexMetadata().name))
 +                                               .filter(Index::shouldBuildBlocking)
 +                                               .collect(Collectors.toSet());
 +        if (toRebuild.isEmpty())
 +        {
 +            logger.info("No defined indexes with the supplied names: {}", Joiner.on(',').join(indexNames));
 +            return;
 +        }
 +
 +        toRebuild.forEach(indexer -> markIndexRemoved(indexer.getIndexMetadata().name));
 +
 +        buildIndexesBlocking(sstables, toRebuild);
 +
 +        toRebuild.forEach(indexer -> markIndexBuilt(indexer.getIndexMetadata().name));
 +    }
 +
 +    public void buildAllIndexesBlocking(Collection<SSTableReader> sstables)
 +    {
 +        buildIndexesBlocking(sstables, indexes.values()
 +                                              .stream()
 +                                              .filter(Index::shouldBuildBlocking)
 +                                              .collect(Collectors.toSet()));
 +    }
 +
 +    // For convenience, may be called directly from Index impls
 +    public void buildIndexBlocking(Index index)
 +    {
 +        if (index.shouldBuildBlocking())
 +        {
 +            try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL));
 +                 Refs<SSTableReader> sstables = viewFragment.refs)
 +            {
 +                buildIndexesBlocking(sstables, Collections.singleton(index));
 +                markIndexBuilt(index.getIndexMetadata().name);
 +            }
 +        }
 +    }
 +
 +    /**
 +     * Checks if the specified {@link ColumnFamilyStore} is a secondary index.
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code> to check.
 +     * @return <code>true</code> if the specified <code>ColumnFamilyStore</code> is a secondary index,
 +     * <code>false</code> otherwise.
 +     */
 +    public static boolean isIndexColumnFamilyStore(ColumnFamilyStore cfs)
 +    {
 +        return isIndexColumnFamily(cfs.name);
 +    }
 +
 +    /**
 +     * Checks if the specified {@link ColumnFamilyStore} is the one secondary index.
 +     *
 +     * @param cfName the name of the <code>ColumnFamilyStore</code> to check.
 +     * @return <code>true</code> if the specified <code>ColumnFamilyStore</code> is a secondary index,
 +     * <code>false</code> otherwise.
 +     */
 +    public static boolean isIndexColumnFamily(String cfName)
 +    {
 +        return cfName.contains(Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    /**
 +     * Returns the parent of the specified {@link ColumnFamilyStore}.
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code>
 +     * @return the parent of the specified <code>ColumnFamilyStore</code>
 +     */
 +    public static ColumnFamilyStore getParentCfs(ColumnFamilyStore cfs)
 +    {
 +        String parentCfs = getParentCfsName(cfs.name);
 +        return cfs.keyspace.getColumnFamilyStore(parentCfs);
 +    }
 +
 +    /**
 +     * Returns the parent name of the specified {@link ColumnFamilyStore}.
 +     *
 +     * @param cfName the <code>ColumnFamilyStore</code> name
 +     * @return the parent name of the specified <code>ColumnFamilyStore</code>
 +     */
 +    public static String getParentCfsName(String cfName)
 +    {
 +        assert isIndexColumnFamily(cfName);
 +        return StringUtils.substringBefore(cfName, Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    /**
 +     * Returns the index name
 +     *
 +     * @param cfs the <code>ColumnFamilyStore</code>
 +     * @return the index name
 +     */
 +    public static String getIndexName(ColumnFamilyStore cfs)
 +    {
 +        return getIndexName(cfs.name);
 +    }
 +
 +    /**
 +     * Returns the index name
 +     *
 +     * @param cfName the <code>ColumnFamilyStore</code> name
 +     * @return the index name
 +     */
 +    public static String getIndexName(String cfName)
 +    {
 +        assert isIndexColumnFamily(cfName);
 +        return StringUtils.substringAfter(cfName, Directories.SECONDARY_INDEX_NAME_SEPARATOR);
 +    }
 +
 +    private void buildIndexesBlocking(Collection<SSTableReader> sstables, Set<Index> indexes)
 +    {
 +        if (indexes.isEmpty())
 +            return;
 +
 +        logger.info("Submitting index build of {} for data in {}",
 +                    indexes.stream().map(i -> i.getIndexMetadata().name).collect(Collectors.joining(",")),
 +                    sstables.stream().map(SSTableReader::toString).collect(Collectors.joining(",")));
 +
 +        SecondaryIndexBuilder builder = new SecondaryIndexBuilder(baseCfs,
 +                                                                  indexes,
 +                                                                  new ReducingKeyIterator(sstables));
 +        Future<?> future = CompactionManager.instance.submitIndexBuild(builder);
 +        FBUtilities.waitOnFuture(future);
 +
 +        flushIndexesBlocking(indexes);
 +        logger.info("Index build of {} complete",
 +                    indexes.stream().map(i -> i.getIndexMetadata().name).collect(Collectors.joining(",")));
 +    }
 +
 +    /**
 +     * Marks the specified index as build.
 +     * <p>This method is public as it need to be accessible from the {@link Index} implementations</p>
 +     * @param indexName the index name
 +     */
 +    public void markIndexBuilt(String indexName)
 +    {
 +        builtIndexes.add(indexName);
-         SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), indexName);
++        if (DatabaseDescriptor.isDaemonInitialized())
++            SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), indexName);
 +    }
 +
 +    /**
 +     * Marks the specified index as removed.
 +     * <p>This method is public as it need to be accessible from the {@link Index} implementations</p>
 +     * @param indexName the index name
 +     */
 +    public void markIndexRemoved(String indexName)
 +    {
 +        SystemKeyspace.setIndexRemoved(baseCfs.keyspace.getName(), indexName);
 +    }
 +
 +    public Index getIndexByName(String indexName)
 +    {
 +        return indexes.get(indexName);
 +    }
 +
 +    private Index createInstance(IndexMetadata indexDef)
 +    {
 +        Index newIndex;
 +        if (indexDef.isCustom())
 +        {
 +            assert indexDef.options != null;
 +            String className = indexDef.options.get(IndexTarget.CUSTOM_INDEX_OPTION_NAME);
 +            assert ! Strings.isNullOrEmpty(className);
 +            try
 +            {
 +                Class<? extends Index> indexClass = FBUtilities.classForName(className, "Index");
 +                Constructor<? extends Index> ctor = indexClass.getConstructor(ColumnFamilyStore.class, IndexMetadata.class);
 +                newIndex = (Index)ctor.newInstance(baseCfs, indexDef);
 +            }
 +            catch (Exception e)
 +            {
 +                throw new RuntimeException(e);
 +            }
 +        }
 +        else
 +        {
 +            newIndex = CassandraIndex.newIndex(baseCfs, indexDef);
 +        }
 +        return newIndex;
 +    }
 +
 +    /**
 +     * Truncate all indexes
 +     */
 +    public void truncateAllIndexesBlocking(final long truncatedAt)
 +    {
 +        executeAllBlocking(indexes.values().stream(), (index) -> index.getTruncateTask(truncatedAt));
 +    }
 +
 +    /**
 +     * Remove all indexes
 +     */
 +    public void invalidateAllIndexesBlocking()
 +    {
 +        markAllIndexesRemoved();
 +        executeAllBlocking(indexes.values().stream(), Index::getInvalidateTask);
 +    }
 +
 +    /**
 +     * Perform a blocking flush all indexes
 +     */
 +    public void flushAllIndexesBlocking()
 +    {
 +       flushIndexesBlocking(ImmutableSet.copyOf(indexes.values()));
 +    }
 +
 +    /**
 +     * Perform a blocking flush of selected indexes
 +     */
 +    public void flushIndexesBlocking(Set<Index> indexes)
 +    {
 +        if (indexes.isEmpty())
 +            return;
 +
 +        List<Future<?>> wait = new ArrayList<>();
 +        List<Index> nonCfsIndexes = new ArrayList<>();
 +
 +        // for each CFS backed index, submit a flush task which we'll wait on for completion
 +        // for the non-CFS backed indexes, we'll flush those while we wait.
 +        synchronized (baseCfs.getTracker())
 +        {
 +            indexes.forEach(index ->
 +                index.getBackingTable()
 +                     .map(cfs -> wait.add(cfs.forceFlush()))
 +                     .orElseGet(() -> nonCfsIndexes.add(index)));
 +        }
 +
 +        executeAllBlocking(nonCfsIndexes.stream(), Index::getBlockingFlushTask);
 +        FBUtilities.waitOnFutures(wait);
 +    }
 +
 +    /**
 +     * Performs a blocking flush of all custom indexes
 +     */
 +    public void flushAllNonCFSBackedIndexesBlocking()
 +    {
 +        executeAllBlocking(indexes.values()
 +                                  .stream()
 +                                  .filter(index -> !index.getBackingTable().isPresent()),
 +                           Index::getBlockingFlushTask);
 +    }
 +
 +    /**
 +     * @return all indexes which are marked as built and ready to use
 +     */
 +    public List<String> getBuiltIndexNames()
 +    {
 +        Set<String> allIndexNames = new HashSet<>();
 +        indexes.values().stream()
 +                .map(i -> i.getIndexMetadata().name)
 +                .forEach(allIndexNames::add);
 +        return SystemKeyspace.getBuiltIndexes(baseCfs.keyspace.getName(), allIndexNames);
 +    }
 +
 +    /**
 +     * @return all backing Tables used by registered indexes
 +     */
 +    public Set<ColumnFamilyStore> getAllIndexColumnFamilyStores()
 +    {
 +        Set<ColumnFamilyStore> backingTables = new HashSet<>();
 +        indexes.values().forEach(index -> index.getBackingTable().ifPresent(backingTables::add));
 +        return backingTables;
 +    }
 +
 +    /**
 +     * @return if there are ANY indexes registered for this table
 +     */
 +    public boolean hasIndexes()
 +    {
 +        return !indexes.isEmpty();
 +    }
 +
 +    /**
 +     * When building an index against existing data in sstables, add the given partition to the index
 +     */
 +    public void indexPartition(DecoratedKey key, Set<Index> indexes, int pageSize)
 +    {
 +        if (logger.isTraceEnabled())
 +            logger.trace("Indexing partition {}", baseCfs.metadata.getKeyValidator().getString(key.getKey()));
 +
 +        if (!indexes.isEmpty())
 +        {
 +            SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(baseCfs.metadata,
 +                                                                                          FBUtilities.nowInSeconds(),
 +                                                                                          key);
 +            int nowInSec = cmd.nowInSec();
 +            boolean readStatic = false;
 +
 +            SinglePartitionPager pager = new SinglePartitionPager(cmd, null, Server.CURRENT_VERSION);
 +            while (!pager.isExhausted())
 +            {
 +                try (ReadOrderGroup readGroup = cmd.startOrderGroup();
 +                     OpOrder.Group writeGroup = Keyspace.writeOrder.start();
 +                     RowIterator partition =
 +                        PartitionIterators.getOnlyElement(pager.fetchPageInternal(pageSize,readGroup),
 +                                                          cmd))
 +                {
 +                    Set<Index.Indexer> indexers = indexes.stream()
 +                                                         .map(index -> index.indexerFor(key,
 +                                                                                        partition.columns(),
 +                                                                                        nowInSec,
 +                                                                                        writeGroup,
 +                                                                                        IndexTransaction.Type.UPDATE))
 +                                                         .filter(Objects::nonNull)
 +                                                         .collect(Collectors.toSet());
 +
 +                    indexers.forEach(Index.Indexer::begin);
 +
 +                    // only process the static row once per partition
 +                    if (!readStatic && !partition.staticRow().isEmpty())
 +                    {
 +                        indexers.forEach(indexer -> indexer.insertRow(partition.staticRow()));
 +                        readStatic = true;
 +                    }
 +
 +                    while (partition.hasNext())
 +                    {
 +                        Row row = partition.next();
 +                        indexers.forEach(indexer -> indexer.insertRow(row));
 +                    }
 +
 +                    indexers.forEach(Index.Indexer::finish);
 +                }
 +            }
 +        }
 +    }
 +
 +    /**
 +     * Return the page size used when indexing an entire partition
 +     */
 +    public int calculateIndexingPageSize()
 +    {
 +        if (Boolean.getBoolean("cassandra.force_default_indexing_page_size"))
 +            return DEFAULT_PAGE_SIZE;
 +
 +        double targetPageSizeInBytes = 32 * 1024 * 1024;
 +        double meanPartitionSize = baseCfs.getMeanPartitionSize();
 +        if (meanPartitionSize <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int meanCellsPerPartition = baseCfs.getMeanColumns();
 +        if (meanCellsPerPartition <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int columnsPerRow = baseCfs.metadata.partitionColumns().regulars.size();
 +        if (meanCellsPerPartition <= 0)
 +            return DEFAULT_PAGE_SIZE;
 +
 +        int meanRowsPerPartition = meanCellsPerPartition / columnsPerRow;
 +        double meanRowSize = meanPartitionSize / meanRowsPerPartition;
 +
 +        int pageSize = (int) Math.max(1, Math.min(DEFAULT_PAGE_SIZE, targetPageSizeInBytes / meanRowSize));
 +
 +        logger.trace("Calculated page size {} for indexing {}.{} ({}/{}/{}/{})",
 +                     pageSize,
 +                     baseCfs.metadata.ksName,
 +                     baseCfs.metadata.cfName,
 +                     meanPartitionSize,
 +                     meanCellsPerPartition,
 +                     meanRowsPerPartition,
 +                     meanRowSize);
 +
 +        return pageSize;
 +    }
 +
 +    /**
 +     * Delete all data from all indexes for this partition.
 +     * For when cleanup rips a partition out entirely.
 +     *
 +     * TODO : improve cleanup transaction to batch updates & perform them async
 +     */
 +    public void deletePartition(UnfilteredRowIterator partition, int nowInSec)
 +    {
 +        // we need to acquire memtable lock because secondary index deletion may
 +        // cause a race (see CASSANDRA-3712). This is done internally by the
 +        // index transaction when it commits
 +        CleanupTransaction indexTransaction = newCleanupTransaction(partition.partitionKey(),
 +                                                                    partition.columns(),
 +                                                                    nowInSec);
 +        indexTransaction.start();
 +        indexTransaction.onPartitionDeletion(new DeletionTime(FBUtilities.timestampMicros(), nowInSec));
 +        indexTransaction.commit();
 +
 +        while (partition.hasNext())
 +        {
 +            Unfiltered unfiltered = partition.next();
 +            if (unfiltered.kind() != Unfiltered.Kind.ROW)
 +                continue;
 +
 +            indexTransaction = newCleanupTransaction(partition.partitionKey(),
 +                                                     partition.columns(),
 +                                                     nowInSec);
 +            indexTransaction.start();
 +            indexTransaction.onRowDelete((Row)unfiltered);
 +            indexTransaction.commit();
 +        }
 +    }
 +
 +    /**
 +     * Called at query time to choose which (if any) of the registered index implementations to use for a given query.
 +     *
 +     * This is a two step processes, firstly compiling the set of searchable indexes then choosing the one which reduces
 +     * the search space the most.
 +     *
 +     * In the first phase, if the command's RowFilter contains any custom index expressions, the indexes that they
 +     * specify are automatically included. Following that, the registered indexes are filtered to include only those
 +     * which support the standard expressions in the RowFilter.
 +     *
 +     * The filtered set then sorted by selectivity, as reported by the Index implementations' getEstimatedResultRows
 +     * method.
 +     *
 +     * Implementation specific validation of the target expression, either custom or standard, by the selected
 +     * index should be performed in the searcherFor method to ensure that we pick the right index regardless of
 +     * the validity of the expression.
 +     *
 +     * This method is only called once during the lifecycle of a ReadCommand and the result is
 +     * cached for future use when obtaining a Searcher, getting the index's underlying CFS for
 +     * ReadOrderGroup, or an estimate of the result size from an average index query.
 +     *
 +     * @param command ReadCommand to be executed
 +     * @return an Index instance, ready to use during execution of the command, or null if none
 +     * of the registered indexes can support the command.
 +     */
 +    public Index getBestIndexFor(ReadCommand command)
 +    {
 +        if (indexes.isEmpty() || command.rowFilter().isEmpty())
 +            return null;
 +
 +        Set<Index> searchableIndexes = new HashSet<>();
 +        for (RowFilter.Expression expression : command.rowFilter())
 +        {
 +            if (expression.isCustom())
 +            {
 +                // Only a single custom expression is allowed per query and, if present,
 +                // we want to always favour the index specified in such an expression
 +                RowFilter.CustomExpression customExpression = (RowFilter.CustomExpression)expression;
 +                logger.trace("Command contains a custom index expression, using target index {}", customExpression.getTargetIndex().name);
 +                Tracing.trace("Command contains a custom index expression, using target index {}", customExpression.getTargetIndex().name);
 +                return indexes.get(customExpression.getTargetIndex().name);
 +            }
 +            else
 +            {
 +                indexes.values().stream()
 +                       .filter(index -> index.supportsExpression(expression.column(), expression.operator()))
 +                       .forEach(searchableIndexes::add);
 +            }
 +        }
 +
 +        if (searchableIndexes.isEmpty())
 +        {
 +            logger.trace("No applicable indexes found");
 +            Tracing.trace("No applicable indexes found");
 +            return null;
 +        }
 +
 +        Index selected = searchableIndexes.size() == 1
 +                         ? Iterables.getOnlyElement(searchableIndexes)
 +                         : searchableIndexes.stream()
 +                                            .min((a, b) -> Longs.compare(a.getEstimatedResultRows(),
 +                                                                         b.getEstimatedResultRows()))
 +                                            .orElseThrow(() -> new AssertionError("Could not select most selective index"));
 +
 +        // pay for an additional threadlocal get() rather than build the strings unnecessarily
 +        if (Tracing.isTracing())
 +        {
 +            Tracing.trace("Index mean cardinalities are {}. Scanning with {}.",
 +                          searchableIndexes.stream().map(i -> i.getIndexMetadata().name + ':' + i.getEstimatedResultRows())
 +                                           .collect(Collectors.joining(",")),
 +                          selected.getIndexMetadata().name);
 +        }
 +        return selected;
 +    }
 +
 +    /**
 +     * Called at write time to ensure that values present in the update
 +     * are valid according to the rules of all registered indexes which
 +     * will process it. The partition key as well as the clustering and
 +     * cell values for each row in the update may be checked by index
 +     * implementations
 +     * @param update PartitionUpdate containing the values to be validated by registered Index implementations
 +     * @throws InvalidRequestException
 +     */
 +    public void validate(PartitionUpdate update) throws InvalidRequestException
 +    {
 +        for (Index index : indexes.values())
 +            index.validate(update);
 +    }
 +
 +    /**
 +     * IndexRegistry methods
 +     */
 +    public void registerIndex(Index index)
 +    {
 +        String name = index.getIndexMetadata().name;
 +        indexes.put(name, index);
 +        logger.trace("Registered index {}", name);
 +    }
 +
 +    public void unregisterIndex(Index index)
 +    {
 +        unregisterIndex(index.getIndexMetadata().name);
 +    }
 +
 +    private Index unregisterIndex(String name)
 +    {
 +        Index removed = indexes.remove(name);
 +        builtIndexes.remove(name);
 +        logger.trace(removed == null ? "Index {} was not registered" : "Removed index {} from registry",
 +                     name);
 +        return removed;
 +    }
 +
 +    public Index getIndex(IndexMetadata metadata)
 +    {
 +        return indexes.get(metadata.name);
 +    }
 +
 +    public Collection<Index> listIndexes()
 +    {
 +        return ImmutableSet.copyOf(indexes.values());
 +    }
 +
 +    /**
 +     * Handling of index updates.
 +     * Implementations of the various IndexTransaction interfaces, for keeping indexes in sync with base data
 +     * during updates, compaction and cleanup. Plus factory methods for obtaining transaction instances.
 +     */
 +
 +    /**
 +     * Transaction for updates on the write path.
 +     */
 +    public UpdateTransaction newUpdateTransaction(PartitionUpdate update, OpOrder.Group opGroup, int nowInSec)
 +    {
 +        if (!hasIndexes())
 +            return UpdateTransaction.NO_OP;
 +
 +        Index.Indexer[] indexers = indexes.values().stream()
 +                                          .map(i -> i.indexerFor(update.partitionKey(),
 +                                                                 update.columns(),
 +                                                                 nowInSec,
 +                                                                 opGroup,
 +                                                                 IndexTransaction.Type.UPDATE))
 +                                          .filter(Objects::nonNull)
 +                                          .toArray(Index.Indexer[]::new);
 +
 +        return indexers.length == 0 ? UpdateTransaction.NO_OP : new WriteTimeTransaction(indexers);
 +    }
 +
 +    /**
 +     * Transaction for use when merging rows during compaction
 +     */
 +    public CompactionTransaction newCompactionTransaction(DecoratedKey key,
 +                                                          PartitionColumns partitionColumns,
 +                                                          int versions,
 +                                                          int nowInSec)
 +    {
 +        // the check for whether there are any registered indexes is already done in CompactionIterator
 +        return new IndexGCTransaction(key, partitionColumns, versions, nowInSec, listIndexes());
 +    }
 +
 +    /**
 +     * Transaction for use when removing partitions during cleanup
 +     */
 +    public CleanupTransaction newCleanupTransaction(DecoratedKey key,
 +                                                    PartitionColumns partitionColumns,
 +                                                    int nowInSec)
 +    {
 +        if (!hasIndexes())
 +            return CleanupTransaction.NO_OP;
 +
 +        return new CleanupGCTransaction(key, partitionColumns, nowInSec, listIndexes());
 +    }
 +
 +    /**
 +     * A single use transaction for processing a partition update on the regular write path
 +     */
 +    private static final class WriteTimeTransaction implements UpdateTransaction
 +    {
 +        private final Index.Indexer[] indexers;
 +
 +        private WriteTimeTransaction(Index.Indexer...indexers)
 +        {
 +            // don't allow null indexers, if we don't need any use a NullUpdater object
 +            for (Index.Indexer indexer : indexers) assert indexer != null;
 +            this.indexers = indexers;
 +        }
 +
 +        public void start()
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.begin();
 +        }
 +
 +        public void onPartitionDeletion(DeletionTime deletionTime)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.partitionDelete(deletionTime);
 +        }
 +
 +        public void onRangeTombstone(RangeTombstone tombstone)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.rangeTombstone(tombstone);
 +        }
 +
 +        public void onInserted(Row row)
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.insertRow(row);
 +        }
 +
 +        public void onUpdated(Row existing, Row updated)
 +        {
 +            final Row.Builder toRemove = BTreeRow.sortedBuilder();
 +            toRemove.newRow(existing.clustering());
 +            toRemove.addPrimaryKeyLivenessInfo(existing.primaryKeyLivenessInfo());
 +            toRemove.addRowDeletion(existing.deletion());
 +            final Row.Builder toInsert = BTreeRow.sortedBuilder();
 +            toInsert.newRow(updated.clustering());
 +            toInsert.addPrimaryKeyLivenessInfo(updated.primaryKeyLivenessInfo());
 +            toInsert.addRowDeletion(updated.deletion());
 +            // diff listener collates the columns to be added & removed from the indexes
 +            RowDiffListener diffListener = new RowDiffListener()
 +            {
 +                public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
 +                {
 +                }
 +
 +                public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original)
 +                {
 +                }
 +
 +                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
 +                {
 +                }
 +
 +                public void onCell(int i, Clustering clustering, Cell merged, Cell original)
 +                {
 +                    if (merged != null && !merged.equals(original))
 +                        toInsert.addCell(merged);
 +
 +                    if (merged == null || (original != null && shouldCleanupOldValue(original, merged)))
 +                        toRemove.addCell(original);
 +
 +                }
 +            };
 +            Rows.diff(diffListener, updated, existing);
 +            Row oldRow = toRemove.build();
 +            Row newRow = toInsert.build();
 +            for (Index.Indexer indexer : indexers)
 +                indexer.updateRow(oldRow, newRow);
 +        }
 +
 +        public void commit()
 +        {
 +            for (Index.Indexer indexer : indexers)
 +                indexer.finish();
 +        }
 +
 +        private boolean shouldCleanupOldValue(Cell oldCell, Cell newCell)
 +        {
 +            // If either the value or timestamp is different, then we
 +            // should delete from the index. If not, then we can infer that
 +            // at least one of the cells is an ExpiringColumn and that the
 +            // difference is in the expiry time. In this case, we don't want to
 +            // delete the old value from the index as the tombstone we insert
 +            // will just hide the inserted value.
 +            // Completely identical cells (including expiring columns with
 +            // identical ttl & localExpirationTime) will not get this far due
 +            // to the oldCell.equals(newCell) in StandardUpdater.update
 +            return !oldCell.value().equals(newCell.value()) || oldCell.timestamp() != newCell.timestamp();
 +        }
 +    }
 +
 +    /**
 +     * A single-use transaction for updating indexes for a single partition during compaction where the only
 +     * operation is to merge rows
 +     * TODO : make this smarter at batching updates so we can use a single transaction to process multiple rows in
 +     * a single partition
 +     */
 +    private static final class IndexGCTransaction implements CompactionTransaction
 +    {
 +        private final DecoratedKey key;
 +        private final PartitionColumns columns;
 +        private final int versions;
 +        private final int nowInSec;
 +        private final Collection<Index> indexes;
 +
 +        private Row[] rows;
 +
 +        private IndexGCTransaction(DecoratedKey key,
 +                                   PartitionColumns columns,
 +                                   int versions,
 +                                   int nowInSec,
 +                                   Collection<Index> indexes)
 +        {
 +            this.key = key;
 +            this.columns = columns;
 +            this.versions = versions;
 +            this.indexes = indexes;
 +            this.nowInSec = nowInSec;
 +        }
 +
 +        public void start()
 +        {
 +            if (versions > 0)
 +                rows = new Row[versions];
 +        }
 +
 +        public void onRowMerge(Row merged, Row...versions)
 +        {
 +            // Diff listener constructs rows representing deltas between the merged and original versions
 +            // These delta rows are then passed to registered indexes for removal processing
 +            final Row.Builder[] builders = new Row.Builder[versions.length];
 +            RowDiffListener diffListener = new RowDiffListener()
 +            {
 +                public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
 +                {
 +                    if (original != null && (merged == null || !merged.isLive(nowInSec)))
 +                        getBuilder(i, clustering).addPrimaryKeyLivenessInfo(original);
 +                }
 +
 +                public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original)
 +                {
 +                }
 +
 +                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
 +                {
 +                }
 +
 +                public void onCell(int i, Clustering clustering, Cell merged, Cell original)
 +                {
 +                    if (original != null && (merged == null || !merged.isLive(nowInSec)))
 +                        getBuilder(i, clustering).addCell(original);
 +                }
 +
 +                private Row.Builder getBuilder(int index, Clustering clustering)
 +                {
 +                    if (builders[index] == null)
 +                    {
 +                        builders[index] = BTreeRow.sortedBuilder();
 +                        builders[index].newRow(clustering);
 +                    }
 +                    return builders[index];
 +                }
 +            };
 +
 +            Rows.diff(diffListener, merged, versions);
 +
 +            for(int i = 0; i < builders.length; i++)
 +                if (builders[i] != null)
 +                    rows[i] = builders[i].build();
 +        }
 +
 +        public void commit()
 +        {
 +            if (rows == null)
 +                return;
 +
 +            try (OpOrder.Group opGroup = Keyspace.writeOrder.start())
 +            {
 +                for (Index index : indexes)
 +                {
 +                    Index.Indexer indexer = index.indexerFor(key, columns, nowInSec, opGroup, Type.COMPACTION);
 +                    if (indexer == null)
 +                        continue;
 +
 +                    indexer.begin();
 +                    for (Row row : rows)
 +                        if (row != null)
 +                            indexer.removeRow(row);
 +                    indexer.finish();
 +                }
 +            }
 +        }
 +    }
 +
 +    /**
 +     * A single-use transaction for updating indexes for a single partition during cleanup, where
 +     * partitions and rows are only removed
 +     * TODO : make this smarter at batching updates so we can use a single transaction to process multiple rows in
 +     * a single partition
 +     */
 +    private static final class CleanupGCTransaction implements CleanupTransaction
 +    {
 +        private final DecoratedKey key;
 +        private final PartitionColumns columns;
 +        private final int nowInSec;
 +        private final Collection<Index> indexes;
 +
 +        private Row row;
 +        private DeletionTime partitionDelete;
 +
 +        private CleanupGCTransaction(DecoratedKey key,
 +                                     PartitionColumns columns,
 +                                     int nowInSec,
 +                                     Collection<Index> indexes)
 +        {
 +            this.key = key;
 +            this.columns = columns;
 +            this.indexes = indexes;
 +            this.nowInSec = nowInSec;
 +        }
 +
 +        public void start()
 +        {
 +        }
 +
 +        public void onPartitionDeletion(DeletionTime deletionTime)
 +        {
 +            partitionDelete = deletionTime;
 +        }
 +
 +        public void onRowDelete(Row row)
 +        {
 +            this.row = row;
 +        }
 +
 +        public void commit()
 +        {
 +            if (row == null && partitionDelete == null)
 +                return;
 +
 +            try (OpOrder.Group opGroup = Keyspace.writeOrder.start())
 +            {
 +                for (Index index : indexes)
 +                {
 +                    Index.Indexer indexer = index.indexerFor(key, columns, nowInSec, opGroup, Type.CLEANUP);
 +                    if (indexer == null)
 +                        continue;
 +
 +                    indexer.begin();
 +
 +                    if (partitionDelete != null)
 +                        indexer.partitionDelete(partitionDelete);
 +
 +                    if (row != null)
 +                        indexer.removeRow(row);
 +
 +                    indexer.finish();
 +                }
 +            }
 +        }
 +    }
 +
 +    private static void executeBlocking(Callable<?> task)
 +    {
 +        if (null != task)
 +            FBUtilities.waitOnFuture(blockingExecutor.submit(task));
 +    }
 +
 +    private static void executeAllBlocking(Stream<Index> indexers, Function<Index, Callable<?>> function)
 +    {
 +        List<Future<?>> waitFor = new ArrayList<>();
 +        indexers.forEach(indexer -> {
 +            Callable<?> task = function.apply(indexer);
 +            if (null != task)
 +                waitFor.add(blockingExecutor.submit(task));
 +        });
 +        FBUtilities.waitOnFutures(waitFor);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index f0cdcf5,ba060d4..3283723
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@@ -2183,10 -2246,10 +2183,10 @@@ public abstract class SSTableReader ext
  
              // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist
              // the read meter when in client mode.
-             if (Schema.isSystemKeyspace(desc.ksname))
 -            if (SystemKeyspace.NAME.equals(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
++            if (Schema.isSystemKeyspace(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
              {
                  readMeter = null;
 -                readMeterSyncFuture = null;
 +                readMeterSyncFuture = NULL;
                  return;
              }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/CQLTester.java
index 3d45393,98b8e23..3d8d03b
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@@ -140,86 -137,7 +140,87 @@@ public abstract class CQLTeste
      // We don't use USE_PREPARED_VALUES in the code below so some test can foce value preparation (if the result
      // is not expected to be the same without preparation)
      private boolean usePrepared = USE_PREPARED_VALUES;
 -    private static final boolean reusePrepared = Boolean.valueOf(System.getProperty("cassandra.test.reuse_prepared", "true"));
 +    private static boolean reusePrepared = REUSE_PREPARED;
 +
 +    public static void prepareServer()
 +    {
 +        if (isServerPrepared)
 +            return;
 +
 +        // Cleanup first
 +        try
 +        {
 +            cleanupAndLeaveDirs();
 +        }
 +        catch (IOException e)
 +        {
 +            logger.error("Failed to cleanup and recreate directories.");
 +            throw new RuntimeException(e);
 +        }
 +
 +        Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()
 +        {
 +            public void uncaughtException(Thread t, Throwable e)
 +            {
 +                logger.error("Fatal exception in thread " + t, e);
 +            }
 +        });
 +
 +        ThreadAwareSecurityManager.install();
 +
++        DatabaseDescriptor.setDaemonInitialized();
 +        Keyspace.setInitialized();
 +        isServerPrepared = true;
 +    }
 +
 +    public static void cleanupAndLeaveDirs() throws IOException
 +    {
 +        // We need to stop and unmap all CLS instances prior to cleanup() or we'll get failures on Windows.
 +        CommitLog.instance.stopUnsafe(true);
 +        mkdirs();
 +        cleanup();
 +        mkdirs();
 +        CommitLog.instance.restartUnsafe();
 +    }
 +
 +    public static void cleanup()
 +    {
 +        // clean up commitlog
 +        String[] directoryNames = { DatabaseDescriptor.getCommitLogLocation(), };
 +        for (String dirName : directoryNames)
 +        {
 +            File dir = new File(dirName);
 +            if (!dir.exists())
 +                throw new RuntimeException("No such directory: " + dir.getAbsolutePath());
 +            FileUtils.deleteRecursive(dir);
 +        }
 +
 +        cleanupSavedCaches();
 +
 +        // clean up data directory which are stored as data directory/keyspace/data files
 +        for (String dirName : DatabaseDescriptor.getAllDataFileLocations())
 +        {
 +            File dir = new File(dirName);
 +            if (!dir.exists())
 +                throw new RuntimeException("No such directory: " + dir.getAbsolutePath());
 +            FileUtils.deleteRecursive(dir);
 +        }
 +    }
 +
 +    public static void mkdirs()
 +    {
 +        DatabaseDescriptor.createAllDirectories();
 +    }
 +
 +    public static void cleanupSavedCaches()
 +    {
 +        File cachesDir = new File(DatabaseDescriptor.getSavedCachesLocation());
 +
 +        if (!cachesDir.exists() || !cachesDir.isDirectory())
 +            return;
 +
 +        FileUtils.delete(cachesDir.listFiles());
 +    }
  
      @BeforeClass
      public static void setUpClass()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 557beba,5e2fffe..e6d18c4
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@@ -32,9 -32,9 +32,10 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.Util;
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.Config;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.config.Schema;
  import org.apache.cassandra.cql3.QueryProcessor;
  import org.apache.cassandra.cql3.UntypedResultSet;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 5ac1b31,bc5be46..37b5fa9
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@@ -28,7 -27,7 +28,8 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.exceptions.ConfigurationException;
  import org.apache.cassandra.gms.ApplicationState;
  import org.apache.cassandra.gms.Gossiper;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
index 6f76db4,0000000..a2c9cf9
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/HintedHandOffMetricsTest.java
@@@ -1,56 -1,0 +1,63 @@@
 +/*
 + * 
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + * 
 + *   http://www.apache.org/licenses/LICENSE-2.0
 + * 
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + * 
 + */
 +package org.apache.cassandra.metrics;
 +
 +import java.net.InetAddress;
 +import java.util.Map;
 +import java.util.UUID;
 +
++import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import com.google.common.collect.Iterators;
 +
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.UntypedResultSet;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.db.marshal.Int32Type;
 +import org.apache.cassandra.db.marshal.UUIDType;
 +import org.apache.cassandra.hints.HintsService;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 +
 +public class HintedHandOffMetricsTest
 +{
++    @BeforeClass
++    public static void initDD()
++    {
++        DatabaseDescriptor.setDaemonInitialized();
++    }
++
 +    @Test
 +    public void testHintsMetrics() throws Exception
 +    {
 +        DatabaseDescriptor.getHintsDirectory().mkdirs();
 +
 +        for (int i = 0; i < 99; i++)
 +            HintsService.instance.metrics.incrPastWindow(InetAddress.getLocalHost());
 +        HintsService.instance.metrics.log();
 +
 +        UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS);
 +        Map<UUID, Integer> returned = rows.one().getMap("hints_dropped", UUIDType.instance, Int32Type.instance);
 +        assertEquals(Iterators.getLast(returned.values().iterator()).intValue(), 99);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
index f0c850d,4e160c2..bb2b9b0
--- a/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
+++ b/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
@@@ -18,14 -18,14 +18,21 @@@
  */
  package org.apache.cassandra.utils.concurrent;
  
++import org.junit.BeforeClass;
  import org.junit.Ignore;
  import org.junit.Test;
  
  import junit.framework.Assert;
++import org.apache.cassandra.config.DatabaseDescriptor;
  
  @Ignore
  public abstract class AbstractTransactionalTest
  {
++    @BeforeClass
++    public static void setupDD()
++    {
++        DatabaseDescriptor.setDaemonInitialized();
++    }
  
      protected abstract TestableTransaction newTest() throws Exception;
  


[13/19] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by yu...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0fe82be8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0fe82be8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0fe82be8

Branch: refs/heads/cassandra-3.X
Commit: 0fe82be83cceceb12172d63913388678253413bc
Parents: e9b7a0f 66f1aaf
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Dec 13 15:55:34 2016 -0800
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:55:34 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/config/DatabaseDescriptor.java    | 12 +++++++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++++--
 .../cassandra/db/PartitionRangeReadCommand.java |  3 +-
 .../cassandra/db/compaction/CompactionTask.java | 18 +++++++----
 .../cassandra/db/lifecycle/LogTransaction.java  |  3 +-
 .../apache/cassandra/db/lifecycle/Tracker.java  | 34 ++++++++++++--------
 .../cassandra/index/SecondaryIndexManager.java  |  4 ++-
 .../io/sstable/format/SSTableReader.java        |  2 +-
 .../cassandra/service/CassandraDaemon.java      |  1 +
 .../service/EmbeddedCassandraService.java       |  2 ++
 .../config/DatabaseDescriptorTest.java          |  6 ++++
 .../org/apache/cassandra/cql3/CQLTester.java    |  1 +
 .../apache/cassandra/db/SystemKeyspaceTest.java |  2 ++
 .../db/context/CounterContextTest.java          |  8 +++++
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 ++-
 .../cassandra/db/lifecycle/TrackerTest.java     |  7 ++--
 .../cassandra/dht/StreamStateStoreTest.java     |  7 ++++
 .../cassandra/gms/FailureDetectorTest.java      |  2 ++
 .../org/apache/cassandra/gms/GossiperTest.java  |  5 +++
 .../io/sstable/CQLSSTableWriterTest.java        |  2 ++
 .../cassandra/locator/CloudstackSnitchTest.java |  2 ++
 .../apache/cassandra/locator/EC2SnitchTest.java |  2 ++
 .../locator/GoogleCloudSnitchTest.java          |  2 ++
 .../metrics/HintedHandOffMetricsTest.java       |  7 ++++
 .../service/StorageServiceServerTest.java       |  1 +
 .../concurrent/AbstractTransactionalTest.java   |  7 ++++
 27 files changed, 124 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 5621c93,8cff097..145afb9
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,18 -1,5 +1,19 @@@
 -2.2.9
 +3.0.11
 + * Mark MVs as built after successful bootstrap (CASSANDRA-12984)
 + * Estimated TS drop-time histogram updated with Cell.NO_DELETION_TIME (CASSANDRA-13040)
 + * Nodetool compactionstats fails with NullPointerException (CASSANDRA-13021)
 + * Thread local pools never cleaned up (CASSANDRA-13033)
 + * Set RPC_READY to false when draining or if a node is marked as shutdown (CASSANDRA-12781)
 + * CQL often queries static columns unnecessarily (CASSANDRA-12768)
 + * Make sure sstables only get committed when it's safe to discard commit log records (CASSANDRA-12956)
 + * Reject default_time_to_live option when creating or altering MVs (CASSANDRA-12868)
 + * Nodetool should use a more sane max heap size (CASSANDRA-12739)
 + * LocalToken ensures token values are cloned on heap (CASSANDRA-12651)
 + * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
 + * Prevent reloading of logback.xml from UDF sandbox (CASSANDRA-12535)
 + * Reenable HeapPool (CASSANDRA-12900)
 +Merged from 2.2:
+  * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
   * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
   * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
   * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 71e1653,4bc46d0..39ed804
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -388,13 -388,17 +388,17 @@@ public class ColumnFamilyStore implemen
  
          logger.info("Initializing {}.{}", keyspace.getName(), name);
  
-         // scan for sstables corresponding to this cf and load them
-         data = new Tracker(this, loadSSTables);
+         // Create Memtable only on online
+         Memtable initialMemtable = null;
+         if (DatabaseDescriptor.isDaemonInitialized())
+             initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
+         data = new Tracker(initialMemtable, loadSSTables);
  
+         // scan for sstables corresponding to this cf and load them
          if (data.loadsstables)
          {
 -            Directories.SSTableLister sstableFiles = directories.sstableLister().skipTemporary(true);
 -            Collection<SSTableReader> sstables = SSTableReader.openAll(sstableFiles.list().entrySet(), metadata, this.partitioner);
 +            Directories.SSTableLister sstableFiles = directories.sstableLister(Directories.OnTxnErr.IGNORE).skipTemporary(true);
 +            Collection<SSTableReader> sstables = SSTableReader.openAll(sstableFiles.list().entrySet(), metadata);
              data.addInitialSSTables(sstables);
          }
  
@@@ -1953,10 -2758,12 +1957,10 @@@
              {
                  public Void call()
                  {
-                     cfs.data.reset();
+                     cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
 -                    cfs.getCompactionStrategy().shutdown();
 -                    cfs.getCompactionStrategy().startup();
                      return null;
                  }
 -            }, true);
 +            }, true, false);
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 99e24c8,0000000..17adef0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -1,322 -1,0 +1,323 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.db;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.List;
 +import java.util.Optional;
 +
 +import com.google.common.collect.Iterables;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.db.filter.*;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.lifecycle.View;
 +import org.apache.cassandra.db.partitions.*;
 +import org.apache.cassandra.db.rows.BaseRowIterator;
 +import org.apache.cassandra.db.transform.Transformation;
 +import org.apache.cassandra.dht.AbstractBounds;
 +import org.apache.cassandra.exceptions.RequestExecutionException;
 +import org.apache.cassandra.index.Index;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.metrics.TableMetrics;
 +import org.apache.cassandra.net.MessageOut;
 +import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.service.StorageProxy;
 +import org.apache.cassandra.service.pager.*;
 +import org.apache.cassandra.thrift.ThriftResultsMerger;
 +import org.apache.cassandra.tracing.Tracing;
 +import org.apache.cassandra.utils.FBUtilities;
 +
 +/**
 + * A read command that selects a (part of a) range of partitions.
 + */
 +public class PartitionRangeReadCommand extends ReadCommand
 +{
 +    protected static final SelectionDeserializer selectionDeserializer = new Deserializer();
 +
 +    private final DataRange dataRange;
 +    private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
 +
 +    public PartitionRangeReadCommand(boolean isDigest,
 +                                     int digestVersion,
 +                                     boolean isForThrift,
 +                                     CFMetaData metadata,
 +                                     int nowInSec,
 +                                     ColumnFilter columnFilter,
 +                                     RowFilter rowFilter,
 +                                     DataLimits limits,
 +                                     DataRange dataRange,
 +                                     Optional<IndexMetadata> index)
 +    {
 +        super(Kind.PARTITION_RANGE, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits);
 +        this.dataRange = dataRange;
 +        this.index = index;
 +    }
 +
 +    public PartitionRangeReadCommand(CFMetaData metadata,
 +                                     int nowInSec,
 +                                     ColumnFilter columnFilter,
 +                                     RowFilter rowFilter,
 +                                     DataLimits limits,
 +                                     DataRange dataRange,
 +                                     Optional<IndexMetadata> index)
 +    {
 +        this(false, 0, false, metadata, nowInSec, columnFilter, rowFilter, limits, dataRange, index);
 +    }
 +
 +    /**
 +     * Creates a new read command that query all the data in the table.
 +     *
 +     * @param metadata the table to query.
 +     * @param nowInSec the time in seconds to use are "now" for this query.
 +     *
 +     * @return a newly created read command that queries everything in the table.
 +     */
 +    public static PartitionRangeReadCommand allDataRead(CFMetaData metadata, int nowInSec)
 +    {
 +        return new PartitionRangeReadCommand(metadata,
 +                                             nowInSec,
 +                                             ColumnFilter.all(metadata),
 +                                             RowFilter.NONE,
 +                                             DataLimits.NONE,
 +                                             DataRange.allData(metadata.partitioner),
 +                                             Optional.empty());
 +    }
 +
 +    public DataRange dataRange()
 +    {
 +        return dataRange;
 +    }
 +
 +    public ClusteringIndexFilter clusteringIndexFilter(DecoratedKey key)
 +    {
 +        return dataRange.clusteringIndexFilter(key);
 +    }
 +
 +    public boolean isNamesQuery()
 +    {
 +        return dataRange.isNamesQuery();
 +    }
 +
 +    public PartitionRangeReadCommand forSubRange(AbstractBounds<PartitionPosition> range)
 +    {
 +        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange().forSubRange(range), index);
 +    }
 +
 +    public PartitionRangeReadCommand copy()
 +    {
 +        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange(), index);
 +    }
 +
 +    public PartitionRangeReadCommand withUpdatedLimit(DataLimits newLimits)
 +    {
 +        return new PartitionRangeReadCommand(metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, dataRange(), index);
 +    }
 +
 +    public long getTimeout()
 +    {
 +        return DatabaseDescriptor.getRangeRpcTimeout();
 +    }
 +
 +    public boolean selectsKey(DecoratedKey key)
 +    {
 +        if (!dataRange().contains(key))
 +            return false;
 +
 +        return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().getKeyValidator());
 +    }
 +
 +    public boolean selectsClustering(DecoratedKey key, Clustering clustering)
 +    {
 +        if (clustering == Clustering.STATIC_CLUSTERING)
 +            return !columnFilter().fetchedColumns().statics.isEmpty();
 +
 +        if (!dataRange().clusteringIndexFilter(key).selects(clustering))
 +            return false;
 +        return rowFilter().clusteringKeyRestrictionsAreSatisfiedBy(clustering);
 +    }
 +
 +    public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState) throws RequestExecutionException
 +    {
 +        return StorageProxy.getRangeSlice(this, consistency);
 +    }
 +
 +    public QueryPager getPager(PagingState pagingState, int protocolVersion)
 +    {
 +            return new PartitionRangeQueryPager(this, pagingState, protocolVersion);
 +    }
 +
 +    protected void recordLatency(TableMetrics metric, long latencyNanos)
 +    {
 +        metric.rangeLatency.addNano(latencyNanos);
 +    }
 +
 +    protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
 +    {
 +        ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
 +        Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));
 +
 +        // fetch data from current memtable, historical memtables, and SSTables in the correct order.
 +        final List<UnfilteredPartitionIterator> iterators = new ArrayList<>(Iterables.size(view.memtables) + view.sstables.size());
 +
 +        try
 +        {
 +            for (Memtable memtable : view.memtables)
 +            {
 +                @SuppressWarnings("resource") // We close on exception and on closing the result returned by this method
 +                Memtable.MemtableUnfilteredPartitionIterator iter = memtable.makePartitionIterator(columnFilter(), dataRange(), isForThrift());
 +                oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, iter.getMinLocalDeletionTime());
 +                iterators.add(isForThrift() ? ThriftResultsMerger.maybeWrap(iter, metadata(), nowInSec()) : iter);
 +            }
 +
 +            for (SSTableReader sstable : view.sstables)
 +            {
 +                @SuppressWarnings("resource") // We close on exception and on closing the result returned by this method
 +                UnfilteredPartitionIterator iter = sstable.getScanner(columnFilter(), dataRange(), isForThrift());
 +                iterators.add(isForThrift() ? ThriftResultsMerger.maybeWrap(iter, metadata(), nowInSec()) : iter);
 +                if (!sstable.isRepaired())
 +                    oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime());
 +            }
-             return checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
++            return iterators.isEmpty() ? EmptyIterators.unfilteredPartition(metadata(), isForThrift())
++                                       : checkCacheFilter(UnfilteredPartitionIterators.mergeLazily(iterators, nowInSec()), cfs);
 +        }
 +        catch (RuntimeException | Error e)
 +        {
 +            try
 +            {
 +                FBUtilities.closeAll(iterators);
 +            }
 +            catch (Exception suppressed)
 +            {
 +                e.addSuppressed(suppressed);
 +            }
 +
 +            throw e;
 +        }
 +    }
 +
 +    @Override
 +    protected int oldestUnrepairedTombstone()
 +    {
 +        return oldestUnrepairedTombstone;
 +    }
 +
 +    private UnfilteredPartitionIterator checkCacheFilter(UnfilteredPartitionIterator iter, final ColumnFamilyStore cfs)
 +    {
 +        class CacheFilter extends Transformation
 +        {
 +            @Override
 +            public BaseRowIterator applyToPartition(BaseRowIterator iter)
 +            {
 +                // Note that we rely on the fact that until we actually advance 'iter', no really costly operation is actually done
 +                // (except for reading the partition key from the index file) due to the call to mergeLazily in queryStorage.
 +                DecoratedKey dk = iter.partitionKey();
 +
 +                // Check if this partition is in the rowCache and if it is, if  it covers our filter
 +                CachedPartition cached = cfs.getRawCachedPartition(dk);
 +                ClusteringIndexFilter filter = dataRange().clusteringIndexFilter(dk);
 +
 +                if (cached != null && cfs.isFilterFullyCoveredBy(filter, limits(), cached, nowInSec()))
 +                {
 +                    // We won't use 'iter' so close it now.
 +                    iter.close();
 +
 +                    return filter.getUnfilteredRowIterator(columnFilter(), cached);
 +                }
 +
 +                return iter;
 +            }
 +        }
 +        return Transformation.apply(iter, new CacheFilter());
 +    }
 +
 +    public MessageOut<ReadCommand> createMessage(int version)
 +    {
 +        return dataRange().isPaging()
 +             ? new MessageOut<>(MessagingService.Verb.PAGED_RANGE, this, pagedRangeSerializer)
 +             : new MessageOut<>(MessagingService.Verb.RANGE_SLICE, this, rangeSliceSerializer);
 +    }
 +
 +    protected void appendCQLWhereClause(StringBuilder sb)
 +    {
 +        if (dataRange.isUnrestricted() && rowFilter().isEmpty())
 +            return;
 +
 +        sb.append(" WHERE ");
 +        // We put the row filter first because the data range can end by "ORDER BY"
 +        if (!rowFilter().isEmpty())
 +        {
 +            sb.append(rowFilter());
 +            if (!dataRange.isUnrestricted())
 +                sb.append(" AND ");
 +        }
 +        if (!dataRange.isUnrestricted())
 +            sb.append(dataRange.toCQLString(metadata()));
 +    }
 +
 +    /**
 +     * Allow to post-process the result of the query after it has been reconciled on the coordinator
 +     * but before it is passed to the CQL layer to return the ResultSet.
 +     *
 +     * See CASSANDRA-8717 for why this exists.
 +     */
 +    public PartitionIterator postReconciliationProcessing(PartitionIterator result)
 +    {
 +        ColumnFamilyStore cfs = Keyspace.open(metadata().ksName).getColumnFamilyStore(metadata().cfName);
 +        Index index = getIndex(cfs);
 +        return index == null ? result : index.postProcessorFor(this).apply(result, this);
 +    }
 +
 +    @Override
 +    public String toString()
 +    {
 +        return String.format("Read(%s.%s columns=%s rowfilter=%s limits=%s %s)",
 +                             metadata().ksName,
 +                             metadata().cfName,
 +                             columnFilter(),
 +                             rowFilter(),
 +                             limits(),
 +                             dataRange().toString(metadata()));
 +    }
 +
 +    protected void serializeSelection(DataOutputPlus out, int version) throws IOException
 +    {
 +        DataRange.serializer.serialize(dataRange(), out, version, metadata());
 +    }
 +
 +    protected long selectionSerializedSize(int version)
 +    {
 +        return DataRange.serializer.serializedSize(dataRange(), version, metadata());
 +    }
 +
 +    private static class Deserializer extends SelectionDeserializer
 +    {
 +        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
 +        throws IOException
 +        {
 +            DataRange range = DataRange.serializer.deserialize(in, version, metadata);
 +            return new PartitionRangeReadCommand(isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, range, index);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 0c4e144,20d3dc0..f0a1f47
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@@ -223,15 -218,20 +223,19 @@@ public class CompactionTask extends Abs
              for (SSTableReader reader : newSStables)
                  newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
  
-             double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
-             Summary mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
-             logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                       taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, mergeSummary.totalSourceRows, totalKeysWritten, mergeSummary.partitionMerge));
-             logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-             logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
- 
              if (offline)
+             {
                  Refs.release(Refs.selfRefs(newSStables));
+             }
+             else
+             {
+                 double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
 -                long totalSourceRows = 0;
 -                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
++                Summary mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, startsize, endsize);
+                 logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
 -                                           taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
++                                           taskId, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, mergeSummary.totalSourceRows, totalKeysWritten, mergeSummary.partitionMerge));
+                 logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
 -                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten));
++                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
+             }
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
index ca644eb,0000000..350477c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
@@@ -1,444 -1,0 +1,445 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.db.lifecycle;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.nio.file.Files;
 +import java.nio.file.NoSuchFileException;
 +import java.util.*;
 +import java.util.concurrent.ConcurrentLinkedQueue;
 +import java.util.concurrent.TimeUnit;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.util.concurrent.Runnables;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.concurrent.ScheduledExecutors;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.db.ColumnFamilyStore;
 +import org.apache.cassandra.db.Directories;
 +import org.apache.cassandra.db.SystemKeyspace;
 +import org.apache.cassandra.db.compaction.OperationType;
 +import org.apache.cassandra.db.lifecycle.LogRecord.Type;
 +import org.apache.cassandra.io.sstable.Component;
 +import org.apache.cassandra.io.sstable.Descriptor;
 +import org.apache.cassandra.io.sstable.SSTable;
 +import org.apache.cassandra.io.sstable.SnapshotDeletingTask;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.utils.*;
 +import org.apache.cassandra.utils.concurrent.Ref;
 +import org.apache.cassandra.utils.concurrent.RefCounted;
 +import org.apache.cassandra.utils.concurrent.Transactional;
 +
 +/**
 + * IMPORTANT: When this object is involved in a transactional graph, and is not encapsulated in a LifecycleTransaction,
 + * for correct behaviour its commit MUST occur before any others, since it may legitimately fail. This is consistent
 + * with the Transactional API, which permits one failing action to occur at the beginning of the commit phase, but also
 + * *requires* that the prepareToCommit() phase only take actions that can be rolled back.
 + *
 + * IMPORTANT: The transaction must complete (commit or abort) before any temporary files are deleted, even though the
 + * txn log file itself will not be deleted until all tracked files are deleted. This is required by FileLister to ensure
 + * a consistent disk state. LifecycleTransaction ensures this requirement, so this class should really never be used
 + * outside of LT. @see FileLister.classifyFiles(TransactionData txn)
 + *
 + * A class that tracks sstable files involved in a transaction across sstables:
 + * if the transaction succeeds the old files should be deleted and the new ones kept; vice-versa if it fails.
 + *
 + * The transaction log file contains new and old sstables as follows:
 + *
 + * add:[sstable-2][CRC]
 + * remove:[sstable-1,max_update_time,num files][CRC]
 + *
 + * where sstable-2 is a new sstable to be retained if the transaction succeeds and sstable-1 is an old sstable to be
 + * removed. CRC is an incremental CRC of the file content up to this point. For old sstable files we also log the
 + * last update time of all files for the sstable descriptor and a checksum of vital properties such as update times
 + * and file sizes.
 + *
 + * Upon commit we add a final line to the log file:
 + *
 + * commit:[commit_time][CRC]
 + *
 + * When the transaction log is cleaned-up by the TransactionTidier, which happens only after any old sstables have been
 + * osoleted, then any sstable files for old sstables are removed before deleting the transaction log if the transaction
 + * was committed, vice-versa if the transaction was aborted.
 + *
 + * On start-up we look for any transaction log files and repeat the cleanup process described above.
 + *
 + * See CASSANDRA-7066 for full details.
 + */
 +class LogTransaction extends Transactional.AbstractTransactional implements Transactional
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(LogTransaction.class);
 +
 +    /**
 +     * If the format of the lines in the transaction log is wrong or the checksum
 +     * does not match, then we throw this exception.
 +     */
 +    public static final class CorruptTransactionLogException extends RuntimeException
 +    {
 +        public final LogFile txnFile;
 +
 +        public CorruptTransactionLogException(String message, LogFile txnFile)
 +        {
 +            super(message);
 +            this.txnFile = txnFile;
 +        }
 +    }
 +
 +    private final Tracker tracker;
 +    private final LogFile txnFile;
 +    private final Ref<LogTransaction> selfRef;
 +    // Deleting sstables is tricky because the mmapping might not have been finalized yet,
 +    // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
 +    // Additionally, we need to make sure to delete the data file first, so on restart the others
 +    // will be recognized as GCable.
 +    private static final Queue<Runnable> failedDeletions = new ConcurrentLinkedQueue<>();
 +
 +    LogTransaction(OperationType opType)
 +    {
 +        this(opType, null);
 +    }
 +
 +    LogTransaction(OperationType opType, Tracker tracker)
 +    {
 +        this.tracker = tracker;
 +        this.txnFile = new LogFile(opType, UUIDGen.getTimeUUID());
 +        this.selfRef = new Ref<>(this, new TransactionTidier(txnFile));
 +
 +        if (logger.isTraceEnabled())
 +            logger.trace("Created transaction logs with id {}", txnFile.id());
 +    }
 +
 +    /**
 +     * Track a reader as new.
 +     **/
 +    void trackNew(SSTable table)
 +    {
 +        txnFile.add(Type.ADD, table);
 +    }
 +
 +    /**
 +     * Stop tracking a reader as new.
 +     */
 +    void untrackNew(SSTable table)
 +    {
 +        txnFile.remove(Type.ADD, table);
 +    }
 +
 +    /**
 +     * Schedule a reader for deletion as soon as it is fully unreferenced.
 +     */
 +    SSTableTidier obsoleted(SSTableReader reader)
 +    {
 +        if (txnFile.contains(Type.ADD, reader))
 +        {
 +            if (txnFile.contains(Type.REMOVE, reader))
 +                throw new IllegalArgumentException();
 +
 +            return new SSTableTidier(reader, true, this);
 +        }
 +
 +        txnFile.add(Type.REMOVE, reader);
 +
 +        if (tracker != null)
 +            tracker.notifyDeleting(reader);
 +
 +        return new SSTableTidier(reader, false, this);
 +    }
 +
 +    OperationType type()
 +    {
 +        return txnFile.type();
 +    }
 +
 +    UUID id()
 +    {
 +        return txnFile.id();
 +    }
 +
 +    @VisibleForTesting
 +    LogFile txnFile()
 +    {
 +        return txnFile;
 +    }
 +
 +    @VisibleForTesting
 +    List<File> logFiles()
 +    {
 +        return txnFile.getFiles();
 +    }
 +
 +    @VisibleForTesting
 +    List<String> logFilePaths()
 +    {
 +        return txnFile.getFilePaths();
 +    }
 +
 +    static void delete(File file)
 +    {
 +        try
 +        {
 +            if (logger.isTraceEnabled())
 +                logger.trace("Deleting {}", file);
 +
 +            Files.delete(file.toPath());
 +        }
 +        catch (NoSuchFileException e)
 +        {
 +            logger.error("Unable to delete {} as it does not exist", file);
 +        }
 +        catch (IOException e)
 +        {
 +            logger.error("Unable to delete {}", file, e);
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    /**
 +     * The transaction tidier.
 +     *
 +     * When the transaction reference is fully released we try to delete all the obsolete files
 +     * depending on the transaction result, as well as the transaction log file.
 +     */
 +    private static class TransactionTidier implements RefCounted.Tidy, Runnable
 +    {
 +        private final LogFile data;
 +
 +        TransactionTidier(LogFile data)
 +        {
 +            this.data = data;
 +        }
 +
 +        public void tidy() throws Exception
 +        {
 +            run();
 +        }
 +
 +        public String name()
 +        {
 +            return data.toString();
 +        }
 +
 +        public void run()
 +        {
 +            if (logger.isTraceEnabled())
 +                logger.trace("Removing files for transaction {}", name());
 +
 +            if (!data.completed())
 +            { // this happens if we forget to close a txn and the garbage collector closes it for us
 +                logger.error("{} was not completed, trying to abort it now", data);
 +                Throwable err = Throwables.perform((Throwable)null, data::abort);
 +                if (err != null)
 +                    logger.error("Failed to abort {}", data, err);
 +            }
 +
 +            Throwable err = data.removeUnfinishedLeftovers(null);
 +
 +            if (err != null)
 +            {
 +                logger.info("Failed deleting files for transaction {}, we'll retry after GC and on on server restart", name(), err);
 +                failedDeletions.add(this);
 +            }
 +            else
 +            {
 +                if (logger.isTraceEnabled())
 +                    logger.trace("Closing file transaction {}", name());
 +
 +                data.close();
 +            }
 +        }
 +    }
 +
 +    static class Obsoletion
 +    {
 +        final SSTableReader reader;
 +        final SSTableTidier tidier;
 +
 +        Obsoletion(SSTableReader reader, SSTableTidier tidier)
 +        {
 +            this.reader = reader;
 +            this.tidier = tidier;
 +        }
 +    }
 +
 +    /**
 +     * The SSTableReader tidier. When a reader is fully released and no longer referenced
 +     * by any one, we run this. It keeps a reference to the parent transaction and releases
 +     * it when done, so that the final transaction cleanup can run when all obsolete readers
 +     * are released.
 +     */
 +    public static class SSTableTidier implements Runnable
 +    {
 +        // must not retain a reference to the SSTableReader, else leak detection cannot kick in
 +        private final Descriptor desc;
 +        private final long sizeOnDisk;
 +        private final Tracker tracker;
 +        private final boolean wasNew;
 +        private final Ref<LogTransaction> parentRef;
 +
 +        public SSTableTidier(SSTableReader referent, boolean wasNew, LogTransaction parent)
 +        {
 +            this.desc = referent.descriptor;
 +            this.sizeOnDisk = referent.bytesOnDisk();
 +            this.tracker = parent.tracker;
 +            this.wasNew = wasNew;
 +            this.parentRef = parent.selfRef.tryRef();
 +        }
 +
 +        public void run()
 +        {
-             SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
++            if (tracker != null && !tracker.isDummy())
++                SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
 +
 +            try
 +            {
 +                // If we can't successfully delete the DATA component, set the task to be retried later: see TransactionTidier
 +                File datafile = new File(desc.filenameFor(Component.DATA));
 +
 +                delete(datafile);
 +                // let the remainder be cleaned up by delete
 +                SSTable.delete(desc, SSTable.discoverComponentsFor(desc));
 +            }
 +            catch (Throwable t)
 +            {
 +                logger.error("Failed deletion for {}, we'll retry after GC and on server restart", desc);
 +                failedDeletions.add(this);
 +                return;
 +            }
 +
 +            if (tracker != null && tracker.cfstore != null && !wasNew)
 +                tracker.cfstore.metric.totalDiskSpaceUsed.dec(sizeOnDisk);
 +
 +            // release the referent to the parent so that the all transaction files can be released
 +            parentRef.release();
 +        }
 +
 +        public void abort()
 +        {
 +            parentRef.release();
 +        }
 +    }
 +
 +
 +    static void rescheduleFailedDeletions()
 +    {
 +        Runnable task;
 +        while ( null != (task = failedDeletions.poll()))
 +            ScheduledExecutors.nonPeriodicTasks.submit(task);
 +
 +        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
 +        SnapshotDeletingTask.rescheduleFailedTasks();
 +    }
 +
 +    static void waitForDeletions()
 +    {
 +        FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(Runnables.doNothing(), 0, TimeUnit.MILLISECONDS));
 +    }
 +
 +    @VisibleForTesting
 +    Throwable complete(Throwable accumulate)
 +    {
 +        try
 +        {
 +            accumulate = selfRef.ensureReleased(accumulate);
 +            return accumulate;
 +        }
 +        catch (Throwable t)
 +        {
 +            logger.error("Failed to complete file transaction {}", id(), t);
 +            return Throwables.merge(accumulate, t);
 +        }
 +    }
 +
 +    protected Throwable doCommit(Throwable accumulate)
 +    {
 +        return complete(Throwables.perform(accumulate, txnFile::commit));
 +    }
 +
 +    protected Throwable doAbort(Throwable accumulate)
 +    {
 +        return complete(Throwables.perform(accumulate, txnFile::abort));
 +    }
 +
 +    protected void doPrepare() { }
 +
 +    /**
 +     * Called on startup to scan existing folders for any unfinished leftovers of
 +     * operations that were ongoing when the process exited. Also called by the standalone
 +     * sstableutil tool when the cleanup option is specified, @see StandaloneSSTableUtil.
 +     *
 +     */
 +    static void removeUnfinishedLeftovers(CFMetaData metadata)
 +    {
 +        removeUnfinishedLeftovers(new Directories(metadata, ColumnFamilyStore.getInitialDirectories()).getCFDirectories());
 +    }
 +
 +    @VisibleForTesting
 +    static void removeUnfinishedLeftovers(List<File> folders)
 +    {
 +        LogFilesByName logFiles = new LogFilesByName();
 +        folders.forEach(logFiles::list);
 +        logFiles.removeUnfinishedLeftovers();
 +    }
 +
 +    private static final class LogFilesByName
 +    {
 +        Map<String, List<File>> files = new HashMap<>();
 +
 +        void list(File folder)
 +        {
 +            Arrays.stream(folder.listFiles(LogFile::isLogFile)).forEach(this::add);
 +        }
 +
 +        void add(File file)
 +        {
 +            List<File> filesByName = files.get(file.getName());
 +            if (filesByName == null)
 +            {
 +                filesByName = new ArrayList<>();
 +                files.put(file.getName(), filesByName);
 +            }
 +
 +            filesByName.add(file);
 +        }
 +
 +        void removeUnfinishedLeftovers()
 +        {
 +            files.forEach(LogFilesByName::removeUnfinishedLeftovers);
 +        }
 +
 +        static void removeUnfinishedLeftovers(String name, List<File> logFiles)
 +        {
 +
 +            try(LogFile txn = LogFile.make(name, logFiles))
 +            {
 +                if (txn.verify())
 +                {
 +                    Throwable failure = txn.removeUnfinishedLeftovers(null);
 +                    if (failure != null)
 +                        logger.error("Failed to remove unfinished transaction leftovers for txn {}", txn, failure);
 +                }
 +                else
 +                {
 +                    logger.error("Unexpected disk state: failed to read transaction txn {}", txn);
 +                }
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0fe82be8/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index 5a3d524,e77ef78..9feaa3e
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@@ -193,15 -204,14 +201,13 @@@ public class Tracke
  
      /** (Re)initializes the tracker, purging all references. */
      @VisibleForTesting
-     public void reset()
+     public void reset(Memtable memtable)
      {
-         view.set(new View(
-                          !isDummy() ? ImmutableList.of(new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfstore))
-                                     : ImmutableList.<Memtable>of(),
-                          ImmutableList.<Memtable>of(),
-                          Collections.<SSTableReader, SSTableReader>emptyMap(),
-                          Collections.<SSTableReader, SSTableReader>emptyMap(),
-                          SSTableIntervalTree.empty()));
 -        view.set(new View(memtable != null ? singletonList(memtable) : Collections.<Memtable>emptyList(),
 -                          Collections.<Memtable>emptyList(),
 -                          Collections.<SSTableReader, SSTableReader>emptyMap(),
 -                          Collections.<SSTableReader>emptySet(),
 -                          Collections.<SSTableReader>emptySet(),
++        view.set(new View(memtable != null ? singletonList(memtable) : Collections.emptyList(),
++                          Collections.emptyList(),
++                          Collections.emptyMap(),
++                          Collections.emptyMap(),
+                           SSTableIntervalTree.empty()));
      }
  
      public Throwable dropSSTablesIfInvalid(Throwable accumulate)


[04/19] cassandra git commit: Temporarily fix bug that creates commit log when running offline tools

Posted by yu...@apache.org.
Temporarily fix bug that creates commit log when running offline tools

patch by yukim; reviewed by thobbs for CASSANDRA-8616


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/66f1aaf8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/66f1aaf8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/66f1aaf8

Branch: refs/heads/cassandra-3.11
Commit: 66f1aaf88d3cde5c52b13d71d3326da5eda16fb1
Parents: fb29400
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Feb 11 19:06:27 2016 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Dec 13 15:51:37 2016 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/config/DatabaseDescriptor.java    | 12 +++++++
 .../apache/cassandra/db/ColumnFamilyStore.java  | 10 ++++--
 .../cassandra/db/compaction/CompactionTask.java | 22 +++++++-----
 .../apache/cassandra/db/lifecycle/Tracker.java  | 36 ++++++++++++--------
 .../io/sstable/format/SSTableReader.java        |  8 +++--
 .../cassandra/service/CassandraDaemon.java      |  1 +
 .../service/EmbeddedCassandraService.java       |  2 ++
 .../io/sstable/CQLSSTableWriterLongTest.java    |  2 ++
 .../unit/org/apache/cassandra/SchemaLoader.java |  1 +
 .../config/DatabaseDescriptorTest.java          |  6 ++++
 .../apache/cassandra/db/CounterCellTest.java    |  3 ++
 .../org/apache/cassandra/db/NativeCellTest.java |  3 ++
 .../apache/cassandra/db/SystemKeyspaceTest.java |  2 ++
 .../db/context/CounterContextTest.java          |  8 +++++
 .../db/lifecycle/LifecycleTransactionTest.java  |  5 ++-
 .../cassandra/db/lifecycle/TrackerTest.java     |  7 ++--
 .../cassandra/dht/StreamStateStoreTest.java     |  7 ++++
 .../cassandra/gms/FailureDetectorTest.java      |  2 ++
 .../org/apache/cassandra/gms/GossiperTest.java  |  5 +++
 .../io/sstable/CQLSSTableWriterTest.java        |  2 ++
 .../cassandra/locator/CloudstackSnitchTest.java |  2 ++
 .../apache/cassandra/locator/EC2SnitchTest.java |  2 ++
 .../locator/GoogleCloudSnitchTest.java          |  2 ++
 .../service/StorageServiceServerTest.java       |  1 +
 25 files changed, 118 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f02350d..8cff097 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.9
+ * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
  * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
  * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
  * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index a3fb79b..35debd0 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -102,6 +102,18 @@ public class DatabaseDescriptor
     private static Comparator<InetAddress> localComparator;
     private static boolean hasLoggedConfig;
 
+    private static boolean daemonInitialized;
+
+    public static boolean isDaemonInitialized()
+    {
+        return daemonInitialized;
+    }
+
+    public static void setDaemonInitialized()
+    {
+        daemonInitialized = true;
+    }
+
     public static void forceStaticInitialization() {}
     static
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c6b69dc..4bc46d0 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -388,9 +388,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         logger.info("Initializing {}.{}", keyspace.getName(), name);
 
-        // scan for sstables corresponding to this cf and load them
-        data = new Tracker(this, loadSSTables);
+        // Create Memtable only on online
+        Memtable initialMemtable = null;
+        if (DatabaseDescriptor.isDaemonInitialized())
+            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
+        data = new Tracker(initialMemtable, loadSSTables);
 
+        // scan for sstables corresponding to this cf and load them
         if (data.loadsstables)
         {
             Directories.SSTableLister sstableFiles = directories.sstableLister().skipTemporary(true);
@@ -2754,7 +2758,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             {
                 public Void call()
                 {
-                    cfs.data.reset();
+                    cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
                     cfs.getCompactionStrategy().shutdown();
                     cfs.getCompactionStrategy().startup();
                     return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 575c326..20d3dc0 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -127,7 +127,7 @@ public class CompactionTask extends AbstractCompactionTask
             }
         });
 
-        UUID taskId = SystemKeyspace.startCompaction(cfs, transaction.originals());
+        UUID taskId = offline ? null : SystemKeyspace.startCompaction(cfs, transaction.originals());
 
         // new sstables from flush can be added during a compaction, but only the compaction can remove them,
         // so in our single-threaded compaction world this is a valid way of determining if we're compacting
@@ -218,16 +218,20 @@ public class CompactionTask extends AbstractCompactionTask
             for (SSTableReader reader : newSStables)
                 newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
 
-            double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
-            long totalSourceRows = 0;
-            String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
-            logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                      taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
-            logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-            logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
-
             if (offline)
+            {
                 Refs.release(Refs.selfRefs(newSStables));
+            }
+            else
+            {
+                double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
+                long totalSourceRows = 0;
+                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
+                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
+                                           taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
+                logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
+                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten));
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index 5d5701f..e77ef78 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -31,7 +31,6 @@ import com.google.common.collect.*;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Memtable;
-import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,6 +51,7 @@ import static com.google.common.base.Predicates.not;
 import static com.google.common.collect.ImmutableSet.copyOf;
 import static com.google.common.collect.Iterables.filter;
 import static java.util.Collections.singleton;
+import static java.util.Collections.singletonList;
 import static org.apache.cassandra.db.lifecycle.Helpers.*;
 import static org.apache.cassandra.db.lifecycle.View.permitCompacting;
 import static org.apache.cassandra.db.lifecycle.View.updateCompacting;
@@ -61,21 +61,29 @@ import static org.apache.cassandra.utils.Throwables.merge;
 import static org.apache.cassandra.utils.concurrent.Refs.release;
 import static org.apache.cassandra.utils.concurrent.Refs.selfRefs;
 
+/**
+ * Tracker tracks live {@link View} of data store for a table.
+ */
 public class Tracker
 {
     private static final Logger logger = LoggerFactory.getLogger(Tracker.class);
 
-    public final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+    private final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+
     public final ColumnFamilyStore cfstore;
     final AtomicReference<View> view;
     public final boolean loadsstables;
 
-    public Tracker(ColumnFamilyStore cfstore, boolean loadsstables)
+    /**
+     * @param memtable Initial Memtable. Can be null.
+     * @param loadsstables true to indicate to load SSTables (TODO: remove as this is only accessed from 2i)
+     */
+    public Tracker(Memtable memtable, boolean loadsstables)
     {
-        this.cfstore = cfstore;
+        this.cfstore = memtable != null ? memtable.cfs : null;
         this.view = new AtomicReference<>();
         this.loadsstables = loadsstables;
-        this.reset();
+        this.reset(memtable);
     }
 
     public LifecycleTransaction tryModify(SSTableReader sstable, OperationType operationType)
@@ -196,16 +204,14 @@ public class Tracker
 
     /** (Re)initializes the tracker, purging all references. */
     @VisibleForTesting
-    public void reset()
+    public void reset(Memtable memtable)
     {
-        view.set(new View(
-                         !isDummy() ? ImmutableList.of(new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfstore))
-                                    : ImmutableList.<Memtable>of(),
-                         ImmutableList.<Memtable>of(),
-                         Collections.<SSTableReader, SSTableReader>emptyMap(),
-                         Collections.<SSTableReader>emptySet(),
-                         Collections.<SSTableReader>emptySet(),
-                         SSTableIntervalTree.empty()));
+        view.set(new View(memtable != null ? singletonList(memtable) : Collections.<Memtable>emptyList(),
+                          Collections.<Memtable>emptyList(),
+                          Collections.<SSTableReader, SSTableReader>emptyMap(),
+                          Collections.<SSTableReader>emptySet(),
+                          Collections.<SSTableReader>emptySet(),
+                          SSTableIntervalTree.empty()));
     }
 
     public Throwable dropSSTablesIfInvalid(Throwable accumulate)
@@ -473,7 +479,7 @@ public class Tracker
 
     public boolean isDummy()
     {
-        return cfstore == null;
+        return cfstore == null || !DatabaseDescriptor.isDaemonInitialized();
     }
 
     public void subscribe(INotificationConsumer consumer)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index fddf058..ba060d4 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -2246,7 +2246,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
             // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist
             // the read meter when in client mode.
-            if (SystemKeyspace.NAME.equals(desc.ksname))
+            if (SystemKeyspace.NAME.equals(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
             {
                 readMeter = null;
                 readMeterSyncFuture = null;
@@ -2272,9 +2272,11 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         {
             lookup.remove(desc);
             if (readMeterSyncFuture != null)
+            {
                 readMeterSyncFuture.cancel(true);
-            if (isCompacted.get())
-                SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+                if (isCompacted.get())
+                    SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+            }
             // don't ideally want to dropPageCache for the file until all instances have been released
             CLibrary.trySkipCache(desc.filenameFor(Component.DATA), 0, 0);
             CLibrary.trySkipCache(desc.filenameFor(Component.PRIMARY_INDEX), 0, 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index cf5f3c2..35e990f 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -505,6 +505,7 @@ public class CassandraDaemon
             try
             {
                 DatabaseDescriptor.forceStaticInitialization();
+                DatabaseDescriptor.setDaemonInitialized();
             }
             catch (ExceptionInInitializerError e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
index 659d851..6c154cd 100644
--- a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
+++ b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.service;
 
 import java.io.IOException;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.service.CassandraDaemon;
 
 /**
@@ -49,6 +50,7 @@ public class EmbeddedCassandraService
     public void start() throws IOException
     {
         cassandraDaemon = new CassandraDaemon();
+        DatabaseDescriptor.setDaemonInitialized();
         cassandraDaemon.init(null);
         cassandraDaemon.start();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
index fcec40d..ee719d1 100644
--- a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
+++ b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
@@ -30,6 +30,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.service.StorageService;
@@ -39,6 +40,7 @@ public class CQLSSTableWriterLongTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index a7cf7b4..2048f74 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -90,6 +90,7 @@ public class SchemaLoader
             }
         });
 
+        DatabaseDescriptor.setDaemonInitialized();
         Keyspace.setInitialized();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 5f231c3..7409535 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -47,6 +47,12 @@ import static org.junit.Assert.assertNull;
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class DatabaseDescriptorTest
 {
+    @BeforeClass
+    public static void setupDatabaseDescriptor()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testCFMetaDataSerialization() throws ConfigurationException, InvalidRequestException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/CounterCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
index 8d75b9a..5d4b8a8 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -30,6 +30,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
 import org.apache.cassandra.db.context.CounterContext;
@@ -57,6 +58,8 @@ public class CounterCellTest
         countLength   = 8; // size of long
 
         stepLength    = idLength + clockLength + countLength;
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/NativeCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NativeCellTest.java b/test/unit/org/apache/cassandra/db/NativeCellTest.java
index 70b7b87..4145a91 100644
--- a/test/unit/org/apache/cassandra/db/NativeCellTest.java
+++ b/test/unit/org/apache/cassandra/db/NativeCellTest.java
@@ -32,6 +32,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -128,6 +129,8 @@ public class NativeCellTest
         {
             throw new AssertionError();
         }
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index b8aa161..c3047b8 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -42,6 +42,8 @@ public class SystemKeyspaceTest
     @BeforeClass
     public static void prepSnapshotTracker()
     {
+        DatabaseDescriptor.setDaemonInitialized();
+
         if (FBUtilities.isWindows())
             WindowsFailedSnapshotTracker.deleteOldSnapshots();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
index a72d30d..4f587c6 100644
--- a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
+++ b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
@@ -22,9 +22,11 @@ package org.apache.cassandra.db.context;
 
 import java.nio.ByteBuffer;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ClockAndCount;
 import org.apache.cassandra.db.context.CounterContext.Relationship;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -48,6 +50,12 @@ public class CounterContextTest
     private static final int countLength = 8;
     private static final int stepLength = idLength + clockLength + countLength;
 
+    @BeforeClass
+    public static void setupDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testAllocate()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
index f13d1b7..737392e 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.lifecycle;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.junit.After;
 import org.junit.Before;
@@ -30,6 +31,8 @@ import junit.framework.Assert;
 import org.apache.cassandra.MockSchema;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
@@ -270,7 +273,7 @@ public class LifecycleTransactionTest extends AbstractTransactionalTest
 
         private static Tracker tracker(ColumnFamilyStore cfs, List<SSTableReader> readers)
         {
-            Tracker tracker = new Tracker(cfs, false);
+            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs), false);
             tracker.addInitialSSTables(readers);
             return tracker;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
index adeb778..04b4e4a 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -75,6 +75,7 @@ public class TrackerTest
     @BeforeClass
     public static void setUp()
     {
+        DatabaseDescriptor.setDaemonInitialized();
         MockSchema.cleanup();
     }
 
@@ -82,7 +83,7 @@ public class TrackerTest
     public void testTryModify()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = new Tracker(null, false);
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, true, cfs), MockSchema.sstable(1, cfs), MockSchema.sstable(2, cfs));
         tracker.addInitialSSTables(copyOf(readers));
         Assert.assertNull(tracker.tryModify(ImmutableList.of(MockSchema.sstable(0, cfs)), OperationType.COMPACTION));
@@ -146,7 +147,7 @@ public class TrackerTest
     public void testAddInitialSSTables()
     {
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = cfs.getTracker();
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),
                                                        MockSchema.sstable(1, 121, cfs),
                                                        MockSchema.sstable(2, 9, cfs));
@@ -163,7 +164,7 @@ public class TrackerTest
         boolean backups = DatabaseDescriptor.isIncrementalBackupsEnabled();
         DatabaseDescriptor.setIncrementalBackupsEnabled(false);
         ColumnFamilyStore cfs = MockSchema.newCFS();
-        Tracker tracker = new Tracker(cfs, false);
+        Tracker tracker = cfs.getTracker();
         MockListener listener = new MockListener(false);
         tracker.subscribe(listener);
         List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
index 86781d9..bdb654a 100644
--- a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
+++ b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
@@ -20,8 +20,10 @@ package org.apache.cassandra.dht;
 import java.net.InetAddress;
 import java.util.Collections;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.streaming.DefaultConnectionFactory;
 import org.apache.cassandra.streaming.StreamEvent;
 import org.apache.cassandra.streaming.StreamSession;
@@ -32,6 +34,11 @@ import static org.junit.Assert.assertTrue;
 
 public class StreamStateStoreTest
 {
+    @BeforeClass
+    public static void initDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
 
     @Test
     public void testUpdateAndQueryAvailableRanges()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
index 9325922..af099b0 100644
--- a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
+++ b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
@@ -45,6 +45,8 @@ public class FailureDetectorTest
     {
         // slow unit tests can cause problems with FailureDetector's GC pause handling
         System.setProperty("cassandra.max_local_pause_in_ms", "20000");
+
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/gms/GossiperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java
index ad07165..eb01305 100644
--- a/test/unit/org/apache/cassandra/gms/GossiperTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java
@@ -33,6 +33,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Token;
@@ -44,6 +45,10 @@ import static org.junit.Assert.assertEquals;
 
 public class GossiperTest
 {
+    static
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
     static final IPartitioner partitioner = new RandomPartitioner();
     StorageService ss = StorageService.instance;
     TokenMetadata tmd = StorageService.instance.getTokenMetadata();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 8a14428..5e2fffe 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -34,6 +34,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -52,6 +53,7 @@ public class CQLSSTableWriterTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 90e63e0..bc5be46 100644
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@ -27,6 +27,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
@@ -43,6 +44,7 @@ public class CloudstackSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
index 56bbb77..32383d9 100644
--- a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
@@ -33,6 +33,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -51,6 +52,7 @@ public class EC2SnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
index 1521454..f2450f4 100644
--- a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
@@ -31,6 +31,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -47,6 +48,7 @@ public class GoogleCloudSnitchTest
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/66f1aaf8/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index 4481501..a693a23 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -63,6 +63,7 @@ public class StorageServiceServerTest
     @BeforeClass
     public static void setUp() throws ConfigurationException
     {
+        DatabaseDescriptor.setDaemonInitialized();
         IEndpointSnitch snitch = new PropertyFileSnitch();
         DatabaseDescriptor.setEndpointSnitch(snitch);
         Keyspace.setInitialized();