You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sp...@apache.org on 2017/03/23 20:01:19 UTC

[01/11] cassandra git commit: Bugs handling range tombstones in the sstable iterators

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 bf0906b92 -> 5978f9d5f
  refs/heads/cassandra-3.0 f53e502c3 -> 631162271
  refs/heads/cassandra-3.11 82d3cdcd6 -> a10b8079e
  refs/heads/trunk 18c6ed25e -> 3048608c6


Bugs handling range tombstones in the sstable iterators

patch by Sylvain Lebresne; reviewed by Branimir Lambov for CASSANDRA-13340


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

Branch: refs/heads/cassandra-3.11
Commit: f53e502c3c484481a296d9fdbff5fde4b709a9fc
Parents: 2836a64
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Mar 16 17:05:15 2017 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Mar 23 17:04:07 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ClusteringPrefix.java   |   2 +-
 .../cassandra/db/UnfilteredDeserializer.java    |   1 -
 .../db/columniterator/SSTableIterator.java      |  11 +-
 .../columniterator/SSTableReversedIterator.java | 124 +++++++++++++++----
 .../cql3/validation/operations/DeleteTest.java  |  70 +++++++++++
 6 files changed, 180 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f53e502c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9140c73..4ee5814 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.13
+ * Bugs handling range tombstones in the sstable iterators (CASSANDRA-13340)
  * Fix CONTAINS filtering for null collections (CASSANDRA-13246)
  * Applying: Use a unique metric reservoir per test run when using Cassandra-wide metrics residing in MBeans (CASSANDRA-13216)
  * Propagate row deletions in 2i tables on upgrade (CASSANDRA-13320)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f53e502c/src/java/org/apache/cassandra/db/ClusteringPrefix.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ClusteringPrefix.java b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
index 7f7f964..3b826c9 100644
--- a/src/java/org/apache/cassandra/db/ClusteringPrefix.java
+++ b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
@@ -451,7 +451,7 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
             }
 
             if (bound.size() == nextSize)
-                return nextKind.compareTo(bound.kind());
+                return Kind.compare(nextKind, bound.kind());
 
             // We know that we'll have exited already if nextSize < bound.size
             return -bound.kind().comparedToClustering;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f53e502c/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
index 42a806a..7bbbfdb 100644
--- a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
+++ b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
@@ -694,6 +694,5 @@ public abstract class UnfilteredDeserializer
                 }
             }
         }
