You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2016/07/18 15:43:16 UTC

[01/50] cassandra git commit: Apply Thrift ordering to legacy remote response where needed [Forced Update!]

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.8 00d50303f -> 2aa76632d (forced update)


Apply Thrift ordering to legacy remote response where needed

Patch by Tyler Hobbs; reviewed by Sylvain Lebresne for CASSANDRA-12123


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

Branch: refs/heads/cassandra-3.8
Commit: 2217695166a61f576b36993b36a6bde8c8952fde
Parents: a9a6e5f
Author: Tyler Hobbs <ty...@gmail.com>
Authored: Tue Jul 5 11:45:45 2016 -0500
Committer: Tyler Hobbs <ty...@gmail.com>
Committed: Tue Jul 5 11:45:45 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                        | 3 +++
 src/java/org/apache/cassandra/db/ReadResponse.java | 8 +++++---
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/22176951/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2df77e1..7f8a3a1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,7 @@
 3.0.9
+ * Fix column ordering of results with static columns for Thrift requests in
+   a mixed 2.x/3.x cluster, also fix potential non-resolved duplication of
+   those static columns in query results (CASSANDRA-12123)
  * Avoid digest mismatch with empty but static rows (CASSANDRA-12090)
  * Fix EOF exception when altering column type (CASSANDRA-11820)
 Merged from 2.2:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/22176951/src/java/org/apache/cassandra/db/ReadResponse.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadResponse.java b/src/java/org/apache/cassandra/db/ReadResponse.java
index a618aa5..8bd1be6 100644
--- a/src/java/org/apache/cassandra/db/ReadResponse.java
+++ b/src/java/org/apache/cassandra/db/ReadResponse.java
@@ -38,6 +38,7 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.thrift.ThriftResultsMerger;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -267,15 +268,16 @@ public abstract class ReadResponse
                 {
                     ImmutableBTreePartition partition = toReturn.get(idx++);
 
-
                     ClusteringIndexFilter filter = command.clusteringIndexFilter(partition.partitionKey());
 
                     // Pre-3.0, we didn't have a way to express exclusivity for non-composite comparators, so all slices were
                     // inclusive on both ends. If we have exclusive slice ends, we need to filter the results here.
                     if (!command.metadata().isCompound())
-                        return filter.filter(partition.sliceableUnfilteredIterator(command.columnFilter(), filter.isReversed()));
+                        return ThriftResultsMerger.maybeWrap(
+                                filter.filter(partition.sliceableUnfilteredIterator(command.columnFilter(), filter.isReversed())), command.nowInSec());
 
-                    return partition.unfilteredIterator(command.columnFilter(), Slices.ALL, filter.isReversed());
+                    return ThriftResultsMerger.maybeWrap(
+                            partition.unfilteredIterator(command.columnFilter(), Slices.ALL, filter.isReversed()), command.nowInSec());
                 }
             };
         }


[47/50] cassandra git commit: Ninja: add assertions for missing partition keys when loading table

Posted by al...@apache.org.
Ninja: add assertions for missing partition keys when loading table

Done to help debug test failures like CASSANDRA-12213


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

Branch: refs/heads/cassandra-3.8
Commit: b0ab12f4d52f78d30bb514d7a5ebb49c34a399b4
Parents: 2764e85
Author: Tyler Hobbs <ty...@gmail.com>
Authored: Fri Jul 15 12:35:59 2016 -0500
Committer: Tyler Hobbs <ty...@gmail.com>
Committed: Fri Jul 15 12:35:59 2016 -0500

----------------------------------------------------------------------
 src/java/org/apache/cassandra/config/CFMetaData.java     | 3 +++
 src/java/org/apache/cassandra/schema/SchemaKeyspace.java | 6 ++++++
 2 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b0ab12f4/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index 4708033..4de4f7b 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -286,6 +286,9 @@ public final class CFMetaData
         // A compact table should always have a clustering
         assert isCQLTable() || !clusteringColumns.isEmpty() : String.format("For table %s.%s, isDense=%b, isCompound=%b, clustering=%s", ksName, cfName, isDense, isCompound, clusteringColumns);
 
+        // All tables should have a partition key
+        assert !partitionKeyColumns.isEmpty() : String.format("Have no partition keys for table %s.%s", ksName, cfName);
+
         this.partitionKeyColumns = partitionKeyColumns;
         this.clusteringColumns = clusteringColumns;
         this.partitionColumns = partitionColumns;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b0ab12f4/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index dd0bb46..8e3961e 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -951,6 +951,12 @@ public final class SchemaKeyspace
         boolean isCompound = flags.contains(CFMetaData.Flag.COMPOUND);
 
         List<ColumnDefinition> columns = fetchColumns(keyspaceName, tableName, types);
+        if (!columns.stream().anyMatch(ColumnDefinition::isPartitionKey))
+        {
+            String msg = String.format("Table %s.%s did not have any partition key columns in the schema tables", keyspaceName, tableName);
+            throw new AssertionError(msg);
+        }
+
         Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns = fetchDroppedColumns(keyspaceName, tableName);
         Indexes indexes = fetchIndexes(keyspaceName, tableName);
         Triggers triggers = fetchTriggers(keyspaceName, tableName);


[09/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 262cd38b0179c8ff2b8a60509f10d669c21c8312
Parents: 574e8df 73c7167
Author: Marcus Eriksson <ma...@apache.org>
Authored: Wed Jul 6 10:11:29 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Wed Jul 6 10:11:29 2016 +0200

----------------------------------------------------------------------
 .../cassandra/db/compaction/DateTieredCompactionStrategyTest.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------



[17/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 59ee46e55a15775a49edde86de81b9b79875731d
Parents: 5ad1763 778f2a4
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Jul 6 12:34:22 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Jul 6 12:34:22 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/streaming/ConnectionHandler.java  |  8 ++--
 .../cassandra/streaming/StreamReceiveTask.java  | 50 +++++++++++++++-----
 .../cassandra/streaming/StreamSession.java      | 17 +++++--
 .../streaming/StreamingTransferTest.java        | 30 ++++++++++--
 5 files changed, 83 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/59ee46e5/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 2861cf7,8118de1..d459e34
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,7 -1,5 +1,8 @@@
 -3.0.9
 +3.9
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
+  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
   * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
   * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
   * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59ee46e5/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59ee46e5/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59ee46e5/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------


[15/50] cassandra git commit: Improve streaming synchronization and fault tolerance

Posted by al...@apache.org.
Improve streaming synchronization and fault tolerance

Patch by Paulo Motta; Reviewed by yukim for CASSANDRA-11414


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

Branch: refs/heads/cassandra-3.8
Commit: 00e7ecf1394f8704e2f13369f7950e129459ce2c
Parents: 43c741e
Author: Paulo Motta <pa...@gmail.com>
Authored: Wed Jul 6 12:16:16 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Jul 6 12:32:39 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                                  | 1 +
 .../org/apache/cassandra/streaming/ConnectionHandler.java    | 8 +++-----
 .../org/apache/cassandra/streaming/StreamReceiveTask.java    | 2 --
 3 files changed, 4 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/00e7ecf1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index bfd8aa2..7d62f97 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.8
+ * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
  * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 Merged from 2.1:
  * Don't write shadowed range tombstone (CASSANDRA-12030)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/00e7ecf1/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/ConnectionHandler.java b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
index c497a39..364435e 100644
--- a/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
+++ b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
@@ -233,6 +233,9 @@ public class ConnectionHandler
 
         protected void signalCloseDone()
         {
+            if (closeFuture == null)
+                close();
+
             closeFuture.get().set(null);
 
             // We can now close the socket
@@ -294,11 +297,6 @@ public class ConnectionHandler
                     }
                 }
             }
-            catch (SocketException e)
-            {
-                // socket is closed
-                close();
-            }
             catch (Throwable t)
             {
                 JVMStabilityInspector.inspectThrowable(t);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/00e7ecf1/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index 6911ec6..b342edc 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -18,8 +18,6 @@
 package org.apache.cassandra.streaming;
 
 import java.io.File;
-import java.io.IOError;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;


[49/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 136f6b1c740184da6377ffc3afc9d3f03ba1a14e
Parents: b0ab12f 557c596
Author: Tyler Hobbs <ty...@gmail.com>
Authored: Fri Jul 15 17:56:32 2016 -0500
Committer: Tyler Hobbs <ty...@gmail.com>
Committed: Fri Jul 15 17:56:32 2016 -0500

----------------------------------------------------------------------
 src/java/org/apache/cassandra/db/ReadResponse.java | 15 ++++++++++-----
 1 file changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/136f6b1c/src/java/org/apache/cassandra/db/ReadResponse.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ReadResponse.java
index 140652d,2304cb4..05ebd00
--- a/src/java/org/apache/cassandra/db/ReadResponse.java
+++ b/src/java/org/apache/cassandra/db/ReadResponse.java
@@@ -282,12 -282,17 +282,17 @@@ public abstract class ReadRespons
  
                      // Pre-3.0, we didn't have a way to express exclusivity for non-composite comparators, so all slices were
                      // inclusive on both ends. If we have exclusive slice ends, we need to filter the results here.
+                     UnfilteredRowIterator iterator;
                      if (!command.metadata().isCompound())
-                         return ThriftResultsMerger.maybeWrap(
-                                 partition.unfilteredIterator(command.columnFilter(), filter.getSlices(command.metadata()), filter.isReversed()), command.nowInSec());
- 
-                     return ThriftResultsMerger.maybeWrap(
-                             partition.unfilteredIterator(command.columnFilter(), Slices.ALL, filter.isReversed()), command.nowInSec());
 -                        iterator = filter.filter(partition.sliceableUnfilteredIterator(command.columnFilter(), filter.isReversed()));
++                        iterator = partition.unfilteredIterator(command.columnFilter(), filter.getSlices(command.metadata()), filter.isReversed());
+                     else
+                         iterator = partition.unfilteredIterator(command.columnFilter(), Slices.ALL, filter.isReversed());
+ 
+                     // Wrap results with a ThriftResultMerger only if they're intended for the thrift command.
+                     if (command.isForThrift())
+                         return ThriftResultsMerger.maybeWrap(iterator, command.nowInSec());
+                     else
+                         return iterator;
                  }
              };
          }


[31/50] cassandra git commit: Fix upgrading sparse tables that are incorrectly marked as dense

Posted by al...@apache.org.
Fix upgrading sparse tables that are incorrectly marked as dense

patch by Aleksey Yeschenko; reviewed by Sylvain Lebresne for
CASSANDRA-11315


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

Branch: refs/heads/cassandra-3.8
Commit: f37b6f145d3db2b14de38b2c75f390f7a3113c85
Parents: 2fa44cd
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed Jun 29 23:15:45 2016 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Mon Jul 11 14:29:55 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/schema/LegacySchemaMigrator.java  | 81 ++++++++++++++------
 2 files changed, 58 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f37b6f14/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0e483f1..f2d11b9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)
  * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
 2.2.8
  * Use dedicated thread for JMX notifications (CASSANDRA-12146)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f37b6f14/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java b/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
index 924bd7a..b6d8d2b 100644
--- a/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
+++ b/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
@@ -284,11 +284,26 @@ public final class LegacySchemaMigrator
         AbstractType<?> subComparator = tableRow.has("subcomparator") ? TypeParser.parse(tableRow.getString("subcomparator")) : null;
 
         boolean isSuper = "super".equals(tableRow.getString("type").toLowerCase());
-        boolean isDense = tableRow.has("is_dense")
-                        ? tableRow.getBoolean("is_dense")
-                        : calculateIsDense(rawComparator, columnRows);
         boolean isCompound = rawComparator instanceof CompositeType;
 
+        /*
+         * Determine whether or not the table is *really* dense
+         * We cannot trust is_dense value of true (see CASSANDRA-11502, that fixed the issue for 2.2 only, and not retroactively),
+         * but we can trust is_dense value of false.
+         */
+        Boolean rawIsDense = tableRow.has("is_dense") ? tableRow.getBoolean("is_dense") : null;
+        boolean isDense;
+        if (rawIsDense != null && !rawIsDense)
+            isDense = false;
+        else
+            isDense = calculateIsDense(rawComparator, columnRows);
+
+        // now, if switched to sparse, remove redundant compact_value column and the last clustering column,
+        // directly copying CASSANDRA-11502 logic. See CASSANDRA-11315.
+        Iterable<UntypedResultSet.Row> filteredColumnRows = !isDense && (rawIsDense == null || rawIsDense)
+                                                          ? filterOutRedundantRowsForSparse(columnRows, isSuper, isCompound)
+                                                          : columnRows;
+
         // We don't really use the default validator but as we have it for backward compatibility, we use it to know if it's a counter table
         AbstractType<?> defaultValidator = TypeParser.parse(tableRow.getString("default_validator"));
         boolean isCounter = defaultValidator instanceof CounterColumnType;
@@ -311,9 +326,9 @@ public final class LegacySchemaMigrator
         // previous versions, they may not have the expected schema, so detect if we need to upgrade and do
         // it in createColumnsFromColumnRows.
         // We can remove this once we don't support upgrade from versions < 3.0.
-        boolean needsUpgrade = !isCQLTable && checkNeedsUpgrade(columnRows, isSuper, isStaticCompactTable);
+        boolean needsUpgrade = !isCQLTable && checkNeedsUpgrade(filteredColumnRows, isSuper, isStaticCompactTable);
 
