You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2015/08/14 16:40:44 UTC

[1/2] cassandra git commit: Fixes race during construction of commit log: - Moves thread start outside of constructor for all commit log classes. - Removes all references to CommitLog.instance from commitlog package.

Repository: cassandra
Updated Branches:
  refs/heads/trunk 90bd5cae4 -> d0ba476c8


Fixes race during construction of commit log:
- Moves thread start outside of constructor for all commit log classes.
- Removes all references to CommitLog.instance from commitlog package.

patch by Branimir Lambov; reviewed by tjake for CASSANDRA-10049


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

Branch: refs/heads/trunk
Commit: f8f6fd2d7e51be7d0450d3a264136d1d93454203
Parents: 7b02696
Author: Branimir Lambov <br...@datastax.com>
Authored: Wed Aug 12 11:59:41 2015 +0300
Committer: T Jake Luciani <ja...@apache.org>
Committed: Fri Aug 14 10:38:30 2015 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/commitlog/AbstractCommitLogService.java  |  2 +-
 .../cassandra/db/commitlog/CommitLog.java       | 22 +++++++-----
 .../db/commitlog/CommitLogArchiver.java         | 38 +++++++++++++-------
 .../db/commitlog/CommitLogDescriptor.java       |  1 -
 .../db/commitlog/CommitLogReplayer.java         | 14 ++++----
 .../db/commitlog/CommitLogSegment.java          |  1 -
 .../db/commitlog/CommitLogSegmentManager.java   | 17 +++------
 .../db/commitlog/CommitLogStressTest.java       |  8 ++---
 .../unit/org/apache/cassandra/SchemaLoader.java |  2 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |  2 +-
 .../org/apache/cassandra/db/CommitLogTest.java  |  4 +--
 .../db/commitlog/CommitLogTestReplayer.java     | 10 +++---
 .../db/commitlog/CommitLogUpgradeTest.java      | 11 ++----
 14 files changed, 70 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e7cc12e..85eb223 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.0-beta1
+ * Fix race during construction of commit log (CASSANDRA-10049)
  * Add option to only purge repaired tombstones (CASSANDRA-6434)
  * Change authorization handling for MVs (CASSANDRA-9927)
  * Add custom JMX enabled executor for UDF sandbox (CASSANDRA-10026)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
index d371571..557bf50 100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
@@ -192,7 +192,7 @@ public abstract class AbstractCommitLogService
     /**
      * FOR TESTING ONLY
      */
-    public void startUnsafe()
+    public void restartUnsafe()
     {
         while (haveWork.availablePermits() < 1)
             haveWork.release();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index e096011..8d74677 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -74,7 +74,7 @@ public class CommitLog implements CommitLogMBean
 
     private static CommitLog construct()
     {
-        CommitLog log = new CommitLog(DatabaseDescriptor.getCommitLogLocation(), new CommitLogArchiver());
+        CommitLog log = new CommitLog(DatabaseDescriptor.getCommitLogLocation(), CommitLogArchiver.construct());
 
         MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
         try
@@ -85,7 +85,7 @@ public class CommitLog implements CommitLogMBean
         {
             throw new RuntimeException(e);
         }
-        return log;
+        return log.start();
     }
 
     @VisibleForTesting
@@ -105,12 +105,18 @@ public class CommitLog implements CommitLogMBean
                 : new PeriodicCommitLogService(this);
 
         allocator = new CommitLogSegmentManager(this);
-        executor.start();
 
         // register metrics
         metrics.attach(executor, allocator);
     }
 
+    CommitLog start()
+    {
+        executor.start();
+        allocator.start();
+        return this;
+    }
+
     /**
      * Perform recovery on commit logs located in the directory specified by the config file.
      *
@@ -177,7 +183,7 @@ public class CommitLog implements CommitLogMBean
      */
     public int recover(File... clogs) throws IOException
     {
-        CommitLogReplayer recovery = CommitLogReplayer.create();
+        CommitLogReplayer recovery = CommitLogReplayer.construct(this);
         recovery.recover(clogs);
         return recovery.blockForWrites();
     }