-
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f53e502c/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
index 0409310..9bcca48 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
@@ -123,7 +123,14 @@ public class SSTableIterator extends AbstractSSTableIterator
         {
             assert deserializer != null;
 
-            if (!deserializer.hasNext() || deserializer.compareNextTo(end) > 0)
+            // We use a same reasoning as in handlePreSliceData regarding the strictness of the inequality below.
+            // We want to exclude deserialized unfiltered equal to end, because 1) we won't miss any rows since those
+            // woudn't be equal to a slice bound and 2) a end bound can be equal to a start bound
+            // (EXCL_END(x) == INCL_START(x) for instance) and in that case we don't want to return start bound because
+            // it's fundamentally excluded. And if the bound is a  end (for a range tombstone), it means it's exactly
+            // our slice end, but in that  case we will properly close the range tombstone anyway as part of our "close
+            // an open marker" code in hasNextInterna
+            if (!deserializer.hasNext() || deserializer.compareNextTo(end) >= 0)
                 return null;
 
             Unfiltered next = deserializer.readNext();
@@ -259,7 +266,7 @@ public class SSTableIterator extends AbstractSSTableIterator
             if (indexState.isDone()
                 || indexState.currentBlockIdx() > lastBlockIdx
                 || !deserializer.hasNext()
-                || (indexState.currentBlockIdx() == lastBlockIdx && deserializer.compareNextTo(end) > 0))
+                || (indexState.currentBlockIdx() == lastBlockIdx && deserializer.compareNextTo(end) >= 0))
                 return null;
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f53e502c/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 3e49a3a..49dc82a 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@ -27,6 +27,7 @@ import org.apache.cassandra.db.partitions.ImmutableBTreePartition;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.utils.AbstractIterator;
 import org.apache.cassandra.utils.btree.BTree;
 
 /**
@@ -66,6 +67,11 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
         protected ReusablePartitionData buffer;
         protected Iterator<Unfiltered> iterator;
 
+        // Set in loadFromDisk () and used in setIterator to handle range tombstone extending on multiple index block. See
+        // loadFromDisk for details. Note that those are always false for non-indexed readers.
+        protected boolean skipFirstIteratedItem;
+        protected boolean skipLastIteratedItem;
+
         private ReverseReader(FileDataInput file, boolean shouldCloseFile)
         {
             super(file, shouldCloseFile);
@@ -109,7 +115,7 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
                 // Note that we can reuse that buffer between slices (we could alternatively re-read from disk
                 // every time, but that feels more wasteful) so we want to include everything from the beginning.
                 // We can stop at the slice end however since any following slice will be before that.
-                loadFromDisk(null, slice.end(), true);
+                loadFromDisk(null, slice.end(), true, false, false);
             }
             setIterator(slice);
         }
@@ -118,6 +124,15 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
         {
             assert buffer != null;
             iterator = buffer.built.unfilteredIterator(columns, Slices.with(metadata().comparator, slice), true);
+
+            if (!iterator.hasNext())
+                return;
+
+            if (skipFirstIteratedItem)
+                iterator.next();
+
+            if (skipLastIteratedItem)
+                iterator = new SkipLastIterator(iterator);
         }
 
         protected boolean hasNextInternal() throws IOException
@@ -143,9 +158,18 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
 
         // Reads the unfiltered from disk and load them into the reader buffer. It stops reading when either the partition
         // is fully read, or when stopReadingDisk() returns true.
-        protected void loadFromDisk(Slice.Bound start, Slice.Bound end, boolean includeFirst) throws IOException
+        protected void loadFromDisk(Slice.Bound start,
+                                    Slice.Bound end,
+                                    boolean includeFirst,
+                                    boolean hasPreviousBlock,
+                                    boolean hasNextBlock) throws IOException
         {
+            // start != null means it's the block covering the beginning of the slice, so it has to be the last block for this slice.
+            assert start == null || !hasNextBlock;
+
             buffer.reset();
+            skipFirstIteratedItem = false;
+            skipLastIteratedItem = false;
 
             boolean isFirst = true;
 
@@ -162,16 +186,30 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
                 }
             }
 
-            // If we have an open marker, it's either one from what we just skipped (if start != null), or it's from the previous index block.
+            // If we have an open marker, it's either one from what we just skipped or it's one that open in the next (or
+            // one of the next) index block (if openMarker == openMarkerAtStartOfBlock).
             if (openMarker != null)
             {
+                // We have to feed a marker to the buffer, because that marker is likely to be close later and ImmtableBTreePartition
+                // doesn't take kindly to marker that comes without their counterpart. If that's the last block we're gonna read (for
+                // the current slice at least) it's easy because we'll want to return that open marker at the end of the data in this
+                // block anyway, so we have nothing more to do than adding it to the buffer.
+                // If it's not the last block however, in which case we know we'll have start == null, it means this marker is really
+                // open in a next block and so while we do need to add it the buffer for the reason mentioned above, we don't
+                // want to "return" it just yet, we'll wait until we reach it in the next blocks. That's why we trigger
+                // skipLastIteratedItem in that case (this is first item of the block, but we're iterating in reverse order
+                // so it will be last returned by the iterator).
                 RangeTombstone.Bound markerStart = start == null ? RangeTombstone.Bound.BOTTOM : RangeTombstone.Bound.fromSliceBound(start);
                 buffer.add(new RangeTombstoneBoundMarker(markerStart, openMarker));
+                if (hasNextBlock)
+                    skipLastIteratedItem = true;
             }
 
             // Now deserialize everything until we reach our requested end (if we have one)
+            // See SSTableIterator.ForwardRead.computeNext() for why this is a strict inequality below: this is the same
+            // reasoning here.
             while (deserializer.hasNext()
-                   && (end == null || deserializer.compareNextTo(end) <= 0)
+                   && (end == null || deserializer.compareNextTo(end) < 0)
                    && !stopReadingDisk())
             {
                 Unfiltered unfiltered = deserializer.readNext();
@@ -187,9 +225,18 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
             // If we have an open marker, we should close it before finishing
             if (openMarker != null)
             {
-                // If we have no end and still an openMarker, this means we're indexed and the marker is closed in a following block.
+                // This is the reverse problem than the one at the start of the block. Namely, if it's the first block
+                // we deserialize for the slice (the one covering the slice end basically), then it's easy, we just want
+                // to add the close marker to the buffer and return it normally.
+                // If it's note our first block (for the slice) however, it means that marker closed in a previously read
+                // block and we have already returned it. So while we should still add it to the buffer for the sake of
+                // not breaking ImmutableBTreePartition, we should skip it when returning from the iterator, hence the
+                // skipFirstIteratedItem (this is the last item of the block, but we're iterating in reverse order so it will
+                // be the first returned by the iterator).
                 RangeTombstone.Bound markerEnd = end == null ? RangeTombstone.Bound.TOP : RangeTombstone.Bound.fromSliceBound(end);
                 buffer.add(new RangeTombstoneBoundMarker(markerEnd, getAndClearOpenMarker()));
+                if (hasPreviousBlock)
+                    skipFirstIteratedItem = true;
             }
 
             buffer.build();
@@ -245,13 +292,13 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
             if (startIdx >= indexState.blocksCount())
                 startIdx = indexState.blocksCount() - 1;
 
-            if (startIdx != indexState.currentBlockIdx())
-            {
-                indexState.setToBlock(startIdx);
-                readCurrentBlock(true);
-            }
+            // Note that even if we were already set on the proper block (which would happen if the previous slice
+            // requested ended on the same block this one start), we can't reuse it because when reading the previous
+            // slice we've only read that block from the previous slice start. Re-reading also handles
+            // skipFirstIteratedItem/skipLastIteratedItem that we would need to handle otherwise.
+            indexState.setToBlock(startIdx);
 
-            setIterator(slice);
+            readCurrentBlock(false, startIdx != lastBlockIdx);
         }
 
         @Override
@@ -260,15 +307,14 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
             if (super.hasNextInternal())
                 return true;
 
-            // We have nothing more for our current block, move the previous one.
-            int previousBlockIdx = indexState.currentBlockIdx() - 1;
-            if (previousBlockIdx < 0 || previousBlockIdx < lastBlockIdx)
+            // We have nothing more for our current block, move the next one (so the one before on disk).
+            int nextBlockIdx = indexState.currentBlockIdx() - 1;
+            if (nextBlockIdx < 0 || nextBlockIdx < lastBlockIdx)
                 return false;
 
             // The slice start can be in 
-            indexState.setToBlock(previousBlockIdx);
-            readCurrentBlock(false);
-            setIterator(slice);
+            indexState.setToBlock(nextBlockIdx);
+            readCurrentBlock(true, nextBlockIdx != lastBlockIdx);
             // since that new block is within the bounds we've computed in setToSlice(), we know there will
             // always be something matching the slice unless we're on the lastBlockIdx (in which case there
             // may or may not be results, but if there isn't, we're done for the slice).
@@ -278,33 +324,42 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
         /**
          * Reads the current block, the last one we've set.
          *
-         * @param canIncludeSliceEnd whether the block can include the slice end.
+         * @param hasPreviousBlock is whether we have already read a previous block for the current slice.
+         * @param hasNextBlock is whether we have more blocks to read for the current slice.
          */
-        private void readCurrentBlock(boolean canIncludeSliceEnd) throws IOException
+        private void readCurrentBlock(boolean hasPreviousBlock, boolean hasNextBlock) throws IOException
         {
             if (buffer == null)
                 buffer = createBuffer(indexState.blocksCount());
 
             int currentBlock = indexState.currentBlockIdx();
 
-            boolean canIncludeSliceStart = currentBlock == lastBlockIdx;
+            // The slice start (resp. slice end) is only meaningful on the last (resp. first) block read (since again,
+            // we read blocks in reverse order).
+            boolean canIncludeSliceStart = !hasNextBlock;
+            boolean canIncludeSliceEnd = !hasPreviousBlock;
 
             // When dealing with old format sstable, we have the problem that a row can span 2 index block, i.e. it can
             // start at the end of a block and end at the beginning of the next one. That's not a problem per se for
             // UnfilteredDeserializer.OldFormatSerializer, since it always read rows entirely, even if they span index
             // blocks, but as we reading index block in reverse we must be careful to not read the end of the row at
             // beginning of a block before we're reading the beginning of that row. So what we do is that if we detect
-            // that the row starting this block is also the row ending the previous one, we skip that first result and
-            // let it be read when we'll read the previous block.
+            // that the row starting this block is also the row ending the next one we're read (previous on disk), then
+            // we'll skip that first result and  let it be read with the next block.
             boolean includeFirst = true;
             if (!sstable.descriptor.version.storeRows() && currentBlock > 0)
             {
-                ClusteringPrefix lastOfPrevious = indexState.index(currentBlock - 1).lastName;
+                ClusteringPrefix lastOfNext = indexState.index(currentBlock - 1).lastName;
                 ClusteringPrefix firstOfCurrent = indexState.index(currentBlock).firstName;
-                includeFirst = metadata().comparator.compare(lastOfPrevious, firstOfCurrent) != 0;
+                includeFirst = metadata().comparator.compare(lastOfNext, firstOfCurrent) != 0;
             }
 
-            loadFromDisk(canIncludeSliceStart ? slice.start() : null, canIncludeSliceEnd ? slice.end() : null, includeFirst);
+            loadFromDisk(canIncludeSliceStart ? slice.start() : null,
+                         canIncludeSliceEnd ? slice.end() : null,
+                         includeFirst,
+                         hasPreviousBlock,
+                         hasNextBlock);
+            setIterator(slice);
         }
 
         @Override
