You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by st...@apache.org on 2016/11/07 02:35:19 UTC

[1/6] cassandra git commit: Fix CommitLogTest.testDeleteIfNotDirty

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 817ba0387 -> ec64cdf4c
  refs/heads/cassandra-3.X 9d9a1a122 -> 80843c987
  refs/heads/trunk 22a3db065 -> 9ed4faceb


Fix CommitLogTest.testDeleteIfNotDirty

patch by Stefania Alborghetti; reviewed by Branimir Lambov for CASSANDRA-12854


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

Branch: refs/heads/cassandra-3.0
Commit: ec64cdf4c6b731a00bea8795247d865d6371157f
Parents: 817ba03
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Fri Oct 28 17:25:15 2016 +0800
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Mon Nov 7 10:30:15 2016 +0800

----------------------------------------------------------------------
 CHANGES.txt                                              |  1 +
 src/java/org/apache/cassandra/schema/KeyspaceParams.java | 11 ++++++++++-
 .../org/apache/cassandra/db/commitlog/CommitLogTest.java |  5 +++++
 3 files changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec64cdf4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c80e045..51d571b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.10
+ * Fix CommitLogTest.testDeleteIfNotDirty (CASSANDRA-12854)
  * Avoid deadlock due to materialized view lock contention (CASSANDRA-12689)
  * Fix for KeyCacheCqlTest flakiness (CASSANDRA-12801)
  * Include SSTable filename in compacting large row message (CASSANDRA-12384)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec64cdf4/src/java/org/apache/cassandra/schema/KeyspaceParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/KeyspaceParams.java b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