@@ -412,7 +418,7 @@ public class CommitLog implements CommitLogMBean
     public int resetUnsafe(boolean deleteSegments) throws IOException
     {
         stopUnsafe(deleteSegments);
-        return startUnsafe();
+        return restartUnsafe();
     }
 
     /**
@@ -435,10 +441,10 @@ public class CommitLog implements CommitLogMBean
     /**
      * FOR TESTING PURPOSES.  See CommitLogAllocator
      */
-    public int startUnsafe() throws IOException
+    public int restartUnsafe() throws IOException
     {
-        allocator.startUnsafe();
-        executor.startUnsafe();
+        allocator.start();
+        executor.restartUnsafe();
         return recover();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
index e7d115d..095e9c5 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
@@ -52,33 +52,45 @@ public class CommitLogArchiver
     }
 
     public final Map<String, Future<?>> archivePending = new ConcurrentHashMap<String, Future<?>>();
-    private final ExecutorService executor = new JMXEnabledThreadPoolExecutor("CommitLogArchiver");
+    private final ExecutorService executor;
     final String archiveCommand;
     final String restoreCommand;
     final String restoreDirectories;
     public long restorePointInTime;
     public final TimeUnit precision;
 
-    public CommitLogArchiver()
+    public CommitLogArchiver(String archiveCommand, String restoreCommand, String restoreDirectories,
+            long restorePointInTime, TimeUnit precision)
+    {
+        this.archiveCommand = archiveCommand;
+        this.restoreCommand = restoreCommand;
+        this.restoreDirectories = restoreDirectories;
+        this.restorePointInTime = restorePointInTime;
+        this.precision = precision;
+        executor = !Strings.isNullOrEmpty(archiveCommand) ? new JMXEnabledThreadPoolExecutor("CommitLogArchiver") : null;
+    }
+
+    public static CommitLogArchiver disabled()
+    {
+        return new CommitLogArchiver(null, null, null, Long.MAX_VALUE, TimeUnit.MICROSECONDS);
+    }
+
+    public static CommitLogArchiver construct()
     {
         Properties commitlog_commands = new Properties();
-        try (InputStream stream = getClass().getClassLoader().getResourceAsStream("commitlog_archiving.properties"))
+        try (InputStream stream = CommitLogArchiver.class.getClassLoader().getResourceAsStream("commitlog_archiving.properties"))
         {
             if (stream == null)
             {
                 logger.debug("No commitlog_archiving properties found; archive + pitr will be disabled");
-                archiveCommand = null;
-                restoreCommand = null;
-                restoreDirectories = null;
-                restorePointInTime = Long.MAX_VALUE;
-                precision = TimeUnit.MICROSECONDS;
+                return disabled();
             }
             else
             {
                 commitlog_commands.load(stream);
-                archiveCommand = commitlog_commands.getProperty("archive_command");
-                restoreCommand = commitlog_commands.getProperty("restore_command");
-                restoreDirectories = commitlog_commands.getProperty("restore_directories");
+                String archiveCommand = commitlog_commands.getProperty("archive_command");
+                String restoreCommand = commitlog_commands.getProperty("restore_command");
+                String restoreDirectories = commitlog_commands.getProperty("restore_directories");
                 if (restoreDirectories != null && !restoreDirectories.isEmpty())
                 {
                     for (String dir : restoreDirectories.split(DELIMITER))
@@ -94,7 +106,8 @@ public class CommitLogArchiver
                     }
                 }
                 String targetTime = commitlog_commands.getProperty("restore_point_in_time");
-                precision = TimeUnit.valueOf(commitlog_commands.getProperty("precision", "MICROSECONDS"));
+                TimeUnit precision = TimeUnit.valueOf(commitlog_commands.getProperty("precision", "MICROSECONDS"));
+                long restorePointInTime;
                 try
                 {
                     restorePointInTime = Strings.isNullOrEmpty(targetTime) ? Long.MAX_VALUE : format.parse(targetTime).getTime();
@@ -103,6 +116,7 @@ public class CommitLogArchiver
                 {
                     throw new RuntimeException("Unable to parse restore target time", e);
                 }
+                return new CommitLogArchiver(archiveCommand, restoreCommand, restoreDirectories, restorePointInTime, precision);
             }
         }
         catch (IOException e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
index c49f813..6774d39 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
@@ -40,7 +40,6 @@ import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.FBUtilities;
 import org.json.simple.JSONValue;
 
 import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index d232a63..358661a 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -79,8 +79,9 @@ public class CommitLogReplayer
     private byte[] uncompressedBuffer;
 
     private final ReplayFilter replayFilter;
+    private final CommitLogArchiver archiver;
 
-    CommitLogReplayer(ReplayPosition globalPosition, Map<UUID, ReplayPosition> cfPositions, ReplayFilter replayFilter)
+    CommitLogReplayer(CommitLog commitLog, ReplayPosition globalPosition, Map<UUID, ReplayPosition> cfPositions, ReplayFilter replayFilter)
     {
         this.keyspacesRecovered = new NonBlockingHashSet<Keyspace>();
         this.futures = new ArrayList<Future<?>>();
@@ -93,9 +94,10 @@ public class CommitLogReplayer
         this.cfPositions = cfPositions;
         this.globalPosition = globalPosition;
         this.replayFilter = replayFilter;
+        this.archiver = commitLog.archiver;
     }
 
-    public static CommitLogReplayer create()
+    public static CommitLogReplayer construct(CommitLog commitLog)
     {
         // compute per-CF and global replay positions
         Map<UUID, ReplayPosition> cfPositions = new HashMap<UUID, ReplayPosition>();
@@ -115,7 +117,7 @@ public class CommitLogReplayer
                 // Point in time restore is taken to mean that the tables need to be recovered even if they were
                 // deleted at a later point in time. Any truncation record after that point must thus be cleared prior
                 // to recovery (CASSANDRA-9195).
-                long restoreTime = CommitLog.instance.archiver.restorePointInTime;
+                long restoreTime = commitLog.archiver.restorePointInTime;
                 long truncatedTime = SystemKeyspace.getTruncatedAt(cfs.metadata.cfId);
                 if (truncatedTime > restoreTime)
                 {
@@ -137,7 +139,7 @@ public class CommitLogReplayer
         }
         ReplayPosition globalPosition = replayPositionOrdering.min(cfPositions.values());
         logger.debug("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPositions));
-        return new CommitLogReplayer(globalPosition, cfPositions, replayFilter);
+        return new CommitLogReplayer(commitLog, globalPosition, cfPositions, replayFilter);
     }
 
     public void recover(File[] clogs) throws IOException
@@ -569,11 +571,11 @@ public class CommitLogReplayer
 
     protected boolean pointInTimeExceeded(Mutation fm)
     {
-        long restoreTarget = CommitLog.instance.archiver.restorePointInTime;
+        long restoreTarget = archiver.restorePointInTime;
 
         for (PartitionUpdate upd : fm.getPartitionUpdates())
         {
-            if (CommitLog.instance.archiver.precision.toMillis(upd.maxTimestamp()) > restoreTarget)
+            if (archiver.precision.toMillis(upd.maxTimestamp()) > restoreTarget)
                 return true;
         }
         return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index b4b7b19..5dd7c9f 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -49,7 +49,6 @@ import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.CLibrary;
-import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
index 3f00e97..19b850f 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
@@ -34,6 +34,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.*;
 
@@ -95,13 +96,14 @@ public class CommitLogSegmentManager
     private volatile boolean run = true;
     private final CommitLog commitLog;
 
+    @VisibleForTesting
     public CommitLogSegmentManager(final CommitLog commitLog)
     {
         this.commitLog = commitLog;
-        start();
     }
 
-    private void start()
+    @VisibleForTesting
+    public void start()
     {
         // The run loop for the manager thread
         Runnable runnable = new WrappedRunnable()
@@ -302,8 +304,7 @@ public class CommitLogSegmentManager
             if (cfs != null)
                 keyspaces.add(cfs.keyspace);
         }
-        for (Keyspace keyspace : keyspaces)
-            keyspace.writeOrder.awaitNewBarrier();
+        Keyspace.writeOrder.awaitNewBarrier();
 
         // flush and wait for all CFs that are dirty in segments up-to and including 'last'
         Future<?> future = flushDataFrom(segmentsToRecycle, true);
@@ -526,14 +527,6 @@ public class CommitLogSegmentManager
     }
 
     /**
-     * Starts CL, for testing purposes. DO NOT USE THIS OUTSIDE OF TESTS.
-     */
-    public void startUnsafe()
-    {
-        start();
-    }
-
-    /**
      * Initiates the shutdown process for the management thread.
      */
     public void shutdown()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index d3ff082..be3abb4 100644
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@ -208,7 +208,7 @@ public class CommitLogStressTest
             for (CommitLogSync sync : CommitLogSync.values())
             {
                 DatabaseDescriptor.setCommitLogSync(sync);
-                CommitLog commitLog = new CommitLog(location, CommitLog.instance.archiver);
+                CommitLog commitLog = new CommitLog(location, CommitLogArchiver.disabled()).start();
                 testLog(commitLog);
             }
         }
@@ -273,7 +273,7 @@ public class CommitLogStressTest
 
         System.out.print("Stopped. Replaying... ");
         System.out.flush();
-        Replayer repl = new Replayer();
+        Replayer repl = new Replayer(commitLog);
         File[] files = new File(location).listFiles();
         repl.recover(files);
 
@@ -443,9 +443,9 @@ public class CommitLogStressTest
 
     class Replayer extends CommitLogReplayer
     {
-        Replayer()
+        Replayer(CommitLog log)
         {
-            super(discardedPos, null, ReplayFilter.create());
+            super(log, discardedPos, null, ReplayFilter.create());
         }
 
         int hash = 0;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/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 ffe0ac8..e914f13 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -464,7 +464,7 @@ public class SchemaLoader
         mkdirs();
         cleanup();
         mkdirs();
-        CommitLog.instance.startUnsafe();
+        CommitLog.instance.restartUnsafe();
     }
 
     public static void cleanup()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 721963d..2bd390f 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -173,7 +173,7 @@ public abstract class CQLTester
         mkdirs();
         cleanup();
         mkdirs();
-        CommitLog.instance.startUnsafe();
+        CommitLog.instance.restartUnsafe();
     }
 
     public static void cleanup()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/test/unit/org/apache/cassandra/db/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CommitLogTest.java b/test/unit/org/apache/cassandra/db/CommitLogTest.java
index 512a3d2..0c5093a 100644
--- a/test/unit/org/apache/cassandra/db/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java
@@ -408,7 +408,7 @@ public class CommitLogTest
             DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore);
 
             //now let's create a commit log segment manager and wait for it to fail