-        List<ColumnDefinition> columnDefs = createColumnsFromColumnRows(columnRows,
+        List<ColumnDefinition> columnDefs = createColumnsFromColumnRows(filteredColumnRows,
                                                                         ksName,
                                                                         cfName,
                                                                         rawComparator,
@@ -323,7 +338,6 @@ public final class LegacySchemaMigrator
                                                                         isStaticCompactTable,
                                                                         needsUpgrade);
 
-
         if (needsUpgrade)
         {
             addDefinitionForUpgrade(columnDefs,
@@ -348,7 +362,7 @@ public final class LegacySchemaMigrator
                                            DatabaseDescriptor.getPartitioner());
 
         Indexes indexes = createIndexesFromColumnRows(cfm,
-                                                      columnRows,
+                                                      filteredColumnRows,
                                                       ksName,
                                                       cfName,
                                                       rawComparator,
@@ -374,7 +388,7 @@ public final class LegacySchemaMigrator
      * information for table just created through thrift, nor for table prior to CASSANDRA-7744, so this
      * method does its best to infer whether the table is dense or not based on other elements.
      */
-    public static boolean calculateIsDense(AbstractType<?> comparator, UntypedResultSet columnRows)
+    private static boolean calculateIsDense(AbstractType<?> comparator, UntypedResultSet columnRows)
     {
         /*
          * As said above, this method is only here because we need to deal with thrift upgrades.
@@ -395,25 +409,44 @@ public final class LegacySchemaMigrator
          * in which case it should not be dense. However, we can limit our margin of error by assuming we are
          * in the latter case only if the comparator is exactly CompositeType(UTF8Type).
          */
-        boolean hasRegular = false;
+        for (UntypedResultSet.Row columnRow : columnRows)
+            if ("regular".equals(columnRow.getString("type")))
+                return false;
+
         int maxClusteringIdx = -1;
+        for (UntypedResultSet.Row columnRow : columnRows)
+            if ("clustering_key".equals(columnRow.getString("type")))
+                maxClusteringIdx = Math.max(maxClusteringIdx, columnRow.has("component_index") ? columnRow.getInt("component_index") : 0);
 
+        return maxClusteringIdx >= 0
+             ? maxClusteringIdx == comparator.componentsCount() - 1
+             : !isCQL3OnlyPKComparator(comparator);
+    }
+
+    private static Iterable<UntypedResultSet.Row> filterOutRedundantRowsForSparse(UntypedResultSet columnRows, boolean isSuper, boolean isCompound)
+    {
+        Collection<UntypedResultSet.Row> filteredRows = new ArrayList<>();
         for (UntypedResultSet.Row columnRow : columnRows)
         {
-            switch (columnRow.getString("type"))
+            String kind = columnRow.getString("type");
+
+            if ("compact_value".equals(kind))
+                continue;
+
+            if ("clustering_key".equals(kind))
             {
-                case "clustering_key":
-                    maxClusteringIdx = Math.max(maxClusteringIdx, columnRow.has("component_index") ? columnRow.getInt("component_index") : 0);
-                    break;
-                case "regular":
-                    hasRegular = true;
-                    break;
+                int position = columnRow.has("component_index") ? columnRow.getInt("component_index") : 0;
+                if (isSuper && position != 0)
+                    continue;
+
+                if (!isSuper && !isCompound)
+                    continue;
             }
+
+            filteredRows.add(columnRow);
         }
 
-        return maxClusteringIdx >= 0
-               ? maxClusteringIdx == comparator.componentsCount() - 1
-               : !hasRegular && !isCQL3OnlyPKComparator(comparator);
+        return filteredRows;
     }
 
     private static boolean isCQL3OnlyPKComparator(AbstractType<?> comparator)
@@ -507,7 +540,7 @@ public final class LegacySchemaMigrator
     }
 
     // Should only be called on compact tables
-    private static boolean checkNeedsUpgrade(UntypedResultSet defs, boolean isSuper, boolean isStaticCompactTable)
+    private static boolean checkNeedsUpgrade(Iterable<UntypedResultSet.Row> defs, boolean isSuper, boolean isStaticCompactTable)
     {
         if (isSuper)
         {
@@ -527,7 +560,7 @@ public final class LegacySchemaMigrator
         return !hasRegularColumns(defs);
     }
 
-    private static boolean hasRegularColumns(UntypedResultSet columnRows)
+    private static boolean hasRegularColumns(Iterable<UntypedResultSet.Row> columnRows)
     {
         for (UntypedResultSet.Row row : columnRows)
         {
@@ -581,7 +614,7 @@ public final class LegacySchemaMigrator
         }
     }
 
-    private static boolean hasKind(UntypedResultSet defs, ColumnDefinition.Kind kind)
+    private static boolean hasKind(Iterable<UntypedResultSet.Row> defs, ColumnDefinition.Kind kind)
     {
         for (UntypedResultSet.Row row : defs)
             if (deserializeKind(row.getString("type")) == kind)
@@ -620,7 +653,7 @@ public final class LegacySchemaMigrator
         }
     }
 
-    private static List<ColumnDefinition> createColumnsFromColumnRows(UntypedResultSet rows,
+    private static List<ColumnDefinition> createColumnsFromColumnRows(Iterable<UntypedResultSet.Row> rows,
                                                                       String keyspace,
                                                                       String table,
                                                                       AbstractType<?> rawComparator,
@@ -689,7 +722,7 @@ public final class LegacySchemaMigrator
     }
 
     private static Indexes createIndexesFromColumnRows(CFMetaData cfm,
-                                                       UntypedResultSet rows,
+                                                       Iterable<UntypedResultSet.Row> rows,
                                                        String keyspace,
                                                        String table,
                                                        AbstractType<?> rawComparator,


[04/50] cassandra git commit: Always select the live sstables when getting sstables in bounds

Posted by al...@apache.org.
Always select the live sstables when getting sstables in bounds

Patch by marcuse; reviewed by benedict for CASSANDRA-11944


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

Branch: refs/heads/cassandra-3.8
Commit: 5b0566a70f373d6eb537c89c0db2a2e224706916
Parents: 2217695
Author: Marcus Eriksson <ma...@apache.org>
Authored: Thu Jun 2 09:37:06 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Wed Jul 6 07:52:28 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                               |  1 +
 src/java/org/apache/cassandra/db/ColumnFamilyStore.java   |  8 ++++----
 .../apache/cassandra/db/PartitionRangeReadCommand.java    |  2 +-
 .../org/apache/cassandra/db/SizeEstimatesRecorder.java    |  5 +++--
 .../db/compaction/AbstractCompactionStrategy.java         |  2 +-
 .../cassandra/db/compaction/CompactionController.java     |  2 +-
 .../db/compaction/DateTieredCompactionStrategy.java       |  2 +-
 .../db/compaction/TimeWindowCompactionStrategy.java       |  2 +-
 src/java/org/apache/cassandra/db/lifecycle/View.java      | 10 +++++-----
 .../org/apache/cassandra/streaming/StreamSession.java     |  8 +++++++-
 test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java |  2 +-
 11 files changed, 26 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b0566a7/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7f8a3a1..99ac3ad 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
  * Fix column ordering of results with static columns for Thrift requests in
    a mixed 2.x/3.x cluster, also fix potential non-resolved duplication of
    those static columns in query results (CASSANDRA-12123)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b0566a7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 3264327..1be3175 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1231,7 +1231,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      * @return sstables whose key range overlaps with that of the given sstables, not including itself.
      * (The given sstables may or may not overlap with each other.)
      */
-    public Collection<SSTableReader> getOverlappingSSTables(SSTableSet sstableSet, Iterable<SSTableReader> sstables)
+    public Collection<SSTableReader> getOverlappingLiveSSTables(Iterable<SSTableReader> sstables)
     {
         logger.trace("Checking for sstables overlapping {}", sstables);
 
@@ -1282,7 +1282,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         Set<SSTableReader> results = new HashSet<>();
 
         for (AbstractBounds<PartitionPosition> bound : bounds)
-            Iterables.addAll(results, view.sstablesInBounds(sstableSet, bound.left, bound.right));
+            Iterables.addAll(results, view.liveSSTablesInBounds(bound.left, bound.right));
 
         return Sets.difference(results, ImmutableSet.copyOf(sstables));
     }
@@ -1290,11 +1290,11 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     /**
      * like getOverlappingSSTables, but acquires references before returning
      */
-    public Refs<SSTableReader> getAndReferenceOverlappingSSTables(SSTableSet sstableSet, Iterable<SSTableReader> sstables)
+    public Refs<SSTableReader> getAndReferenceOverlappingLiveSSTables(Iterable<SSTableReader> sstables)
     {
         while (true)
         {
-            Iterable<SSTableReader> overlapped = getOverlappingSSTables(sstableSet, sstables);
+            Iterable<SSTableReader> overlapped = getOverlappingLiveSSTables(sstables);
             Refs<SSTableReader> refs = Refs.tryRef(overlapped);
             if (refs != null)
                 return refs;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b0566a7/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 9585b59..842ad5f 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@ -175,7 +175,7 @@ public class PartitionRangeReadCommand extends ReadCommand
 
     protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
     {
-        ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, dataRange().keyRange()));
+        ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
         Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));
 
         // fetch data from current memtable, historical memtables, and SSTables in the correct order.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b0566a7/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
index 2a74ea9..3461aef 100644
--- a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
+++ b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
@@ -103,8 +103,9 @@ public class SizeEstimatesRecorder extends MigrationListener implements Runnable
             {
                 while (refs == null)
                 {
-                    ColumnFamilyStore.ViewFragment view = table.select(View.select(SSTableSet.CANONICAL, Range.makeRowRange(range)));
-                    refs = Refs.tryRef(view.sstables);
+                    // note that this is not guaranteed to return all sstables within the ranges, but for an estimated size, that should be fine
+                    Iterable<SSTableReader> canonicalSSTables = table.getTracker().getView().select(SSTableSet.CANONICAL, table.select(View.selectLive(Range.makeRowRange(range))).sstables);
+                    refs = Refs.tryRef(canonicalSSTables);
                 }
 
                 // calculate the estimates.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b0566a7/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index e30e4f7..0dce52b 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -380,7 +380,7 @@ public abstract class AbstractCompactionStrategy
         if (uncheckedTombstoneCompaction)
             return true;
 
-        Collection<SSTableReader> overlaps = cfs.getOverlappingSSTables(SSTableSet.CANONICAL, Collections.singleton(sstable));
+        Collection<SSTableReader> overlaps = cfs.getOverlappingLiveSSTables(Collections.singleton(sstable));
         if (overlaps.isEmpty())
         {
             // there is no overlap, tombstones are safely droppable

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b0566a7/src/java/org/apache/cassandra/db/compaction/CompactionController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
index a5b8308..fbf29e3 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -103,7 +103,7 @@ public class CompactionController implements AutoCloseable
         if (compacting == null)
             overlappingSSTables = Refs.tryRef(Collections.<SSTableReader>emptyList());
         else
-            overlappingSSTables = cfs.getAndReferenceOverlappingSSTables(SSTableSet.LIVE, compacting);
+            overlappingSSTables = cfs.getAndReferenceOverlappingLiveSSTables(compacting);
         this.overlapIterator = new OverlapIterator<>(buildIntervals(overlappingSSTables));
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b0566a7/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
index 8571906..3e6ae61 100644
--- a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
@@ -99,7 +99,7 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
         if (System.currentTimeMillis() - lastExpiredCheck > options.expiredSSTableCheckFrequency)
         {
             // Find fully expired SSTables. Those will be included no matter what.
-            expired = CompactionController.getFullyExpiredSSTables(cfs, uncompacting, cfs.getOverlappingSSTables(SSTableSet.CANONICAL, uncompacting), gcBefore);
+            expired = CompactionController.getFullyExpiredSSTables(cfs, uncompacting, cfs.getOverlappingLiveSSTables(uncompacting), gcBefore);
             lastExpiredCheck = System.currentTimeMillis();
         }
         Set<SSTableReader> candidates = Sets.newHashSet(filterSuspectSSTables(uncompacting));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b0566a7/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java
index d1630c5..e2ab7dc 100644
--- a/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java
@@ -104,7 +104,7 @@ public class TimeWindowCompactionStrategy extends AbstractCompactionStrategy
         if (System.currentTimeMillis() - lastExpiredCheck > options.expiredSSTableCheckFrequency)
         {
             logger.debug("TWCS expired check sufficiently far in the past, checking for fully expired SSTables");
-            expired = CompactionController.getFullyExpiredSSTables(cfs, uncompacting, cfs.getOverlappingSSTables(SSTableSet.CANONICAL, uncompacting), gcBefore);
+            expired = CompactionController.getFullyExpiredSSTables(cfs, uncompacting, cfs.getOverlappingLiveSSTables(uncompacting), gcBefore);
             lastExpiredCheck = System.currentTimeMillis();
         }
         else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b0566a7/src/java/org/apache/cassandra/db/lifecycle/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/View.java b/src/java/org/apache/cassandra/db/lifecycle/View.java
index 99903fc..96aaa49 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/View.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/View.java
@@ -136,7 +136,7 @@ public class View
         return Iterables.concat(sstables, filterOut(compacting, sstables));
     }
 
-    private Iterable<SSTableReader> select(SSTableSet sstableSet, Iterable<SSTableReader> sstables)
+    public Iterable<SSTableReader> select(SSTableSet sstableSet, Iterable<SSTableReader> sstables)
     {
         switch (sstableSet)
         {
@@ -182,7 +182,7 @@ public class View
      * Returns the sstables that have any partition between {@code left} and {@code right}, when both bounds are taken inclusively.
      * The interval formed by {@code left} and {@code right} shouldn't wrap.
      */
-    public Iterable<SSTableReader> sstablesInBounds(SSTableSet sstableSet, PartitionPosition left, PartitionPosition right)
+    public Iterable<SSTableReader> liveSSTablesInBounds(PartitionPosition left, PartitionPosition right)
     {
         assert !AbstractBounds.strictlyWrapsAround(left, right);
 
@@ -190,7 +190,7 @@ public class View
             return Collections.emptyList();
 
         PartitionPosition stopInTree = right.isMinimum() ? intervalTree.max() : right;
-        return select(sstableSet, intervalTree.search(Interval.create(left, stopInTree)));
+        return select(SSTableSet.LIVE, intervalTree.search(Interval.create(left, stopInTree)));
     }
 
     public static List<SSTableReader> sstablesInBounds(PartitionPosition left, PartitionPosition right, SSTableIntervalTree intervalTree)
@@ -228,14 +228,14 @@ public class View
      * @return a ViewFragment containing the sstables and memtables that may need to be merged
      * for rows within @param rowBounds, inclusive, according to the interval tree.
      */
-    public static Function<View, Iterable<SSTableReader>> select(SSTableSet sstableSet, AbstractBounds<PartitionPosition> rowBounds)
+    public static Function<View, Iterable<SSTableReader>> selectLive(AbstractBounds<PartitionPosition> rowBounds)
     {
         // Note that View.sstablesInBounds always includes it's bound while rowBounds may not. This is ok however
         // because the fact we restrict the sstables returned by this function is an optimization in the first
         // place and the returned sstables will (almost) never cover *exactly* rowBounds anyway. It's also
         // *very* unlikely that a sstable is included *just* because we consider one of the bound inclusively
         // instead of exclusively, so the performance impact is negligible in practice.
-        return (view) -> view.sstablesInBounds(sstableSet, rowBounds.left, rowBounds.right);
+        return (view) -> view.liveSSTablesInBounds(rowBounds.left, rowBounds.right);
     }
 
     // METHODS TO CONSTRUCT FUNCTIONS FOR MODIFYING A VIEW:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b0566a7/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index d5c060e..a14f815 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -29,8 +29,8 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.*;
 
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
-import org.apache.cassandra.db.lifecycle.SSTableSet;
 import org.apache.cassandra.db.lifecycle.SSTableIntervalTree;
+import org.apache.cassandra.db.lifecycle.SSTableSet;
 import org.apache.cassandra.db.lifecycle.View;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
@@ -332,6 +332,12 @@ public class StreamSession implements IEndpointStateChangeSubscriber
                     SSTableIntervalTree intervalTree = SSTableIntervalTree.build(view.sstables(SSTableSet.CANONICAL));
                     for (Range<PartitionPosition> keyRange : keyRanges)
                     {
+                        // keyRange excludes its start, while sstableInBounds is inclusive (of both start and end).
+                        // This is fine however, because keyRange has been created from a token range through Range.makeRowRange (see above).
+                        // And that later method uses the Token.maxKeyBound() method to creates the range, which return a "fake" key that
+                        // sort after all keys having the token. That "fake" key cannot however be equal to any real key, so that even
+                        // including keyRange.left will still exclude any key having the token of the original token range, and so we're
+                        // still actually selecting what we wanted.
                         for (SSTableReader sstable : View.sstablesInBounds(keyRange.left, keyRange.right, intervalTree))
                         {
                             if (!isIncremental || !sstable.isRepaired())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b0566a7/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
index 8a5e00e..98f9300 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
@@ -71,7 +71,7 @@ public class ViewTest
                             continue;
 
                         AbstractBounds<PartitionPosition> bounds = AbstractBounds.bounds(min, minInc, max, maxInc);
-                        List<SSTableReader> r = ImmutableList.copyOf(initialView.sstablesInBounds(SSTableSet.LIVE,bounds.left, bounds.right));
+                        List<SSTableReader> r = ImmutableList.copyOf(initialView.liveSSTablesInBounds(bounds.left, bounds.right));
                         Assert.assertEquals(String.format("%d(%s) %d(%s)", i, minInc, j, maxInc), j - i + (minInc ? 0 : -1) + (maxInc ? 1 : 0), r.size());
                     }
                 }


[42/50] cassandra git commit: Merge branch cassandra-3.0 into cassandra-3.9

Posted by al...@apache.org.
Merge branch cassandra-3.0 into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: fb4eb5ac1898bb47fb0d3f5d608bee92df2c4256
Parents: 76188e9 fbd287a
Author: Benjamin Lerer <b....@gmail.com>
Authored: Thu Jul 14 11:37:20 2016 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Thu Jul 14 11:37:20 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  5 ++
 .../cassandra/config/DatabaseDescriptor.java    |  1 +
 .../cassandra/db/PartitionRangeReadCommand.java |  7 +--
 .../org/apache/cassandra/db/ReadCommand.java    | 65 ++++++++++----------
 .../org/apache/cassandra/db/ReadResponse.java   | 43 +++++--------
 .../db/SinglePartitionReadCommand.java          |  2 +-
 .../io/ForwardingVersionedSerializer.java       | 57 +++++++++++++++++
 .../apache/cassandra/net/MessagingService.java  |  6 +-
 8 files changed, 117 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fb4eb5ac/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index da8216f,3829046..bedba6d
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,9 -1,5 +1,14 @@@
++<<<<<<< HEAD
 +3.9
 + * Partial revert of CASSANDRA-11971, cannot recycle buffer in SP.sendMessagesToNonlocalDC (CASSANDRA-11950)
 + * Fix hdr logging for single operation workloads (CASSANDRA-12145)
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
++=======
+ 3.0.9
+  * Wait until the message is being send to decide which serializer must be used (CASSANDRA-11393)
++>>>>>>> asf/cassandra-3.0
   * Fix migration of static thrift column names with non-text comparators (CASSANDRA-12147)
   * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)
   * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fb4eb5ac/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fb4eb5ac/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ReadCommand.java
index c842814,36969f8..68c9e3b
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@@ -56,14 -54,44 +57,44 @@@ import org.apache.cassandra.utils.Pair
   * <p>
   * This contains all the informations needed to do a local read.
   */
 -public abstract class ReadCommand implements ReadQuery
 +public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
  {
 +    private static final int TEST_ITERATION_DELAY_MILLIS = Integer.valueOf(System.getProperty("cassandra.test.read_iteration_delay_ms", "0"));
      protected static final Logger logger = LoggerFactory.getLogger(ReadCommand.class);
 -
      public static final IVersionedSerializer<ReadCommand> serializer = new Serializer();
+ 
+     // For READ verb: will either dispatch on 'serializer' for 3.0 or 'legacyReadCommandSerializer' for earlier version.
+     // Can be removed (and replaced by 'serializer') once we drop pre-3.0 backward compatibility.
+     public static final IVersionedSerializer<ReadCommand> readSerializer = new ForwardingVersionedSerializer<ReadCommand>()
+     {
+         protected IVersionedSerializer<ReadCommand> delegate(int version)
+         {
+             return version < MessagingService.VERSION_30
+                     ? legacyReadCommandSerializer : serializer;
+         }
+     };
+ 
      // For RANGE_SLICE verb: will either dispatch on 'serializer' for 3.0 or 'legacyRangeSliceCommandSerializer' for earlier version.
      // Can be removed (and replaced by 'serializer') once we drop pre-3.0 backward compatibility.
-     public static final IVersionedSerializer<ReadCommand> rangeSliceSerializer = new RangeSliceSerializer();
+     public static final IVersionedSerializer<ReadCommand> rangeSliceSerializer = new ForwardingVersionedSerializer<ReadCommand>()
+     {
+         protected IVersionedSerializer<ReadCommand> delegate(int version)
+         {
+             return version < MessagingService.VERSION_30
+                     ? legacyRangeSliceCommandSerializer : serializer;
+         }
+     };
+ 
+     // For PAGED_RANGE verb: will either dispatch on 'serializer' for 3.0 or 'legacyPagedRangeCommandSerializer' for earlier version.
+     // Can be removed (and replaced by 'serializer') once we drop pre-3.0 backward compatibility.
+     public static final IVersionedSerializer<ReadCommand> pagedRangeSerializer = new ForwardingVersionedSerializer<ReadCommand>()
+     {
+         protected IVersionedSerializer<ReadCommand> delegate(int version)
+         {
+             return version < MessagingService.VERSION_30
+                     ? legacyPagedRangeCommandSerializer : serializer;
+         }
+     };
  
      public static final IVersionedSerializer<ReadCommand> legacyRangeSliceCommandSerializer = new LegacyRangeSliceCommandSerializer();
      public static final IVersionedSerializer<ReadCommand> legacyPagedRangeCommandSerializer = new LegacyPagedRangeCommandSerializer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fb4eb5ac/src/java/org/apache/cassandra/db/ReadResponse.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fb4eb5ac/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fb4eb5ac/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------


[21/50] cassandra git commit: Don't ignore deletion info in sstable on reverse queries

Posted by al...@apache.org.
Don't ignore deletion info in sstable on reverse queries

patch by Sylvain Lebresne; reviewed by Aleksey Yeschenko for CASSANDRA-11733


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

Branch: refs/heads/cassandra-3.8
Commit: 76e68e9b49b1fbcb601633e6e2b8d8e1f71c7402
Parents: 30f5d44
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jun 30 15:13:24 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:54:52 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../columniterator/SSTableReversedIterator.java |  2 +-
 .../cql3/validation/operations/DeleteTest.java  | 26 ++++++++++++++++++++
 3 files changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8118de1..20ed6e0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
  * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
  * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
index 14cec36..3e49a3a 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@ -356,7 +356,7 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
         {
             deletionInfo = deletionBuilder.build();
             built = new ImmutableBTreePartition(metadata, partitionKey, columns, Rows.EMPTY_STATIC_ROW, rowBuilder.build(),
-                                                DeletionInfo.LIVE, EncodingStats.NO_STATS);
+                                                deletionInfo, EncodingStats.NO_STATS);
             deletionBuilder = null;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/76e68e9b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
index 76351ee..814e822 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -1057,4 +1057,30 @@ public class DeleteTest extends CQLTester
         if (forceFlush)
             flush();
     }
+
+    @Test
+    public void testDeleteAndReverseQueries() throws Throwable
+    {
+        // This test insert rows in one sstable and a range tombstone covering some of those rows in another, and it
+        // validates we correctly get only the non-removed rows when doing reverse queries.
+
+        createTable("CREATE TABLE %s (k text, i int, PRIMARY KEY (k, i))");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s(k, i) values (?, ?)", "a", i);
+
+        flush();
+
+        execute("DELETE FROM %s WHERE k = ? AND i >= ? AND i <= ?", "a", 2, 7);
+
+        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+            row(9), row(8), row(1), row(0)
+        );
+
+        flush();
+
+        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+            row(9), row(8), row(1), row(0)
+        );
+    }
 }


[30/50] cassandra git commit: Partial revert of CASSANDRA-11971, cannot recycle buffer in SP.sendMessagesToNonlocalDC

Posted by al...@apache.org.
Partial revert of CASSANDRA-11971, cannot recycle buffer in SP.sendMessagesToNonlocalDC

patch by Stefania Alborghetti; reviewed by Jake Luciani for CASSANDRA-11950


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

Branch: refs/heads/cassandra-3.8
Commit: d20c765a7d962b55ce905f393589869dab63d996
Parents: 1417a51
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Fri Jul 8 10:26:47 2016 +0800
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Mon Jul 11 10:57:27 2016 +0800

----------------------------------------------------------------------
 CHANGES.txt                                             | 1 +
 src/java/org/apache/cassandra/service/StorageProxy.java | 7 +------
 2 files changed, 2 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d20c765a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b094b00..9df49f3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.9
+ * Partial revert of CASSANDRA-11971, cannot recycle buffer in SP.sendMessagesToNonlocalDC (CASSANDRA-11950)
  * Fix hdr logging for single operation workloads (CASSANDRA-12145)
  * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
  * Increase size of flushExecutor thread pool (CASSANDRA-12071)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d20c765a/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index c88c449..3ce8013 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -1284,8 +1284,7 @@ public class StorageProxy implements StorageProxyMBean
         InetAddress target = iter.next();
 
         // Add the other destinations of the same message as a FORWARD_HEADER entry
-        DataOutputBuffer out = null;
-        try (DataOutputBuffer ignored = out = DataOutputBuffer.RECYCLER.get())
+        try(DataOutputBuffer out = new DataOutputBuffer())
         {
             out.writeInt(targets.size() - 1);
             while (iter.hasNext())
@@ -1311,10 +1310,6 @@ public class StorageProxy implements StorageProxyMBean
             // DataOutputBuffer is in-memory, doesn't throw IOException
             throw new AssertionError(e);
         }
-        finally
-        {
-            out.recycle();
-        }
     }
 
     private static void performLocally(Stage stage, final Runnable runnable)


[25/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 8475f891c7576e3816ac450178344a5232b72738
Parents: a006f57 a227cc6
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Jul 7 11:00:37 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu Jul 7 11:00:37 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                               | 2 ++
 src/java/org/apache/cassandra/service/StorageService.java | 3 +++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8475f891/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 1d11149,0e483f1..34e7587
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,8 -1,7 +1,10 @@@
 -3.0.9
 +3.9
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
   * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
+ 2.2.8
+  * Use dedicated thread for JMX notifications (CASSANDRA-12146)
   * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
   * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
   * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8475f891/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------


[24/50] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

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


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

Branch: refs/heads/cassandra-3.8
Commit: a227cc61c501ff81d5dfeba3f6f9c2f214d19c30
Parents: 76e68e9 f28409b
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Jul 7 11:00:31 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu Jul 7 11:00:31 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                               | 2 ++
 src/java/org/apache/cassandra/service/StorageService.java | 3 +++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a227cc61/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 20ed6e0,9fef5a2..0e483f1
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,28 -1,16 +1,30 @@@
 +3.0.9
 + * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
+ 2.2.8
+  * Use dedicated thread for JMX notifications (CASSANDRA-12146)
 - * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
   * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
 + * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
 + * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
 + * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
 + * Fix column ordering of results with static columns for Thrift requests in
 +   a mixed 2.x/3.x cluster, also fix potential non-resolved duplication of
 +   those static columns in query results (CASSANDRA-12123)
 + * Avoid digest mismatch with empty but static rows (CASSANDRA-12090)
 + * Fix EOF exception when altering column type (CASSANDRA-11820)
 +Merged from 2.2:
   * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
  Merged from 2.1:
 - * Don't write shadowed range tombstone (CASSANDRA-12030)
 - * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)
   * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
 - * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
  
  
 -2.2.7
 +3.0.8
 + * Fix potential race in schema during new table creation (CASSANDRA-12083)
 + * cqlsh: fix error handling in rare COPY FROM failure scenario (CASSANDRA-12070)
 + * Disable autocompaction during drain (CASSANDRA-11878)
 + * Add a metrics timer to MemtablePool and use it to track time spent blocked on memory in MemtableAllocator (CASSANDRA-11327)
 + * Fix upgrading schema with super columns with non-text subcomparators (CASSANDRA-12023)
 + * Add TimeWindowCompactionStrategy (CASSANDRA-9666)
 +Merged from 2.2:
   * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
   * Validate bloom_filter_fp_chance against lowest supported
     value when the table is created (CASSANDRA-11920)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a227cc61/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------


[32/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 314f60a44e969fb7da62a21275d94524277c4eb3
Parents: d20c765 f37b6f1
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Mon Jul 11 14:32:30 2016 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Mon Jul 11 14:32:30 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../cassandra/schema/LegacySchemaMigrator.java  | 81 ++++++++++++++------
 2 files changed, 59 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/314f60a4/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 9df49f3,f2d11b9..44fe26c
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,9 -1,5 +1,11 @@@
 +3.9
 + * Partial revert of CASSANDRA-11971, cannot recycle buffer in SP.sendMessagesToNonlocalDC (CASSANDRA-11950)
 + * Fix hdr logging for single operation workloads (CASSANDRA-12145)
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
+ 3.0.9
+  * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)
   * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
  2.2.8
   * Use dedicated thread for JMX notifications (CASSANDRA-12146)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/314f60a4/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
----------------------------------------------------------------------


[40/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 76188e9520d0aed8da287cffd80122e1069ddcae
Parents: 56abaca e99ee19
Author: Josh McKenzie <jm...@apache.org>
Authored: Mon Jul 11 16:29:58 2016 -0400
Committer: Josh McKenzie <jm...@apache.org>
Committed: Mon Jul 11 16:29:58 2016 -0400

----------------------------------------------------------------------
 conf/cassandra.yaml                                       |  6 ++++++
 src/java/org/apache/cassandra/config/Config.java          |  1 +
 .../org/apache/cassandra/config/DatabaseDescriptor.java   | 10 ++++++++++
 src/java/org/apache/cassandra/service/GCInspector.java    |  2 +-
 4 files changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/76188e95/conf/cassandra.yaml
----------------------------------------------------------------------
diff --cc conf/cassandra.yaml
index 076a729,09d2094..d79423e
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@@ -1051,6 -924,16 +1051,12 @@@ inter_dc_tcp_nodelay: fals
  tracetype_query_ttl: 86400
  tracetype_repair_ttl: 604800
  
+ # By default, Cassandra logs GC Pauses greater than 200 ms at INFO level
+ # This threshold can be adjusted to minimize logging if necessary
+ # gc_log_threshold_in_ms: 200
+ 
 -# GC Pauses greater than gc_warn_threshold_in_ms will be logged at WARN level
+ # If unset, all GC Pauses greater than gc_log_threshold_in_ms will log at
+ # INFO level
 -# Adjust the threshold based on your application throughput requirement
 -gc_warn_threshold_in_ms: 1000
 -
  # UDFs (user defined functions) are disabled by default.
  # As of Cassandra 3.0 there is a sandbox in place that should prevent execution of evil code.
  enable_user_defined_functions: false

http://git-wip-us.apache.org/repos/asf/cassandra/blob/76188e95/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/76188e95/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 1375a39,100bcf4..38dce11
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@@ -2169,11 -1984,11 +2174,16 @@@ public class DatabaseDescripto
          conf.user_function_timeout_policy = userFunctionTimeoutPolicy;
      }
  
+     public static long getGCLogThreshold()
+     {
+         return conf.gc_log_threshold_in_ms;
+     }
+ 
 +    public static EncryptionContext getEncryptionContext()
 +    {
 +        return encryptionContext;
 +    }
 +    
      public static long getGCWarnThreshold()
      {
          return conf.gc_warn_threshold_in_ms;


[23/50] cassandra git commit: Use dedicated thread for sending JMX notifications

Posted by al...@apache.org.
Use dedicated thread for sending JMX notifications

patch by Stefan Podkowinski; reviewed by yukim for CASSANDRA-12146


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

Branch: refs/heads/cassandra-3.8
Commit: f28409bb9730c0318c3243f9d0febbb05ec0c2dc
Parents: ef18a17
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Wed Jul 6 16:58:47 2016 +0200
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu Jul 7 10:59:44 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                               | 1 +
 src/java/org/apache/cassandra/service/StorageService.java | 3 +++
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f28409bb/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e10af6f..9fef5a2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.8
+ * Use dedicated thread for JMX notifications (CASSANDRA-12146)
  * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
  * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f28409bb/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index a877074..fa04595 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -210,6 +210,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public StorageService()
     {
+        // use dedicated executor for sending JMX notifications
+        super(Executors.newSingleThreadExecutor());
+
         MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
         try
         {


[20/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9

* cassandra-3.0:
  NPE when trying to remove purgable tombstones from result


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

Branch: refs/heads/cassandra-3.8
Commit: 376dae26833591303cd3140001666f23aa216a11
Parents: 59ee46e 30f5d44
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jul 7 12:50:26 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:50:26 2016 +0200

----------------------------------------------------------------------

----------------------------------------------------------------------



[38/50] cassandra git commit: Make GCInspector min log duration configurable

Posted by al...@apache.org.
Make GCInspector min log duration configurable

Patch by jjirsa; reviewed by jmckenzie for CASSANDRA-11715


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

Branch: refs/heads/cassandra-3.8
Commit: f0d1d75ebf10beff6d24323c03c57e29dcd38c15
Parents: 9a8406f
Author: Jeff Jirsa <je...@jeffjirsa.net>
Authored: Mon Jul 11 16:27:04 2016 -0400
Committer: Josh McKenzie <jm...@apache.org>
Committed: Mon Jul 11 16:27:04 2016 -0400

----------------------------------------------------------------------
 conf/cassandra.yaml                                       |  7 ++++++-
 src/java/org/apache/cassandra/config/Config.java          |  1 +
 .../org/apache/cassandra/config/DatabaseDescriptor.java   | 10 ++++++++++
 src/java/org/apache/cassandra/service/GCInspector.java    |  2 +-
 4 files changed, 18 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0d1d75e/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 35e94d2..4ad798a 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -858,9 +858,14 @@ inter_dc_tcp_nodelay: false
 tracetype_query_ttl: 86400
 tracetype_repair_ttl: 604800
 
+# By default, Cassandra logs GC Pauses greater than 200 ms at INFO level
+# This threshold can be adjusted to minimize logging if necessary
+# gc_log_threshold_in_ms: 200
+
 # GC Pauses greater than gc_warn_threshold_in_ms will be logged at WARN level
+# If unset, all GC Pauses greater than gc_log_threshold_in_ms will log at
+# INFO level
 # Adjust the threshold based on your application throughput requirement
-# By default, Cassandra logs GC Pauses greater than 200 ms at INFO level
 # gc_warn_threshold_in_ms: 1000
 
 # UDFs (user defined functions) are disabled by default.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0d1d75e/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 9736a03..ede4560 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -254,6 +254,7 @@ public class Config
     public volatile Long index_summary_capacity_in_mb;
     public volatile int index_summary_resize_interval_in_minutes = 60;
 
+    public int gc_log_threshold_in_ms = 200;
     public int gc_warn_threshold_in_ms = 0;
 
     private static final CsvPreference STANDARD_SURROUNDING_SPACES_NEED_QUOTES = new CsvPreference.Builder(CsvPreference.STANDARD_PREFERENCE)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0d1d75e/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index d3a5028..f1acfc4 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -366,6 +366,11 @@ public class DatabaseDescriptor
         }
         paritionerName = partitioner.getClass().getCanonicalName();
 
+        if (config.gc_log_threshold_in_ms < 0)
+        {
+            throw new ConfigurationException("gc_log_threshold_in_ms must be a positive integer");
+        }
+
         if (conf.gc_warn_threshold_in_ms < 0)
         {
             throw new ConfigurationException("gc_warn_threshold_in_ms must be a positive integer");
@@ -1801,6 +1806,11 @@ public class DatabaseDescriptor
         return conf.windows_timer_interval;
     }
 
+    public static long getGCLogThreshold()
+    {
+        return conf.gc_log_threshold_in_ms;
+    }
+
     public static long getGCWarnThreshold()
     {
         return conf.gc_warn_threshold_in_ms;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0d1d75e/src/java/org/apache/cassandra/service/GCInspector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/GCInspector.java b/src/java/org/apache/cassandra/service/GCInspector.java
index de5acc0..31de151 100644
--- a/src/java/org/apache/cassandra/service/GCInspector.java
+++ b/src/java/org/apache/cassandra/service/GCInspector.java
@@ -48,7 +48,7 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean
 {
     public static final String MBEAN_NAME = "org.apache.cassandra.service:type=GCInspector";
     private static final Logger logger = LoggerFactory.getLogger(GCInspector.class);
-    final static long MIN_LOG_DURATION = 200;
+    final static long MIN_LOG_DURATION = DatabaseDescriptor.getGCLogThreshold();
     final static long GC_WARN_THRESHOLD_IN_MS = DatabaseDescriptor.getGCWarnThreshold();
     final static long STAT_THRESHOLD = GC_WARN_THRESHOLD_IN_MS != 0 ? GC_WARN_THRESHOLD_IN_MS : MIN_LOG_DURATION;
 


[05/50] cassandra git commit: Avoid missing sstables when getting the canonical sstables

Posted by al...@apache.org.
Avoid missing sstables when getting the canonical sstables

Patch by marcuse; reviewed by Stefania Alborghetti for CASSANDRA-11996


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

Branch: refs/heads/cassandra-3.8
Commit: bc23632f201f760147d8bd1fbee68533fc3f6dfa
Parents: 5b0566a
Author: Marcus Eriksson <ma...@apache.org>
Authored: Mon Jun 13 15:29:08 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Wed Jul 6 07:57:24 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  6 +--
 .../cassandra/db/SizeEstimatesRecorder.java     |  7 +++-
 .../apache/cassandra/db/lifecycle/Tracker.java  |  2 +-
 .../org/apache/cassandra/db/lifecycle/View.java | 30 +++++++-------
 .../apache/cassandra/db/view/ViewBuilder.java   |  4 +-
 .../cassandra/index/SecondaryIndexManager.java  |  2 +-
 .../index/internal/CassandraIndex.java          |  2 +-
 .../io/sstable/IndexSummaryManager.java         |  2 +-
 .../cassandra/streaming/StreamSession.java      |  4 +-
 .../apache/cassandra/db/lifecycle/ViewTest.java |  6 +--
 .../index/internal/CustomCassandraIndex.java    |  2 +-
 .../io/sstable/SSTableRewriterTest.java         | 41 +++++++++++++++++++-
 13 files changed, 78 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 99ac3ad..b3063b4 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
  * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
  * Fix column ordering of results with static columns for Thrift requests in
    a mixed 2.x/3.x cluster, also fix potential non-resolved duplication of

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 1be3175..b95e88d 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1482,7 +1482,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public Iterable<SSTableReader> getSSTables(SSTableSet sstableSet)
     {
-        return data.getView().sstables(sstableSet);
+        return data.getView().select(sstableSet);
     }
 
     public Iterable<SSTableReader> getUncompactingSSTables()
@@ -1916,7 +1916,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public Iterable<DecoratedKey> keySamples(Range<Token> range)
     {
-        try (RefViewFragment view = selectAndReference(View.select(SSTableSet.CANONICAL)))
+        try (RefViewFragment view = selectAndReference(View.selectFunction(SSTableSet.CANONICAL)))
         {
             Iterable<DecoratedKey>[] samples = new Iterable[view.sstables.size()];
             int i = 0;
@@ -1930,7 +1930,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public long estimatedKeysForRange(Range<Token> range)
     {
-        try (RefViewFragment view = selectAndReference(View.select(SSTableSet.CANONICAL)))
+        try (RefViewFragment view = selectAndReference(View.selectFunction(SSTableSet.CANONICAL)))
         {
             long count = 0;
             for (SSTableReader sstable : view.sstables)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
index 3461aef..0b31b87 100644
--- a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
+++ b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
@@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.db.lifecycle.SSTableIntervalTree;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
 import org.apache.cassandra.db.lifecycle.View;
 import org.apache.cassandra.dht.Range;
@@ -103,8 +104,10 @@ public class SizeEstimatesRecorder extends MigrationListener implements Runnable
             {
                 while (refs == null)
                 {
-                    // note that this is not guaranteed to return all sstables within the ranges, but for an estimated size, that should be fine
-                    Iterable<SSTableReader> canonicalSSTables = table.getTracker().getView().select(SSTableSet.CANONICAL, table.select(View.selectLive(Range.makeRowRange(range))).sstables);
+                    Iterable<SSTableReader> sstables = table.getTracker().getView().select(SSTableSet.CANONICAL);
+                    SSTableIntervalTree tree = SSTableIntervalTree.build(sstables);
+                    Range<PartitionPosition> r = Range.makeRowRange(range);
+                    Iterable<SSTableReader> canonicalSSTables = View.sstablesInBounds(r.left, r.right, tree);
                     refs = Refs.tryRef(canonicalSSTables);
                 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index 16090a1..c94b88f 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -401,7 +401,7 @@ public class Tracker
 
     public Iterable<SSTableReader> getUncompacting()
     {
-        return view.get().sstables(SSTableSet.NONCOMPACTING);
+        return view.get().select(SSTableSet.NONCOMPACTING);
     }
 
     public Iterable<SSTableReader> getUncompacting(Iterable<SSTableReader> candidates)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/src/java/org/apache/cassandra/db/lifecycle/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/View.java b/src/java/org/apache/cassandra/db/lifecycle/View.java
index 96aaa49..3fa197f 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/View.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/View.java
@@ -118,14 +118,9 @@ public class View
         return sstables;
     }
 
-    public Iterable<SSTableReader> sstables(SSTableSet sstableSet)
-    {
-        return select(sstableSet, sstables);
-    }
-
     public Iterable<SSTableReader> sstables(SSTableSet sstableSet, Predicate<SSTableReader> filter)
     {
-        return select(sstableSet, filter(sstables, filter));
+        return filter(select(sstableSet), filter);
     }
 
     // any sstable known by this tracker in any form; we have a special method here since it's only used for testing/debug
@@ -136,7 +131,7 @@ public class View
         return Iterables.concat(sstables, filterOut(compacting, sstables));
     }
 
-    public Iterable<SSTableReader> select(SSTableSet sstableSet, Iterable<SSTableReader> sstables)
+    public Iterable<SSTableReader> select(SSTableSet sstableSet)
     {
         switch (sstableSet)
         {
@@ -145,9 +140,18 @@ public class View
             case NONCOMPACTING:
                 return filter(sstables, (s) -> !compacting.contains(s));
             case CANONICAL:
-                return transform(filter(sstables,
-                                        (s) -> s.openReason != SSTableReader.OpenReason.EARLY),
-                                 (s) -> s.openReason != SSTableReader.OpenReason.MOVED_START ? s : compactingMap.get(s));
+                Set<SSTableReader> canonicalSSTables = new HashSet<>();
+                for (SSTableReader sstable : compacting)
+                    if (sstable.openReason != SSTableReader.OpenReason.EARLY)
+                        canonicalSSTables.add(sstable);
+                // reason for checking if compacting contains the sstable is that if compacting has an EARLY version
+                // of a NORMAL sstable, we still have the canonical version of that sstable in sstables.
+                // note that the EARLY version is equal, but not == since it is a different instance of the same sstable.
+                for (SSTableReader sstable : sstables)
+                    if (!compacting.contains(sstable) && sstable.openReason != SSTableReader.OpenReason.EARLY)
+                        canonicalSSTables.add(sstable);
+
+                return canonicalSSTables;
             default:
                 throw new IllegalStateException();
         }
@@ -190,7 +194,7 @@ public class View
             return Collections.emptyList();
 
         PartitionPosition stopInTree = right.isMinimum() ? intervalTree.max() : right;
-        return select(SSTableSet.LIVE, intervalTree.search(Interval.create(left, stopInTree)));
+        return intervalTree.search(Interval.create(left, stopInTree));
     }
 
     public static List<SSTableReader> sstablesInBounds(PartitionPosition left, PartitionPosition right, SSTableIntervalTree intervalTree)
@@ -204,9 +208,9 @@ public class View
         return intervalTree.search(Interval.create(left, stopInTree));
     }
 
-    public static Function<View, Iterable<SSTableReader>> select(SSTableSet sstableSet)
+    public static Function<View, Iterable<SSTableReader>> selectFunction(SSTableSet sstableSet)
     {
-        return (view) -> view.sstables(sstableSet);
+        return (view) -> view.select(sstableSet);
     }
 
     public static Function<View, Iterable<SSTableReader>> select(SSTableSet sstableSet, Predicate<SSTableReader> filter)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/src/java/org/apache/cassandra/db/view/ViewBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilder.java b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
index b2b409b..b55eda0 100644
--- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
@@ -108,7 +108,7 @@ public class ViewBuilder extends CompactionInfo.Holder
         if (buildStatus == null)
         {
             baseCfs.forceBlockingFlush();
-            function = org.apache.cassandra.db.lifecycle.View.select(SSTableSet.CANONICAL);
+            function = org.apache.cassandra.db.lifecycle.View.selectFunction(SSTableSet.CANONICAL);
             int generation = Integer.MIN_VALUE;
 
             try (Refs<SSTableReader> temp = baseCfs.selectAndReference(function).refs)
@@ -129,7 +129,7 @@ public class ViewBuilder extends CompactionInfo.Holder
                 @Nullable
                 public Iterable<SSTableReader> apply(org.apache.cassandra.db.lifecycle.View view)
                 {
-                    Iterable<SSTableReader> readers = org.apache.cassandra.db.lifecycle.View.select(SSTableSet.CANONICAL).apply(view);
+                    Iterable<SSTableReader> readers = org.apache.cassandra.db.lifecycle.View.selectFunction(SSTableSet.CANONICAL).apply(view);
                     if (readers != null)
                         return Iterables.filter(readers, ssTableReader -> ssTableReader.descriptor.generation <= buildStatus.left);
                     return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index 9635c59..6dfdeee 100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@ -271,7 +271,7 @@ public class SecondaryIndexManager implements IndexRegistry
     {
         if (index.shouldBuildBlocking())
         {
-            try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.select(SSTableSet.CANONICAL));
+            try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL));
                  Refs<SSTableReader> sstables = viewFragment.refs)
             {
                 buildIndexesBlocking(sstables, Collections.singleton(index));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
index 9d997a7..2a0dec0 100644
--- a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
@@ -699,7 +699,7 @@ public abstract class CassandraIndex implements Index
     {
         baseCfs.forceBlockingFlush();
 
-        try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.select(SSTableSet.CANONICAL));
+        try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL));
              Refs<SSTableReader> sstables = viewFragment.refs)
         {
             if (sstables.isEmpty())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
index aed35c9..ddda430 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
@@ -208,7 +208,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
                 do
                 {
                     View view = cfStore.getTracker().getView();
-                    allSSTables = ImmutableSet.copyOf(view.sstables(SSTableSet.CANONICAL));
+                    allSSTables = ImmutableSet.copyOf(view.select(SSTableSet.CANONICAL));
                     nonCompacting = ImmutableSet.copyOf(view.getUncompacting(allSSTables));
                 }
                 while (null == (txn = cfStore.getTracker().tryModify(nonCompacting, OperationType.UNKNOWN)));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index a14f815..bfbedc7 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -329,7 +329,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
                     keyRanges.add(Range.makeRowRange(range));
                 refs.addAll(cfStore.selectAndReference(view -> {
                     Set<SSTableReader> sstables = Sets.newHashSet();
-                    SSTableIntervalTree intervalTree = SSTableIntervalTree.build(view.sstables(SSTableSet.CANONICAL));
+                    SSTableIntervalTree intervalTree = SSTableIntervalTree.build(view.select(SSTableSet.CANONICAL));
                     for (Range<PartitionPosition> keyRange : keyRanges)
                     {
                         // keyRange excludes its start, while sstableInBounds is inclusive (of both start and end).
@@ -346,7 +346,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
                     }
 
                     if (logger.isDebugEnabled())
-                        logger.debug("ViewFilter for {}/{} sstables", sstables.size(), Iterables.size(view.sstables(SSTableSet.CANONICAL)));
+                        logger.debug("ViewFilter for {}/{} sstables", sstables.size(), Iterables.size(view.select(SSTableSet.CANONICAL)));
                     return sstables;
                 }).refs);
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
index 98f9300..a5dceca 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
@@ -100,8 +100,8 @@ public class ViewTest
         Assert.assertFalse(View.permitCompacting(readers.subList(1, 2)).apply(cur));
         Assert.assertTrue(readers.subList(2, 5).containsAll(copyOf(cur.getUncompacting(readers))));
         Assert.assertEquals(3, copyOf(cur.getUncompacting(readers)).size());
-        Assert.assertTrue(ImmutableSet.copyOf(cur.sstables(SSTableSet.NONCOMPACTING)).containsAll(readers.subList(2, 5)));
-        Assert.assertEquals(3, ImmutableSet.copyOf(cur.sstables(SSTableSet.NONCOMPACTING)).size());
+        Assert.assertTrue(ImmutableSet.copyOf(cur.select(SSTableSet.NONCOMPACTING)).containsAll(readers.subList(2, 5)));
+        Assert.assertEquals(3, ImmutableSet.copyOf(cur.select(SSTableSet.NONCOMPACTING)).size());
 
         // check marking already compacting readers fails with an exception
         testFailure(View.updateCompacting(emptySet(), readers.subList(0, 1)), cur);
@@ -129,7 +129,7 @@ public class ViewTest
         testFailure(View.updateCompacting(copyOf(readers.subList(0, 2)), emptySet()), cur);
         Assert.assertTrue(copyOf(concat(readers.subList(0, 1), readers.subList(2, 5))).containsAll(copyOf(cur.getUncompacting(readers))));
         Assert.assertEquals(4, copyOf(cur.getUncompacting(readers)).size());
-        Set<SSTableReader> nonCompacting = ImmutableSet.copyOf(cur.sstables(SSTableSet.NONCOMPACTING));
+        Set<SSTableReader> nonCompacting = ImmutableSet.copyOf(cur.select(SSTableSet.NONCOMPACTING));
         Assert.assertTrue(nonCompacting.containsAll(readers.subList(2, 5)));
         Assert.assertTrue(nonCompacting.containsAll(readers.subList(0, 1)));
         Assert.assertEquals(4, nonCompacting.size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
index 6aaefb7..2124abe 100644
--- a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
+++ b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
@@ -623,7 +623,7 @@ public class CustomCassandraIndex implements Index
     {
         baseCfs.forceBlockingFlush();
 
-        try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.select(SSTableSet.CANONICAL));
+        try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL));
              Refs<SSTableReader> sstables = viewFragment.refs)
         {
             if (sstables.isEmpty())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc23632f/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
index 18bc760..c842b7f 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -759,7 +759,7 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
                 if (!checked && writer.currentWriter().getFilePointer() > 15000000)
                 {
                     checked = true;
-                    ColumnFamilyStore.ViewFragment viewFragment = cfs.select(View.select(SSTableSet.CANONICAL));
+                    ColumnFamilyStore.ViewFragment viewFragment = cfs.select(View.selectFunction(SSTableSet.CANONICAL));
                     // canonical view should have only one SSTable which is not opened early.
                     assertEquals(1, viewFragment.sstables.size());
                     SSTableReader sstable = viewFragment.sstables.get(0);
@@ -872,6 +872,45 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
         validateCFS(cfs);
     }
 
+    @Test
+    public void testCanonicalSSTables() throws ExecutionException, InterruptedException
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
+        truncate(cfs);
+
+        cfs.addSSTable(writeFile(cfs, 100));
+        Collection<SSTableReader> allSSTables = cfs.getSSTables();
+        assertEquals(1, allSSTables.size());
+        final AtomicBoolean done = new AtomicBoolean(false);
+        final AtomicBoolean failed = new AtomicBoolean(false);
+        Runnable r = () -> {
+            while (!done.get())
+            {
+                Iterable<SSTableReader> sstables = cfs.getSSTables(SSTableSet.CANONICAL);
+                if (Iterables.size(sstables) != 1)
+                {
+                    failed.set(true);
+                    return;
+                }
+            }
+        };
+        Thread t = new Thread(r);
+        try
+        {
+            t.start();
+            cfs.forceMajorCompaction();
+        }
+        finally
+        {
+            done.set(true);
+            t.join(20);
+        }
+        assertFalse(failed.get());
+
+
+    }
+
     private void validateKeys(Keyspace ks)
     {
         for (int i = 0; i < 100; i++)


[14/50] cassandra git commit: Merge commit '43c741e251102bf5651ff8aa1b5ca078eb0ddc0b' into cassandra-3.0

Posted by al...@apache.org.
Merge commit '43c741e251102bf5651ff8aa1b5ca078eb0ddc0b' into cassandra-3.0

* commit '43c741e251102bf5651ff8aa1b5ca078eb0ddc0b':
  Range tombstones that are masked by row tombstones should not be written out


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

Branch: refs/heads/cassandra-3.8
Commit: 9ed3b42d3b50237f99485233857a7b34d5238d9a
Parents: dd05e46 43c741e
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Jul 6 14:39:52 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jul 6 14:39:52 2016 +0200

----------------------------------------------------------------------

----------------------------------------------------------------------



[16/50] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

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


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

Branch: refs/heads/cassandra-3.8
Commit: 778f2a46e2df52aa8451aceaf17046e6b8c86ace
Parents: 9ed3b42 00e7ecf
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Jul 6 12:33:54 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Jul 6 12:33:54 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/streaming/ConnectionHandler.java  |  8 ++--
 .../cassandra/streaming/StreamReceiveTask.java  | 50 +++++++++++++++-----
 .../cassandra/streaming/StreamSession.java      | 17 +++++--
 .../streaming/StreamingTransferTest.java        | 30 ++++++++++--
 5 files changed, 83 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/778f2a46/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 02786c5,7d62f97..8118de1
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,26 -1,14 +1,27 @@@
 -2.2.8
 +3.0.9
+  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
 + * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
 + * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
 + * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
 + * Fix column ordering of results with static columns for Thrift requests in
 +   a mixed 2.x/3.x cluster, also fix potential non-resolved duplication of
 +   those static columns in query results (CASSANDRA-12123)
 + * Avoid digest mismatch with empty but static rows (CASSANDRA-12090)
 + * Fix EOF exception when altering column type (CASSANDRA-11820)
 +Merged from 2.2:
   * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
  Merged from 2.1:
 - * Don't write shadowed range tombstone (CASSANDRA-12030)
 - * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)
   * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
 - * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
  
  
 -2.2.7
 +3.0.8
 + * Fix potential race in schema during new table creation (CASSANDRA-12083)
 + * cqlsh: fix error handling in rare COPY FROM failure scenario (CASSANDRA-12070)
 + * Disable autocompaction during drain (CASSANDRA-11878)
 + * Add a metrics timer to MemtablePool and use it to track time spent blocked on memory in MemtableAllocator (CASSANDRA-11327)
 + * Fix upgrading schema with super columns with non-text subcomparators (CASSANDRA-12023)
 + * Add TimeWindowCompactionStrategy (CASSANDRA-9666)
 +Merged from 2.2:
   * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
   * Validate bloom_filter_fp_chance against lowest supported
     value when the table is created (CASSANDRA-11920)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/778f2a46/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index 6280f3a,b342edc..040906b
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@@ -17,9 -17,7 +17,6 @@@
   */
  package org.apache.cassandra.streaming;
  
--import java.io.File;
- import java.io.IOError;
- import java.io.IOException;
  import java.util.ArrayList;
  import java.util.Collection;
  import java.util.List;
@@@ -36,19 -33,13 +33,20 @@@ import org.apache.cassandra.concurrent.
  import org.apache.cassandra.config.Schema;
  import org.apache.cassandra.db.ColumnFamilyStore;
  import org.apache.cassandra.db.Keyspace;
 +import org.apache.cassandra.db.Mutation;
 +import org.apache.cassandra.db.compaction.OperationType;
 +import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 +import org.apache.cassandra.db.view.View;
  import org.apache.cassandra.dht.Bounds;
  import org.apache.cassandra.dht.Token;
 +import org.apache.cassandra.io.sstable.ISSTableScanner;
 +import org.apache.cassandra.io.sstable.SSTableMultiWriter;
  import org.apache.cassandra.io.sstable.format.SSTableReader;
 -import org.apache.cassandra.io.sstable.format.SSTableWriter;
  import org.apache.cassandra.utils.JVMStabilityInspector;
  import org.apache.cassandra.utils.Pair;
 -
++import org.apache.cassandra.utils.Throwables;
  import org.apache.cassandra.utils.concurrent.Refs;
  
  /**
@@@ -65,16 -55,11 +63,16 @@@ public class StreamReceiveTask extends 
      // total size of files to receive
      private final long totalSize;
  
 +    // Transaction tracking new files received
-     public final LifecycleTransaction txn;
++    private final LifecycleTransaction txn;
 +
      // true if task is done (either completed or aborted)
--    private boolean done = false;
++    private volatile boolean done = false;
  
      //  holds references to SSTables received
 -    protected Collection<SSTableWriter> sstables;
 +    protected Collection<SSTableReader> sstables;
 +
 +    private int remoteSSTablesReceived = 0;
  
      public StreamReceiveTask(StreamSession session, UUID cfId, int totalFiles, long totalSize)
      {
@@@ -92,18 -74,16 +90,32 @@@
       *
       * @param sstable SSTable file received.
       */
 -    public synchronized void received(SSTableWriter sstable)
 +    public synchronized void received(SSTableMultiWriter sstable)
      {
          if (done)
++        {
++            logger.warn("[{}] Received sstable {} on already finished stream received task. Aborting sstable.", session.planId(),
++                        sstable.getFilename());
++            Throwables.maybeFail(sstable.abort(null));
              return;
++        }
+ 
 -        assert cfId.equals(sstable.metadata.cfId);
 +        remoteSSTablesReceived++;
 +        assert cfId.equals(sstable.getCfId());
  
-         Collection<SSTableReader> finished = sstable.finish(true);
 -        sstables.add(sstable);
++        Collection<SSTableReader> finished = null;
++        try
++        {
++            finished = sstable.finish(true);
++        }
++        catch (Throwable t)
++        {
++            Throwables.maybeFail(sstable.abort(t));
++        }
 +        txn.update(finished, false);
 +        sstables.addAll(finished);
  
 -        if (sstables.size() == totalFiles)
 +        if (remoteSSTablesReceived == totalFiles)
          {
              done = true;
              executor.submit(new OnCompletionRunnable(this));
@@@ -120,6 -100,6 +132,13 @@@
          return totalSize;
      }
  
++    public synchronized LifecycleTransaction getTransaction()
++    {
++        if (done)
++            throw new RuntimeException(String.format("Stream receive task {} of cf {} already finished.", session.planId(), cfId));
++        return txn;
++    }
++
      private static class OnCompletionRunnable implements Runnable
      {
          private final StreamReceiveTask task;
@@@ -139,71 -117,52 +158,71 @@@
                  if (kscf == null)
                  {
                      // schema was dropped during streaming
 -                    for (SSTableWriter writer : task.sstables)
 -                        writer.abort();
                      task.sstables.clear();
-                     task.txn.abort();
++                    task.abortTransaction();
 +                    task.session.taskCompleted(task);
                      return;
                  }
 -                ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
 +                cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
 +                hasViews = !Iterables.isEmpty(View.findAll(kscf.left, kscf.right));
  
 -                File lockfiledir = cfs.directories.getWriteableLocationAsFile(task.sstables.size() * 256L);
 -                StreamLockfile lockfile = new StreamLockfile(lockfiledir, UUID.randomUUID());
 -                lockfile.create(task.sstables);
 -                List<SSTableReader> readers = new ArrayList<>();
 -                for (SSTableWriter writer : task.sstables)
 -                    readers.add(writer.finish(true));
 -                lockfile.delete();
 -                task.sstables.clear();
 +                Collection<SSTableReader> readers = task.sstables;
  
                  try (Refs<SSTableReader> refs = Refs.ref(readers))
                  {
 -                    // add sstables and build secondary indexes
 -                    cfs.addSSTables(readers);
 -                    cfs.indexManager.maybeBuildSecondaryIndexes(readers, cfs.indexManager.allIndexesNames());
 -
 -                    //invalidate row and counter cache
 -                    if (cfs.isRowCacheEnabled() || cfs.metadata.isCounter())
 +                    //We have a special path for views.
 +                    //Since the view requires cleaning up any pre-existing state, we must put
 +                    //all partitions through the same write path as normal mutations.
 +                    //This also ensures any 2is are also updated
 +                    if (hasViews)
                      {
 -                        List<Bounds<Token>> boundsToInvalidate = new ArrayList<>(readers.size());
 -                        for (SSTableReader sstable : readers)
 -                            boundsToInvalidate.add(new Bounds<Token>(sstable.first.getToken(), sstable.last.getToken()));
 -                        Set<Bounds<Token>> nonOverlappingBounds = Bounds.getNonOverlappingBounds(boundsToInvalidate);
 -
 -                        if (cfs.isRowCacheEnabled())
 +                        for (SSTableReader reader : readers)
                          {
 -                            int invalidatedKeys = cfs.invalidateRowCache(nonOverlappingBounds);
 -                            if (invalidatedKeys > 0)
 -                                logger.debug("[Stream #{}] Invalidated {} row cache entries on table {}.{} after stream " +
 -                                             "receive task completed.", task.session.planId(), invalidatedKeys,
 -                                             cfs.keyspace.getName(), cfs.getColumnFamilyName());
 +                            try (ISSTableScanner scanner = reader.getScanner())
 +                            {
 +                                while (scanner.hasNext())
 +                                {
 +                                    try (UnfilteredRowIterator rowIterator = scanner.next())
 +                                    {
 +                                        //Apply unsafe (we will flush below before transaction is done)
 +                                        new Mutation(PartitionUpdate.fromIterator(rowIterator)).applyUnsafe();
 +                                    }
 +                                }
 +                            }
                          }
 +                    }
 +                    else
 +                    {
-                         task.txn.finish();
++                        task.finishTransaction();
  
 -                        if (cfs.metadata.isCounter())
 +                        // add sstables and build secondary indexes
 +                        cfs.addSSTables(readers);
 +                        cfs.indexManager.buildAllIndexesBlocking(readers);
 +
 +                        //invalidate row and counter cache
 +                        if (cfs.isRowCacheEnabled() || cfs.metadata.isCounter())
                          {
 -                            int invalidatedKeys = cfs.invalidateCounterCache(nonOverlappingBounds);
 -                            if (invalidatedKeys > 0)
 -                                logger.debug("[Stream #{}] Invalidated {} counter cache entries on table {}.{} after stream " +
 -                                             "receive task completed.", task.session.planId(), invalidatedKeys,
 -                                             cfs.keyspace.getName(), cfs.getColumnFamilyName());
 +                            List<Bounds<Token>> boundsToInvalidate = new ArrayList<>(readers.size());
 +                            readers.forEach(sstable -> boundsToInvalidate.add(new Bounds<Token>(sstable.first.getToken(), sstable.last.getToken())));
 +                            Set<Bounds<Token>> nonOverlappingBounds = Bounds.getNonOverlappingBounds(boundsToInvalidate);
 +
 +                            if (cfs.isRowCacheEnabled())
 +                            {
 +                                int invalidatedKeys = cfs.invalidateRowCache(nonOverlappingBounds);
 +                                if (invalidatedKeys > 0)
 +                                    logger.debug("[Stream #{}] Invalidated {} row cache entries on table {}.{} after stream " +
 +                                                 "receive task completed.", task.session.planId(), invalidatedKeys,
 +                                                 cfs.keyspace.getName(), cfs.getTableName());
 +                            }
 +
 +                            if (cfs.metadata.isCounter())
 +                            {
 +                                int invalidatedKeys = cfs.invalidateCounterCache(nonOverlappingBounds);
 +                                if (invalidatedKeys > 0)
 +                                    logger.debug("[Stream #{}] Invalidated {} counter cache entries on table {}.{} after stream " +
 +                                                 "receive task completed.", task.session.planId(), invalidatedKeys,
 +                                                 cfs.keyspace.getName(), cfs.getTableName());
 +                            }
                          }
                      }
                  }
@@@ -211,21 -171,10 +230,20 @@@
              }
              catch (Throwable t)
              {
--                logger.error("Error applying streamed data: ", t);
                  JVMStabilityInspector.inspectThrowable(t);
                  task.session.onError(t);
              }
 +            finally
 +            {
 +                //We don't keep the streamed sstables since we've applied them manually
 +                //So we abort the txn and delete the streamed sstables
 +                if (hasViews)
 +                {
 +                    if (cfs != null)
 +                        cfs.forceBlockingFlush();
-                     task.txn.abort();
++                    task.abortTransaction();
 +                }
 +            }
          }
      }
  
@@@ -241,7 -190,8 +259,17 @@@
              return;
  
          done = true;
-         txn.abort();
 -        for (SSTableWriter writer : sstables)
 -            writer.abort();
++        abortTransaction();
          sstables.clear();
      }
++
++    private synchronized void abortTransaction()
++    {
++        txn.abort();
++    }
++
++    private synchronized void finishTransaction()
++    {
++        txn.finish();
++    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/778f2a46/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/StreamSession.java
index bfbedc7,f4c900e..12f561b
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@@ -211,12 -212,6 +211,12 @@@ public class StreamSession implements I
      }
  
  
 +    public LifecycleTransaction getTransaction(UUID cfId)
 +    {
 +        assert receivers.containsKey(cfId);
-         return receivers.get(cfId).txn;
++        return receivers.get(cfId).getTransaction();
 +    }
 +
      /**
       * Bind this session to report to specific {@link StreamResultFuture} and
       * perform pre-streaming initialization.
@@@ -281,8 -276,8 +281,9 @@@
       * @param flushTables flush tables?
       * @param repairedAt the time the repair started.
       */
--    public void addTransferRanges(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies, boolean flushTables, long repairedAt)
++    public synchronized void addTransferRanges(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies, boolean flushTables, long repairedAt)
      {
++        failIfFinished();
          Collection<ColumnFamilyStore> stores = getColumnFamilyStores(keyspace, columnFamilies);
          if (flushTables)
              flushSSTables(stores);
@@@ -300,6 -295,6 +301,12 @@@
          }
      }
  
++    private void failIfFinished()
++    {
++        if (state() == State.COMPLETE || state() == State.FAILED)
++            throw new RuntimeException(String.format("Stream %s is finished with state %s", planId(), state().name()));
++    }
++
      private Collection<ColumnFamilyStore> getColumnFamilyStores(String keyspace, Collection<String> columnFamilies)
      {
          Collection<ColumnFamilyStore> stores = new HashSet<>();
@@@ -371,8 -369,8 +378,9 @@@
          }
      }
  
--    public void addTransferFiles(Collection<SSTableStreamingSections> sstableDetails)
++    public synchronized void addTransferFiles(Collection<SSTableStreamingSections> sstableDetails)
      {
++        failIfFinished();
          Iterator<SSTableStreamingSections> iter = sstableDetails.iterator();
          while (iter.hasNext())
          {
@@@ -745,8 -743,8 +753,9 @@@
          FBUtilities.waitOnFutures(flushes);
      }
  
--    private void prepareReceiving(StreamSummary summary)
++    private synchronized void prepareReceiving(StreamSummary summary)
      {
++        failIfFinished();
          if (summary.files > 0)
              receivers.put(summary.cfId, new StreamReceiveTask(this, summary.cfId, summary.files, summary.totalSize));
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/778f2a46/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index 7223e76,2b16267..6be880c
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@@ -229,14 -238,14 +229,38 @@@ public class StreamingTransferTes
          List<Range<Token>> ranges = new ArrayList<>();
          // wrapped range
          ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("key1")), p.getToken(ByteBufferUtil.bytes("key0"))));
--        new StreamPlan("StreamingTransferTest").transferRanges(LOCAL, cfs.keyspace.getName(), ranges, cfs.getColumnFamilyName()).execute().get();
++        StreamPlan streamPlan = new StreamPlan("StreamingTransferTest").transferRanges(LOCAL, cfs.keyspace.getName(), ranges, cfs.getColumnFamilyName());
++        streamPlan.execute().get();
          verifyConnectionsAreClosed();
++
++        //cannot add ranges after stream session is finished
++        try
++        {
++            streamPlan.transferRanges(LOCAL, cfs.keyspace.getName(), ranges, cfs.getColumnFamilyName());
++            fail("Should have thrown exception");
++        }
++        catch (RuntimeException e)
++        {
++            //do nothing
++        }
      }
  
      private void transfer(SSTableReader sstable, List<Range<Token>> ranges) throws Exception
      {
--        new StreamPlan("StreamingTransferTest").transferFiles(LOCAL, makeStreamingDetails(ranges, Refs.tryRef(Arrays.asList(sstable)))).execute().get();
++        StreamPlan streamPlan = new StreamPlan("StreamingTransferTest").transferFiles(LOCAL, makeStreamingDetails(ranges, Refs.tryRef(Arrays.asList(sstable))));
++        streamPlan.execute().get();
          verifyConnectionsAreClosed();
++
++        //cannot add files after stream session is finished
++        try
++        {
++            streamPlan.transferFiles(LOCAL, makeStreamingDetails(ranges, Refs.tryRef(Arrays.asList(sstable))));
++            fail("Should have thrown exception");
++        }
++        catch (RuntimeException e)
++        {
++            //do nothing
++        }
      }
  
      /**
@@@ -312,36 -325,27 +336,36 @@@
          String cfname = "StandardInteger1";
          Keyspace keyspace = Keyspace.open(ks);
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
 +        ClusteringComparator comparator = cfs.getComparator();
  
 -        String key = "key0";
 -        Mutation rm = new Mutation(ks, ByteBufferUtil.bytes(key));
 -        // add columns of size slightly less than column_index_size to force insert column index
 -        rm.add(cfname, cellname(1), ByteBuffer.wrap(new byte[DatabaseDescriptor.getColumnIndexSize() - 64]), 2);
 -        rm.add(cfname, cellname(6), ByteBuffer.wrap(new byte[DatabaseDescriptor.getColumnIndexSize()]), 2);
 -        ColumnFamily cf = rm.addOrGet(cfname);
 -        // add RangeTombstones
 -        cf.delete(new DeletionInfo(cellname(2), cellname(3), cf.getComparator(), 1, (int) (System.currentTimeMillis() / 1000)));
 -        cf.delete(new DeletionInfo(cellname(5), cellname(7), cf.getComparator(), 1, (int) (System.currentTimeMillis() / 1000)));
 -        cf.delete(new DeletionInfo(cellname(8), cellname(10), cf.getComparator(), 1, (int) (System.currentTimeMillis() / 1000)));
 -        rm.applyUnsafe();
 +        String key = "key1";
 +
 +
 +        RowUpdateBuilder updates = new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros(), key);
  
 -        key = "key1";
 -        rm = new Mutation(ks, ByteBufferUtil.bytes(key));
          // add columns of size slightly less than column_index_size to force insert column index
 -        rm.add(cfname, cellname(1), ByteBuffer.wrap(new byte[DatabaseDescriptor.getColumnIndexSize() - 64]), 2);
 -        cf = rm.addOrGet(cfname);
 +        updates.clustering(1)
 +                .add("val", ByteBuffer.wrap(new byte[DatabaseDescriptor.getColumnIndexSize() - 64]))
 +                .build()
 +                .apply();
 +
 +        updates = new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros(), key);
 +        updates.clustering(6)
 +                .add("val", ByteBuffer.wrap(new byte[DatabaseDescriptor.getColumnIndexSize()]))
-                .build()
++                .build()
 +                .apply();
 +
          // add RangeTombstones
 -        cf.delete(new DeletionInfo(cellname(2), cellname(3), cf.getComparator(), 1, (int) (System.currentTimeMillis() / 1000)));
 -        rm.applyUnsafe();
 +        //updates = new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros() + 1 , key);
 +        //updates.addRangeTombstone(Slice.make(comparator, comparator.make(2), comparator.make(4)))
 +        //        .build()
 +        //        .apply();
 +
 +
 +        updates = new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros() + 1, key);
 +        updates.addRangeTombstone(Slice.make(comparator.make(5), comparator.make(7)))
 +                .build()
 +                .apply();
  
          cfs.forceBlockingFlush();
  


[41/50] cassandra git commit: Wait until the message is being send to decide which serializer must be used

Posted by al...@apache.org.
Wait until the message is being send to decide which serializer must be used

patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-11393


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

Branch: refs/heads/cassandra-3.8
Commit: fbd287ad2ed09190dd9c6e152b82215e81020847
Parents: e99ee19
Author: Benjamin Lerer <b....@gmail.com>
Authored: Thu Jul 14 11:33:08 2016 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Thu Jul 14 11:33:08 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/config/DatabaseDescriptor.java    |  1 +
 .../cassandra/db/PartitionRangeReadCommand.java |  7 +--
 .../org/apache/cassandra/db/ReadCommand.java    | 65 ++++++++++----------
 .../org/apache/cassandra/db/ReadResponse.java   | 43 +++++--------
 .../db/SinglePartitionReadCommand.java          |  2 +-
 .../io/ForwardingVersionedSerializer.java       | 57 +++++++++++++++++
 .../apache/cassandra/net/MessagingService.java  |  6 +-
 8 files changed, 113 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fbd287ad/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 70210a8..3829046 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Wait until the message is being send to decide which serializer must be used (CASSANDRA-11393)
  * Fix migration of static thrift column names with non-text comparators (CASSANDRA-12147)
  * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)
  * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fbd287ad/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 100bcf4..b71ebf6 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -1124,6 +1124,7 @@ public class DatabaseDescriptor
             case READ:
                 return getReadRpcTimeout();
             case RANGE_SLICE:
+            case PAGED_RANGE:
                 return getRangeRpcTimeout();
             case TRUNCATE:
                 return getTruncateRpcTimeout();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fbd287ad/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 842ad5f..99e24c8 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@ -253,12 +253,9 @@ public class PartitionRangeReadCommand extends ReadCommand
 
     public MessageOut<ReadCommand> createMessage(int version)
     {
-        if (version >= MessagingService.VERSION_30)
-            return new MessageOut<>(MessagingService.Verb.RANGE_SLICE, this, serializer);
-
         return dataRange().isPaging()
-             ? new MessageOut<>(MessagingService.Verb.PAGED_RANGE, this, legacyPagedRangeCommandSerializer)
-             : new MessageOut<>(MessagingService.Verb.RANGE_SLICE, this, legacyRangeSliceCommandSerializer);
+             ? new MessageOut<>(MessagingService.Verb.PAGED_RANGE, this, pagedRangeSerializer)
+             : new MessageOut<>(MessagingService.Verb.RANGE_SLICE, this, rangeSliceSerializer);
     }
 
     protected void appendCQLWhereClause(StringBuilder sb)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fbd287ad/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index c792a5a..36969f8 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -34,6 +34,7 @@ import org.apache.cassandra.db.transform.Transformation;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.index.IndexNotAvailableException;
+import org.apache.cassandra.io.ForwardingVersionedSerializer;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -58,9 +59,39 @@ public abstract class ReadCommand implements ReadQuery
     protected static final Logger logger = LoggerFactory.getLogger(ReadCommand.class);
 
     public static final IVersionedSerializer<ReadCommand> serializer = new Serializer();
+
+    // For READ verb: will either dispatch on 'serializer' for 3.0 or 'legacyReadCommandSerializer' for earlier version.
+    // Can be removed (and replaced by 'serializer') once we drop pre-3.0 backward compatibility.
+    public static final IVersionedSerializer<ReadCommand> readSerializer = new ForwardingVersionedSerializer<ReadCommand>()
+    {
+        protected IVersionedSerializer<ReadCommand> delegate(int version)
+        {
+            return version < MessagingService.VERSION_30
+                    ? legacyReadCommandSerializer : serializer;
+        }
+    };
+
     // For RANGE_SLICE verb: will either dispatch on 'serializer' for 3.0 or 'legacyRangeSliceCommandSerializer' for earlier version.
     // Can be removed (and replaced by 'serializer') once we drop pre-3.0 backward compatibility.
-    public static final IVersionedSerializer<ReadCommand> rangeSliceSerializer = new RangeSliceSerializer();
+    public static final IVersionedSerializer<ReadCommand> rangeSliceSerializer = new ForwardingVersionedSerializer<ReadCommand>()
+    {
+        protected IVersionedSerializer<ReadCommand> delegate(int version)
+        {
+            return version < MessagingService.VERSION_30
+                    ? legacyRangeSliceCommandSerializer : serializer;
+        }
+    };
+
+    // For PAGED_RANGE verb: will either dispatch on 'serializer' for 3.0 or 'legacyPagedRangeCommandSerializer' for earlier version.
+    // Can be removed (and replaced by 'serializer') once we drop pre-3.0 backward compatibility.
+    public static final IVersionedSerializer<ReadCommand> pagedRangeSerializer = new ForwardingVersionedSerializer<ReadCommand>()
+    {
+        protected IVersionedSerializer<ReadCommand> delegate(int version)
+        {
+            return version < MessagingService.VERSION_30
+                    ? legacyPagedRangeCommandSerializer : serializer;
+        }
+    };
 
     public static final IVersionedSerializer<ReadCommand> legacyRangeSliceCommandSerializer = new LegacyRangeSliceCommandSerializer();
     public static final IVersionedSerializer<ReadCommand> legacyPagedRangeCommandSerializer = new LegacyPagedRangeCommandSerializer();
@@ -567,7 +598,6 @@ public abstract class ReadCommand implements ReadQuery
 
         public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
         {
-            // for serialization, createLegacyMessage() should cause legacyReadCommandSerializer to be used directly
             assert version >= MessagingService.VERSION_30;
 
             out.writeByte(command.kind.ordinal());
@@ -587,8 +617,7 @@ public abstract class ReadCommand implements ReadQuery
 
         public ReadCommand deserialize(DataInputPlus in, int version) throws IOException
         {
-            if (version < MessagingService.VERSION_30)
-                return legacyReadCommandSerializer.deserialize(in, version);
+            assert version >= MessagingService.VERSION_30;
 
             Kind kind = Kind.values()[in.readByte()];
             int flags = in.readByte();
@@ -628,7 +657,6 @@ public abstract class ReadCommand implements ReadQuery
 
         public long serializedSize(ReadCommand command, int version)
         {
-            // for serialization, createLegacyMessage() should cause legacyReadCommandSerializer to be used directly
             assert version >= MessagingService.VERSION_30;
 
             return 2 // kind + flags
@@ -643,33 +671,6 @@ public abstract class ReadCommand implements ReadQuery
         }
     }
 
-    // Dispatch to either Serializer or LegacyRangeSliceCommandSerializer. Only useful as long as we maintain pre-3.0
-    // compatibility
-    private static class RangeSliceSerializer implements IVersionedSerializer<ReadCommand>
-    {
-        public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
-        {
-            if (version < MessagingService.VERSION_30)
-                legacyRangeSliceCommandSerializer.serialize(command, out, version);
-            else
-                serializer.serialize(command, out, version);
-        }
-
-        public ReadCommand deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return version < MessagingService.VERSION_30
-                 ? legacyRangeSliceCommandSerializer.deserialize(in, version)
-                 : serializer.deserialize(in, version);
-        }
-
-        public long serializedSize(ReadCommand command, int version)
-        {
-            return version < MessagingService.VERSION_30
-                 ? legacyRangeSliceCommandSerializer.serializedSize(command, version)
-                 : serializer.serializedSize(command, version);
-        }
-    }
-
     private enum LegacyType
     {
         GET_BY_NAMES((byte)1),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fbd287ad/src/java/org/apache/cassandra/db/ReadResponse.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadResponse.java b/src/java/org/apache/cassandra/db/ReadResponse.java
index 8bd1be6..12a200f 100644
--- a/src/java/org/apache/cassandra/db/ReadResponse.java
+++ b/src/java/org/apache/cassandra/db/ReadResponse.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.io.ForwardingVersionedSerializer;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -46,11 +47,20 @@ public abstract class ReadResponse
 {
     // Serializer for single partition read response
     public static final IVersionedSerializer<ReadResponse> serializer = new Serializer();
-    // Serializer for partition range read response (this actually delegate to 'serializer' in 3.0 and to
-    // 'legacyRangeSliceReplySerializer' in older version.
-    public static final IVersionedSerializer<ReadResponse> rangeSliceSerializer = new RangeSliceSerializer();
     // Serializer for the pre-3.0 rang slice responses.
     public static final IVersionedSerializer<ReadResponse> legacyRangeSliceReplySerializer = new LegacyRangeSliceReplySerializer();
+    // Serializer for partition range read response (this actually delegate to 'serializer' in 3.0 and to
+    // 'legacyRangeSliceReplySerializer' in older version.
+    public static final IVersionedSerializer<ReadResponse> rangeSliceSerializer = new ForwardingVersionedSerializer<ReadResponse>()
+    {
+        @Override
+        protected IVersionedSerializer<ReadResponse> delegate(int version)
+        {
+            return version < MessagingService.VERSION_30
+                    ? legacyRangeSliceReplySerializer
+                    : serializer;
+        }
+    };
 
     // This is used only when serializing data responses and we can't it easily in other cases. So this can be null, which is slighly
     // hacky, but as this hack doesn't escape this class, and it's easy enough to validate that it's not null when we need, it's "good enough".
@@ -411,31 +421,6 @@ public abstract class ReadResponse
         }
     }
 
-    private static class RangeSliceSerializer implements IVersionedSerializer<ReadResponse>
-    {
-        public void serialize(ReadResponse response, DataOutputPlus out, int version) throws IOException
-        {
-            if (version < MessagingService.VERSION_30)
-                legacyRangeSliceReplySerializer.serialize(response, out, version);
-            else
-                serializer.serialize(response, out, version);
-        }
-
-        public ReadResponse deserialize(DataInputPlus in, int version) throws IOException
-        {
-            return version < MessagingService.VERSION_30
-                 ? legacyRangeSliceReplySerializer.deserialize(in, version)
-                 : serializer.deserialize(in, version);
-        }
-
-        public long serializedSize(ReadResponse response, int version)
-        {
-            return version < MessagingService.VERSION_30
-                 ? legacyRangeSliceReplySerializer.serializedSize(response, version)
-                 : serializer.serializedSize(response, version);
-        }
-    }
-
     private static class LegacyRangeSliceReplySerializer implements IVersionedSerializer<ReadResponse>
     {
         public void serialize(ReadResponse response, DataOutputPlus out, int version) throws IOException
@@ -477,6 +462,8 @@ public abstract class ReadResponse
 
         public ReadResponse deserialize(DataInputPlus in, int version) throws IOException
         {
+            assert version < MessagingService.VERSION_30;
+
             int partitionCount = in.readInt();
             ArrayList<ImmutableBTreePartition> partitions = new ArrayList<>(partitionCount);
             for (int i = 0; i < partitionCount; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fbd287ad/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 6784770..73eb9bd 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -894,7 +894,7 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     public MessageOut<ReadCommand> createMessage(int version)
     {
-        return new MessageOut<>(MessagingService.Verb.READ, this, version < MessagingService.VERSION_30 ? legacyReadCommandSerializer : serializer);
+        return new MessageOut<>(MessagingService.Verb.READ, this, readSerializer);
     }
 
     protected void appendCQLWhereClause(StringBuilder sb)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fbd287ad/src/java/org/apache/cassandra/io/ForwardingVersionedSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/ForwardingVersionedSerializer.java b/src/java/org/apache/cassandra/io/ForwardingVersionedSerializer.java
new file mode 100644
index 0000000..64f91d7
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/ForwardingVersionedSerializer.java
@@ -0,0 +1,57 @@
+/*
+ * 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.io;
+
+import java.io.IOException;
+
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * A serializer which forwards all its method calls to another serializer. Subclasses should override one or more
+ * methods to modify the behavior of the backing serializer as desired per the decorator pattern.
+ */
+public abstract class ForwardingVersionedSerializer<T> implements IVersionedSerializer<T>
+{
+    protected ForwardingVersionedSerializer()
+    {
+    }
+
+    /**
+     * Returns the backing delegate instance that methods are forwarded to.
+     *
+     * @param version the server version
+     * @return the backing delegate instance that methods are forwarded to.
+     */
+    protected abstract IVersionedSerializer<T> delegate(int version);
+
+    public void serialize(T t, DataOutputPlus out, int version) throws IOException
+    {
+        delegate(version).serialize(t, out, version);
+    }
+
+    public T deserialize(DataInputPlus in, int version) throws IOException
+    {
+        return delegate(version).deserialize(in, version);
+    }
+
+    public long serializedSize(T t, int version)
+    {
+        return delegate(version).serializedSize(t, version);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fbd287ad/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index fac46eb..d01419f 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -215,9 +215,9 @@ public final class MessagingService implements MessagingServiceMBean
 
         put(Verb.MUTATION, Mutation.serializer);
         put(Verb.READ_REPAIR, Mutation.serializer);
-        put(Verb.READ, ReadCommand.serializer);
+        put(Verb.READ, ReadCommand.readSerializer);
         put(Verb.RANGE_SLICE, ReadCommand.rangeSliceSerializer);
-        put(Verb.PAGED_RANGE, ReadCommand.legacyPagedRangeCommandSerializer);
+        put(Verb.PAGED_RANGE, ReadCommand.pagedRangeSerializer);
         put(Verb.BOOTSTRAP_TOKEN, BootStrapper.StringSerializer.instance);
         put(Verb.REPAIR_MESSAGE, RepairMessage.serializer);
         put(Verb.GOSSIP_DIGEST_ACK, GossipDigestAck.serializer);
@@ -247,7 +247,7 @@ public final class MessagingService implements MessagingServiceMBean
         put(Verb.READ_REPAIR, WriteResponse.serializer);
         put(Verb.COUNTER_MUTATION, WriteResponse.serializer);
         put(Verb.RANGE_SLICE, ReadResponse.rangeSliceSerializer);
-        put(Verb.PAGED_RANGE, ReadResponse.legacyRangeSliceReplySerializer);
+        put(Verb.PAGED_RANGE, ReadResponse.rangeSliceSerializer);
         put(Verb.READ, ReadResponse.serializer);
         put(Verb.TRUNCATE, TruncateResponse.serializer);
         put(Verb.SNAPSHOT, null);


[10/50] cassandra git commit: Avoid potential race when rebuilding CFMetaData.columnMetadata

Posted by al...@apache.org.
Avoid potential race when rebuilding CFMetaData.columnMetadata

Patch by Sam Tunnicliffe; reviewed by Aleksey Yeschenko for
CASSANDRA-12098


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

Branch: refs/heads/cassandra-3.8
Commit: dd05e46f0cb5475edf72676230474e0ad0f9cdbf
Parents: 73c7167
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Tue Jul 5 13:02:20 2016 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Jul 6 12:08:22 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                         |  1 +
 .../org/apache/cassandra/config/CFMetaData.java     | 16 +++++++---------
 2 files changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd05e46f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b3063b4..02786c5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
  * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
  * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
  * Fix column ordering of results with static columns for Thrift requests in

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd05e46f/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index e263697..5678ada 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -22,7 +22,6 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
@@ -53,9 +52,6 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.utils.*;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.UUIDGen;
 import org.github.jamm.Unmetered;
 
 /**
@@ -109,7 +105,7 @@ public final class CFMetaData
      * clustering key ones, those list are ordered by the "component index" of the
      * elements.
      */
-    private final Map<ByteBuffer, ColumnDefinition> columnMetadata = new ConcurrentHashMap<>(); // not on any hot path
+    private volatile Map<ByteBuffer, ColumnDefinition> columnMetadata = new HashMap<>();
     private volatile List<ColumnDefinition> partitionKeyColumns;  // Always of size keyValidator.componentsCount, null padded if necessary
     private volatile List<ColumnDefinition> clusteringColumns;    // Of size comparator.componentsCount or comparator.componentsCount -1, null padded if necessary
     private volatile PartitionColumns partitionColumns;           // Always non-PK, non-clustering columns
@@ -298,16 +294,18 @@ public final class CFMetaData
     {
         this.comparator = new ClusteringComparator(extractTypes(clusteringColumns));
 
-        this.columnMetadata.clear();
+        Map<ByteBuffer, ColumnDefinition> newColumnMetadata = new HashMap<>();
         for (ColumnDefinition def : partitionKeyColumns)
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
         for (ColumnDefinition def : clusteringColumns)
         {
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
             def.type.checkComparable();
         }
         for (ColumnDefinition def : partitionColumns)
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
+
+        this.columnMetadata = newColumnMetadata;
 
         List<AbstractType<?>> keyTypes = extractTypes(partitionKeyColumns);
         this.keyValidator = keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes);


[43/50] cassandra git commit: Fix CHANGES.txt

Posted by al...@apache.org.
Fix CHANGES.txt


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

Branch: refs/heads/cassandra-3.8
Commit: e3f9b7a3b6ec5df1c65bd6324d2c312209183cf8
Parents: fb4eb5a
Author: Benjamin Lerer <b....@gmail.com>
Authored: Thu Jul 14 11:46:14 2016 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Thu Jul 14 11:46:14 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3f9b7a3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index bedba6d..ba8e299 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,14 +1,10 @@
-<<<<<<< HEAD
 3.9
  * Partial revert of CASSANDRA-11971, cannot recycle buffer in SP.sendMessagesToNonlocalDC (CASSANDRA-11950)
  * Fix hdr logging for single operation workloads (CASSANDRA-12145)
  * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
  * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 Merged from 3.0:
-=======
-3.0.9
  * Wait until the message is being send to decide which serializer must be used (CASSANDRA-11393)
->>>>>>> asf/cassandra-3.0
  * Fix migration of static thrift column names with non-text comparators (CASSANDRA-12147)
  * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)
  * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)