@@ -360,4 +415,23 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
             deletionBuilder = null;
         }
     }
+
+    private static class SkipLastIterator extends AbstractIterator<Unfiltered>
+    {
+        private final Iterator<Unfiltered> iterator;
+
+        private SkipLastIterator(Iterator<Unfiltered> iterator)
+        {
+            this.iterator = iterator;
+        }
+
+        protected Unfiltered computeNext()
+        {
+            if (!iterator.hasNext())
+                return endOfData();
+
+            Unfiltered next = iterator.next();
+            return iterator.hasNext() ? next : endOfData();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f53e502c/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 9d7d4a3..9f770a5 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -1291,6 +1291,76 @@ public class DeleteTest extends CQLTester
         assertTrue("The memtable should be empty but is not", isMemtableEmpty());
     }
 
+    @Test
+    public void testQueryingOnRangeTombstoneBoundForward() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k text, i int, PRIMARY KEY (k, i))");
+
+        execute("INSERT INTO %s (k, i) VALUES (?, ?)", "a", 0);
+
+        execute("DELETE FROM %s WHERE k = ? AND i > ? AND i <= ?", "a", 0, 1);
+        execute("DELETE FROM %s WHERE k = ? AND i > ?", "a", 1);
+
+        flush();
+
+        assertEmpty(execute("SELECT i FROM %s WHERE k = ? AND i = ?", "a", 1));
+    }
+
+    @Test
+    public void testQueryingOnRangeTombstoneBoundReverse() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k text, i int, PRIMARY KEY (k, i))");
+
+        execute("INSERT INTO %s (k, i) VALUES (?, ?)", "a", 0);
+
+        execute("DELETE FROM %s WHERE k = ? AND i > ? AND i <= ?", "a", 0, 1);
+        execute("DELETE FROM %s WHERE k = ? AND i > ?", "a", 1);
+
+        flush();
+
+        assertRows(execute("SELECT i FROM %s WHERE k = ? AND i <= ? ORDER BY i DESC", "a", 1), row(0));
+    }
+
+    @Test
+    public void testReverseQueryWithRangeTombstoneOnMultipleBlocks() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k text, i int, v text, PRIMARY KEY (k, i))");
+
+        StringBuilder sb = new StringBuilder();
+        for (int i = 0; i < 1200; i++)
+            sb.append('a');
+        String longText = sb.toString();
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s(k, i, v) VALUES (?, ?, ?) USING TIMESTAMP 3", "a", i*2, longText);
+
+        execute("DELETE FROM %s USING TIMESTAMP 1 WHERE k = ? AND i >= ? AND i <= ?", "a", 12, 16);
+
+        flush();
+
+        execute("INSERT INTO %s(k, i, v) VALUES (?, ?, ?) USING TIMESTAMP 0", "a", 3, longText);
+        execute("INSERT INTO %s(k, i, v) VALUES (?, ?, ?) USING TIMESTAMP 3", "a", 11, longText);
+        execute("INSERT INTO %s(k, i, v) VALUES (?, ?, ?) USING TIMESTAMP 0", "a", 15, longText);
+        execute("INSERT INTO %s(k, i, v) VALUES (?, ?, ?) USING TIMESTAMP 0", "a", 17, longText);
+
+        flush();
+
+        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
+                   row(18),
+                   row(17),
+                   row(16),
+                   row(14),
+                   row(12),
+                   row(11),
+                   row(10),
+                   row(8),
+                   row(6),
+                   row(4),
+                   row(3),
+                   row(2),
+                   row(0));
+    }
+
     /**
      * Test for CASSANDRA-13305
      */


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

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

Branch: refs/heads/cassandra-3.11
Commit: 631162271c9bbaca6b48dc4e2223dbba97bf51d4
Parents: f53e502 5978f9d
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Thu Mar 23 20:49:05 2017 +0100
Committer: Stefan Podkowinski <s....@gmail.com>
Committed: Thu Mar 23 20:50:13 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../src/org/apache/cassandra/stress/settings/SettingsTransport.java | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/63116227/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 4ee5814,a415395..2c5573a
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,11 -1,5 +1,12 @@@
 -2.2.10
 +3.0.13
 + * Bugs handling range tombstones in the sstable iterators (CASSANDRA-13340)
 + * Fix CONTAINS filtering for null collections (CASSANDRA-13246)
 + * Applying: Use a unique metric reservoir per test run when using Cassandra-wide metrics residing in MBeans (CASSANDRA-13216)
 + * Propagate row deletions in 2i tables on upgrade (CASSANDRA-13320)
 + * Slice.isEmpty() returns false for some empty slices (CASSANDRA-13305)
 + * Add formatted row output to assertEmpty in CQL Tester (CASSANDRA-13238)
 +Merged from 2.2:
+  * Honor truststore-password parameter in cassandra-stress (CASSANDRA-12773)
   * Discard in-flight shadow round responses (CASSANDRA-12653)
   * Don't anti-compact repaired data to avoid inconsistencies (CASSANDRA-13153)
   * Wrong logger name in AnticompactionTask (CASSANDRA-13343)


[09/11] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

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


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

Branch: refs/heads/trunk
Commit: a10b8079ef713d2ee59fb4af27f65c148d68d900
Parents: 82d3cdc 6311622
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Thu Mar 23 20:51:17 2017 +0100
Committer: Stefan Podkowinski <s....@gmail.com>
Committed: Thu Mar 23 20:52:05 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../src/org/apache/cassandra/stress/settings/SettingsTransport.java | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a10b8079/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 6644796,2c5573a..8b13109
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -41,144 -51,6 +41,145 @@@ Merged from 3.0
     live rows in sstabledump (CASSANDRA-13177)
   * Provide user workaround when system_schema.columns does not contain entries
     for a table that's in system_schema.tables (CASSANDRA-13180)
 +Merged from 2.2:
++ * Honor truststore-password parameter in cassandra-stress (CASSANDRA-12773)
 + * Discard in-flight shadow round responses (CASSANDRA-12653)
 + * Don't anti-compact repaired data to avoid inconsistencies (CASSANDRA-13153)
 + * Wrong logger name in AnticompactionTask (CASSANDRA-13343)
 + * Commitlog replay may fail if last mutation is within 4 bytes of end of segment (CASSANDRA-13282)
 + * Fix queries updating multiple time the same list (CASSANDRA-13130)
 + * Fix GRANT/REVOKE when keyspace isn't specified (CASSANDRA-13053)
 + * Fix flaky LongLeveledCompactionStrategyTest (CASSANDRA-12202)
 + * Fix failing COPY TO STDOUT (CASSANDRA-12497)
 + * Fix ColumnCounter::countAll behaviour for reverse queries (CASSANDRA-13222)
 + * Exceptions encountered calling getSeeds() breaks OTC thread (CASSANDRA-13018)
 + * Fix negative mean latency metric (CASSANDRA-12876)
 + * Use only one file pointer when creating commitlog segments (CASSANDRA-12539)
 +Merged from 2.1:
 + * Remove unused repositories (CASSANDRA-13278)
 + * Log stacktrace of uncaught exceptions (CASSANDRA-13108)
 + * Use portable stderr for java error in startup (CASSANDRA-13211)
 + * Fix Thread Leak in OutboundTcpConnection (CASSANDRA-13204)
 + * Coalescing strategy can enter infinite loop (CASSANDRA-13159)
 +
 +
 +3.10
 + * Fix secondary index queries regression (CASSANDRA-13013)
 + * Add duration type to the protocol V5 (CASSANDRA-12850)
 + * Fix duration type validation (CASSANDRA-13143)
 + * Fix flaky GcCompactionTest (CASSANDRA-12664)
 + * Fix TestHintedHandoff.hintedhandoff_decom_test (CASSANDRA-13058)
 + * Fixed query monitoring for range queries (CASSANDRA-13050)
 + * Remove outboundBindAny configuration property (CASSANDRA-12673)
 + * Use correct bounds for all-data range when filtering (CASSANDRA-12666)
 + * Remove timing window in test case (CASSANDRA-12875)
 + * Resolve unit testing without JCE security libraries installed (CASSANDRA-12945)
 + * Fix inconsistencies in cassandra-stress load balancing policy (CASSANDRA-12919)
 + * Fix validation of non-frozen UDT cells (CASSANDRA-12916)
 + * Don't shut down socket input/output on StreamSession (CASSANDRA-12903)
 + * Fix Murmur3PartitionerTest (CASSANDRA-12858)
 + * Move cqlsh syntax rules into separate module and allow easier customization (CASSANDRA-12897)
 + * Fix CommitLogSegmentManagerTest (CASSANDRA-12283)
 + * Fix cassandra-stress truncate option (CASSANDRA-12695)
 + * Fix crossNode value when receiving messages (CASSANDRA-12791)
 + * Don't load MX4J beans twice (CASSANDRA-12869)
 + * Extend native protocol request flags, add versions to SUPPORTED, and introduce ProtocolVersion enum (CASSANDRA-12838)
 + * Set JOINING mode when running pre-join tasks (CASSANDRA-12836)
 + * remove net.mintern.primitive library due to license issue (CASSANDRA-12845)
 + * Properly format IPv6 addresses when logging JMX service URL (CASSANDRA-12454)
 + * Optimize the vnode allocation for single replica per DC (CASSANDRA-12777)
 + * Use non-token restrictions for bounds when token restrictions are overridden (CASSANDRA-12419)
 + * Fix CQLSH auto completion for PER PARTITION LIMIT (CASSANDRA-12803)
 + * Use different build directories for Eclipse and Ant (CASSANDRA-12466)
 + * Avoid potential AttributeError in cqlsh due to no table metadata (CASSANDRA-12815)
 + * Fix RandomReplicationAwareTokenAllocatorTest.testExistingCluster (CASSANDRA-12812)
 + * Upgrade commons-codec to 1.9 (CASSANDRA-12790)
 + * Make the fanout size for LeveledCompactionStrategy to be configurable (CASSANDRA-11550)
 + * Add duration data type (CASSANDRA-11873)
 + * Fix timeout in ReplicationAwareTokenAllocatorTest (CASSANDRA-12784)
 + * Improve sum aggregate functions (CASSANDRA-12417)
 + * Make cassandra.yaml docs for batch_size_*_threshold_in_kb reflect changes in CASSANDRA-10876 (CASSANDRA-12761)
 + * cqlsh fails to format collections when using aliases (CASSANDRA-11534)
 + * Check for hash conflicts in prepared statements (CASSANDRA-12733)
 + * Exit query parsing upon first error (CASSANDRA-12598)
 + * Fix cassandra-stress to use single seed in UUID generation (CASSANDRA-12729)
 + * CQLSSTableWriter does not allow Update statement (CASSANDRA-12450)
 + * Config class uses boxed types but DD exposes primitive types (CASSANDRA-12199)
 + * Add pre- and post-shutdown hooks to Storage Service (CASSANDRA-12461)
 + * Add hint delivery metrics (CASSANDRA-12693)
 + * Remove IndexInfo cache from FileIndexInfoRetriever (CASSANDRA-12731)
 + * ColumnIndex does not reuse buffer (CASSANDRA-12502)
 + * cdc column addition still breaks schema migration tasks (CASSANDRA-12697)
 + * Upgrade metrics-reporter dependencies (CASSANDRA-12089)
 + * Tune compaction thread count via nodetool (CASSANDRA-12248)
 + * Add +=/-= shortcut syntax for update queries (CASSANDRA-12232)
 + * Include repair session IDs in repair start message (CASSANDRA-12532)
 + * Add a blocking task to Index, run before joining the ring (CASSANDRA-12039)
 + * Fix NPE when using CQLSSTableWriter (CASSANDRA-12667)
 + * Support optional backpressure strategies at the coordinator (CASSANDRA-9318)
 + * Make randompartitioner work with new vnode allocation (CASSANDRA-12647)
 + * Fix cassandra-stress graphing (CASSANDRA-12237)
 + * Allow filtering on partition key columns for queries without secondary indexes (CASSANDRA-11031)
 + * Fix Cassandra Stress reporting thread model and precision (CASSANDRA-12585)
 + * Add JMH benchmarks.jar (CASSANDRA-12586)
 + * Cleanup uses of AlterTableStatementColumn (CASSANDRA-12567)
 + * Add keep-alive to streaming (CASSANDRA-11841)
 + * Tracing payload is passed through newSession(..) (CASSANDRA-11706)
 + * avoid deleting non existing sstable files and improve related log messages (CASSANDRA-12261)
 + * json/yaml output format for nodetool compactionhistory (CASSANDRA-12486)
 + * Retry all internode messages once after a connection is
 +   closed and reopened (CASSANDRA-12192)
 + * Add support to rebuild from targeted replica (CASSANDRA-9875)
 + * Add sequence distribution type to cassandra stress (CASSANDRA-12490)
 + * "SELECT * FROM foo LIMIT ;" does not error out (CASSANDRA-12154)
 + * Define executeLocally() at the ReadQuery Level (CASSANDRA-12474)
 + * Extend read/write failure messages with a map of replica addresses
 +   to error codes in the v5 native protocol (CASSANDRA-12311)
 + * Fix rebuild of SASI indexes with existing index files (CASSANDRA-12374)
 + * Let DatabaseDescriptor not implicitly startup services (CASSANDRA-9054, 12550)
 + * Fix clustering indexes in presence of static columns in SASI (CASSANDRA-12378)
 + * Fix queries on columns with reversed type on SASI indexes (CASSANDRA-12223)
 + * Added slow query log (CASSANDRA-12403)
 + * Count full coordinated request against timeout (CASSANDRA-12256)
 + * Allow TTL with null value on insert and update (CASSANDRA-12216)
 + * Make decommission operation resumable (CASSANDRA-12008)
 + * Add support to one-way targeted repair (CASSANDRA-9876)
 + * Remove clientutil jar (CASSANDRA-11635)
 + * Fix compaction throughput throttle (CASSANDRA-12366, CASSANDRA-12717)
 + * Delay releasing Memtable memory on flush until PostFlush has finished running (CASSANDRA-12358)
 + * Cassandra stress should dump all setting on startup (CASSANDRA-11914)
 + * Make it possible to compact a given token range (CASSANDRA-10643)
 + * Allow updating DynamicEndpointSnitch properties via JMX (CASSANDRA-12179)
 + * Collect metrics on queries by consistency level (CASSANDRA-7384)
 + * Add support for GROUP BY to SELECT statement (CASSANDRA-10707)
 + * Deprecate memtable_cleanup_threshold and update default for memtable_flush_writers (CASSANDRA-12228)
 + * Upgrade to OHC 0.4.4 (CASSANDRA-12133)
 + * Add version command to cassandra-stress (CASSANDRA-12258)
 + * Create compaction-stress tool (CASSANDRA-11844)
 + * Garbage-collecting compaction operation and schema option (CASSANDRA-7019)
 + * Add beta protocol flag for v5 native protocol (CASSANDRA-12142)
 + * Support filtering on non-PRIMARY KEY columns in the CREATE
 +   MATERIALIZED VIEW statement's WHERE clause (CASSANDRA-10368)
 + * Unify STDOUT and SYSTEMLOG logback format (CASSANDRA-12004)
 + * COPY FROM should raise error for non-existing input files (CASSANDRA-12174)
 + * Faster write path (CASSANDRA-12269)
 + * Option to leave omitted columns in INSERT JSON unset (CASSANDRA-11424)
 + * Support json/yaml output in nodetool tpstats (CASSANDRA-12035)
 + * Expose metrics for successful/failed authentication attempts (CASSANDRA-10635)
 + * Prepend snapshot name with "truncated" or "dropped" when a snapshot
 +   is taken before truncating or dropping a table (CASSANDRA-12178)
 + * Optimize RestrictionSet (CASSANDRA-12153)
 + * cqlsh does not automatically downgrade CQL version (CASSANDRA-12150)
 + * Omit (de)serialization of state variable in UDAs (CASSANDRA-9613)
 + * Create a system table to expose prepared statements (CASSANDRA-8831)
 + * Reuse DataOutputBuffer from ColumnIndex (CASSANDRA-11970)
 + * Remove DatabaseDescriptor dependency from SegmentedFile (CASSANDRA-11580)
 + * Add supplied username to authentication error messages (CASSANDRA-12076)
 + * Remove pre-startup check for open JMX port (CASSANDRA-12074)
 + * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738)
 + * Restore resumable hints delivery (CASSANDRA-11960)
 + * Properly report LWT contention (CASSANDRA-12626)
 +Merged from 3.0:
   * Dump threads when unit tests time out (CASSANDRA-13117)
   * Better error when modifying function permissions without explicit keyspace (CASSANDRA-12925)
   * Indexer is not correctly invoked when building indexes over sstables (CASSANDRA-13075)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a10b8079/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
