You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jm...@apache.org on 2015/03/24 19:25:53 UTC

[1/5] cassandra git commit: Fix potential data loss in CompressedSequentialWriter

Repository: cassandra
Updated Branches:
  refs/heads/trunk dadc206c6 -> 507a36305


Fix potential data loss in CompressedSequentialWriter

Patch by jmckenzie; reviewed by tjake for CASSANDRA-8949


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

Branch: refs/heads/trunk
Commit: f5f97d81539a931653e8cbb8ba6d6be8421f6c29
Parents: 061ea33
Author: Joshua McKenzie <jm...@apache.org>
Authored: Tue Mar 24 13:07:58 2015 -0500
Committer: Joshua McKenzie <jm...@apache.org>
Committed: Tue Mar 24 13:07:58 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../io/compress/CompressedSequentialWriter.java |   3 +
 .../cassandra/io/compress/LZ4Compressor.java    |   6 +-
 .../CompressedSequentialWriterTest.java         | 126 +++++++++++++++++++
 4 files changed, 135 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f5f97d81/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2073b00..f0555dd 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.14:
+ * Fix potential data loss in CompressedSequentialWriter (CASSANDRA-8949)
  * (cqlsh) Allow increasing CSV field size limit through
    cqlshrc config option (CASSANDRA-8934)
  * Stop logging range tombstones when exceeding the threshold

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f5f97d81/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 909d822..97200e0 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -229,6 +229,9 @@ public class CompressedSequentialWriter extends SequentialWriter
         bufferOffset = current - validBufferBytes;
         chunkCount = realMark.nextChunkIndex - 1;
 
+        // mark as dirty so we don't lose the bytes on subsequent reBuffer calls
+        isDirty = true;
+
         // truncate data and index file
         truncate(chunkOffset);
         metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f5f97d81/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java b/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java
index 0cf36c1..6f56f30 100644
--- a/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java
+++ b/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java
@@ -23,6 +23,8 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import net.jpountz.lz4.LZ4Exception;
 import net.jpountz.lz4.LZ4Factory;
 