[45/50] cassandra git commit: Fix paging logic for deleted partitions with static columns

Posted by al...@apache.org.
Fix paging logic for deleted partitions with static columns

patch by Sharvanath Pathak; reviewed by Benjamin Lerer for CASSANDRA-12107


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

Branch: refs/heads/cassandra-3.8
Commit: 84426d183ae095107bb264b92d828f231d0a9826
Parents: fbd287a
Author: Sharvanath Pathak <sh...@gmail.com>
Authored: Thu Jul 14 21:38:14 2016 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Thu Jul 14 21:38:14 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/filter/DataLimits.java  |  3 +-
 .../validation/operations/SelectLimitTest.java  | 31 ++++++++++++++++++++
 3 files changed, 33 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/84426d18/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3829046..59f0a5f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Fix paging logic for deleted partitions with static columns (CASSANDRA-12107)
  * Wait until the message is being send to decide which serializer must be used (CASSANDRA-11393)
  * Fix migration of static thrift column names with non-text comparators (CASSANDRA-12147)
  * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/84426d18/src/java/org/apache/cassandra/db/filter/DataLimits.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/DataLimits.java b/src/java/org/apache/cassandra/db/filter/DataLimits.java
index f6fdcdd..94f43dc 100644
--- a/src/java/org/apache/cassandra/db/filter/DataLimits.java
+++ b/src/java/org/apache/cassandra/db/filter/DataLimits.java
@@ -360,8 +360,7 @@ public abstract class DataLimits
             public void applyToPartition(DecoratedKey partitionKey, Row staticRow)
             {
                 rowInCurrentPartition = 0;
-                if (!staticRow.isEmpty() && (assumeLiveData || staticRow.hasLiveData(nowInSec)))
-                    hasLiveStaticRow = true;
+                hasLiveStaticRow = !staticRow.isEmpty() && (assumeLiveData || staticRow.hasLiveData(nowInSec));
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/84426d18/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
index a21ef3c..aeb3d56 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
@@ -133,4 +133,35 @@ public class SelectLimitTest extends CQLTester
                    row(2, 2),
                    row(2, 3));
     }