----------------------------------------------------------------------


[04/11] cassandra git commit: Honor truststore-password parameter in stress

Posted by sp...@apache.org.
Honor truststore-password parameter in stress

patch by Jane Deng; reviewed by Robert Stupp for CASSANDRA-12773


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

Branch: refs/heads/cassandra-3.11
Commit: 5978f9d5f719455ceb79d5f077cdd1b72b4e1876
Parents: bf0906b
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Thu Mar 23 20:48:03 2017 +0100
Committer: Stefan Podkowinski <s....@gmail.com>
Committed: Thu Mar 23 20:48:03 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../src/org/apache/cassandra/stress/settings/SettingsTransport.java | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5978f9d5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index df2421d..a415395 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.10
+ * Honor truststore-password parameter in cassandra-stress (CASSANDRA-12773)
  * Discard in-flight shadow round responses (CASSANDRA-12653)
  * Don't anti-compact repaired data to avoid inconsistencies (CASSANDRA-13153)
  * Wrong logger name in AnticompactionTask (CASSANDRA-13343)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5978f9d5/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
index b6d1d90..a253c07 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
@@ -115,6 +115,7 @@ public class SettingsTransport implements Serializable
             {
                 // mandatory for SSLFactory.createSSLContext(), see CASSANDRA-9325
                 encOptions.keystore = encOptions.truststore;
+                encOptions.keystore_password = encOptions.truststore_password;
             }
             encOptions.algorithm = options.alg.value();
             encOptions.protocol = options.protocol.value();


[05/11] cassandra git commit: Honor truststore-password parameter in stress

Posted by sp...@apache.org.
Honor truststore-password parameter in stress

