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 2017/04/25 16:03:54 UTC

[1/3] cassandra git commit: Fix compaction-stress by using daemonInitialization

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.11 1a37992e9 -> 2369faab7
  refs/heads/trunk 2381be314 -> d45cd7e65


Fix compaction-stress by using daemonInitialization

Patch by Jay Zhuang; reviewed by tjake for CASSANDRA-13188


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

Branch: refs/heads/cassandra-3.11
Commit: 2369faab7959d57c8f6bc1f324de47c5aeaf19b9
Parents: 1a37992
Author: Jay Zhuang <ja...@yahoo.com>
Authored: Tue Apr 25 09:21:00 2017 -0400
Committer: T Jake Luciani <ja...@apache.org>
Committed: Tue Apr 25 09:25:44 2017 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +-
 build.xml                                       |  2 +-
 test/resources/blogpost.yaml                    | 69 ++++++++++++++++++++
 .../cassandra/tools/CompactionStressTest.java   | 60 +++++++++++++++++
 .../io/sstable/StressCQLSSTableWriter.java      |  4 +-
 .../cassandra/stress/CompactionStress.java      |  2 +-
 6 files changed, 135 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c0aceb3..1690e31 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.0
+ * Fix compaction-stress by using daemonInitialization (CASSANDRA-13188)
  * V5 protocol flags decoding broken (CASSANDRA-13443)
  * Use write lock not read lock for removing sstables from compaction strategies. (CASSANDRA-13422)
  * Use corePoolSize equal to maxPoolSize in JMXEnabledThreadPoolExecutors (CASSANDRA-13329)
@@ -101,7 +102,6 @@ Merged from 2.1:
  * Fix Thread Leak in OutboundTcpConnection (CASSANDRA-13204)
  * Coalescing strategy can enter infinite loop (CASSANDRA-13159)
 
-
 3.10
  * Fix secondary index queries regression (CASSANDRA-13013)
  * Add duration type to the protocol V5 (CASSANDRA-12850)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 1af8958..bf1577e 100644
--- a/build.xml
+++ b/build.xml
@@ -1719,7 +1719,7 @@
     ]]>
   </scriptdef>
 
-  <target name="test" depends="build-test" description="Parallel Test Runner">
+  <target name="test" depends="build-test,stress-build" description="Parallel Test Runner">
     <path id="all-test-classes-path">
       <fileset dir="${test.unit.src}" includes="**/${test.name}.java" />
     </path>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/test/resources/blogpost.yaml