+
+    @Test
+    public void testLimitWithDeletedRowsAndStaticColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int, c int, v int, s int static, PRIMARY KEY (pk, c))");
+
+        execute("INSERT INTO %s (pk, c, v, s) VALUES (1, -1, 1, 1)");
+        execute("INSERT INTO %s (pk, c, v, s) VALUES (2, -1, 1, 1)");
+        execute("INSERT INTO %s (pk, c, v, s) VALUES (3, -1, 1, 1)");
+        execute("INSERT INTO %s (pk, c, v, s) VALUES (4, -1, 1, 1)");
+        execute("INSERT INTO %s (pk, c, v, s) VALUES (5, -1, 1, 1)");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, -1, 1, 1),
+                   row(2, -1, 1, 1),
+                   row(3, -1, 1, 1),
+                   row(4, -1, 1, 1),
+                   row(5, -1, 1, 1));
+
+        execute("DELETE FROM %s WHERE pk = 2");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, -1, 1, 1),
+                   row(3, -1, 1, 1),
+                   row(4, -1, 1, 1),
+                   row(5, -1, 1, 1));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 2"),
+                   row(1, -1, 1, 1),
+                   row(3, -1, 1, 1));
+    }
 }


[18/50] cassandra git commit: NPE when trying to remove purgable tombstones from result