patch by Jane Deng; reviewed by Robert Stupp for CASSANDRA-12773


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

Branch: refs/heads/trunk
Commit: 5978f9d5f719455ceb79d5f077cdd1b72b4e1876
Parents: bf0906b
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Thu Mar 23 20:48:03 2017 +0100
Committer: Stefan Podkowinski <s....@gmail.com>
Committed: Thu Mar 23 20:48:03 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../src/org/apache/cassandra/stress/settings/SettingsTransport.java | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5978f9d5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index df2421d..a415395 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.10
+ * Honor truststore-password parameter in cassandra-stress (CASSANDRA-12773)
  * Discard in-flight shadow round responses (CASSANDRA-12653)
  * Don't anti-compact repaired data to avoid inconsistencies (CASSANDRA-13153)
  * Wrong logger name in AnticompactionTask (CASSANDRA-13343)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5978f9d5/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
index b6d1d90..a253c07 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
@@ -115,6 +115,7 @@ public class SettingsTransport implements Serializable
             {
                 // mandatory for SSLFactory.createSSLContext(), see CASSANDRA-9325
                 encOptions.keystore = encOptions.truststore;
+                encOptions.keystore_password = encOptions.truststore_password;
             }
             encOptions.algorithm = options.alg.value();
             encOptions.protocol = options.protocol.value();


[03/11] cassandra git commit: Honor truststore-password parameter in stress

Posted by sp...@apache.org.
Honor truststore-password parameter in stress

patch by Jane Deng; reviewed by Robert Stupp for CASSANDRA-12773


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

Branch: refs/heads/cassandra-3.0
Commit: 5978f9d5f719455ceb79d5f077cdd1b72b4e1876
Parents: bf0906b
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Thu Mar 23 20:48:03 2017 +0100
Committer: Stefan Podkowinski <s....@gmail.com>
Committed: Thu Mar 23 20:48:03 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../src/org/apache/cassandra/stress/settings/SettingsTransport.java | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5978f9d5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index df2421d..a415395 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.10
+ * Honor truststore-password parameter in cassandra-stress (CASSANDRA-12773)
  * Discard in-flight shadow round responses (CASSANDRA-12653)
  * Don't anti-compact repaired data to avoid inconsistencies (CASSANDRA-13153)
  * Wrong logger name in AnticompactionTask (CASSANDRA-13343)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5978f9d5/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
index b6d1d90..a253c07 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
@@ -115,6 +115,7 @@ public class SettingsTransport implements Serializable
             {
                 // mandatory for SSLFactory.createSSLContext(), see CASSANDRA-9325
                 encOptions.keystore = encOptions.truststore;
+                encOptions.keystore_password = encOptions.truststore_password;
             }
             encOptions.algorithm = options.alg.value();
             encOptions.protocol = options.protocol.value();


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

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

Branch: refs/heads/trunk
Commit: 631162271c9bbaca6b48dc4e2223dbba97bf51d4
Parents: f53e502 5978f9d
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Thu Mar 23 20:49:05 2017 +0100
Committer: Stefan Podkowinski <s....@gmail.com>
Committed: Thu Mar 23 20:50:13 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../src/org/apache/cassandra/stress/settings/SettingsTransport.java | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/63116227/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 4ee5814,a415395..2c5573a
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,11 -1,5 +1,12 @@@
 -2.2.10
 +3.0.13
 + * Bugs handling range tombstones in the sstable iterators (CASSANDRA-13340)
 + * Fix CONTAINS filtering for null collections (CASSANDRA-13246)
 + * Applying: Use a unique metric reservoir per test run when using Cassandra-wide metrics residing in MBeans (CASSANDRA-13216)
 + * Propagate row deletions in 2i tables on upgrade (CASSANDRA-13320)
 + * Slice.isEmpty() returns false for some empty slices (CASSANDRA-13305)
 + * Add formatted row output to assertEmpty in CQL Tester (CASSANDRA-13238)
 +Merged from 2.2:
+  * Honor truststore-password parameter in cassandra-stress (CASSANDRA-12773)
   * Discard in-flight shadow round responses (CASSANDRA-12653)
   * Don't anti-compact repaired data to avoid inconsistencies (CASSANDRA-13153)
   * Wrong logger name in AnticompactionTask (CASSANDRA-13343)


[02/11] cassandra git commit: Honor truststore-password parameter in stress

Posted by sp...@apache.org.
Honor truststore-password parameter in stress

patch by Jane Deng; reviewed by Robert Stupp for CASSANDRA-12773


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

Branch: refs/heads/cassandra-2.2
Commit: 5978f9d5f719455ceb79d5f077cdd1b72b4e1876
Parents: bf0906b
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Thu Mar 23 20:48:03 2017 +0100
Committer: Stefan Podkowinski <s....@gmail.com>
Committed: Thu Mar 23 20:48:03 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../src/org/apache/cassandra/stress/settings/SettingsTransport.java | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5978f9d5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index df2421d..a415395 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.10
+ * Honor truststore-password parameter in cassandra-stress (CASSANDRA-12773)
  * Discard in-flight shadow round responses (CASSANDRA-12653)
  * Don't anti-compact repaired data to avoid inconsistencies (CASSANDRA-13153)
  * Wrong logger name in AnticompactionTask (CASSANDRA-13343)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5978f9d5/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
index b6d1d90..a253c07 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
@@ -115,6 +115,7 @@ public class SettingsTransport implements Serializable
             {
                 // mandatory for SSLFactory.createSSLContext(), see CASSANDRA-9325
                 encOptions.keystore = encOptions.truststore;
+                encOptions.keystore_password = encOptions.truststore_password;
             }
             encOptions.algorithm = options.alg.value();
             encOptions.protocol = options.protocol.value();


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

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


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

Branch: refs/heads/trunk
Commit: 3048608c6099fd5c3bcc9bd72d3265307283bc41
Parents: 18c6ed2 a10b807
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Thu Mar 23 20:56:22 2017 +0100
Committer: Stefan Podkowinski <s....@gmail.com>
Committed: Thu Mar 23 20:57:10 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../src/org/apache/cassandra/stress/settings/SettingsTransport.java | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3048608c/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
----------------------------------------------------------------------


[10/11] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

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


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