index c0e8916..2ea18ca 100644
--- a/src/java/org/apache/cassandra/schema/KeyspaceParams.java
+++ b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.schema;
 
 import java.util.Map;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
 
 /**
@@ -28,6 +29,14 @@ public final class KeyspaceParams
 {
     public static final boolean DEFAULT_DURABLE_WRITES = true;
 
+    /**
+     * This determines durable writes for the {@link org.apache.cassandra.db.SystemKeyspace#NAME}
+     * and {@link SchemaKeyspace#NAME} keyspaces, the only reason it is not final is for commitlog
+     * unit tests. It should only be changed for testing purposes.
+     */
+    @VisibleForTesting
+    public static boolean DEFAULT_LOCAL_DURABLE_WRITES = true;
+
     public enum Option
     {
         DURABLE_WRITES,
@@ -56,7 +65,7 @@ public final class KeyspaceParams
 
     public static KeyspaceParams local()
     {
-        return new KeyspaceParams(true, ReplicationParams.local());
+        return new KeyspaceParams(DEFAULT_LOCAL_DURABLE_WRITES, ReplicationParams.local());
     }
 
     public static KeyspaceParams simple(int replicationFactor)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec64cdf4/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 9a0ddb8..c4ab6ab 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -97,6 +97,11 @@ public class CommitLogTest
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
+        // Disable durable writes for system keyspaces to prevent system mutations, e.g. sstable_activity,
+        // to end up in CL segments and cause unexpected results in this test wrt counting CL segments,
+        // see CASSANDRA-12854
+        KeyspaceParams.DEFAULT_LOCAL_DURABLE_WRITES = false;
+
         SchemaLoader.prepareServer();
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     KeyspaceParams.simple(1),


[4/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.X

Posted by st...@apache.org.
Merge branch 'cassandra-3.0' 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/80843c98
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/80843c98
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/80843c98

Branch: refs/heads/trunk
Commit: 80843c987ea3cf70aabc4def8139caf319efbc7c
Parents: 9d9a1a1 ec64cdf
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Mon Nov 7 10:31:29 2016 +0800
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Mon Nov 7 10:32:23 2016 +0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/schema/KeyspaceParams.java | 11 +++++++-
 .../cassandra/db/commitlog/CommitLogTest.java   | 29 ++++++++++++++++++--
 3 files changed, 38 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/80843c98/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 40aace3,51d571b..3157e8e
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,105 -1,6 +1,106 @@@
 -3.0.10
 +3.10
 + * Fix crossNode value when receiving messages (CASSANDRA-12791)
 + * Don't load MX4J beans twice (CASSANDRA-12869)
 + * Extend native protocol request flags, add versions to SUPPORTED, and introduce ProtocolVersion enum (CASSANDRA-12838)
 + * Set JOINING mode when running pre-join tasks (CASSANDRA-12836)
 + * remove net.mintern.primitive library due to license issue (CASSANDRA-12845)
 + * Properly format IPv6 addresses when logging JMX service URL (CASSANDRA-12454)
 + * Optimize the vnode allocation for single replica per DC (CASSANDRA-12777)
 + * Use non-token restrictions for bounds when token restrictions are overridden (CASSANDRA-12419)
 + * Fix CQLSH auto completion for PER PARTITION LIMIT (CASSANDRA-12803)
 + * Use different build directories for Eclipse and Ant (CASSANDRA-12466)
 + * Avoid potential AttributeError in cqlsh due to no table metadata (CASSANDRA-12815)
 + * Fix RandomReplicationAwareTokenAllocatorTest.testExistingCluster (CASSANDRA-12812)
 + * Upgrade commons-codec to 1.9 (CASSANDRA-12790)
 + * Make the fanout size for LeveledCompactionStrategy to be configurable (CASSANDRA-11550)
 + * Add duration data type (CASSANDRA-11873)
 + * Fix timeout in ReplicationAwareTokenAllocatorTest (CASSANDRA-12784)
 + * Improve sum aggregate functions (CASSANDRA-12417)
 + * Make cassandra.yaml docs for batch_size_*_threshold_in_kb reflect changes in CASSANDRA-10876 (CASSANDRA-12761)
 + * cqlsh fails to format collections when using aliases (CASSANDRA-11534)
 + * Check for hash conflicts in prepared statements (CASSANDRA-12733)
 + * Exit query parsing upon first error (CASSANDRA-12598)
 + * Fix cassandra-stress to use single seed in UUID generation (CASSANDRA-12729)
 + * CQLSSTableWriter does not allow Update statement (CASSANDRA-12450)
 + * Config class uses boxed types but DD exposes primitive types (CASSANDRA-12199)
 + * Add pre- and post-shutdown hooks to Storage Service (CASSANDRA-12461)
 + * Add hint delivery metrics (CASSANDRA-12693)
 + * Remove IndexInfo cache from FileIndexInfoRetriever (CASSANDRA-12731)
 + * ColumnIndex does not reuse buffer (CASSANDRA-12502)
 + * cdc column addition still breaks schema migration tasks (CASSANDRA-12697)
 + * Upgrade metrics-reporter dependencies (CASSANDRA-12089)
 + * Tune compaction thread count via nodetool (CASSANDRA-12248)
 + * Add +=/-= shortcut syntax for update queries (CASSANDRA-12232)
 + * Include repair session IDs in repair start message (CASSANDRA-12532)
 + * Add a blocking task to Index, run before joining the ring (CASSANDRA-12039)
 + * Fix NPE when using CQLSSTableWriter (CASSANDRA-12667)
 + * Support optional backpressure strategies at the coordinator (CASSANDRA-9318)
 + * Make randompartitioner work with new vnode allocation (CASSANDRA-12647)
 + * Fix cassandra-stress graphing (CASSANDRA-12237)
 + * Allow filtering on partition key columns for queries without secondary indexes (CASSANDRA-11031)
 + * Fix Cassandra Stress reporting thread model and precision (CASSANDRA-12585)
 + * Add JMH benchmarks.jar (CASSANDRA-12586)
 + * Add row offset support to SASI (CASSANDRA-11990)
 + * Cleanup uses of AlterTableStatementColumn (CASSANDRA-12567)
 + * Add keep-alive to streaming (CASSANDRA-11841)
 + * Tracing payload is passed through newSession(..) (CASSANDRA-11706)
 + * avoid deleting non existing sstable files and improve related log messages (CASSANDRA-12261)
 + * json/yaml output format for nodetool compactionhistory (CASSANDRA-12486)
 + * Retry all internode messages once after a connection is
 +   closed and reopened (CASSANDRA-12192)
 + * Add support to rebuild from targeted replica (CASSANDRA-9875)
 + * Add sequence distribution type to cassandra stress (CASSANDRA-12490)
 + * "SELECT * FROM foo LIMIT ;" does not error out (CASSANDRA-12154)
 + * Define executeLocally() at the ReadQuery Level (CASSANDRA-12474)
 + * Extend read/write failure messages with a map of replica addresses
 +   to error codes in the v5 native protocol (CASSANDRA-12311)
 + * Fix rebuild of SASI indexes with existing index files (CASSANDRA-12374)
 + * Let DatabaseDescriptor not implicitly startup services (CASSANDRA-9054, 12550)
 + * Fix clustering indexes in presence of static columns in SASI (CASSANDRA-12378)
 + * Fix queries on columns with reversed type on SASI indexes (CASSANDRA-12223)
 + * Added slow query log (CASSANDRA-12403)
 + * Count full coordinated request against timeout (CASSANDRA-12256)
 + * Allow TTL with null value on insert and update (CASSANDRA-12216)
 + * Make decommission operation resumable (CASSANDRA-12008)
 + * Add support to one-way targeted repair (CASSANDRA-9876)
 + * Remove clientutil jar (CASSANDRA-11635)
 + * Fix compaction throughput throttle (CASSANDRA-12366, CASSANDRA-12717)
 + * Delay releasing Memtable memory on flush until PostFlush has finished running (CASSANDRA-12358)
 + * Cassandra stress should dump all setting on startup (CASSANDRA-11914)
 + * Make it possible to compact a given token range (CASSANDRA-10643)
 + * Allow updating DynamicEndpointSnitch properties via JMX (CASSANDRA-12179)
 + * Collect metrics on queries by consistency level (CASSANDRA-7384)
 + * Add support for GROUP BY to SELECT statement (CASSANDRA-10707)
 + * Deprecate memtable_cleanup_threshold and update default for memtable_flush_writers (CASSANDRA-12228)
 + * Upgrade to OHC 0.4.4 (CASSANDRA-12133)
 + * Add version command to cassandra-stress (CASSANDRA-12258)
 + * Create compaction-stress tool (CASSANDRA-11844)
 + * Garbage-collecting compaction operation and schema option (CASSANDRA-7019)
 + * Add beta protocol flag for v5 native protocol (CASSANDRA-12142)
 + * Support filtering on non-PRIMARY KEY columns in the CREATE
 +   MATERIALIZED VIEW statement's WHERE clause (CASSANDRA-10368)
 + * Unify STDOUT and SYSTEMLOG logback format (CASSANDRA-12004)
 + * COPY FROM should raise error for non-existing input files (CASSANDRA-12174)
 + * Faster write path (CASSANDRA-12269)
 + * Option to leave omitted columns in INSERT JSON unset (CASSANDRA-11424)
 + * Support json/yaml output in nodetool tpstats (CASSANDRA-12035)
 + * Expose metrics for successful/failed authentication attempts (CASSANDRA-10635)
 + * Prepend snapshot name with "truncated" or "dropped" when a snapshot
 +   is taken before truncating or dropping a table (CASSANDRA-12178)
 + * Optimize RestrictionSet (CASSANDRA-12153)
 + * cqlsh does not automatically downgrade CQL version (CASSANDRA-12150)
 + * Omit (de)serialization of state variable in UDAs (CASSANDRA-9613)
 + * Create a system table to expose prepared statements (CASSANDRA-8831)
 + * Reuse DataOutputBuffer from ColumnIndex (CASSANDRA-11970)
 + * Remove DatabaseDescriptor dependency from SegmentedFile (CASSANDRA-11580)
 + * Add supplied username to authentication error messages (CASSANDRA-12076)
 + * Remove pre-startup check for open JMX port (CASSANDRA-12074)
 + * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738)
 + * Restore resumable hints delivery (CASSANDRA-11960)
 + * Properly report LWT contention (CASSANDRA-12626)
 +Merged from 3.0:
+  * Fix CommitLogTest.testDeleteIfNotDirty (CASSANDRA-12854)
 - * Avoid deadlock due to materialized view lock contention (CASSANDRA-12689)
 + * Avoid deadlock due to MV lock contention (CASSANDRA-12689)
   * Fix for KeyCacheCqlTest flakiness (CASSANDRA-12801)
   * Include SSTable filename in compacting large row message (CASSANDRA-12384)
   * Fix potential socket leak (CASSANDRA-12329, CASSANDRA-12330)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/80843c98/src/java/org/apache/cassandra/schema/KeyspaceParams.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/schema/KeyspaceParams.java
index 63775fa,2ea18ca..1deaa29
--- a/src/java/org/apache/cassandra/schema/KeyspaceParams.java
+++ b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
@@@ -19,7 -19,7 +19,8 @@@ package org.apache.cassandra.schema
  
  import java.util.Map;
  
+ import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.base.MoreObjects;
  import com.google.common.base.Objects;
  
  /**
@@@ -29,6 -29,14 +30,14 @@@ public final class KeyspaceParam
  {
      public static final boolean DEFAULT_DURABLE_WRITES = true;
  
+     /**
 -     * This determines durable writes for the {@link org.apache.cassandra.db.SystemKeyspace#NAME}
 -     * and {@link SchemaKeyspace#NAME} keyspaces, the only reason it is not final is for commitlog
 -     * unit tests. It should only be changed for testing purposes.
++     * This determines durable writes for the {@link org.apache.cassandra.config.SchemaConstants#SCHEMA_KEYSPACE_NAME}
++     * and {@link org.apache.cassandra.config.SchemaConstants#SYSTEM_KEYSPACE_NAME} keyspaces,
++     * the only reason it is not final is for commitlog unit tests. It should only be changed for testing purposes.
+      */
+     @VisibleForTesting
+     public static boolean DEFAULT_LOCAL_DURABLE_WRITES = true;
+ 
      public enum Option
      {
          DURABLE_WRITES,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/80843c98/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 30dffe5,c4ab6ab..5476d03
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@@ -24,6 -27,6 +24,7 @@@ import java.util.*
  import java.util.concurrent.Callable;
  import java.util.concurrent.ExecutionException;
  import java.util.function.BiConsumer;
++import java.util.stream.Collectors;
  import java.util.zip.CRC32;
  import java.util.zip.Checksum;
  
@@@ -97,8 -95,13 +98,13 @@@ public class CommitLogTes
      }
  
      @BeforeClass
 -    public static void defineSchema() throws ConfigurationException
 +    public static void beforeClass() throws ConfigurationException
      {
+         // Disable durable writes for system keyspaces to prevent system mutations, e.g. sstable_activity,
+         // to end up in CL segments and cause unexpected results in this test wrt counting CL segments,
+         // see CASSANDRA-12854
+         KeyspaceParams.DEFAULT_LOCAL_DURABLE_WRITES = false;
+ 
          SchemaLoader.prepareServer();
          SchemaLoader.createKeyspace(KEYSPACE1,
                                      KeyspaceParams.simple(1),
@@@ -293,16 -277,17 +299,35 @@@
          CommitLog.instance.add(rm2);
          CommitLog.instance.add(rm2);
  
-         assertEquals(3, CommitLog.instance.segmentManager.getActiveSegments().size());
 -        assert CommitLog.instance.activeSegments() == 3 : "Expecting 3 segments, got " + CommitLog.instance.activeSegments();
++        Collection<CommitLogSegment> segments = CommitLog.instance.segmentManager.getActiveSegments();
+ 
++        assertEquals(String.format("Expected 3 segments but got %d (%s)", segments.size(), getDirtyCFIds(segments)),
++                     3,
++                     segments.size());
  
          // "Flush" second cf: The first segment should be deleted since we
          // didn't write anything on cf1 since last flush (and we flush cf2)
  
          UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
 -        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
 +        CommitLog.instance.discardCompletedSegments(cfid2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
 +
++        segments = CommitLog.instance.segmentManager.getActiveSegments();
+ 
          // Assert we still have both our segment
-         assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
 -        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
++        assertEquals(String.format("Expected 1 segment but got %d (%s)", segments.size(), getDirtyCFIds(segments)),
++                     1,
++                     segments.size());
++    }
++
++    private String getDirtyCFIds(Collection<CommitLogSegment> segments)
++    {
++        return "Dirty cfIds: <"
++               + String.join(", ", segments.stream()
++                                           .map(CommitLogSegment::getDirtyCFIDs)
++                                           .flatMap(uuids -> uuids.stream())
++                                           .distinct()
++                                           .map(uuid -> uuid.toString()).collect(Collectors.toList()))
++               + ">";
      }
  
      private static int getMaxRecordDataSize(String keyspace, ByteBuffer key, String cfName, String colName)


[3/6] cassandra git commit: Fix CommitLogTest.testDeleteIfNotDirty

Posted by st...@apache.org.
Fix CommitLogTest.testDeleteIfNotDirty

patch by Stefania Alborghetti; reviewed by Branimir Lambov for CASSANDRA-12854


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

Branch: refs/heads/trunk
Commit: ec64cdf4c6b731a00bea8795247d865d6371157f
Parents: 817ba03
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Fri Oct 28 17:25:15 2016 +0800
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Mon Nov 7 10:30:15 2016 +0800

----------------------------------------------------------------------
 CHANGES.txt                                              |  1 +
 src/java/org/apache/cassandra/schema/KeyspaceParams.java | 11 ++++++++++-
 .../org/apache/cassandra/db/commitlog/CommitLogTest.java |  5 +++++
 3 files changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec64cdf4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c80e045..51d571b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.10
+ * Fix CommitLogTest.testDeleteIfNotDirty (CASSANDRA-12854)
  * Avoid deadlock due to materialized view lock contention (CASSANDRA-12689)
  * Fix for KeyCacheCqlTest flakiness (CASSANDRA-12801)
  * Include SSTable filename in compacting large row message (CASSANDRA-12384)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec64cdf4/src/java/org/apache/cassandra/schema/KeyspaceParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/KeyspaceParams.java b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
index c0e8916..2ea18ca 100644
--- a/src/java/org/apache/cassandra/schema/KeyspaceParams.java
+++ b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.schema;
 
 import java.util.Map;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
 
 /**
@@ -28,6 +29,14 @@ public final class KeyspaceParams
 {
     public static final boolean DEFAULT_DURABLE_WRITES = true;
 
+    /**
+     * This determines durable writes for the {@link org.apache.cassandra.db.SystemKeyspace#NAME}
+     * and {@link SchemaKeyspace#NAME} keyspaces, the only reason it is not final is for commitlog
+     * unit tests. It should only be changed for testing purposes.
+     */
+    @VisibleForTesting
+    public static boolean DEFAULT_LOCAL_DURABLE_WRITES = true;
+
     public enum Option
     {
         DURABLE_WRITES,
@@ -56,7 +65,7 @@ public final class KeyspaceParams
 
     public static KeyspaceParams local()
     {
-        return new KeyspaceParams(true, ReplicationParams.local());
+        return new KeyspaceParams(DEFAULT_LOCAL_DURABLE_WRITES, ReplicationParams.local());
     }
 
     public static KeyspaceParams simple(int replicationFactor)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec64cdf4/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 9a0ddb8..c4ab6ab 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -97,6 +97,11 @@ public class CommitLogTest
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
+        // Disable durable writes for system keyspaces to prevent system mutations, e.g. sstable_activity,
+        // to end up in CL segments and cause unexpected results in this test wrt counting CL segments,
+        // see CASSANDRA-12854
+        KeyspaceParams.DEFAULT_LOCAL_DURABLE_WRITES = false;
+
         SchemaLoader.prepareServer();
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     KeyspaceParams.simple(1),


[2/6] cassandra git commit: Fix CommitLogTest.testDeleteIfNotDirty

Posted by st...@apache.org.
Fix CommitLogTest.testDeleteIfNotDirty

patch by Stefania Alborghetti; reviewed by Branimir Lambov for CASSANDRA-12854


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

Branch: refs/heads/cassandra-3.X
Commit: ec64cdf4c6b731a00bea8795247d865d6371157f
Parents: 817ba03
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Fri Oct 28 17:25:15 2016 +0800
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Mon Nov 7 10:30:15 2016 +0800

----------------------------------------------------------------------
 CHANGES.txt                                              |  1 +
 src/java/org/apache/cassandra/schema/KeyspaceParams.java | 11 ++++++++++-
 .../org/apache/cassandra/db/commitlog/CommitLogTest.java |  5 +++++
 3 files changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec64cdf4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c80e045..51d571b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.10
+ * Fix CommitLogTest.testDeleteIfNotDirty (CASSANDRA-12854)
  * Avoid deadlock due to materialized view lock contention (CASSANDRA-12689)
  * Fix for KeyCacheCqlTest flakiness (CASSANDRA-12801)
  * Include SSTable filename in compacting large row message (CASSANDRA-12384)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec64cdf4/src/java/org/apache/cassandra/schema/KeyspaceParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/KeyspaceParams.java b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
index c0e8916..2ea18ca 100644
--- a/src/java/org/apache/cassandra/schema/KeyspaceParams.java
+++ b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.schema;
 
 import java.util.Map;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
 
 /**
@@ -28,6 +29,14 @@ public final class KeyspaceParams
 {
     public static final boolean DEFAULT_DURABLE_WRITES = true;
 
+    /**
+     * This determines durable writes for the {@link org.apache.cassandra.db.SystemKeyspace#NAME}
+     * and {@link SchemaKeyspace#NAME} keyspaces, the only reason it is not final is for commitlog
+     * unit tests. It should only be changed for testing purposes.
+     */
+    @VisibleForTesting
+    public static boolean DEFAULT_LOCAL_DURABLE_WRITES = true;
+
     public enum Option
     {
         DURABLE_WRITES,
@@ -56,7 +65,7 @@ public final class KeyspaceParams
 
     public static KeyspaceParams local()
     {
-        return new KeyspaceParams(true, ReplicationParams.local());
+        return new KeyspaceParams(DEFAULT_LOCAL_DURABLE_WRITES, ReplicationParams.local());
     }
 
     public static KeyspaceParams simple(int replicationFactor)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec64cdf4/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 9a0ddb8..c4ab6ab 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -97,6 +97,11 @@ public class CommitLogTest
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
+        // Disable durable writes for system keyspaces to prevent system mutations, e.g. sstable_activity,
+        // to end up in CL segments and cause unexpected results in this test wrt counting CL segments,
+        // see CASSANDRA-12854
+        KeyspaceParams.DEFAULT_LOCAL_DURABLE_WRITES = false;
+
         SchemaLoader.prepareServer();
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     KeyspaceParams.simple(1),


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

Posted by st...@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/9ed4face
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/9ed4face
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/9ed4face

Branch: refs/heads/trunk
Commit: 9ed4faceb36b35a04e32d8adc4d7f7c35d6ff74d
Parents: 22a3db0 80843c9
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Mon Nov 7 10:33:43 2016 +0800
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Mon Nov 7 10:33:43 2016 +0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/schema/KeyspaceParams.java | 11 +++++++-
 .../cassandra/db/commitlog/CommitLogTest.java   | 29 ++++++++++++++++++--
 3 files changed, 38 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9ed4face/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index fb16d4b,3157e8e..5eb2850
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -105,8 -97,9 +105,9 @@@
   * Remove pre-startup check for open JMX port (CASSANDRA-12074)
   * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738)
   * Restore resumable hints delivery (CASSANDRA-11960)
 - * Properly report LWT contention (CASSANDRA-12626)
 + * Properly record CAS contention (CASSANDRA-12626)
  Merged from 3.0:
+  * Fix CommitLogTest.testDeleteIfNotDirty (CASSANDRA-12854)
   * Avoid deadlock due to MV lock contention (CASSANDRA-12689)
   * Fix for KeyCacheCqlTest flakiness (CASSANDRA-12801)
   * Include SSTable filename in compacting large row message (CASSANDRA-12384)


[5/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.X

Posted by st...@apache.org.
Merge branch 'cassandra-3.0' 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/80843c98
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/80843c98
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/80843c98

Branch: refs/heads/cassandra-3.X
Commit: 80843c987ea3cf70aabc4def8139caf319efbc7c
Parents: 9d9a1a1 ec64cdf
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Mon Nov 7 10:31:29 2016 +0800
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Mon Nov 7 10:32:23 2016 +0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/schema/KeyspaceParams.java | 11 +++++++-
 .../cassandra/db/commitlog/CommitLogTest.java   | 29 ++++++++++++++++++--
 3 files changed, 38 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/80843c98/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 40aace3,51d571b..3157e8e
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,105 -1,6 +1,106 @@@
 -3.0.10
 +3.10
 + * Fix crossNode value when receiving messages (CASSANDRA-12791)
 + * Don't load MX4J beans twice (CASSANDRA-12869)
 + * Extend native protocol request flags, add versions to SUPPORTED, and introduce ProtocolVersion enum (CASSANDRA-12838)
 + * Set JOINING mode when running pre-join tasks (CASSANDRA-12836)
 + * remove net.mintern.primitive library due to license issue (CASSANDRA-12845)
 + * Properly format IPv6 addresses when logging JMX service URL (CASSANDRA-12454)
 + * Optimize the vnode allocation for single replica per DC (CASSANDRA-12777)
 + * Use non-token restrictions for bounds when token restrictions are overridden (CASSANDRA-12419)
 + * Fix CQLSH auto completion for PER PARTITION LIMIT (CASSANDRA-12803)
 + * Use different build directories for Eclipse and Ant (CASSANDRA-12466)
 + * Avoid potential AttributeError in cqlsh due to no table metadata (CASSANDRA-12815)
 + * Fix RandomReplicationAwareTokenAllocatorTest.testExistingCluster (CASSANDRA-12812)
 + * Upgrade commons-codec to 1.9 (CASSANDRA-12790)
 + * Make the fanout size for LeveledCompactionStrategy to be configurable (CASSANDRA-11550)
 + * Add duration data type (CASSANDRA-11873)
 + * Fix timeout in ReplicationAwareTokenAllocatorTest (CASSANDRA-12784)
 + * Improve sum aggregate functions (CASSANDRA-12417)
 + * Make cassandra.yaml docs for batch_size_*_threshold_in_kb reflect changes in CASSANDRA-10876 (CASSANDRA-12761)
 + * cqlsh fails to format collections when using aliases (CASSANDRA-11534)
 + * Check for hash conflicts in prepared statements (CASSANDRA-12733)
 + * Exit query parsing upon first error (CASSANDRA-12598)
 + * Fix cassandra-stress to use single seed in UUID generation (CASSANDRA-12729)
 + * CQLSSTableWriter does not allow Update statement (CASSANDRA-12450)
 + * Config class uses boxed types but DD exposes primitive types (CASSANDRA-12199)
 + * Add pre- and post-shutdown hooks to Storage Service (CASSANDRA-12461)
 + * Add hint delivery metrics (CASSANDRA-12693)
 + * Remove IndexInfo cache from FileIndexInfoRetriever (CASSANDRA-12731)
 + * ColumnIndex does not reuse buffer (CASSANDRA-12502)
 + * cdc column addition still breaks schema migration tasks (CASSANDRA-12697)
 + * Upgrade metrics-reporter dependencies (CASSANDRA-12089)
 + * Tune compaction thread count via nodetool (CASSANDRA-12248)
 + * Add +=/-= shortcut syntax for update queries (CASSANDRA-12232)
 + * Include repair session IDs in repair start message (CASSANDRA-12532)
 + * Add a blocking task to Index, run before joining the ring (CASSANDRA-12039)
 + * Fix NPE when using CQLSSTableWriter (CASSANDRA-12667)
 + * Support optional backpressure strategies at the coordinator (CASSANDRA-9318)
 + * Make randompartitioner work with new vnode allocation (CASSANDRA-12647)
 + * Fix cassandra-stress graphing (CASSANDRA-12237)
 + * Allow filtering on partition key columns for queries without secondary indexes (CASSANDRA-11031)
 + * Fix Cassandra Stress reporting thread model and precision (CASSANDRA-12585)
 + * Add JMH benchmarks.jar (CASSANDRA-12586)
 + * Add row offset support to SASI (CASSANDRA-11990)
 + * Cleanup uses of AlterTableStatementColumn (CASSANDRA-12567)
 + * Add keep-alive to streaming (CASSANDRA-11841)
 + * Tracing payload is passed through newSession(..) (CASSANDRA-11706)
 + * avoid deleting non existing sstable files and improve related log messages (CASSANDRA-12261)
 + * json/yaml output format for nodetool compactionhistory (CASSANDRA-12486)
 + * Retry all internode messages once after a connection is
 +   closed and reopened (CASSANDRA-12192)
 + * Add support to rebuild from targeted replica (CASSANDRA-9875)
 + * Add sequence distribution type to cassandra stress (CASSANDRA-12490)
 + * "SELECT * FROM foo LIMIT ;" does not error out (CASSANDRA-12154)
 + * Define executeLocally() at the ReadQuery Level (CASSANDRA-12474)
 + * Extend read/write failure messages with a map of replica addresses
 +   to error codes in the v5 native protocol (CASSANDRA-12311)
 + * Fix rebuild of SASI indexes with existing index files (CASSANDRA-12374)
 + * Let DatabaseDescriptor not implicitly startup services (CASSANDRA-9054, 12550)
 + * Fix clustering indexes in presence of static columns in SASI (CASSANDRA-12378)
 + * Fix queries on columns with reversed type on SASI indexes (CASSANDRA-12223)
 + * Added slow query log (CASSANDRA-12403)
 + * Count full coordinated request against timeout (CASSANDRA-12256)
 + * Allow TTL with null value on insert and update (CASSANDRA-12216)
 + * Make decommission operation resumable (CASSANDRA-12008)
 + * Add support to one-way targeted repair (CASSANDRA-9876)
 + * Remove clientutil jar (CASSANDRA-11635)
 + * Fix compaction throughput throttle (CASSANDRA-12366, CASSANDRA-12717)
 + * Delay releasing Memtable memory on flush until PostFlush has finished running (CASSANDRA-12358)
 + * Cassandra stress should dump all setting on startup (CASSANDRA-11914)
 + * Make it possible to compact a given token range (CASSANDRA-10643)
 + * Allow updating DynamicEndpointSnitch properties via JMX (CASSANDRA-12179)
 + * Collect metrics on queries by consistency level (CASSANDRA-7384)
 + * Add support for GROUP BY to SELECT statement (CASSANDRA-10707)
 + * Deprecate memtable_cleanup_threshold and update default for memtable_flush_writers (CASSANDRA-12228)
 + * Upgrade to OHC 0.4.4 (CASSANDRA-12133)
 + * Add version command to cassandra-stress (CASSANDRA-12258)
 + * Create compaction-stress tool (CASSANDRA-11844)
 + * Garbage-collecting compaction operation and schema option (CASSANDRA-7019)
 + * Add beta protocol flag for v5 native protocol (CASSANDRA-12142)
 + * Support filtering on non-PRIMARY KEY columns in the CREATE
 +   MATERIALIZED VIEW statement's WHERE clause (CASSANDRA-10368)
 + * Unify STDOUT and SYSTEMLOG logback format (CASSANDRA-12004)
 + * COPY FROM should raise error for non-existing input files (CASSANDRA-12174)
 + * Faster write path (CASSANDRA-12269)
 + * Option to leave omitted columns in INSERT JSON unset (CASSANDRA-11424)
 + * Support json/yaml output in nodetool tpstats (CASSANDRA-12035)
 + * Expose metrics for successful/failed authentication attempts (CASSANDRA-10635)
 + * Prepend snapshot name with "truncated" or "dropped" when a snapshot
 +   is taken before truncating or dropping a table (CASSANDRA-12178)
 + * Optimize RestrictionSet (CASSANDRA-12153)
 + * cqlsh does not automatically downgrade CQL version (CASSANDRA-12150)
 + * Omit (de)serialization of state variable in UDAs (CASSANDRA-9613)
 + * Create a system table to expose prepared statements (CASSANDRA-8831)
 + * Reuse DataOutputBuffer from ColumnIndex (CASSANDRA-11970)
 + * Remove DatabaseDescriptor dependency from SegmentedFile (CASSANDRA-11580)
 + * Add supplied username to authentication error messages (CASSANDRA-12076)
 + * Remove pre-startup check for open JMX port (CASSANDRA-12074)
 + * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738)
 + * Restore resumable hints delivery (CASSANDRA-11960)
 + * Properly report LWT contention (CASSANDRA-12626)
 +Merged from 3.0:
+  * Fix CommitLogTest.testDeleteIfNotDirty (CASSANDRA-12854)
 - * Avoid deadlock due to materialized view lock contention (CASSANDRA-12689)
 + * Avoid deadlock due to MV lock contention (CASSANDRA-12689)
   * Fix for KeyCacheCqlTest flakiness (CASSANDRA-12801)
   * Include SSTable filename in compacting large row message (CASSANDRA-12384)
   * Fix potential socket leak (CASSANDRA-12329, CASSANDRA-12330)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/80843c98/src/java/org/apache/cassandra/schema/KeyspaceParams.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/schema/KeyspaceParams.java
index 63775fa,2ea18ca..1deaa29
--- a/src/java/org/apache/cassandra/schema/KeyspaceParams.java
+++ b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
@@@ -19,7 -19,7 +19,8 @@@ package org.apache.cassandra.schema
  
  import java.util.Map;
  
+ import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.base.MoreObjects;
  import com.google.common.base.Objects;
  
  /**
@@@ -29,6 -29,14 +30,14 @@@ public final class KeyspaceParam
  {
      public static final boolean DEFAULT_DURABLE_WRITES = true;
  
+     /**
 -     * This determines durable writes for the {@link org.apache.cassandra.db.SystemKeyspace#NAME}
 -     * and {@link SchemaKeyspace#NAME} keyspaces, the only reason it is not final is for commitlog
 -     * unit tests. It should only be changed for testing purposes.
++     * This determines durable writes for the {@link org.apache.cassandra.config.SchemaConstants#SCHEMA_KEYSPACE_NAME}
++     * and {@link org.apache.cassandra.config.SchemaConstants#SYSTEM_KEYSPACE_NAME} keyspaces,
++     * the only reason it is not final is for commitlog unit tests. It should only be changed for testing purposes.
+      */
+     @VisibleForTesting
+     public static boolean DEFAULT_LOCAL_DURABLE_WRITES = true;
+ 
      public enum Option
      {
          DURABLE_WRITES,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/80843c98/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 30dffe5,c4ab6ab..5476d03
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@@ -24,6 -27,6 +24,7 @@@ import java.util.*
  import java.util.concurrent.Callable;
  import java.util.concurrent.ExecutionException;
  import java.util.function.BiConsumer;
++import java.util.stream.Collectors;
  import java.util.zip.CRC32;
  import java.util.zip.Checksum;
  
@@@ -97,8 -95,13 +98,13 @@@ public class CommitLogTes
      }
  
      @BeforeClass
 -    public static void defineSchema() throws ConfigurationException
 +    public static void beforeClass() throws ConfigurationException
      {
+         // Disable durable writes for system keyspaces to prevent system mutations, e.g. sstable_activity,
+         // to end up in CL segments and cause unexpected results in this test wrt counting CL segments,
+         // see CASSANDRA-12854
+         KeyspaceParams.DEFAULT_LOCAL_DURABLE_WRITES = false;
+ 
          SchemaLoader.prepareServer();
          SchemaLoader.createKeyspace(KEYSPACE1,
                                      KeyspaceParams.simple(1),
@@@ -293,16 -277,17 +299,35 @@@
          CommitLog.instance.add(rm2);
          CommitLog.instance.add(rm2);
  
-         assertEquals(3, CommitLog.instance.segmentManager.getActiveSegments().size());
 -        assert CommitLog.instance.activeSegments() == 3 : "Expecting 3 segments, got " + CommitLog.instance.activeSegments();
++        Collection<CommitLogSegment> segments = CommitLog.instance.segmentManager.getActiveSegments();
+ 
++        assertEquals(String.format("Expected 3 segments but got %d (%s)", segments.size(), getDirtyCFIds(segments)),
++                     3,
++                     segments.size());
  
          // "Flush" second cf: The first segment should be deleted since we
          // didn't write anything on cf1 since last flush (and we flush cf2)
  
          UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
 -        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
 +        CommitLog.instance.discardCompletedSegments(cfid2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
 +
++        segments = CommitLog.instance.segmentManager.getActiveSegments();
+ 
          // Assert we still have both our segment
-         assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
 -        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
++        assertEquals(String.format("Expected 1 segment but got %d (%s)", segments.size(), getDirtyCFIds(segments)),
++                     1,
++                     segments.size());
++    }
++
++    private String getDirtyCFIds(Collection<CommitLogSegment> segments)
++    {
++        return "Dirty cfIds: <"
++               + String.join(", ", segments.stream()
++                                           .map(CommitLogSegment::getDirtyCFIDs)
++                                           .flatMap(uuids -> uuids.stream())
++                                           .distinct()
++                                           .map(uuid -> uuid.toString()).collect(Collectors.toList()))
++               + ">";
      }
  
      private static int getMaxRecordDataSize(String keyspace, ByteBuffer key, String cfName, String colName)