Posted by al...@apache.org.
NPE when trying to remove purgable tombstones from result

patch by mck; reviewed by Sylvain Lebresne for CASSANDRA-12143


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

Branch: refs/heads/cassandra-3.8
Commit: ef18a1768a6589eac212a7f320f9748ca6dc8371
Parents: 00e7ecf
Author: mck <mi...@semb.wever.org>
Authored: Thu Jul 7 11:17:40 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:49:12 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  3 +-
 .../cassandra/db/ColumnFamilyStoreTest.java     | 50 ++++++++++++++++----
 3 files changed, 44 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ef18a176/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7d62f97..e10af6f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.8
+ * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
  * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 Merged from 2.1:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ef18a176/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index d86f941..ff63163 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -2347,7 +2347,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 }
 
                 // remove purgable tombstones from result - see CASSANDRA-11427
-                data.purgeTombstones(gcBefore(filter.timestamp));
+                if (data != null)
+                    data.purgeTombstones(gcBefore(filter.timestamp));
 
                 rows.add(new Row(rawRow.key, data));
                 if (!ignoreTombstonedPartitions || !data.hasOnlyTombstones(filter.timestamp))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ef18a176/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index 5419ef5..2d67baf 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -66,6 +66,7 @@ import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.db.composites.Composites;
 import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.filter.ExtendedFilter;
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.db.filter.QueryFilter;
@@ -94,7 +95,6 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.WrappedRunnable;
-import org.apache.thrift.TException;
 
 import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.Util.column;