Branch: refs/heads/cassandra-3.11
Commit: a10b8079ef713d2ee59fb4af27f65c148d68d900
Parents: 82d3cdc 6311622
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Thu Mar 23 20:51:17 2017 +0100
Committer: Stefan Podkowinski <s....@gmail.com>
Committed: Thu Mar 23 20:52:05 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../src/org/apache/cassandra/stress/settings/SettingsTransport.java | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a10b8079/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 6644796,2c5573a..8b13109
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -41,144 -51,6 +41,145 @@@ Merged from 3.0
     live rows in sstabledump (CASSANDRA-13177)
   * Provide user workaround when system_schema.columns does not contain entries
     for a table that's in system_schema.tables (CASSANDRA-13180)
 +Merged from 2.2:
++ * Honor truststore-password parameter in cassandra-stress (CASSANDRA-12773)
 + * Discard in-flight shadow round responses (CASSANDRA-12653)
 + * Don't anti-compact repaired data to avoid inconsistencies (CASSANDRA-13153)
 + * Wrong logger name in AnticompactionTask (CASSANDRA-13343)
 + * Commitlog replay may fail if last mutation is within 4 bytes of end of segment (CASSANDRA-13282)
 + * Fix queries updating multiple time the same list (CASSANDRA-13130)
 + * Fix GRANT/REVOKE when keyspace isn't specified (CASSANDRA-13053)
 + * Fix flaky LongLeveledCompactionStrategyTest (CASSANDRA-12202)
 + * Fix failing COPY TO STDOUT (CASSANDRA-12497)
 + * Fix ColumnCounter::countAll behaviour for reverse queries (CASSANDRA-13222)
 + * Exceptions encountered calling getSeeds() breaks OTC thread (CASSANDRA-13018)
 + * Fix negative mean latency metric (CASSANDRA-12876)
 + * Use only one file pointer when creating commitlog segments (CASSANDRA-12539)
 +Merged from 2.1:
 + * Remove unused repositories (CASSANDRA-13278)
 + * Log stacktrace of uncaught exceptions (CASSANDRA-13108)
 + * Use portable stderr for java error in startup (CASSANDRA-13211)
 + * Fix Thread Leak in OutboundTcpConnection (CASSANDRA-13204)
 + * Coalescing strategy can enter infinite loop (CASSANDRA-13159)
 +
 +
 +3.10
 + * Fix secondary index queries regression (CASSANDRA-13013)
 + * Add duration type to the protocol V5 (CASSANDRA-12850)
 + * Fix duration type validation (CASSANDRA-13143)
 + * Fix flaky GcCompactionTest (CASSANDRA-12664)
 + * Fix TestHintedHandoff.hintedhandoff_decom_test (CASSANDRA-13058)
 + * Fixed query monitoring for range queries (CASSANDRA-13050)
 + * Remove outboundBindAny configuration property (CASSANDRA-12673)
 + * Use correct bounds for all-data range when filtering (CASSANDRA-12666)
 + * Remove timing window in test case (CASSANDRA-12875)
 + * Resolve unit testing without JCE security libraries installed (CASSANDRA-12945)
 + * Fix inconsistencies in cassandra-stress load balancing policy (CASSANDRA-12919)
 + * Fix validation of non-frozen UDT cells (CASSANDRA-12916)
 + * Don't shut down socket input/output on StreamSession (CASSANDRA-12903)
 + * Fix Murmur3PartitionerTest (CASSANDRA-12858)
 + * Move cqlsh syntax rules into separate module and allow easier customization (CASSANDRA-12897)
 + * Fix CommitLogSegmentManagerTest (CASSANDRA-12283)
 + * Fix cassandra-stress truncate option (CASSANDRA-12695)
 + * Fix crossNode value when receiving messages (CASSANDRA-12791)
 + * Don't load MX4J beans twice (CASSANDRA-12869)
 + * Extend native protocol request flags, add versions to SUPPORTED, and introduce ProtocolVersion enum (CASSANDRA-12838)
 + * Set JOINING mode when running pre-join tasks (CASSANDRA-12836)
 + * remove net.mintern.primitive library due to license issue (CASSANDRA-12845)
 + * Properly format IPv6 addresses when logging JMX service URL (CASSANDRA-12454)
 + * Optimize the vnode allocation for single replica per DC (CASSANDRA-12777)
 + * Use non-token restrictions for bounds when token restrictions are overridden (CASSANDRA-12419)
 + * Fix CQLSH auto completion for PER PARTITION LIMIT (CASSANDRA-12803)
 + * Use different build directories for Eclipse and Ant (CASSANDRA-12466)
 + * Avoid potential AttributeError in cqlsh due to no table metadata (CASSANDRA-12815)
 + * Fix RandomReplicationAwareTokenAllocatorTest.testExistingCluster (CASSANDRA-12812)
 + * Upgrade commons-codec to 1.9 (CASSANDRA-12790)
 + * Make the fanout size for LeveledCompactionStrategy to be configurable (CASSANDRA-11550)
 + * Add duration data type (CASSANDRA-11873)
 + * Fix timeout in ReplicationAwareTokenAllocatorTest (CASSANDRA-12784)
 + * Improve sum aggregate functions (CASSANDRA-12417)
 + * Make cassandra.yaml docs for batch_size_*_threshold_in_kb reflect changes in CASSANDRA-10876 (CASSANDRA-12761)
 + * cqlsh fails to format collections when using aliases (CASSANDRA-11534)
 + * Check for hash conflicts in prepared statements (CASSANDRA-12733)
 + * Exit query parsing upon first error (CASSANDRA-12598)
 + * Fix cassandra-stress to use single seed in UUID generation (CASSANDRA-12729)
 + * CQLSSTableWriter does not allow Update statement (CASSANDRA-12450)
 + * Config class uses boxed types but DD exposes primitive types (CASSANDRA-12199)
 + * Add pre- and post-shutdown hooks to Storage Service (CASSANDRA-12461)
 + * Add hint delivery metrics (CASSANDRA-12693)
 + * Remove IndexInfo cache from FileIndexInfoRetriever (CASSANDRA-12731)
 + * ColumnIndex does not reuse buffer (CASSANDRA-12502)
 + * cdc column addition still breaks schema migration tasks (CASSANDRA-12697)
 + * Upgrade metrics-reporter dependencies (CASSANDRA-12089)
 + * Tune compaction thread count via nodetool (CASSANDRA-12248)
 + * Add +=/-= shortcut syntax for update queries (CASSANDRA-12232)
 + * Include repair session IDs in repair start message (CASSANDRA-12532)
 + * Add a blocking task to Index, run before joining the ring (CASSANDRA-12039)
 + * Fix NPE when using CQLSSTableWriter (CASSANDRA-12667)
 + * Support optional backpressure strategies at the coordinator (CASSANDRA-9318)
 + * Make randompartitioner work with new vnode allocation (CASSANDRA-12647)
 + * Fix cassandra-stress graphing (CASSANDRA-12237)
 + * Allow filtering on partition key columns for queries without secondary indexes (CASSANDRA-11031)
 + * Fix Cassandra Stress reporting thread model and precision (CASSANDRA-12585)
 + * Add JMH benchmarks.jar (CASSANDRA-12586)
 + * Cleanup uses of AlterTableStatementColumn (CASSANDRA-12567)
 + * Add keep-alive to streaming (CASSANDRA-11841)
 + * Tracing payload is passed through newSession(..) (CASSANDRA-11706)
 + * avoid deleting non existing sstable files and improve related log messages (CASSANDRA-12261)
 + * json/yaml output format for nodetool compactionhistory (CASSANDRA-12486)
 + * Retry all internode messages once after a connection is
 +   closed and reopened (CASSANDRA-12192)
 + * Add support to rebuild from targeted replica (CASSANDRA-9875)
 + * Add sequence distribution type to cassandra stress (CASSANDRA-12490)
 + * "SELECT * FROM foo LIMIT ;" does not error out (CASSANDRA-12154)
 + * Define executeLocally() at the ReadQuery Level (CASSANDRA-12474)
 + * Extend read/write failure messages with a map of replica addresses
 +   to error codes in the v5 native protocol (CASSANDRA-12311)
 + * Fix rebuild of SASI indexes with existing index files (CASSANDRA-12374)
 + * Let DatabaseDescriptor not implicitly startup services (CASSANDRA-9054, 12550)
 + * Fix clustering indexes in presence of static columns in SASI (CASSANDRA-12378)
 + * Fix queries on columns with reversed type on SASI indexes (CASSANDRA-12223)
 + * Added slow query log (CASSANDRA-12403)
 + * Count full coordinated request against timeout (CASSANDRA-12256)
 + * Allow TTL with null value on insert and update (CASSANDRA-12216)
 + * Make decommission operation resumable (CASSANDRA-12008)
 + * Add support to one-way targeted repair (CASSANDRA-9876)
 + * Remove clientutil jar (CASSANDRA-11635)
 + * Fix compaction throughput throttle (CASSANDRA-12366, CASSANDRA-12717)
 + * Delay releasing Memtable memory on flush until PostFlush has finished running (CASSANDRA-12358)
 + * Cassandra stress should dump all setting on startup (CASSANDRA-11914)
 + * Make it possible to compact a given token range (CASSANDRA-10643)
 + * Allow updating DynamicEndpointSnitch properties via JMX (CASSANDRA-12179)
 + * Collect metrics on queries by consistency level (CASSANDRA-7384)
 + * Add support for GROUP BY to SELECT statement (CASSANDRA-10707)
 + * Deprecate memtable_cleanup_threshold and update default for memtable_flush_writers (CASSANDRA-12228)
 + * Upgrade to OHC 0.4.4 (CASSANDRA-12133)
 + * Add version command to cassandra-stress (CASSANDRA-12258)
 + * Create compaction-stress tool (CASSANDRA-11844)
 + * Garbage-collecting compaction operation and schema option (CASSANDRA-7019)
 + * Add beta protocol flag for v5 native protocol (CASSANDRA-12142)
 + * Support filtering on non-PRIMARY KEY columns in the CREATE
 +   MATERIALIZED VIEW statement's WHERE clause (CASSANDRA-10368)
 + * Unify STDOUT and SYSTEMLOG logback format (CASSANDRA-12004)
 + * COPY FROM should raise error for non-existing input files (CASSANDRA-12174)
 + * Faster write path (CASSANDRA-12269)
 + * Option to leave omitted columns in INSERT JSON unset (CASSANDRA-11424)
 + * Support json/yaml output in nodetool tpstats (CASSANDRA-12035)
 + * Expose metrics for successful/failed authentication attempts (CASSANDRA-10635)
 + * Prepend snapshot name with "truncated" or "dropped" when a snapshot
 +   is taken before truncating or dropping a table (CASSANDRA-12178)
 + * Optimize RestrictionSet (CASSANDRA-12153)
 + * cqlsh does not automatically downgrade CQL version (CASSANDRA-12150)
 + * Omit (de)serialization of state variable in UDAs (CASSANDRA-9613)
 + * Create a system table to expose prepared statements (CASSANDRA-8831)
 + * Reuse DataOutputBuffer from ColumnIndex (CASSANDRA-11970)
 + * Remove DatabaseDescriptor dependency from SegmentedFile (CASSANDRA-11580)
 + * Add supplied username to authentication error messages (CASSANDRA-12076)
 + * Remove pre-startup check for open JMX port (CASSANDRA-12074)
 + * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738)
 + * Restore resumable hints delivery (CASSANDRA-11960)
 + * Properly report LWT contention (CASSANDRA-12626)
 +Merged from 3.0:
   * Dump threads when unit tests time out (CASSANDRA-13117)
   * Better error when modifying function permissions without explicit keyspace (CASSANDRA-12925)
   * Indexer is not correctly invoked when building indexes over sstables (CASSANDRA-13075)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a10b8079/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