@@ -30,7 +32,9 @@ public class LZ4Compressor implements ICompressor
 {
 
     private static final int INTEGER_BYTES = 4;
-    private static final LZ4Compressor instance = new LZ4Compressor();
+
+    @VisibleForTesting
+    public static final LZ4Compressor instance = new LZ4Compressor();
 
     public static LZ4Compressor create(Map<String, String> args)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f5f97d81/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
new file mode 100644
index 0000000..acc4faa
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.compress;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.SSTableMetadata;
+import org.junit.Test;
+
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.io.util.FileMark;
+
+public class CompressedSequentialWriterTest
+{
+    private ICompressor compressor;
+
+    private void runTests(String testName) throws IOException, ConfigurationException
+    {
+        // Test small < 1 chunk data set
+        testWrite(File.createTempFile(testName + "_small", "1"), 25);
+
+        // Test to confirm pipeline w/chunk-aligned data writes works
+        testWrite(File.createTempFile(testName + "_chunkAligned", "1"), CompressionParameters.DEFAULT_CHUNK_LENGTH);
+
+        // Test to confirm pipeline on non-chunk boundaries works
+        testWrite(File.createTempFile(testName + "_large", "1"), CompressionParameters.DEFAULT_CHUNK_LENGTH * 3 + 100);
+    }
+
+    @Test
+    public void testLZ4Writer() throws IOException, ConfigurationException
+    {
+        compressor = LZ4Compressor.instance;
+        runTests("LZ4");
+    }
+
+    @Test
+    public void testDeflateWriter() throws IOException, ConfigurationException
+    {
+        compressor = DeflateCompressor.instance;
+        runTests("Deflate");
+    }
+
+    @Test
+    public void testSnappyWriter() throws IOException, ConfigurationException
+    {
+        compressor = SnappyCompressor.instance;
+        runTests("Snappy");
+    }
+
+    private void testWrite(File f, int bytesToTest) throws IOException, ConfigurationException
+    {
+        try
+        {
+            final String filename = f.getAbsolutePath();
+
+            SSTableMetadata.Collector sstableMetadataCollector = SSTableMetadata.createCollector(BytesType.instance).replayPosition(null);
+            CompressedSequentialWriter writer = new CompressedSequentialWriter( f, filename + ".metadata", false, new CompressionParameters( compressor, 32, Collections.<String, String>emptyMap() ), sstableMetadataCollector);
+
+            byte[] dataPre = new byte[bytesToTest];
+            byte[] rawPost = new byte[bytesToTest];
+            Random r = new Random();
+
+            // Test both write with byte[] and ByteBuffer
+            r.nextBytes(dataPre);
+            r.nextBytes(rawPost);
+
+            writer.write(dataPre);
+            FileMark mark = writer.mark();
+
+            // Write enough garbage to transition chunk
+            for (int i = 0; i < CompressionParameters.DEFAULT_CHUNK_LENGTH; i++)
+            {
+                writer.write((byte)i);
+            }
+            writer.resetAndTruncate(mark);
+            writer.write(rawPost);
+            writer.close();
+
+            assert f.exists();
+            CompressedRandomAccessReader reader = CompressedRandomAccessReader.open(filename, new CompressionMetadata(filename + ".metadata", f.length(), true));
+            assertEquals(dataPre.length + rawPost.length, reader.length());
+            byte[] result = new byte[(int)reader.length()];
+
+            reader.readFully(result);
+
+            assert(reader.isEOF());
+            reader.close();
+
+            byte[] fullInput = new byte[bytesToTest * 2];
+            System.arraycopy(dataPre, 0, fullInput, 0, dataPre.length);
+            System.arraycopy(rawPost, 0, fullInput, bytesToTest, rawPost.length);
+            assert Arrays.equals(result, fullInput);
+        }
+        finally
+        {
+            // cleanup
+            if (f.exists())
+                f.delete();
+            File metadata = new File(f + ".metadata");
+            if (metadata.exists())
+                metadata.delete();
+        }
+    }
+}


[2/5] cassandra git commit: Fix potential data loss in CompressedSequentialWriter (2.1 patch)

Posted by jm...@apache.org.
Fix potential data loss in CompressedSequentialWriter (2.1 patch)

Patch by jmckenzie; reviewed by tjake for CASSANDRA-8949


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

Branch: refs/heads/trunk
Commit: 9ff82706f206d135b1795bf84cbcab23915617f4
Parents: 9d2b273
Author: Joshua McKenzie <jm...@apache.org>
Authored: Tue Mar 24 13:19:45 2015 -0500
Committer: Joshua McKenzie <jm...@apache.org>
Committed: Tue Mar 24 13:19:45 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../io/compress/CompressedSequentialWriter.java |   3 +
 .../cassandra/io/compress/LZ4Compressor.java    |   6 +-
 .../CompressedSequentialWriterTest.java         | 125 +++++++++++++++++++
 4 files changed, 134 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9ff82706/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 06a3c3f..2000bd0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.4
+ * Fix potential data loss in CompressedSequentialWriter (CASSANDRA-8949)
  * Make PasswordAuthenticator number of hashing rounds configurable (CASSANDRA-8085)
  * Fix AssertionError when binding nested collections in DELETE (CASSANDRA-8900)
  * Check for overlap with non-early sstables in LCS (CASSANDRA-8739)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9ff82706/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 6152c5f..c004232 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -229,6 +229,9 @@ public class CompressedSequentialWriter extends SequentialWriter
         bufferOffset = current - validBufferBytes;
         chunkCount = realMark.nextChunkIndex - 1;
 
+        // mark as dirty so we don't lose the bytes on subsequent reBuffer calls
+        isDirty = true;
+
         // truncate data and index file
         truncate(chunkOffset);
         metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9ff82706/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java b/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java
index 0cf36c1..6f56f30 100644
--- a/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java
+++ b/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java
@@ -23,6 +23,8 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import net.jpountz.lz4.LZ4Exception;
 import net.jpountz.lz4.LZ4Factory;
 
@@ -30,7 +32,9 @@ public class LZ4Compressor implements ICompressor
 {
 
     private static final int INTEGER_BYTES = 4;
-    private static final LZ4Compressor instance = new LZ4Compressor();
+
+    @VisibleForTesting
+    public static final LZ4Compressor instance = new LZ4Compressor();
 
     public static LZ4Compressor create(Map<String, String> args)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9ff82706/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
new file mode 100644
index 0000000..fe3ba46
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
@@ -0,0 +1,125 @@
+/*
+ * 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.compress;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
+
+import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.util.FileMark;
+
+public class CompressedSequentialWriterTest
+{
+    private ICompressor compressor;
+
+    private void runTests(String testName) throws IOException, ConfigurationException
+    {
+        // Test small < 1 chunk data set
+        testWrite(File.createTempFile(testName + "_small", "1"), 25);
+
+        // Test to confirm pipeline w/chunk-aligned data writes works
+        testWrite(File.createTempFile(testName + "_chunkAligned", "1"), CompressionParameters.DEFAULT_CHUNK_LENGTH);
+
+        // Test to confirm pipeline on non-chunk boundaries works
+        testWrite(File.createTempFile(testName + "_large", "1"), CompressionParameters.DEFAULT_CHUNK_LENGTH * 3 + 100);
+    }
+
+    @Test
+    public void testLZ4Writer() throws IOException, ConfigurationException
+    {
+        compressor = LZ4Compressor.instance;
+        runTests("LZ4");
+    }
+
+    @Test
+    public void testDeflateWriter() throws IOException, ConfigurationException
+    {
+        compressor = DeflateCompressor.instance;
+        runTests("Deflate");
+    }
+
+    @Test
+    public void testSnappyWriter() throws IOException, ConfigurationException
+    {
+        compressor = SnappyCompressor.instance;
+        runTests("Snappy");
+    }
+
+    private void testWrite(File f, int bytesToTest) throws IOException, ConfigurationException
+    {
+        try
+        {
+            final String filename = f.getAbsolutePath();
+
+            MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance)).replayPosition(null);
+            CompressedSequentialWriter writer = new CompressedSequentialWriter(f, filename + ".metadata", new CompressionParameters(compressor), sstableMetadataCollector);
+
+            byte[] dataPre = new byte[bytesToTest];
+            byte[] rawPost = new byte[bytesToTest];
+            Random r = new Random();
+
+            // Test both write with byte[] and ByteBuffer
+            r.nextBytes(dataPre);
+            r.nextBytes(rawPost);
+
+            writer.write(dataPre);
+            FileMark mark = writer.mark();
+
+            // Write enough garbage to transition chunk
+            for (int i = 0; i < CompressionParameters.DEFAULT_CHUNK_LENGTH; i++)
+            {
+                writer.write((byte)i);
+            }
+            writer.resetAndTruncate(mark);
+            writer.write(rawPost);
+            writer.close();
+
+            assert f.exists();
+            CompressedRandomAccessReader reader = CompressedRandomAccessReader.open(filename, new CompressionMetadata(filename + ".metadata", f.length(), true));
+            assertEquals(dataPre.length + rawPost.length, reader.length());
+            byte[] result = new byte[(int)reader.length()];
+
+            reader.readFully(result);
+
+            assert(reader.isEOF());
+            reader.close();
+
+            byte[] fullInput = new byte[bytesToTest * 2];
+            System.arraycopy(dataPre, 0, fullInput, 0, dataPre.length);
+            System.arraycopy(rawPost, 0, fullInput, bytesToTest, rawPost.length);
+            assert Arrays.equals(result, fullInput);
+        }
+        finally
+        {
+            // cleanup
+            if (f.exists())
+                f.delete();
+            File metadata = new File(f + ".metadata");
+            if (metadata.exists())
+                metadata.delete();
+        }
+    }
+}


[3/5] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by jm...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1


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

Branch: refs/heads/trunk
Commit: 5639eac1af7b24a3a0584b2a8dc25d8d3d7aa1f2
Parents: 9ff8270 f5f97d8
Author: Joshua McKenzie <jm...@apache.org>
Authored: Tue Mar 24 13:20:17 2015 -0500
Committer: Joshua McKenzie <jm...@apache.org>
Committed: Tue Mar 24 13:20:17 2015 -0500

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

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



[4/5] cassandra git commit: Merge branch 'cassandra-2.1' into trunk

Posted by jm...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 7d62a731f343df880793c10fa127b6101eb19876
Parents: dadc206 5639eac
Author: Joshua McKenzie <jm...@apache.org>
Authored: Tue Mar 24 13:21:46 2015 -0500
Committer: Joshua McKenzie <jm...@apache.org>
Committed: Tue Mar 24 13:21:46 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                                       | 1 +
 .../apache/cassandra/io/compress/CompressedSequentialWriter.java  | 3 +++
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7d62a731/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index b3752f7,2000bd0..1648b76
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,81 -1,5 +1,82 @@@
 +3.0
 + * Add a key-value payload for third party usage (CASSANDRA-8553)
 + * Bump metrics-reporter-config dependency for metrics 3.0 (CASSANDRA-8149)
 + * Partition intra-cluster message streams by size, not type (CASSANDRA-8789)
 + * Add WriteFailureException to native protocol, notify coordinator of
 +   write failures (CASSANDRA-8592)
 + * Convert SequentialWriter to nio (CASSANDRA-8709)
 + * Add role based access control (CASSANDRA-7653, 8650, 7216, 8760, 8849, 8761, 8850)
 + * Record client ip address in tracing sessions (CASSANDRA-8162)
 + * Indicate partition key columns in response metadata for prepared
 +   statements (CASSANDRA-7660)
 + * Merge UUIDType and TimeUUIDType parse logic (CASSANDRA-8759)
 + * Avoid memory allocation when searching index summary (CASSANDRA-8793)
 + * Optimise (Time)?UUIDType Comparisons (CASSANDRA-8730)
 + * Make CRC32Ex into a separate maven dependency (CASSANDRA-8836)
 + * Use preloaded jemalloc w/ Unsafe (CASSANDRA-8714)
 + * Avoid accessing partitioner through StorageProxy (CASSANDRA-8244, 8268)
 + * Upgrade Metrics library and remove depricated metrics (CASSANDRA-5657)
 + * Serializing Row cache alternative, fully off heap (CASSANDRA-7438)
 + * Duplicate rows returned when in clause has repeated values (CASSANDRA-6707)
 + * Make CassandraException unchecked, extend RuntimeException (CASSANDRA-8560)
 + * Support direct buffer decompression for reads (CASSANDRA-8464)
 + * DirectByteBuffer compatible LZ4 methods (CASSANDRA-7039)
 + * Group sstables for anticompaction correctly (CASSANDRA-8578)
 + * Add ReadFailureException to native protocol, respond
 +   immediately when replicas encounter errors while handling
 +   a read request (CASSANDRA-7886)
 + * Switch CommitLogSegment from RandomAccessFile to nio (CASSANDRA-8308)
 + * Allow mixing token and partition key restrictions (CASSANDRA-7016)
 + * Support index key/value entries on map collections (CASSANDRA-8473)
 + * Modernize schema tables (CASSANDRA-8261)
 + * Support for user-defined aggregation functions (CASSANDRA-8053)
 + * Fix NPE in SelectStatement with empty IN values (CASSANDRA-8419)
 + * Refactor SelectStatement, return IN results in natural order instead
 +   of IN value list order and ignore duplicate values in partition key IN restrictions (CASSANDRA-7981)
 + * Support UDTs, tuples, and collections in user-defined
 +   functions (CASSANDRA-7563)
 + * Fix aggregate fn results on empty selection, result column name,
 +   and cqlsh parsing (CASSANDRA-8229)
 + * Mark sstables as repaired after full repair (CASSANDRA-7586)
 + * Extend Descriptor to include a format value and refactor reader/writer
 +   APIs (CASSANDRA-7443)
 + * Integrate JMH for microbenchmarks (CASSANDRA-8151)
 + * Keep sstable levels when bootstrapping (CASSANDRA-7460)
 + * Add Sigar library and perform basic OS settings check on startup (CASSANDRA-7838)
 + * Support for aggregation functions (CASSANDRA-4914)
 + * Remove cassandra-cli (CASSANDRA-7920)
 + * Accept dollar quoted strings in CQL (CASSANDRA-7769)
 + * Make assassinate a first class command (CASSANDRA-7935)
 + * Support IN clause on any partition key column (CASSANDRA-7855)
 + * Support IN clause on any clustering column (CASSANDRA-4762)
 + * Improve compaction logging (CASSANDRA-7818)
 + * Remove YamlFileNetworkTopologySnitch (CASSANDRA-7917)
 + * Do anticompaction in groups (CASSANDRA-6851)
 + * Support user-defined functions (CASSANDRA-7395, 7526, 7562, 7740, 7781, 7929,
 +   7924, 7812, 8063, 7813, 7708)
 + * Permit configurable timestamps with cassandra-stress (CASSANDRA-7416)
 + * Move sstable RandomAccessReader to nio2, which allows using the
 +   FILE_SHARE_DELETE flag on Windows (CASSANDRA-4050)
 + * Remove CQL2 (CASSANDRA-5918)
 + * Add Thrift get_multi_slice call (CASSANDRA-6757)
 + * Optimize fetching multiple cells by name (CASSANDRA-6933)
 + * Allow compilation in java 8 (CASSANDRA-7028)
 + * Make incremental repair default (CASSANDRA-7250)
 + * Enable code coverage thru JaCoCo (CASSANDRA-7226)
 + * Switch external naming of 'column families' to 'tables' (CASSANDRA-4369) 
 + * Shorten SSTable path (CASSANDRA-6962)
 + * Use unsafe mutations for most unit tests (CASSANDRA-6969)
 + * Fix race condition during calculation of pending ranges (CASSANDRA-7390)
 + * Fail on very large batch sizes (CASSANDRA-8011)
 + * Improve concurrency of repair (CASSANDRA-6455, 8208)
 + * Select optimal CRC32 implementation at runtime (CASSANDRA-8614)
 + * Evaluate MurmurHash of Token once per query (CASSANDRA-7096)
 + * Generalize progress reporting (CASSANDRA-8901)
 + * Resumable bootstrap streaming (CASSANDRA-8838)
 + * Allow scrub for secondary index (CASSANDRA-5174)
 +
  2.1.4
+  * Fix potential data loss in CompressedSequentialWriter (CASSANDRA-8949)
   * Make PasswordAuthenticator number of hashing rounds configurable (CASSANDRA-8085)
   * Fix AssertionError when binding nested collections in DELETE (CASSANDRA-8900)
   * Check for overlap with non-early sstables in LCS (CASSANDRA-8739)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7d62a731/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index b4d9dcc,c004232..50dca93
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@@ -248,13 -224,14 +248,16 @@@ public class CompressedSequentialWrite
              throw new FSReadError(e, getPath());
          }
  
 -        // reset buffer
 -        validBufferBytes = realMark.bufferOffset;
 -        bufferOffset = current - validBufferBytes;
 +        // Mark as dirty so we can guarantee the newly buffered bytes won't be lost on a rebuffer
 +        buffer.position(realMark.validBufferBytes);
 +        isDirty = true;
 +
 +        bufferOffset = truncateTarget - buffer.position();
          chunkCount = realMark.nextChunkIndex - 1;
  
+         // mark as dirty so we don't lose the bytes on subsequent reBuffer calls
+         isDirty = true;
+ 
          // truncate data and index file
          truncate(chunkOffset);
          metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);


[5/5] cassandra git commit: ninja - clean up merge diff

Posted by jm...@apache.org.
ninja - clean up merge diff


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

Branch: refs/heads/trunk
Commit: 507a36305481c9dff8f4784626d34a871faa5106
Parents: 7d62a73
Author: Joshua McKenzie <jm...@apache.org>
Authored: Tue Mar 24 13:24:16 2015 -0500
Committer: Joshua McKenzie <jm...@apache.org>
Committed: Tue Mar 24 13:24:16 2015 -0500

----------------------------------------------------------------------
 .../apache/cassandra/io/compress/CompressedSequentialWriter.java  | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/507a3630/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 50dca93..b4d9dcc 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -255,9 +255,6 @@ public class CompressedSequentialWriter extends SequentialWriter
         bufferOffset = truncateTarget - buffer.position();
         chunkCount = realMark.nextChunkIndex - 1;
 
-        // mark as dirty so we don't lose the bytes on subsequent reBuffer calls
-        isDirty = true;
-
         // truncate data and index file
         truncate(chunkOffset);
         metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);