-            new CommitLogSegmentManager(CommitLog.instance);
+            new CommitLogSegmentManager(CommitLog.instance).start();
 
             //busy wait since commitlogsegmentmanager spawns another thread
             int retries = 0;
@@ -447,7 +447,7 @@ public class CommitLogTest
             DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore);
 
             //now let's create a commit log segment manager and wait for it to fail
-            new CommitLogSegmentManager(CommitLog.instance);
+            new CommitLogSegmentManager(CommitLog.instance).start();
 
             //wait commit log segment manager thread to execute
             Thread.sleep(50);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java
index 994ee19..1ab9ca7 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java
@@ -39,21 +39,21 @@ public class CommitLogTestReplayer extends CommitLogReplayer
     {
         CommitLog.instance.sync(true);
 
-        CommitLogTestReplayer replayer = new CommitLogTestReplayer(processor);
+        CommitLogTestReplayer replayer = new CommitLogTestReplayer(CommitLog.instance, processor);
         File commitLogDir = new File(DatabaseDescriptor.getCommitLogLocation());
         replayer.recover(commitLogDir.listFiles());
     }
 
     final private Predicate<Mutation> processor;
 
-    public CommitLogTestReplayer(Predicate<Mutation> processor)
+    public CommitLogTestReplayer(CommitLog log, Predicate<Mutation> processor)
     {
-        this(ReplayPosition.NONE, processor);
+        this(log, ReplayPosition.NONE, processor);
     }
 