@@ -246,6 +246,38 @@ public class ColumnFamilyStoreTest
     }
 
     @Test
+    public void testFilterWithNullCF() throws Exception
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
+        final Row row = new Row(Util.dk("key1"), null);
+
+        ColumnFamilyStore.AbstractScanIterator iterator = new ColumnFamilyStore.AbstractScanIterator()
+        {
+            Iterator<Row> it = Collections.singletonList(row).iterator();
+
+            protected Row computeNext()
+            {
+                return it.hasNext() ? it.next() : endOfData();
+            }
+
+            @Override
+            public void close()
+            {
+            }
+        };
+
+        ExtendedFilter filter = ExtendedFilter.create(
+                cfs,
+                DataRange.allData(DatabaseDescriptor.getPartitioner()), null, 1, true, System.currentTimeMillis());
+
+        List<Row> list = cfs.filter(iterator, filter);
+        assert 1 == list.size();
+        assert list.get(0).key == row.key;
+        assert null == list.get(0).cf;
+    }
+
+    @Test
     public void testSkipStartKey()
     {
         ColumnFamilyStore cfs = insertKey1Key2();
@@ -571,7 +603,7 @@ public class ColumnFamilyStoreTest
         cfs.truncateBlocking();
 
         ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
-        CellName colName = cellname("birthdate"); 
+        CellName colName = cellname("birthdate");
         ByteBuffer val1 = ByteBufferUtil.bytes(1L);
         ByteBuffer val2 = ByteBufferUtil.bytes(2L);
 
@@ -635,7 +667,7 @@ public class ColumnFamilyStoreTest
 
         ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
         ByteBuffer clusterKey = ByteBufferUtil.bytes("ck1");
-        ByteBuffer colName = ByteBufferUtil.bytes("col1"); 
+        ByteBuffer colName = ByteBufferUtil.bytes("col1");
 
         CellNameType baseComparator = cfs.getComparator();
         CellName compositeName = baseComparator.makeCellName(clusterKey, colName);
@@ -2291,7 +2323,7 @@ public class ColumnFamilyStoreTest
         });
         System.err.println("Row key: " + rowKey + " Cols: " + transformed);
     }
-    
+
     @Test
     public void testRebuildSecondaryIndex() throws IOException
     {
@@ -2303,19 +2335,19 @@ public class ColumnFamilyStoreTest
 
         rm.apply();
         assertTrue(Arrays.equals("k1".getBytes(), PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY.array()));
-        
+
         Keyspace.open("PerRowSecondaryIndex").getColumnFamilyStore("Indexed1").forceBlockingFlush();
-        
+
         PerRowSecondaryIndexTest.TestIndex.reset();
-        
+
         ColumnFamilyStore.rebuildSecondaryIndex("PerRowSecondaryIndex", "Indexed1", PerRowSecondaryIndexTest.TestIndex.class.getSimpleName());
         assertTrue(Arrays.equals("k1".getBytes(), PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY.array()));
-        
+
         PerRowSecondaryIndexTest.TestIndex.reset();
         PerRowSecondaryIndexTest.TestIndex.ACTIVE = false;
         ColumnFamilyStore.rebuildSecondaryIndex("PerRowSecondaryIndex", "Indexed1", PerRowSecondaryIndexTest.TestIndex.class.getSimpleName());
         assertNull(PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY);
-        
+
         PerRowSecondaryIndexTest.TestIndex.reset();
     }
 }


[44/50] cassandra git commit: Fix potential deadlock in CDC state tracking

Posted by al...@apache.org.
Fix potential deadlock in CDC state tracking

Patch by jmckenzie; reviewed by cyeksigian for CASSANDRA-12198


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

Branch: refs/heads/cassandra-3.8
Commit: 90afc58d3df912c720aff63de0506019b8b9af48
Parents: e3f9b7a
Author: Josh McKenzie <jm...@apache.org>
Authored: Wed Jul 13 18:30:40 2016 -0400
Committer: Josh McKenzie <jm...@apache.org>
Committed: Thu Jul 14 10:36:43 2016 -0400

----------------------------------------------------------------------
 src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java | 3 ++-
 .../cassandra/db/commitlog/CommitLogSegmentManagerCDC.java       | 4 ++--
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/90afc58d/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 2e97fd5..a1158be 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -61,6 +61,7 @@ public abstract class CommitLogSegment
         FORBIDDEN,
         CONTAINS
     }
+    Object cdcStateLock = new Object();
 
     private final static AtomicInteger nextId = new AtomicInteger(1);
     private static long replayLimitId;
@@ -614,7 +615,7 @@ public abstract class CommitLogSegment
             return;
 
         // Also synchronized in CDCSizeTracker.processNewSegment and .processDiscardedSegment
-        synchronized(this)
+        synchronized(cdcStateLock)
         {
             if (cdcState == CDCState.CONTAINS && newState != CDCState.CONTAINS)
                 throw new IllegalArgumentException("Cannot transition from CONTAINS to any other state.");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90afc58d/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
index 5c6fd3f..04beb20 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
@@ -187,7 +187,7 @@ public class CommitLogSegmentManagerCDC extends AbstractCommitLogSegmentManager
         void processNewSegment(CommitLogSegment segment)
         {
             // See synchronization in CommitLogSegment.setCDCState
-            synchronized(segment)
+            synchronized(segment.cdcStateLock)
             {
                 segment.setCDCState(defaultSegmentSize() + totalCDCSizeOnDisk() > allowableCDCBytes()
                                     ? CDCState.FORBIDDEN
@@ -203,7 +203,7 @@ public class CommitLogSegmentManagerCDC extends AbstractCommitLogSegmentManager
         void processDiscardedSegment(CommitLogSegment segment)
         {
             // See synchronization in CommitLogSegment.setCDCState
-            synchronized(segment)
+            synchronized(segment.cdcStateLock)
             {
                 // Add to flushed size before decrementing unflushed so we don't have a window of false generosity
                 if (segment.getCDCState() == CDCState.CONTAINS)


[28/50] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

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


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

Branch: refs/heads/cassandra-3.8
Commit: 2fa44cd88119c24489836e0c0c91fd3eed86ce3c
Parents: a227cc6 e983590
Author: Yuki Morishita <yu...@apache.org>
Authored: Fri Jul 8 19:05:46 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Fri Jul 8 19:05:46 2016 -0500

----------------------------------------------------------------------
 src/java/org/apache/cassandra/streaming/ConnectionHandler.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[29/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 1417a516cf7cc89dc456eb8c9c7e2759811a6991
Parents: c1dcc9c 2fa44cd
Author: Yuki Morishita <yu...@apache.org>
Authored: Fri Jul 8 19:12:34 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Fri Jul 8 19:12:34 2016 -0500

----------------------------------------------------------------------
 src/java/org/apache/cassandra/streaming/ConnectionHandler.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1417a516/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
----------------------------------------------------------------------


[37/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 56abaca0466411739895523d0c3a81a7630ab9f0
Parents: 371a147 5861cd8
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Mon Jul 11 15:00:33 2016 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Mon Jul 11 15:01:04 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                            | 7 +++----
 src/java/org/apache/cassandra/thrift/ThriftServer.java | 2 +-
 2 files changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/56abaca0/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index f65b1f4,70210a8..da8216f
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,10 -1,4 +1,9 @@@
 -3.0.9
 +3.9
 + * Partial revert of CASSANDRA-11971, cannot recycle buffer in SP.sendMessagesToNonlocalDC (CASSANDRA-11950)
 + * Fix hdr logging for single operation workloads (CASSANDRA-12145)
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
- 3.0.9
   * Fix migration of static thrift column names with non-text comparators (CASSANDRA-12147)
   * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)
   * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)


[34/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 371a147945feeaea9d39a714d59847eed1b3dc22
Parents: 314f60a 4d7141f
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Mon Jul 11 14:37:10 2016 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Mon Jul 11 14:37:10 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                         | 1 +
 src/java/org/apache/cassandra/db/CompactTables.java | 7 ++-----
 2 files changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/371a1479/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 44fe26c,4241da8..f65b1f4
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,10 -1,5 +1,11 @@@
 +3.9
 + * Partial revert of CASSANDRA-11971, cannot recycle buffer in SP.sendMessagesToNonlocalDC (CASSANDRA-11950)
 + * Fix hdr logging for single operation workloads (CASSANDRA-12145)
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
  3.0.9
+  * Fix migration of static thrift column names with non-text comparators (CASSANDRA-12147)
   * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)
   * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
  2.2.8

http://git-wip-us.apache.org/repos/asf/cassandra/blob/371a1479/src/java/org/apache/cassandra/db/CompactTables.java
----------------------------------------------------------------------


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

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


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

Branch: refs/heads/cassandra-3.8
Commit: 73c71672bdb48af976cd17909162ea2b3c0ce007
Parents: bc23632 d5a15e4
Author: Marcus Eriksson <ma...@apache.org>
Authored: Wed Jul 6 10:11:21 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Wed Jul 6 10:11:21 2016 +0200

----------------------------------------------------------------------
 .../cassandra/db/compaction/DateTieredCompactionStrategyTest.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/73c71672/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
----------------------------------------------------------------------


[33/50] cassandra git commit: Fix migration of static thrift column names with non-text comparators

Posted by al...@apache.org.
Fix migration of static thrift column names with non-text comparators

patch by Aleksey Yeschenko; reviewed by Sylvain Lebresne for
CASSANDRA-12147


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

Branch: refs/heads/cassandra-3.8
Commit: 4d7141f75866939b8a55a0d3ee09153ee75f80f9
Parents: f37b6f1
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed Jul 6 17:56:51 2016 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Mon Jul 11 14:36:24 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                         | 1 +
 src/java/org/apache/cassandra/db/CompactTables.java | 7 ++-----
 2 files changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4d7141f7/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f2d11b9..4241da8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Fix migration of static thrift column names with non-text comparators (CASSANDRA-12147)
  * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)
  * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
 2.2.8

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4d7141f7/src/java/org/apache/cassandra/db/CompactTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CompactTables.java b/src/java/org/apache/cassandra/db/CompactTables.java
index a73b865..e31fda3 100644
--- a/src/java/org/apache/cassandra/db/CompactTables.java
+++ b/src/java/org/apache/cassandra/db/CompactTables.java
@@ -93,13 +93,10 @@ public abstract class CompactTables
 
     public static AbstractType<?> columnDefinitionComparator(String kind, boolean isSuper, AbstractType<?> rawComparator, AbstractType<?> rawSubComparator)
     {
-        if ("compact_value".equals(kind))
+        if (!"regular".equals(kind))
             return UTF8Type.instance;
 
-        if (isSuper)
-            return "regular".equals(kind) ? rawSubComparator : UTF8Type.instance;
-
-        return "static".equals(kind) ? rawComparator : UTF8Type.instance;
+        return isSuper ? rawSubComparator : rawComparator;
     }
 
     public static boolean hasEmptyCompactValue(CFMetaData metadata)


[48/50] cassandra git commit: Avoid wrapping results with ThriftResultsMerger if command is not for thrift

Posted by al...@apache.org.
Avoid wrapping results with ThriftResultsMerger if command is not for thrift

Patch by Alex Petrov; reviewed by Tyler Hobbs for CASSANDRA-12193.


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

Branch: refs/heads/cassandra-3.8
Commit: 557c59607fa9eb2014a7905cfc0eb3dbcd043cab
Parents: 84426d1
Author: Alex Petrov <ol...@gmail.com>
Authored: Fri Jul 15 17:49:16 2016 -0500
Committer: Tyler Hobbs <ty...@gmail.com>
Committed: Fri Jul 15 17:49:16 2016 -0500

----------------------------------------------------------------------
 src/java/org/apache/cassandra/db/ReadResponse.java | 15 ++++++++++-----
 1 file changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/557c5960/src/java/org/apache/cassandra/db/ReadResponse.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadResponse.java b/src/java/org/apache/cassandra/db/ReadResponse.java
index 12a200f..2304cb4 100644
--- a/src/java/org/apache/cassandra/db/ReadResponse.java
+++ b/src/java/org/apache/cassandra/db/ReadResponse.java
@@ -282,12 +282,17 @@ public abstract class ReadResponse
 
                     // Pre-3.0, we didn't have a way to express exclusivity for non-composite comparators, so all slices were
                     // inclusive on both ends. If we have exclusive slice ends, we need to filter the results here.
+                    UnfilteredRowIterator iterator;
                     if (!command.metadata().isCompound())
-                        return ThriftResultsMerger.maybeWrap(
-                                filter.filter(partition.sliceableUnfilteredIterator(command.columnFilter(), filter.isReversed())), command.nowInSec());
-
-                    return ThriftResultsMerger.maybeWrap(
-                            partition.unfilteredIterator(command.columnFilter(), Slices.ALL, filter.isReversed()), command.nowInSec());
+                        iterator = filter.filter(partition.sliceableUnfilteredIterator(command.columnFilter(), filter.isReversed()));
+                    else
+                        iterator = partition.unfilteredIterator(command.columnFilter(), Slices.ALL, filter.isReversed());
+
+                    // Wrap results with a ThriftResultMerger only if they're intended for the thrift command.
+                    if (command.isForThrift())
+                        return ThriftResultsMerger.maybeWrap(iterator, command.nowInSec());
+                    else
+                        return iterator;
                 }
             };
         }


[19/50] cassandra git commit: Merge commit 'ef18a17' into cassandra-3.0

Posted by al...@apache.org.
Merge commit 'ef18a17' into cassandra-3.0

* commit 'ef18a17':
  NPE when trying to remove purgable tombstones from result


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

Branch: refs/heads/cassandra-3.8
Commit: 30f5d44d8cc53726fc9a17b6df4928ccd23af977
Parents: 778f2a4 ef18a17
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jul 7 12:50:03 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:50:03 2016 +0200

----------------------------------------------------------------------

----------------------------------------------------------------------



[26/50] cassandra git commit: Fix hdr logging for single operation workloads

Posted by al...@apache.org.
Fix hdr logging for single operation workloads

Patch by Nitsan Wakart; reviewed by tjake for CASSANDRA-12145


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

Branch: refs/heads/cassandra-3.8
Commit: c1dcc9ce46f53dd89b80a08d363d5eacac1b9e23
Parents: 8475f89
Author: nitsanw <ni...@yahoo.com>
Authored: Wed Jul 6 14:45:18 2016 +0200
Committer: T Jake Luciani <ja...@apache.org>
Committed: Fri Jul 8 10:38:40 2016 -0400

----------------------------------------------------------------------
 CHANGES.txt                                            |  1 +
 .../src/org/apache/cassandra/stress/StressMetrics.java | 13 ++++++++-----
 2 files changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c1dcc9ce/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 34e7587..b094b00 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.9
+ * Fix hdr logging for single operation workloads (CASSANDRA-12145)
  * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
  * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 Merged from 3.0:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c1dcc9ce/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java b/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
index 668518c..86e9a7a 100644
--- a/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
+++ b/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
@@ -191,15 +191,18 @@ public class StressMetrics
         rowRateUncertainty.update(current.adjustedRowRate());
         if (current.operationCount() != 0)
         {
-            if (result.intervals.intervals().size() > 1)
+            // if there's a single operation we only print the total
+            final boolean logPerOpSummaryLine = result.intervals.intervals().size() > 1;
+
+            for (Map.Entry<String, TimingInterval> type : result.intervals.intervals().entrySet())
             {
-                for (Map.Entry<String, TimingInterval> type : result.intervals.intervals().entrySet())
+                final String opName = type.getKey();
+                final TimingInterval opInterval = type.getValue();
+                if (logPerOpSummaryLine)
                 {
-                    final String opName = type.getKey();
-                    final TimingInterval opInterval = type.getValue();
                     printRow("", opName, opInterval, timing.getHistory().get(type.getKey()), result.extra, rowRateUncertainty, output);
-                    logHistograms(opName, opInterval);
                 }
+                logHistograms(opName, opInterval);
             }
 
             printRow("", "total", current, history, result.extra, rowRateUncertainty, output);


[22/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9

* cassandra-3.0:
  Don't ignore deletion info in sstable on reverse queries


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

Branch: refs/heads/cassandra-3.8
Commit: a006f577bdba7c4b248ef9f4cbd02a6c35a03162
Parents: 376dae2 76e68e9
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jul 7 12:59:34 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jul 7 12:59:34 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../columniterator/SSTableReversedIterator.java |  2 +-
 .../cql3/validation/operations/DeleteTest.java  | 26 ++++++++++++++++++++
 3 files changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a006f577/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index d459e34,20ed6e0..1d11149
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,7 -1,5 +1,8 @@@
 -3.0.9
 +3.9
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
+  * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
   * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
   * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
   * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a006f577/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a006f577/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
index 9ead942,814e822..9b92ebb
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@@ -1105,4 -1051,36 +1105,30 @@@ public class DeleteTest extends CQLTest
          compact();
          assertRows(execute("SELECT * FROM %s"), row(0, null));
      }
+ 
 -    private void flush(boolean forceFlush)
 -    {
 -        if (forceFlush)
 -            flush();
 -    }
 -
+     @Test
+     public void testDeleteAndReverseQueries() throws Throwable
+     {
+         // This test insert rows in one sstable and a range tombstone covering some of those rows in another, and it
+         // validates we correctly get only the non-removed rows when doing reverse queries.
+ 
+         createTable("CREATE TABLE %s (k text, i int, PRIMARY KEY (k, i))");
+ 
+         for (int i = 0; i < 10; i++)
+             execute("INSERT INTO %s(k, i) values (?, ?)", "a", i);
+ 
+         flush();
+ 
+         execute("DELETE FROM %s WHERE k = ? AND i >= ? AND i <= ?", "a", 2, 7);
+ 
+         assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+             row(9), row(8), row(1), row(0)
+         );
+ 
+         flush();
+ 
+         assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+             row(9), row(8), row(1), row(0)
+         );
+     }
  }


[07/50] cassandra git commit: fix flapping DTCS test

Posted by al...@apache.org.
fix flapping DTCS test


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

Branch: refs/heads/cassandra-3.8
Commit: d5a15e45b080756bb553ad4f03ecb51edfed8400
Parents: 3c84361
Author: Marcus Eriksson <ma...@apache.org>
Authored: Wed Jul 6 10:11:13 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Wed Jul 6 10:11:13 2016 +0200

----------------------------------------------------------------------
 .../cassandra/db/compaction/DateTieredCompactionStrategyTest.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d5a15e45/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
index 2fab014..3bc830b 100644
--- a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
@@ -368,5 +368,6 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
         AbstractCompactionTask task = dtcs.getNextBackgroundTask(0);
         assertEquals(20, task.transaction.originals().size());
         task.transaction.abort();
+        cfs.truncateBlocking();
     }
 }


[46/50] cassandra git commit: Merge branch cassandra-3.0 into cassandra-3.9