----------------------------------------------------------------------


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

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

Branch: refs/heads/cassandra-3.0
Commit: 631162271c9bbaca6b48dc4e2223dbba97bf51d4
Parents: f53e502 5978f9d
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Thu Mar 23 20:49:05 2017 +0100
Committer: Stefan Podkowinski <s....@gmail.com>
Committed: Thu Mar 23 20:50:13 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../src/org/apache/cassandra/stress/settings/SettingsTransport.java | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/63116227/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 4ee5814,a415395..2c5573a
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,11 -1,5 +1,12 @@@
 -2.2.10
 +3.0.13
 + * Bugs handling range tombstones in the sstable iterators (CASSANDRA-13340)
 + * Fix CONTAINS filtering for null collections (CASSANDRA-13246)
 + * Applying: Use a unique metric reservoir per test run when using Cassandra-wide metrics residing in MBeans (CASSANDRA-13216)
 + * Propagate row deletions in 2i tables on upgrade (CASSANDRA-13320)
 + * Slice.isEmpty() returns false for some empty slices (CASSANDRA-13305)
 + * Add formatted row output to assertEmpty in CQL Tester (CASSANDRA-13238)
 +Merged from 2.2:
+  * Honor truststore-password parameter in cassandra-stress (CASSANDRA-12773)
   * Discard in-flight shadow round responses (CASSANDRA-12653)
   * Don't anti-compact repaired data to avoid inconsistencies (CASSANDRA-13153)
   * Wrong logger name in AnticompactionTask (CASSANDRA-13343)