-    public CommitLogTestReplayer(ReplayPosition discardedPos, Predicate<Mutation> processor)
+    public CommitLogTestReplayer(CommitLog log, ReplayPosition discardedPos, Predicate<Mutation> processor)
     {
-        super(discardedPos, null, ReplayFilter.create());
+        super(log, discardedPos, null, ReplayFilter.create());
         this.processor = processor;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8f6fd2d/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
index 81693c7..7b0ab06 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
@@ -121,15 +121,8 @@ public class CommitLogUpgradeTest
         }
 
         Hasher hasher = new Hasher();
-        CommitLogTestReplayer replayer = new CommitLogTestReplayer(hasher);
-        File[] files = new File(location).listFiles(new FilenameFilter()
-        {
-            @Override
-            public boolean accept(File dir, String name)
-            {
-                return name.endsWith(".log");
-            }
-        });
+        CommitLogTestReplayer replayer = new CommitLogTestReplayer(CommitLog.instance, hasher);
+        File[] files = new File(location).listFiles((file, name) -> name.endsWith(".log"));
         replayer.recover(files);
 
         Assert.assertEquals(cells, hasher.cells);


[2/2] cassandra git commit: Merge branch 'cassandra-3.0' into trunk

Posted by ja...@apache.org.
Merge branch 'cassandra-3.0' into trunk


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