----------------------------------------------------------------------
diff --git a/test/resources/blogpost.yaml b/test/resources/blogpost.yaml
new file mode 100644
index 0000000..13987e0
--- /dev/null
+++ b/test/resources/blogpost.yaml
@@ -0,0 +1,69 @@
+# Copied from https://gist.github.com/tjake/8995058fed11d9921e31
+### DML ###
+
+# Keyspace Name
+keyspace: stresscql
+
+# The CQL for creating a keyspace (optional if it already exists)
+keyspace_definition: |
+  CREATE KEYSPACE stresscql WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};
+
+# Table name
+table: blogposts
+
+# The CQL for creating a table you wish to stress (optional if it already exists)
+table_definition: |
+  CREATE TABLE blogposts (
+        domain text,
+        published_date timeuuid,
+        url text,
+        author text,
+        title text,
+        body text,
+        PRIMARY KEY(domain, published_date)
+  ) WITH CLUSTERING ORDER BY (published_date DESC)
+    AND compaction = { 'class':'LeveledCompactionStrategy' }
+    AND comment='A table to hold blog posts'
+
+### Column Distribution Specifications ###
+
+columnspec:
+  - name: domain
+    size: gaussian(5..100)       #domain names are relatively short
+    population: uniform(1..10M)  #10M possible domains to pick from
+
+  - name: published_date
+    cluster: fixed(1000)         #under each domain we will have max 1000 posts
+
+  - name: url
+    size: uniform(30..300)
+
+  - name: title                  #titles shouldn't go beyond 200 chars
+    size: gaussian(10..200)
+
+  - name: author
+    size: uniform(5..20)         #author names should be short
+
+  - name: body
+    size: gaussian(100..5000)    #the body of the blog post can be long
+
+### Batch Ratio Distribution Specifications ###
+
+insert:
+  partitions: fixed(1)            # Our partition key is the domain so only insert one per batch
+
+  select:    fixed(1)/1000        # We have 1000 posts per domain so 1/1000 will allow 1 post per batch
+
+  batchtype: UNLOGGED             # Unlogged batches
+
+
+#
+# A list of queries you wish to run against the schema
+#
+queries:
+   singlepost:
+      cql: select * from blogposts where domain = ? LIMIT 1
+      fields: samerow
+   timeline:
+      cql: select url, title, published_date from blogposts where domain = ? LIMIT 10
+      fields: samerow
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/test/unit/org/apache/cassandra/tools/CompactionStressTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/CompactionStressTest.java b/test/unit/org/apache/cassandra/tools/CompactionStressTest.java
new file mode 100644
index 0000000..c8b0b97
--- /dev/null
+++ b/test/unit/org/apache/cassandra/tools/CompactionStressTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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 org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+
+@RunWith(OrderedJUnit4ClassRunner.class)
+public class CompactionStressTest extends ToolsTester
+{
+    @Test
+    public void testNoArgs()
+    {
+        runTool(0, "org.apache.cassandra.stress.CompactionStress");
+    }
+
+    @Test
+    public void testWriteAndCompact()
+    {
+        ClassLoader classLoader = getClass().getClassLoader();
+        File file = new File(classLoader.getResource("blogpost.yaml").getFile());
+        String profileFile = file.getAbsolutePath();
+
+        runTool(0,
+                "org.apache.cassandra.stress.CompactionStress",
+                "write",
+                "-d", "build/test/cassandra",
+                "-g", "0",
+                "-p", profileFile,
+                "-t", "4");
+
+        runTool(0,
+                "org.apache.cassandra.stress.CompactionStress",
+                "compact",
+                "-d", "build/test/cassandra",
+                "-p", profileFile,
+                "-t", "4");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
index 41a0d6f..56f0297 100644
--- a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
+++ b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
@@ -609,7 +609,9 @@ public class StressCQLSSTableWriter implements Closeable
             KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
 
             CFMetaData cfMetaData = ksm.tables.getNullable(schemaStatement.columnFamily());
-            assert cfMetaData == null;
+
+            if (cfMetaData != null)
+                return Schema.instance.getColumnFamilyStoreInstance(cfMetaData.cfId);
 
             CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(ksm.types).statement;
             statement.validate(ClientState.forInternalCalls());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
index 4180524..1860fef 100644
--- a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
+++ b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
@@ -74,7 +74,7 @@ public abstract class CompactionStress implements Runnable
 
     static
     {
-        DatabaseDescriptor.toolInitialization();
+        DatabaseDescriptor.daemonInitialization();
     }
 
     List<File> getDataDirectories()


[2/3] cassandra git commit: Fix compaction-stress by using daemonInitialization

Posted by ja...@apache.org.
Fix compaction-stress by using daemonInitialization

Patch by Jay Zhuang; reviewed by tjake for CASSANDRA-13188


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

Branch: refs/heads/trunk
Commit: 2369faab7959d57c8f6bc1f324de47c5aeaf19b9
Parents: 1a37992
Author: Jay Zhuang <ja...@yahoo.com>
Authored: Tue Apr 25 09:21:00 2017 -0400
Committer: T Jake Luciani <ja...@apache.org>
Committed: Tue Apr 25 09:25:44 2017 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +-
 build.xml                                       |  2 +-
 test/resources/blogpost.yaml                    | 69 ++++++++++++++++++++
 .../cassandra/tools/CompactionStressTest.java   | 60 +++++++++++++++++
 .../io/sstable/StressCQLSSTableWriter.java      |  4 +-
 .../cassandra/stress/CompactionStress.java      |  2 +-
 6 files changed, 135 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c0aceb3..1690e31 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.0
+ * Fix compaction-stress by using daemonInitialization (CASSANDRA-13188)
  * V5 protocol flags decoding broken (CASSANDRA-13443)
  * Use write lock not read lock for removing sstables from compaction strategies. (CASSANDRA-13422)
  * Use corePoolSize equal to maxPoolSize in JMXEnabledThreadPoolExecutors (CASSANDRA-13329)
@@ -101,7 +102,6 @@ Merged from 2.1:
  * Fix Thread Leak in OutboundTcpConnection (CASSANDRA-13204)
  * Coalescing strategy can enter infinite loop (CASSANDRA-13159)
 
-
 3.10
  * Fix secondary index queries regression (CASSANDRA-13013)
  * Add duration type to the protocol V5 (CASSANDRA-12850)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 1af8958..bf1577e 100644
--- a/build.xml
+++ b/build.xml
@@ -1719,7 +1719,7 @@
     ]]>
   </scriptdef>
 