Posted by al...@apache.org.
Merge branch cassandra-3.0 into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 2764e85a557c140f44ffc08c09e4b06a61e1ef4e
Parents: 90afc58 84426d1
Author: Benjamin Lerer <b....@gmail.com>
Authored: Thu Jul 14 21:44:34 2016 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Thu Jul 14 21:44:34 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/filter/DataLimits.java  |  3 +-
 .../validation/operations/SelectLimitTest.java  | 32 +++++++++++++++++++-
 3 files changed, 33 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2764e85a/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index ba8e299,59f0a5f..4c46695
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,9 -1,5 +1,10 @@@
 -3.0.9
 +3.9
 + * Partial revert of CASSANDRA-11971, cannot recycle buffer in SP.sendMessagesToNonlocalDC (CASSANDRA-11950)
 + * Fix hdr logging for single operation workloads (CASSANDRA-12145)
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
+  * Fix paging logic for deleted partitions with static columns (CASSANDRA-12107)
   * Wait until the message is being send to decide which serializer must be used (CASSANDRA-11393)
   * Fix migration of static thrift column names with non-text comparators (CASSANDRA-12147)
   * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2764e85a/src/java/org/apache/cassandra/db/filter/DataLimits.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2764e85a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
index 528d9f6,aeb3d56..21c48dd
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
@@@ -26,7 -26,7 +26,6 @@@ import org.junit.Test
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.cql3.CQLTester;
  import org.apache.cassandra.dht.ByteOrderedPartitioner;
--import org.apache.cassandra.exceptions.InvalidRequestException;
  
  public class SelectLimitTest extends CQLTester
  {
@@@ -135,114 -135,33 +134,145 @@@
      }
  
      @Test
 +    public void testPerPartitionLimit() throws Throwable
 +    {
 +        perPartitionLimitTest(false);
 +    }
 +
 +    @Test
 +    public void testPerPartitionLimitWithCompactStorage() throws Throwable
 +    {
 +        perPartitionLimitTest(true);
 +    }
 +
 +    private void perPartitionLimitTest(boolean withCompactStorage) throws Throwable
 +    {
 +        String query = "CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))";
 +
 +        if (withCompactStorage)
 +            createTable(query + " WITH COMPACT STORAGE");
 +        else
 +            createTable(query);
 +
 +        for (int i = 0; i < 5; i++)
 +        {
 +            for (int j = 0; j < 5; j++)
 +            {
 +                execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", i, j, j);
 +            }
 +        }
 +
 +        assertInvalidMessage("LIMIT must be strictly positive",
 +                             "SELECT * FROM %s PER PARTITION LIMIT ?", 0);
 +        assertInvalidMessage("LIMIT must be strictly positive",
 +                             "SELECT * FROM %s PER PARTITION LIMIT ?", -1);
 +
 +        assertRowsIgnoringOrder(execute("SELECT * FROM %s PER PARTITION LIMIT ?", 2),
 +                                row(0, 0, 0),
 +                                row(0, 1, 1),
 +                                row(1, 0, 0),
 +                                row(1, 1, 1),
 +                                row(2, 0, 0),
 +                                row(2, 1, 1),
 +                                row(3, 0, 0),
 +                                row(3, 1, 1),
 +                                row(4, 0, 0),
 +                                row(4, 1, 1));
 +
 +        // Combined Per Partition and "global" limit
 +        assertRowCount(execute("SELECT * FROM %s PER PARTITION LIMIT ? LIMIT ?", 2, 6),
 +                       6);
 +
 +        // odd amount of results
 +        assertRowCount(execute("SELECT * FROM %s PER PARTITION LIMIT ? LIMIT ?", 2, 5),
 +                       5);
 +
 +        // IN query
 +        assertRows(execute("SELECT * FROM %s WHERE a IN (2,3) PER PARTITION LIMIT ?", 2),
 +                   row(2, 0, 0),
 +                   row(2, 1, 1),
 +                   row(3, 0, 0),
 +                   row(3, 1, 1));
 +
 +        assertRows(execute("SELECT * FROM %s WHERE a IN (2,3) PER PARTITION LIMIT ? LIMIT 3", 2),
 +                   row(2, 0, 0),
 +                   row(2, 1, 1),
 +                   row(3, 0, 0));
 +
 +        assertRows(execute("SELECT * FROM %s WHERE a IN (1,2,3) PER PARTITION LIMIT ? LIMIT 3", 2),
 +                   row(1, 0, 0),
 +                   row(1, 1, 1),
 +                   row(2, 0, 0));
 +
 +        // with restricted partition key
 +        assertRows(execute("SELECT * FROM %s WHERE a = ? PER PARTITION LIMIT ?", 2, 3),
 +                   row(2, 0, 0),
 +                   row(2, 1, 1),
 +                   row(2, 2, 2));
 +
 +        // with ordering
 +        assertRows(execute("SELECT * FROM %s WHERE a IN (3, 2) ORDER BY b DESC PER PARTITION LIMIT ?", 2),
 +                   row(2, 4, 4),
 +                   row(3, 4, 4),
 +                   row(2, 3, 3),
 +                   row(3, 3, 3));
 +
 +        assertRows(execute("SELECT * FROM %s WHERE a IN (3, 2) ORDER BY b DESC PER PARTITION LIMIT ? LIMIT ?", 3, 4),
 +                   row(2, 4, 4),
 +                   row(3, 4, 4),
 +                   row(2, 3, 3),
 +                   row(3, 3, 3));
 +
 +        assertRows(execute("SELECT * FROM %s WHERE a = ? ORDER BY b DESC PER PARTITION LIMIT ?", 2, 3),
 +                   row(2, 4, 4),
 +                   row(2, 3, 3),
 +                   row(2, 2, 2));
 +
 +        // with filtering
 +        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b > ? PER PARTITION LIMIT ? ALLOW FILTERING", 2, 0, 2),
 +                   row(2, 1, 1),
 +                   row(2, 2, 2));
 +
 +        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b > ? ORDER BY b DESC PER PARTITION LIMIT ? ALLOW FILTERING", 2, 2, 2),
 +                   row(2, 4, 4),
 +                   row(2, 3, 3));
 +
 +        assertInvalidMessage("PER PARTITION LIMIT is not allowed with SELECT DISTINCT queries",
 +                             "SELECT DISTINCT a FROM %s PER PARTITION LIMIT ?", 3);
 +        assertInvalidMessage("PER PARTITION LIMIT is not allowed with SELECT DISTINCT queries",
 +                             "SELECT DISTINCT a FROM %s PER PARTITION LIMIT ? LIMIT ?", 3, 4);
 +        assertInvalidMessage("PER PARTITION LIMIT is not allowed with aggregate queries.",
 +                             "SELECT COUNT(*) FROM %s PER PARTITION LIMIT ?", 3);
 +    }
++
++    @Test
+     public void testLimitWithDeletedRowsAndStaticColumns() throws Throwable
+     {
+         createTable("CREATE TABLE %s (pk int, c int, v int, s int static, PRIMARY KEY (pk, c))");
+ 
+         execute("INSERT INTO %s (pk, c, v, s) VALUES (1, -1, 1, 1)");
+         execute("INSERT INTO %s (pk, c, v, s) VALUES (2, -1, 1, 1)");
+         execute("INSERT INTO %s (pk, c, v, s) VALUES (3, -1, 1, 1)");
+         execute("INSERT INTO %s (pk, c, v, s) VALUES (4, -1, 1, 1)");
+         execute("INSERT INTO %s (pk, c, v, s) VALUES (5, -1, 1, 1)");
+ 
+         assertRows(execute("SELECT * FROM %s"),
+                    row(1, -1, 1, 1),
+                    row(2, -1, 1, 1),
+                    row(3, -1, 1, 1),
+                    row(4, -1, 1, 1),
+                    row(5, -1, 1, 1));
+ 
+         execute("DELETE FROM %s WHERE pk = 2");
+ 
+         assertRows(execute("SELECT * FROM %s"),
+                    row(1, -1, 1, 1),
+                    row(3, -1, 1, 1),
+                    row(4, -1, 1, 1),
+                    row(5, -1, 1, 1));
+ 
+         assertRows(execute("SELECT * FROM %s LIMIT 2"),
+                    row(1, -1, 1, 1),
+                    row(3, -1, 1, 1));
+     }
  }


[06/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 574e8df207f46d1af361a4a651e6b6cdde18688c
Parents: 78f077d bc23632
Author: Marcus Eriksson <ma...@apache.org>
Authored: Wed Jul 6 08:08:42 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Wed Jul 6 08:08:42 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 14 +++----
 .../cassandra/db/PartitionRangeReadCommand.java |  2 +-
 .../cassandra/db/SizeEstimatesRecorder.java     |  8 +++-
 .../compaction/AbstractCompactionStrategy.java  |  2 +-
 .../db/compaction/CompactionController.java     |  2 +-
 .../DateTieredCompactionStrategy.java           |  2 +-
 .../TimeWindowCompactionStrategy.java           |  2 +-
 .../apache/cassandra/db/lifecycle/Tracker.java  |  2 +-
 .../org/apache/cassandra/db/lifecycle/View.java | 36 +++++++++--------
 .../apache/cassandra/db/view/ViewBuilder.java   |  4 +-
 .../cassandra/index/SecondaryIndexManager.java  |  2 +-
 .../index/internal/CassandraIndex.java          |  2 +-
 .../io/sstable/IndexSummaryManager.java         |  2 +-
 .../cassandra/streaming/StreamSession.java      | 12 ++++--
 .../apache/cassandra/db/lifecycle/ViewTest.java |  8 ++--
 .../index/internal/CustomCassandraIndex.java    |  2 +-
 .../io/sstable/SSTableRewriterTest.java         | 41 +++++++++++++++++++-
 18 files changed, 100 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index aa3e939,b3063b4..afeef2c
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,7 -1,6 +1,9 @@@
 -3.0.9
 +3.9
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
+  * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
+  * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
   * Fix column ordering of results with static columns for Thrift requests in
     a mixed 2.x/3.x cluster, also fix potential non-resolved duplication of
     those static columns in query results (CASSANDRA-12123)

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index bca1877,842ad5f..a8959c2
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -173,9 -173,9 +173,9 @@@ public class PartitionRangeReadCommand 
          metric.rangeLatency.addNano(latencyNanos);
      }
  
 -    protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
 +    protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController executionController)
      {
-         ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, dataRange().keyRange()));
+         ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
          Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));
  
          // fetch data from current memtable, historical memtables, and SSTables in the correct order.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/src/java/org/apache/cassandra/db/compaction/CompactionController.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/src/java/org/apache/cassandra/db/lifecycle/View.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/src/java/org/apache/cassandra/db/view/ViewBuilder.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/574e8df2/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------


[50/50] cassandra git commit: Netty 4.0.39-Final update post .37 fix

Posted by al...@apache.org.
Netty 4.0.39-Final update post .37 fix


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

Branch: refs/heads/cassandra-3.8
Commit: 2aa76632d61c60f1bc115289750fd594949ff98f
Parents: 136f6b1
Author: T Jake Luciani <ja...@apache.org>
Authored: Mon Jul 18 10:46:17 2016 -0400
Committer: T Jake Luciani <ja...@apache.org>
Committed: Mon Jul 18 10:53:22 2016 -0400

----------------------------------------------------------------------
 CHANGES.txt                             |   3 +-
 build.xml                               |   2 +-
 lib/licenses/netty-all-4.0.36.Final.txt | 202 ---------------------------
 lib/licenses/netty-all-4.0.39.Final.txt | 202 +++++++++++++++++++++++++++
 lib/netty-all-4.0.36.Final.jar          | Bin 2195921 -> 0 bytes
 lib/netty-all-4.0.39.Final.jar          | Bin 0 -> 2271610 bytes
 6 files changed, 205 insertions(+), 204 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2aa76632/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4c46695..3307fb3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,4 @@
 3.9
- * Partial revert of CASSANDRA-11971, cannot recycle buffer in SP.sendMessagesToNonlocalDC (CASSANDRA-11950)
  * Fix hdr logging for single operation workloads (CASSANDRA-12145)
  * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
  * Increase size of flushExecutor thread pool (CASSANDRA-12071)
@@ -29,6 +28,8 @@ Merged from 2.1:
 
 
 3.8
+ * Partial revert of CASSANDRA-11971, cannot recycle buffer in SP.sendMessagesToNonlocalDC (CASSANDRA-11950)
+ * Upgrade netty to 4.0.39 (CASSANDRA-12032, CASSANDRA-12034)
  * Improve details in compaction log message (CASSANDRA-12080)
  * Allow unset values in CQLSSTableWriter (CASSANDRA-11911)
  * Chunk cache to request compressor-compatible buffers if pool space is exhausted (CASSANDRA-11993)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2aa76632/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index d151431..e13fdc0 100644
--- a/build.xml
+++ b/build.xml
@@ -426,7 +426,7 @@
           <dependency groupId="com.addthis.metrics" artifactId="reporter-config3" version="3.0.0" />
           <dependency groupId="org.mindrot" artifactId="jbcrypt" version="0.3m" />
           <dependency groupId="io.airlift" artifactId="airline" version="0.6" />
-          <dependency groupId="io.netty" artifactId="netty-all" version="4.0.36.Final" />
+          <dependency groupId="io.netty" artifactId="netty-all" version="4.0.39.Final" />
           <dependency groupId="com.google.code.findbugs" artifactId="jsr305" version="2.0.2" />
           <dependency groupId="com.clearspring.analytics" artifactId="stream" version="2.5.2" />
           <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" version="3.0.1" classifier="shaded">

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2aa76632/lib/licenses/netty-all-4.0.36.Final.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/netty-all-4.0.36.Final.txt b/lib/licenses/netty-all-4.0.36.Final.txt
deleted file mode 100644
index d645695..0000000
--- a/lib/licenses/netty-all-4.0.36.Final.txt
+++ /dev/null
@@ -1,202 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed 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.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2aa76632/lib/licenses/netty-all-4.0.39.Final.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/netty-all-4.0.39.Final.txt b/lib/licenses/netty-all-4.0.39.Final.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/lib/licenses/netty-all-4.0.39.Final.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2aa76632/lib/netty-all-4.0.36.Final.jar
----------------------------------------------------------------------
diff --git a/lib/netty-all-4.0.36.Final.jar b/lib/netty-all-4.0.36.Final.jar
deleted file mode 100644
index 5e278c4..0000000
Binary files a/lib/netty-all-4.0.36.Final.jar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2aa76632/lib/netty-all-4.0.39.Final.jar
----------------------------------------------------------------------
diff --git a/lib/netty-all-4.0.39.Final.jar b/lib/netty-all-4.0.39.Final.jar
new file mode 100644
index 0000000..3f5b3e6
Binary files /dev/null and b/lib/netty-all-4.0.39.Final.jar differ


[36/50] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

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


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

Branch: refs/heads/cassandra-3.8
Commit: 5861cd8fe2e5427b791672c6db61d2d43d925646
Parents: 4d7141f 9a8406f
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Mon Jul 11 14:55:05 2016 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Mon Jul 11 15:00:03 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                            | 6 +++---
 src/java/org/apache/cassandra/thrift/ThriftServer.java | 2 +-
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5861cd8f/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 4241da8,837c3fb..70210a8
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,32 -1,17 +1,32 @@@
 -2.2.8
 +3.0.9
 + * Fix migration of static thrift column names with non-text comparators (CASSANDRA-12147)
 + * Fix upgrading sparse tables that are incorrectly marked as dense (CASSANDRA-11315)
 + * Fix reverse queries ignoring range tombstones (CASSANDRA-11733)
- 2.2.8
-  * Use dedicated thread for JMX notifications (CASSANDRA-12146)
-  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
 + * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
 + * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
 + * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
 + * Fix column ordering of results with static columns for Thrift requests in
 +   a mixed 2.x/3.x cluster, also fix potential non-resolved duplication of
 +   those static columns in query results (CASSANDRA-12123)
 + * Avoid digest mismatch with empty but static rows (CASSANDRA-12090)
 + * Fix EOF exception when altering column type (CASSANDRA-11820)
 +Merged from 2.2:
+  * Synchronize ThriftServer::stop() (CASSANDRA-12105)
+  * Use dedicated thread for JMX notifications (CASSANDRA-12146)
 - * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
+  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
   * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
  Merged from 2.1:
 - * Don't write shadowed range tombstone (CASSANDRA-12030)
 - * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)
   * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
 - * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
  
  
 -2.2.7
 +3.0.8
 + * Fix potential race in schema during new table creation (CASSANDRA-12083)
 + * cqlsh: fix error handling in rare COPY FROM failure scenario (CASSANDRA-12070)
 + * Disable autocompaction during drain (CASSANDRA-11878)
 + * Add a metrics timer to MemtablePool and use it to track time spent blocked on memory in MemtableAllocator (CASSANDRA-11327)
 + * Fix upgrading schema with super columns with non-text subcomparators (CASSANDRA-12023)
 + * Add TimeWindowCompactionStrategy (CASSANDRA-9666)
 +Merged from 2.2:
   * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
   * Validate bloom_filter_fp_chance against lowest supported
     value when the table is created (CASSANDRA-11920)


[12/50] cassandra git commit: Range tombstones that are masked by row tombstones should not be written out

Posted by al...@apache.org.
Range tombstones that are masked by row tombstones should not be written out

patch by Nachiket Patil; reviewed by Sylvain Lebresne for CASSANDRA-12030


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

Branch: refs/heads/cassandra-3.8
Commit: 98f5f77bb3c5d50e52cbb6f577a463ca8a5134ad
Parents: 3c1653f
Author: Nachiket Patil <na...@apple.com>
Authored: Wed Jul 6 11:22:56 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jul 6 14:35:10 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/compaction/LazilyCompactedRow.java       |  3 +-
 .../apache/cassandra/db/RangeTombstoneTest.java | 40 ++++++++++++++++++++
 3 files changed, 43 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b1dcbe1..7fa995d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.16
+ * Don't write shadowed range tombstone (CASSANDRA-12030)
  * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
  * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
  * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
index f912da2..dab5eeb 100644
--- a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
+++ b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
@@ -286,7 +286,8 @@ public class LazilyCompactedRow extends AbstractCompactedRow
                 RangeTombstone t = tombstone;
                 tombstone = null;
 
-                if (t.data.isGcAble(controller.gcBefore) && t.timestamp() < getMaxPurgeableTimestamp())
+                if (t.data.isGcAble(controller.gcBefore) && t.timestamp() < getMaxPurgeableTimestamp() ||
+                    maxRowTombstone.markedForDeleteAt >= t.timestamp())
                 {
                     indexBuilder.tombstoneTracker().update(t, true);
                     return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/98f5f77b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 3292422..dfd6960 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.IndexType;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
 import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNames;
@@ -543,6 +544,45 @@ public class RangeTombstoneTest extends SchemaLoader
     }
 
     @Test