Branch: refs/heads/trunk
Commit: d0ba476c896c79a8dbd1bf32fc9f7a2198c09524
Parents: 90bd5ca f8f6fd2
Author: T Jake Luciani <ja...@apache.org>
Authored: Fri Aug 14 10:40:25 2015 -0400
Committer: T Jake Luciani <ja...@apache.org>
Committed: Fri Aug 14 10:40:25 2015 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/commitlog/AbstractCommitLogService.java  |  2 +-
 .../cassandra/db/commitlog/CommitLog.java       | 22 +++++++-----
 .../db/commitlog/CommitLogArchiver.java         | 38 +++++++++++++-------
 .../db/commitlog/CommitLogDescriptor.java       |  1 -
 .../db/commitlog/CommitLogReplayer.java         | 14 ++++----
 .../db/commitlog/CommitLogSegment.java          |  1 -
 .../db/commitlog/CommitLogSegmentManager.java   | 17 +++------
 .../db/commitlog/CommitLogStressTest.java       |  8 ++---
 .../unit/org/apache/cassandra/SchemaLoader.java |  2 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |  2 +-
 .../org/apache/cassandra/db/CommitLogTest.java  |  4 +--
 .../db/commitlog/CommitLogTestReplayer.java     | 10 +++---
 .../db/commitlog/CommitLogUpgradeTest.java      | 11 ++----
 14 files changed, 70 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d0ba476c/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c3d7b67,85eb223..4f7d896
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,8 -1,5 +1,9 @@@
 +3.2
 + * Add transparent data encryption core classes (CASSANDRA-9945)
 +
 +
  3.0.0-beta1
+  * Fix race during construction of commit log (CASSANDRA-10049)
   * Add option to only purge repaired tombstones (CASSANDRA-6434)
   * Change authorization handling for MVs (CASSANDRA-9927)
   * Add custom JMX enabled executor for UDF sandbox (CASSANDRA-10026)