-  <target name="test" depends="build-test" description="Parallel Test Runner">
+  <target name="test" depends="build-test,stress-build" description="Parallel Test Runner">
     <path id="all-test-classes-path">
       <fileset dir="${test.unit.src}" includes="**/${test.name}.java" />
     </path>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/test/resources/blogpost.yaml
----------------------------------------------------------------------
diff --git a/test/resources/blogpost.yaml b/test/resources/blogpost.yaml
new file mode 100644
index 0000000..13987e0
--- /dev/null
+++ b/test/resources/blogpost.yaml
@@ -0,0 +1,69 @@
+# Copied from https://gist.github.com/tjake/8995058fed11d9921e31
+### DML ###
+
+# Keyspace Name
+keyspace: stresscql
+
+# The CQL for creating a keyspace (optional if it already exists)
+keyspace_definition: |
+  CREATE KEYSPACE stresscql WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};
+
+# Table name
+table: blogposts
+
+# The CQL for creating a table you wish to stress (optional if it already exists)
+table_definition: |
+  CREATE TABLE blogposts (
+        domain text,
+        published_date timeuuid,
+        url text,
+        author text,
+        title text,
+        body text,
+        PRIMARY KEY(domain, published_date)
+  ) WITH CLUSTERING ORDER BY (published_date DESC)
+    AND compaction = { 'class':'LeveledCompactionStrategy' }
+    AND comment='A table to hold blog posts'
+
+### Column Distribution Specifications ###
+
+columnspec:
+  - name: domain
+    size: gaussian(5..100)       #domain names are relatively short
+    population: uniform(1..10M)  #10M possible domains to pick from
+
+  - name: published_date
+    cluster: fixed(1000)         #under each domain we will have max 1000 posts
+
+  - name: url
+    size: uniform(30..300)
+
+  - name: title                  #titles shouldn't go beyond 200 chars
+    size: gaussian(10..200)
+
+  - name: author
+    size: uniform(5..20)         #author names should be short
+
+  - name: body
+    size: gaussian(100..5000)    #the body of the blog post can be long
+
+### Batch Ratio Distribution Specifications ###
+
+insert:
+  partitions: fixed(1)            # Our partition key is the domain so only insert one per batch
+
+  select:    fixed(1)/1000        # We have 1000 posts per domain so 1/1000 will allow 1 post per batch
+
+  batchtype: UNLOGGED             # Unlogged batches
+
+
+#
+# A list of queries you wish to run against the schema
+#
+queries:
+   singlepost:
+      cql: select * from blogposts where domain = ? LIMIT 1
+      fields: samerow
+   timeline:
+      cql: select url, title, published_date from blogposts where domain = ? LIMIT 10
+      fields: samerow
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/test/unit/org/apache/cassandra/tools/CompactionStressTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/CompactionStressTest.java b/test/unit/org/apache/cassandra/tools/CompactionStressTest.java
new file mode 100644
index 0000000..c8b0b97
--- /dev/null
+++ b/test/unit/org/apache/cassandra/tools/CompactionStressTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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 org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+
+@RunWith(OrderedJUnit4ClassRunner.class)
+public class CompactionStressTest extends ToolsTester
+{
+    @Test
+    public void testNoArgs()
+    {
+        runTool(0, "org.apache.cassandra.stress.CompactionStress");
+    }
+
+    @Test
+    public void testWriteAndCompact()
+    {
+        ClassLoader classLoader = getClass().getClassLoader();
+        File file = new File(classLoader.getResource("blogpost.yaml").getFile());
+        String profileFile = file.getAbsolutePath();
+
+        runTool(0,
+                "org.apache.cassandra.stress.CompactionStress",
+                "write",
+                "-d", "build/test/cassandra",
+                "-g", "0",
+                "-p", profileFile,
+                "-t", "4");
+
+        runTool(0,
+                "org.apache.cassandra.stress.CompactionStress",
+                "compact",
+                "-d", "build/test/cassandra",
+                "-p", profileFile,
+                "-t", "4");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
index 41a0d6f..56f0297 100644
--- a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
+++ b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
@@ -609,7 +609,9 @@ public class StressCQLSSTableWriter implements Closeable
             KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
 
             CFMetaData cfMetaData = ksm.tables.getNullable(schemaStatement.columnFamily());
-            assert cfMetaData == null;
+
+            if (cfMetaData != null)
+                return Schema.instance.getColumnFamilyStoreInstance(cfMetaData.cfId);
 
             CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(ksm.types).statement;
             statement.validate(ClientState.forInternalCalls());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2369faab/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
index 4180524..1860fef 100644
--- a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
+++ b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
@@ -74,7 +74,7 @@ public abstract class CompactionStress implements Runnable
 
     static
     {
-        DatabaseDescriptor.toolInitialization();
+        DatabaseDescriptor.daemonInitialization();
     }
 
     List<File> getDataDirectories()


[3/3] cassandra git commit: Merge branch 'cassandra-3.11' into trunk

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


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

Branch: refs/heads/trunk
Commit: d45cd7e6587098adc53b9fdaa483546eff9c2345
Parents: 2381be3 2369faa
Author: T Jake Luciani <ja...@apache.org>
Authored: Tue Apr 25 09:30:25 2017 -0400
Committer: T Jake Luciani <ja...@apache.org>
Committed: Tue Apr 25 09:30:25 2017 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +-
 build.xml                                       |  2 +-
 test/resources/blogpost.yaml                    | 69 ++++++++++++++++++++
 .../cassandra/tools/CompactionStressTest.java   | 60 +++++++++++++++++
 .../io/sstable/StressCQLSSTableWriter.java      | 10 +--
 .../cassandra/stress/CompactionStress.java      |  2 +-
 6 files changed, 138 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d45cd7e6/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 9c933e1,1690e31..7ead5ff
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,64 -1,5 +1,65 @@@
 +4.0
 + * Start compaction when incremental repair finishes (CASSANDRA-13454)
 + * Add repair streaming preview (CASSANDRA-13257)
 + * Cleanup isIncremental/repairedAt usage (CASSANDRA-13430)
 + * Change protocol to allow sending key space independent of query string (CASSANDRA-10145)
 + * Make gc_log and gc_warn settable at runtime (CASSANDRA-12661)
 + * Take number of files in L0 in account when estimating remaining compaction tasks (CASSANDRA-13354)
 + * Skip building views during base table streams on range movements (CASSANDRA-13065)
 + * Improve error messages for +/- operations on maps and tuples (CASSANDRA-13197)
 + * Remove deprecated repair JMX APIs (CASSANDRA-11530)
 + * Fix version check to enable streaming keep-alive (CASSANDRA-12929)
 + * Make it possible to monitor an ideal consistency level separate from actual consistency level (CASSANDRA-13289)
 + * Outbound TCP connections ignore internode authenticator (CASSANDRA-13324)
 + * Upgrade junit from 4.6 to 4.12 (CASSANDRA-13360)
 + * Cleanup ParentRepairSession after repairs (CASSANDRA-13359)
 + * Incremental repair not streaming correct sstables (CASSANDRA-13328)
 + * Upgrade the jna version to 4.3.0 (CASSANDRA-13300)
 + * Add the currentTimestamp, currentDate, currentTime and currentTimeUUID functions (CASSANDRA-13132)
 + * Remove config option index_interval (CASSANDRA-10671)
 + * Reduce lock contention for collection types and serializers (CASSANDRA-13271)
 + * Make it possible to override MessagingService.Verb ids (CASSANDRA-13283)
 + * Avoid synchronized on prepareForRepair in ActiveRepairService (CASSANDRA-9292)
 + * Adds the ability to use uncompressed chunks in compressed files (CASSANDRA-10520)
 + * Don't flush sstables when streaming for incremental repair (CASSANDRA-13226)
 + * Remove unused method (CASSANDRA-13227)
 + * Fix minor bugs related to #9143 (CASSANDRA-13217)
 + * Output warning if user increases RF (CASSANDRA-13079)
 + * Remove pre-3.0 streaming compatibility code for 4.0 (CASSANDRA-13081)
 + * Add support for + and - operations on dates (CASSANDRA-11936)
 + * Fix consistency of incrementally repaired data (CASSANDRA-9143)
 + * Increase commitlog version (CASSANDRA-13161)
 + * Make TableMetadata immutable, optimize Schema (CASSANDRA-9425)
 + * Refactor ColumnCondition (CASSANDRA-12981)
 + * Parallelize streaming of different keyspaces (CASSANDRA-4663)
 + * Improved compactions metrics (CASSANDRA-13015)
 + * Speed-up start-up sequence by avoiding un-needed flushes (CASSANDRA-13031)
 + * Use Caffeine (W-TinyLFU) for on-heap caches (CASSANDRA-10855)
 + * Thrift removal (CASSANDRA-11115)
 + * Remove pre-3.0 compatibility code for 4.0 (CASSANDRA-12716)
 + * Add column definition kind to dropped columns in schema (CASSANDRA-12705)
 + * Add (automate) Nodetool Documentation (CASSANDRA-12672)
 + * Update bundled cqlsh python driver to 3.7.0 (CASSANDRA-12736)
 + * Reject invalid replication settings when creating or altering a keyspace (CASSANDRA-12681)
 + * Clean up the SSTableReader#getScanner API wrt removal of RateLimiter (CASSANDRA-12422)
 + * Use new token allocation for non bootstrap case as well (CASSANDRA-13080)
 + * Avoid byte-array copy when key cache is disabled (CASSANDRA-13084)
 + * Require forceful decommission if number of nodes is less than replication factor (CASSANDRA-12510)
 + * Allow IN restrictions on column families with collections (CASSANDRA-12654)
 + * Log message size in trace message in OutboundTcpConnection (CASSANDRA-13028)
 + * Add timeUnit Days for cassandra-stress (CASSANDRA-13029)
 + * Add mutation size and batch metrics (CASSANDRA-12649)
 + * Add method to get size of endpoints to TokenMetadata (CASSANDRA-12999)
 + * Expose time spent waiting in thread pool queue (CASSANDRA-8398)
 + * Conditionally update index built status to avoid unnecessary flushes (CASSANDRA-12969)
 + * cqlsh auto completion: refactor definition of compaction strategy options (CASSANDRA-12946)
 + * Add support for arithmetic operators (CASSANDRA-11935)
 + * Add histogram for delay to deliver hints (CASSANDRA-13234)
 + * Fix cqlsh automatic protocol downgrade regression (CASSANDRA-13307)
 +
 +
  3.11.0
+  * Fix compaction-stress by using daemonInitialization (CASSANDRA-13188)
   * V5 protocol flags decoding broken (CASSANDRA-13443)
   * Use write lock not read lock for removing sstables from compaction strategies. (CASSANDRA-13422)
   * Use corePoolSize equal to maxPoolSize in JMXEnabledThreadPoolExecutors (CASSANDRA-13329)
@@@ -158,10 -100,8 +159,9 @@@ Merged from 2.1
   * Log stacktrace of uncaught exceptions (CASSANDRA-13108)
   * Use portable stderr for java error in startup (CASSANDRA-13211)
   * Fix Thread Leak in OutboundTcpConnection (CASSANDRA-13204)
 + * Upgrade netty version to fix memory leak with client encryption (CASSANDRA-13114)
   * Coalescing strategy can enter infinite loop (CASSANDRA-13159)
  
- 
  3.10
   * Fix secondary index queries regression (CASSANDRA-13013)
   * Add duration type to the protocol V5 (CASSANDRA-12850)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d45cd7e6/build.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d45cd7e6/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
----------------------------------------------------------------------
diff --cc tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
index 89fd5f9,56f0297..ae72f21
--- a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
+++ b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
@@@ -610,26 -606,31 +610,28 @@@ public class StressCQLSSTableWriter imp
  
              createTypes(keyspace, typeStatements);
  
 -            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
 -
 -            CFMetaData cfMetaData = ksm.tables.getNullable(schemaStatement.columnFamily());
 +            KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace);
  
-             assert ksm.tables.getNullable(schemaStatement.columnFamily()) == null;
 -            if (cfMetaData != null)
 -                return Schema.instance.getColumnFamilyStoreInstance(cfMetaData.cfId);
++            TableMetadata tableMetadata = ksm.tables.getNullable(schemaStatement.columnFamily());
++            if (tableMetadata != null)
++                return Schema.instance.getColumnFamilyStoreInstance(tableMetadata.id);
  
              CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(ksm.types).statement;
              statement.validate(ClientState.forInternalCalls());
  
 -            //Build metatdata with a portable cfId
 -            cfMetaData = statement.metadataBuilder()
 -                                  .withId(CFMetaData.generateLegacyCfId(keyspace, statement.columnFamily()))
 -                                  .build()
 -                                  .params(statement.params());
 +            //Build metadata with a portable tableId
-             TableMetadata tableMetadata = statement.builder()
-                                                    .id(deterministicId(statement.keyspace(), statement.columnFamily()))
-                                                    .build();
++            tableMetadata = statement.builder()
++                                     .id(deterministicId(statement.keyspace(), statement.columnFamily()))
++                                     .build();
  
              Keyspace.setInitialized();
 -            Directories directories = new Directories(cfMetaData, directoryList.stream().map(Directories.DataDirectory::new).collect(Collectors.toList()));
 +            Directories directories = new Directories(tableMetadata, directoryList.stream().map(Directories.DataDirectory::new).collect(Collectors.toList()));
  
              Keyspace ks = Keyspace.openWithoutSSTables(keyspace);
 -            ColumnFamilyStore cfs =  ColumnFamilyStore.createColumnFamilyStore(ks, cfMetaData.cfName, cfMetaData, directories, false, false, true);
 +            ColumnFamilyStore cfs =  ColumnFamilyStore.createColumnFamilyStore(ks, tableMetadata.name, TableMetadataRef.forOfflineTools(tableMetadata), directories, false, false, true);
  
              ks.initCfCustom(cfs);
 -            Schema.instance.load(cfs.metadata);
 -            Schema.instance.setKeyspaceMetadata(ksm.withSwapped(ksm.tables.with(cfs.metadata)));
 +            Schema.instance.load(ksm.withSwapped(ksm.tables.with(cfs.metadata())));
  
              return cfs;
          }