+    public void testCompactionOfRangeTombstonesCoveredByRowTombstone() throws Exception
+    {
+        long testTimeStamp = 1451606400L; // 01/01/2016 : 00:00:00 GMT
+        Keyspace table = Keyspace.open(KSNAME);
+        ColumnFamilyStore cfs = table.getColumnFamilyStore(CFNAME);
+        ByteBuffer key = ByteBufferUtil.bytes("k4");
+
+        // remove any existing sstables before starting
+        cfs.truncateBlocking();
+        cfs.disableAutoCompaction();
+        cfs.setCompactionStrategyClass(LeveledCompactionStrategy.class.getCanonicalName());
+
+        Mutation rm = new Mutation(KSNAME, key);
+        for (int i = 1; i < 11; i += 2, testTimeStamp += i * 10)
+            add(rm, i, testTimeStamp);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        rm = new Mutation(KSNAME, key);
+        ColumnFamily cf = rm.addOrGet(CFNAME);
+
+        // Write the covering row tombstone
+        cf.delete(new DeletionTime(++testTimeStamp, (int) testTimeStamp));
+
+        // Create range tombstones covered by row tombstone above.
+        for (int i = 1; i < 11; i += 2, testTimeStamp -= i * 5)
+            delete(cf, 0, 7, testTimeStamp);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        // there should be 2 sstables
+        assertEquals(2, cfs.getSSTables().size());
+
+        // compact down to nothing
+        CompactionManager.instance.performMaximal(cfs);
+        assertEquals(0, cfs.getSSTables().size());
+    }
+
+    @Test
     public void testOverwritesToDeletedColumns() throws Exception
     {
         Keyspace table = Keyspace.open(KSNAME);


[27/50] cassandra git commit: ninja fix condition to ensure close in ConnectionHandler

Posted by al...@apache.org.
ninja fix condition to ensure close in ConnectionHandler


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

Branch: refs/heads/cassandra-3.8
Commit: e983590d303c9c19577b3bd5b5c95adc9f5abb8a
Parents: f28409b
Author: Yuki Morishita <yu...@apache.org>
Authored: Fri Jul 8 19:05:06 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Fri Jul 8 19:05:06 2016 -0500

----------------------------------------------------------------------
 src/java/org/apache/cassandra/streaming/ConnectionHandler.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e983590d/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/ConnectionHandler.java b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
index 364435e..d3d8ed2 100644
--- a/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
+++ b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
@@ -233,7 +233,7 @@ public class ConnectionHandler
 
         protected void signalCloseDone()
         {
-            if (closeFuture == null)
+            if (!isClosed())
                 close();
 
             closeFuture.get().set(null);


[13/50] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by al...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2

* cassandra-2.1:
  Range tombstones that are masked by row tombstones should not be written out


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

Branch: refs/heads/cassandra-3.8
Commit: 43c741e251102bf5651ff8aa1b5ca078eb0ddc0b
Parents: d5a15e4 98f5f77
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Jul 6 14:39:13 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jul 6 14:39:13 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/compaction/LazilyCompactedRow.java       |  3 +-
 .../apache/cassandra/db/RangeTombstoneTest.java | 40 ++++++++++++++++++++
 3 files changed, 43 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/43c741e2/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 65c7c1f,7fa995d..bfd8aa2
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,43 -1,11 +1,44 @@@
 -2.1.16
 +2.2.8
 + * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 +Merged from 2.1:
+  * Don't write shadowed range tombstone (CASSANDRA-12030)
 - * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
 - * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
   * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)
 -
 -2.1.15
 + * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
   * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
 +
 +
 +2.2.7
 + * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
 + * Validate bloom_filter_fp_chance against lowest supported
 +   value when the table is created (CASSANDRA-11920)
 + * RandomAccessReader: call isEOF() only when rebuffering, not for every read operation (CASSANDRA-12013)
 + * Don't send erroneous NEW_NODE notifications on restart (CASSANDRA-11038)
 + * StorageService shutdown hook should use a volatile variable (CASSANDRA-11984)
 + * Persist local metadata earlier in startup sequence (CASSANDRA-11742)
 + * Run CommitLog tests with different compression settings (CASSANDRA-9039)
 + * cqlsh: fix tab completion for case-sensitive identifiers (CASSANDRA-11664)
 + * Avoid showing estimated key as -1 in tablestats (CASSANDRA-11587)
 + * Fix possible race condition in CommitLog.recover (CASSANDRA-11743)
 + * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
 + * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
 + * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
 + * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
 + * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
 + * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
 + * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
 + * Produce a heap dump when exiting on OOM (CASSANDRA-9861)
 + * Avoid read repairing purgeable tombstones on range slices (CASSANDRA-11427)
 + * Restore ability to filter on clustering columns when using a 2i (CASSANDRA-11510)
 + * JSON datetime formatting needs timezone (CASSANDRA-11137)
 + * Fix is_dense recalculation for Thrift-updated tables (CASSANDRA-11502)
 + * Remove unnescessary file existence check during anticompaction (CASSANDRA-11660)
 + * Add missing files to debian packages (CASSANDRA-11642)
 + * Avoid calling Iterables::concat in loops during ModificationStatement::getFunctions (CASSANDRA-11621)
 + * cqlsh: COPY FROM should use regular inserts for single statement batches and
 +   report errors correctly if workers processes crash on initialization (CASSANDRA-11474)
 + * Always close cluster with connection in CqlRecordWriter (CASSANDRA-11553)
 + * Fix slice queries on ordered COMPACT tables (CASSANDRA-10988)
 +Merged from 2.1:
   * Avoid stalling paxos when the paxos state expires (CASSANDRA-12043)
   * Remove finished incoming streaming connections from MessagingService (CASSANDRA-11854)
   * Don't try to get sstables for non-repairing column families (CASSANDRA-12077)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43c741e2/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43c741e2/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 9ce1236,dfd6960..bff0ddf
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@@ -37,10 -33,13 +37,11 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.config.*;
  import org.apache.cassandra.Util;
 -import org.apache.cassandra.config.ColumnDefinition;
 -import org.apache.cassandra.config.DatabaseDescriptor;
 -import org.apache.cassandra.config.IndexType;
  import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
  import org.apache.cassandra.db.compaction.CompactionManager;
+ import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
  import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
  import org.apache.cassandra.db.composites.CellName;
  import org.apache.cassandra.db.composites.CellNames;
@@@ -559,6 -543,46 +560,45 @@@ public class RangeTombstoneTes
      }
  
      @Test
+     public void testCompactionOfRangeTombstonesCoveredByRowTombstone() throws Exception
+     {
+         long testTimeStamp = 1451606400L; // 01/01/2016 : 00:00:00 GMT
+         Keyspace table = Keyspace.open(KSNAME);
+         ColumnFamilyStore cfs = table.getColumnFamilyStore(CFNAME);
+         ByteBuffer key = ByteBufferUtil.bytes("k4");
+ 
+         // remove any existing sstables before starting
+         cfs.truncateBlocking();
+         cfs.disableAutoCompaction();
+         cfs.setCompactionStrategyClass(LeveledCompactionStrategy.class.getCanonicalName());
+ 
+         Mutation rm = new Mutation(KSNAME, key);
+         for (int i = 1; i < 11; i += 2, testTimeStamp += i * 10)
+             add(rm, i, testTimeStamp);
+         rm.apply();
+         cfs.forceBlockingFlush();
+ 
+         rm = new Mutation(KSNAME, key);
+         ColumnFamily cf = rm.addOrGet(CFNAME);
+ 
+         // Write the covering row tombstone
+         cf.delete(new DeletionTime(++testTimeStamp, (int) testTimeStamp));
+ 
+         // Create range tombstones covered by row tombstone above.
+         for (int i = 1; i < 11; i += 2, testTimeStamp -= i * 5)
+             delete(cf, 0, 7, testTimeStamp);
+         rm.apply();
+         cfs.forceBlockingFlush();
+ 
+         // there should be 2 sstables
+         assertEquals(2, cfs.getSSTables().size());
+ 
+         // compact down to nothing
 -        CompactionManager.instance.performMaximal(cfs);
++        CompactionManager.instance.performMaximal(cfs, false);
+         assertEquals(0, cfs.getSSTables().size());
+     }
+ 
+     @Test
      public void testOverwritesToDeletedColumns() throws Exception
      {
          Keyspace table = Keyspace.open(KSNAME);


[11/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.8
Commit: 5ad17634a1fb75d09ab5a6cb234d7460fed0c3e0
Parents: 262cd38 dd05e46
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Wed Jul 6 12:10:41 2016 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Jul 6 12:10:41 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                         |  1 +
 .../org/apache/cassandra/config/CFMetaData.java     | 16 +++++++---------
 2 files changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5ad17634/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index afeef2c,02786c5..2861cf7
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,7 -1,5 +1,8 @@@
 -3.0.9
 +3.9
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
+  * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
   * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
   * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
   * Fix column ordering of results with static columns for Thrift requests in

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5ad17634/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/CFMetaData.java
index caa2059,5678ada..4708033
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@@ -22,10 -22,8 +22,9 @@@ import java.lang.reflect.Constructor
  import java.lang.reflect.InvocationTargetException;
  import java.nio.ByteBuffer;
  import java.util.*;
- import java.util.concurrent.ConcurrentHashMap;
  import java.util.concurrent.ThreadLocalRandom;
  import java.util.concurrent.TimeUnit;
 +import java.util.regex.Pattern;
  import java.util.stream.Collectors;
  
  import com.google.common.annotations.VisibleForTesting;


[39/50] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

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


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

Branch: refs/heads/cassandra-3.8
Commit: e99ee19950e764ca55331d7c814e965bef359a4f
Parents: 5861cd8 f0d1d75
Author: Josh McKenzie <jm...@apache.org>
Authored: Mon Jul 11 16:28:13 2016 -0400
Committer: Josh McKenzie <jm...@apache.org>
Committed: Mon Jul 11 16:28:22 2016 -0400

----------------------------------------------------------------------
 conf/cassandra.yaml                                       |  7 ++++++-
 src/java/org/apache/cassandra/config/Config.java          |  1 +
 .../org/apache/cassandra/config/DatabaseDescriptor.java   | 10 ++++++++++
 src/java/org/apache/cassandra/service/GCInspector.java    |  2 +-
 4 files changed, 18 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e99ee199/conf/cassandra.yaml
----------------------------------------------------------------------
diff --cc conf/cassandra.yaml
index 4b92f64,4ad798a..09d2094
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@@ -924,21 -858,23 +924,26 @@@ inter_dc_tcp_nodelay: fals
  tracetype_query_ttl: 86400
  tracetype_repair_ttl: 604800
  
+ # By default, Cassandra logs GC Pauses greater than 200 ms at INFO level
+ # This threshold can be adjusted to minimize logging if necessary
+ # gc_log_threshold_in_ms: 200
+ 
  # GC Pauses greater than gc_warn_threshold_in_ms will be logged at WARN level
+ # If unset, all GC Pauses greater than gc_log_threshold_in_ms will log at
+ # INFO level
  # Adjust the threshold based on your application throughput requirement
- # By default, Cassandra logs GC Pauses greater than 200 ms at INFO level
 -# gc_warn_threshold_in_ms: 1000
 +gc_warn_threshold_in_ms: 1000
  
  # UDFs (user defined functions) are disabled by default.
 -# As of Cassandra 2.2, there is no security manager or anything else in place that
 -# prevents execution of evil code. CASSANDRA-9402 will fix this issue for Cassandra 3.0.
 -# This will inherently be backwards-incompatible with any 2.2 UDF that perform insecure
 -# operations such as opening a socket or writing to the filesystem.
 +# As of Cassandra 3.0 there is a sandbox in place that should prevent execution of evil code.
  enable_user_defined_functions: false
  
 +# Enables scripted UDFs (JavaScript UDFs).
 +# Java UDFs are always enabled, if enable_user_defined_functions is true.
 +# Enable this option to be able to use UDFs with "language javascript" or any custom JSR-223 provider.
 +# This option has no effect, if enable_user_defined_functions is false.
 +enable_scripted_user_defined_functions: false
 +
  # The default Windows kernel timer and scheduling resolution is 15.6ms for power conservation.
  # Lowering this value on Windows can provide much tighter latency and better throughput, however
  # some virtualized environments may see a negative performance impact from changing this setting

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e99ee199/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/Config.java
index b49e14c,ede4560..2bd23b5
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@@ -265,8 -254,12 +265,9 @@@ public class Confi
      public volatile Long index_summary_capacity_in_mb;
      public volatile int index_summary_resize_interval_in_minutes = 60;
  
+     public int gc_log_threshold_in_ms = 200;
      public int gc_warn_threshold_in_ms = 0;
  
 -    private static final CsvPreference STANDARD_SURROUNDING_SPACES_NEED_QUOTES = new CsvPreference.Builder(CsvPreference.STANDARD_PREFERENCE)
 -                                                                                                  .surroundingSpacesNeedQuotes(true).build();
 -
      // TTL for different types of trace events.
      public int tracetype_query_ttl = (int) TimeUnit.DAYS.toSeconds(1);
      public int tracetype_repair_ttl = (int) TimeUnit.DAYS.toSeconds(7);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e99ee199/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 2083e42f,f1acfc4..100bcf4
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@@ -1934,51 -1801,16 +1939,56 @@@ public class DatabaseDescripto
          return conf.enable_user_defined_functions;
      }
  
 -    public static int getWindowsTimerInterval()
 +    public static boolean enableScriptedUserDefinedFunctions()
      {
 -        return conf.windows_timer_interval;
 +        return conf.enable_scripted_user_defined_functions;
 +    }
 +
 +    public static void enableScriptedUserDefinedFunctions(boolean enableScriptedUserDefinedFunctions)
 +    {
 +        conf.enable_scripted_user_defined_functions = enableScriptedUserDefinedFunctions;
 +    }
 +
 +    public static boolean enableUserDefinedFunctionsThreads()
 +    {
 +        return conf.enable_user_defined_functions_threads;
 +    }
 +
 +    public static long getUserDefinedFunctionWarnTimeout()
 +    {
 +        return conf.user_defined_function_warn_timeout;
 +    }
 +
 +    public static void setUserDefinedFunctionWarnTimeout(long userDefinedFunctionWarnTimeout)
 +    {
 +        conf.user_defined_function_warn_timeout = userDefinedFunctionWarnTimeout;
 +    }
 +
 +    public static long getUserDefinedFunctionFailTimeout()
 +    {
 +        return conf.user_defined_function_fail_timeout;
 +    }
 +
 +    public static void setUserDefinedFunctionFailTimeout(long userDefinedFunctionFailTimeout)
 +    {
 +        conf.user_defined_function_fail_timeout = userDefinedFunctionFailTimeout;
 +    }
 +
 +    public static Config.UserFunctionTimeoutPolicy getUserFunctionTimeoutPolicy()
 +    {
 +        return conf.user_function_timeout_policy;
 +    }
 +
 +    public static void setUserFunctionTimeoutPolicy(Config.UserFunctionTimeoutPolicy userFunctionTimeoutPolicy)
 +    {
 +        conf.user_function_timeout_policy = userFunctionTimeoutPolicy;
      }
  
+     public static long getGCLogThreshold()
+     {
+         return conf.gc_log_threshold_in_ms;
+     }
+ 
      public static long getGCWarnThreshold()
      {
          return conf.gc_warn_threshold_in_ms;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e99ee199/src/java/org/apache/cassandra/service/GCInspector.java
----------------------------------------------------------------------


[35/50] cassandra git commit: Synchronize ThriftServer::stop()

Posted by al...@apache.org.
Synchronize ThriftServer::stop()

patch by Brian Wawok; reviewed by Aleksey Yeschenko for CASSANDRA-12105


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

Branch: refs/heads/cassandra-3.8
Commit: 9a8406f2f7f03fb1794e6cb0b5585579cb93cc92
Parents: e983590
Author: Brian Wawok <br...@datasax.com>
Authored: Mon Jul 11 14:52:20 2016 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Mon Jul 11 14:53:24 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                            | 1 +
 src/java/org/apache/cassandra/thrift/ThriftServer.java | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9a8406f2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9fef5a2..837c3fb 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.8
+ * Synchronize ThriftServer::stop() (CASSANDRA-12105)
  * Use dedicated thread for JMX notifications (CASSANDRA-12146)
  * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
  * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9a8406f2/src/java/org/apache/cassandra/thrift/ThriftServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftServer.java b/src/java/org/apache/cassandra/thrift/ThriftServer.java
index 2409606..44ec524 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftServer.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftServer.java
@@ -60,7 +60,7 @@ public class ThriftServer implements CassandraDaemon.Server
         }
     }
 
-    public void stop()
+    public synchronized void stop()
     {
         if (server != null)
         {


[02/50] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9

Conflicts:
	CHANGES.txt
	src/java/org/apache/cassandra/db/ReadResponse.java


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

Branch: refs/heads/cassandra-3.8
Commit: 32825a9b4d9b20620f0536e4acd0e43d7a1737c0
Parents: 237330a 2217695
Author: Tyler Hobbs <ty...@gmail.com>
Authored: Tue Jul 5 11:47:34 2016 -0500
Committer: Tyler Hobbs <ty...@gmail.com>
Committed: Tue Jul 5 11:47:34 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                        | 3 +++
 src/java/org/apache/cassandra/db/ReadResponse.java | 7 +++++--
 2 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/32825a9b/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 68854b3,7f8a3a1..aa3e939
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,7 -1,7 +1,10 @@@
 -3.0.9
 +3.9
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
+  * Fix column ordering of results with static columns for Thrift requests in
+    a mixed 2.x/3.x cluster, also fix potential non-resolved duplication of
+    those static columns in query results (CASSANDRA-12123)
   * Avoid digest mismatch with empty but static rows (CASSANDRA-12090)
   * Fix EOF exception when altering column type (CASSANDRA-11820)
  Merged from 2.2:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/32825a9b/src/java/org/apache/cassandra/db/ReadResponse.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ReadResponse.java
index 6d0b2e1,8bd1be6..3f21480
--- a/src/java/org/apache/cassandra/db/ReadResponse.java
+++ b/src/java/org/apache/cassandra/db/ReadResponse.java
@@@ -272,9 -273,11 +273,11 @@@ public abstract class ReadRespons
                      // Pre-3.0, we didn't have a way to express exclusivity for non-composite comparators, so all slices were
                      // inclusive on both ends. If we have exclusive slice ends, we need to filter the results here.
                      if (!command.metadata().isCompound())
-                         return partition.unfilteredIterator(command.columnFilter(), filter.getSlices(command.metadata()), filter.isReversed());
+                         return ThriftResultsMerger.maybeWrap(
 -                                filter.filter(partition.sliceableUnfilteredIterator(command.columnFilter(), filter.isReversed())), command.nowInSec());
++                                partition.unfilteredIterator(command.columnFilter(), filter.getSlices(command.metadata()), filter.isReversed()), command.nowInSec());
  
-                     return partition.unfilteredIterator(command.columnFilter(), Slices.ALL, filter.isReversed());
+                     return ThriftResultsMerger.maybeWrap(
+                             partition.unfilteredIterator(command.columnFilter(), Slices.ALL, filter.isReversed()), command.nowInSec());
                  }
              };
          }


[03/50] cassandra git commit: Revert "Update to Netty 4.0.37"

Posted by al...@apache.org.
Revert "Update to Netty 4.0.37"

This reverts commit e8d7fe8a289e16d18a83e598c589347ec080029f.


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

Branch: refs/heads/cassandra-3.8
Commit: 78f077de079d0b3760a06a7f944be070772b0c83
Parents: 32825a9
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jul 6 09:09:49 2016 +0800
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jul 6 12:58:06 2016 +0800

----------------------------------------------------------------------
 build.xml                               |   2 +-
 conf/cassandra-env.ps1                  |   9 --
 conf/cassandra-env.sh                   |   9 --
 lib/licenses/netty-all-4.0.36.Final.txt | 202 +++++++++++++++++++++++++++
 lib/licenses/netty-all-4.0.37.Final.txt | 202 ---------------------------
 lib/netty-all-4.0.36.Final.jar          | Bin 0 -> 2195921 bytes
 lib/netty-all-4.0.37.Final.jar          | Bin 2204062 -> 0 bytes
 7 files changed, 203 insertions(+), 221 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/78f077de/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 5ddf40d..d151431 100644
--- a/build.xml
+++ b/build.xml
@@ -426,7 +426,7 @@
           <dependency groupId="com.addthis.metrics" artifactId="reporter-config3" version="3.0.0" />
           <dependency groupId="org.mindrot" artifactId="jbcrypt" version="0.3m" />
           <dependency groupId="io.airlift" artifactId="airline" version="0.6" />
-          <dependency groupId="io.netty" artifactId="netty-all" version="4.0.37.Final" />
+          <dependency groupId="io.netty" artifactId="netty-all" version="4.0.36.Final" />
           <dependency groupId="com.google.code.findbugs" artifactId="jsr305" version="2.0.2" />
           <dependency groupId="com.clearspring.analytics" artifactId="stream" version="2.5.2" />
           <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" version="3.0.1" classifier="shaded">

http://git-wip-us.apache.org/repos/asf/cassandra/blob/78f077de/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1
index 8c4311c..9373ba6 100644
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@ -350,15 +350,6 @@ Function SetCassandraEnvironment
     #$env:HEAP_NEWSIZE="800M"
     CalculateHeapSizes
 
-    # Direct memory used for native-protocol network I/O is no longer
-    # managed by the JVM. Instead, Netty allows three options to
-    # manage it via the system property io.netty.maxDirectMemory:
-    #     == 0  behavior as before, uses JVM to manage direct memory (slowest).
-    #     < 0   manages direct memory directly, max direct memory as -XX:MaxDirectMemorySize.
-    #     > 0   manages direct memory directly, max direct memory as specified.
-    #           Note, that appreviations like 2g or 500m are NOT accepted.
-    #$env:JVM_OPTS="$env:JVM_OPTS -Dio.netty.maxDirectMemory=2147483648"
-
     ParseJVMInfo
 
     # We only set -Xms and -Xmx if they were not defined on jvm.options file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/78f077de/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index 6d5de21..93434c9 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -167,15 +167,6 @@ USING_G1=$?
 # Set this to control the amount of arenas per-thread in glibc
 #export MALLOC_ARENA_MAX=4
 
-# Direct memory used for native-protocol network I/O is no longer
-# managed by the JVM. Instead, Netty allows three options to
-# manage it via the system property io.netty.maxDirectMemory:
-#     == 0  behavior as before, uses JVM to manage direct memory (slowest).
-#     < 0   manages direct memory directly, max direct memory as -XX:MaxDirectMemorySize.
-#     > 0   manages direct memory directly, max direct memory as specified.
-#           Note, that appreviations like 2g or 500m are NOT accepted.
-#export JVM_OPTS="$JVM_OPTS -Dio.netty.maxDirectMemory=2147483648"
-
 # only calculate the size if it's not set manually
 if [ "x$MAX_HEAP_SIZE" = "x" ] && [ "x$HEAP_NEWSIZE" = "x" -o $USING_G1 -eq 0 ]; then
     calculate_heap_sizes

http://git-wip-us.apache.org/repos/asf/cassandra/blob/78f077de/lib/licenses/netty-all-4.0.36.Final.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/netty-all-4.0.36.Final.txt b/lib/licenses/netty-all-4.0.36.Final.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/lib/licenses/netty-all-4.0.36.Final.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/78f077de/lib/licenses/netty-all-4.0.37.Final.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/netty-all-4.0.37.Final.txt b/lib/licenses/netty-all-4.0.37.Final.txt
deleted file mode 100644
index d645695..0000000
--- a/lib/licenses/netty-all-4.0.37.Final.txt
+++ /dev/null
@@ -1,202 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed 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.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/78f077de/lib/netty-all-4.0.36.Final.jar
----------------------------------------------------------------------
diff --git a/lib/netty-all-4.0.36.Final.jar b/lib/netty-all-4.0.36.Final.jar
new file mode 100644
index 0000000..5e278c4
Binary files /dev/null and b/lib/netty-all-4.0.36.Final.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/78f077de/lib/netty-all-4.0.37.Final.jar
----------------------------------------------------------------------
diff --git a/lib/netty-all-4.0.37.Final.jar b/lib/netty-all-4.0.37.Final.jar
deleted file mode 100644
index f0bcdac..0000000
Binary files a/lib/netty-all-4.0.37.Final.jar and /dev/null differ