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/0b16546f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0b16546f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0b16546f

Branch: refs/heads/trunk
Commit: 0b16546f6500f7c33db2f94957d6b5a8e0c108d1
Parents: b65b28a c1020d6
Author: Benjamin Lerer <b.le...@gmail.com>
Authored: Tue Apr 10 10:09:05 2018 +0200
Committer: Benjamin Lerer <b.le...@gmail.com>
Committed: Tue Apr 10 10:10:47 2018 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../io/util/CompressedChunkReader.java          | 83 ++++++++++++--------
 2 files changed, 52 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0b16546f/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index d191810,c4f05d5..e68518d
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -236,6 -21,11 +236,8 @@@ Merged from 3.0
   * Chain commit log marker potential performance regression in batch commit 
mode (CASSANDRA-14194)
   * Fully utilise specified compaction threads (CASSANDRA-14210)
   * Pre-create deletion log records to finish compactions quicker 
(CASSANDRA-12763)
 -Merged from 2.2:
 - * Backport circleci yaml (CASSANDRA-14240)
 -Merged from 2.1:
++ Merged from 2.1:
+  * Check checksum before decompressing data (CASSANDRA-14284)
 - * CVE-2017-5929 Security vulnerability in Logback warning in NEWS.txt 
(CASSANDRA-14183)
  
  
  3.11.2

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0b16546f/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/CompressedChunkReader.java
index 5ae083b,177afb0..daec6c4
--- a/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java
+++ b/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java
@@@ -96,8 -94,7 +96,12 @@@ public abstract class CompressedChunkRe
  
          public ByteBuffer allocateBuffer()
          {
-             return allocateBuffer(Math.min(maxCompressedLength,
-                                            
metadata.compressor().initialCompressedBufferLength(metadata.chunkLength())));
 -            return 
allocateBuffer(metadata.compressor().initialCompressedBufferLength(metadata.chunkLength()));
++            int compressedLength = Math.min(maxCompressedLength,
++                                            
metadata.compressor().initialCompressedBufferLength(metadata.chunkLength()));
++
++            int checksumLength = Integer.BYTES;
++
++            return allocateBuffer(compressedLength + checksumLength);
          }
  
          public ByteBuffer allocateBuffer(int size)
@@@ -115,35 -112,54 +119,63 @@@
                  assert position <= fileLength;
  
                  CompressionMetadata.Chunk chunk = metadata.chunkFor(position);
 -                ByteBuffer compressed = compressedHolder.get();
 -
+                 boolean shouldCheckCrc = shouldCheckCrc();
++                int length = shouldCheckCrc ? chunk.length + Integer.BYTES // 
compressed length + checksum length
++                                            : chunk.length;
+ 
 -                int length = shouldCheckCrc ? chunk.length + Integer.BYTES : 
chunk.length;
 -
 -                if (compressed.capacity() < length)
 +                if (chunk.length < maxCompressedLength)
                  {
 -                    compressed = allocateBuffer(length);
 -                    compressedHolder.set(compressed);
 -                }
 -                else
 -                {
 -                    compressed.clear();
 -                }
 +                    ByteBuffer compressed = compressedHolder.get();
-                     assert compressed.capacity() >= chunk.length;
-                     compressed.clear().limit(chunk.length);
-                     if (channel.read(compressed, chunk.offset) != 
chunk.length)
+ 
 -                compressed.limit(length);
 -                if (channel.read(compressed, chunk.offset) != length)
 -                    throw new CorruptBlockException(channel.filePath(), 
chunk);
 -
 -                compressed.flip();
 -                uncompressed.clear();
 -
 -                compressed.position(0).limit(chunk.length);
++                    assert compressed.capacity() >= length;
++                    compressed.clear().limit(length);
++                    if (channel.read(compressed, chunk.offset) != length)
 +                        throw new CorruptBlockException(channel.filePath(), 
chunk);
  
 -                if (shouldCheckCrc)
 +                    compressed.flip();
++                    compressed.limit(chunk.length);
 +                    uncompressed.clear();
 +
++                    if (shouldCheckCrc)
++                    {
++                        int checksum = (int) 
ChecksumType.CRC32.of(compressed);
++
++                        compressed.limit(length);
++                        if (compressed.getInt() != checksum)
++                            throw new 
CorruptBlockException(channel.filePath(), chunk);
++
++                        compressed.position(0).limit(chunk.length);
++                    }
++
 +                    try
 +                    {
 +                        metadata.compressor().uncompress(compressed, 
uncompressed);
 +                    }
 +                    catch (IOException e)
 +                    {
 +                        throw new CorruptBlockException(channel.filePath(), 
chunk, e);
 +                    }
-                     maybeCheckCrc(chunk, compressed);
 +                }
 +                else
                  {
 -                    int checksum = (int) metadata.checksumType.of(compressed);
 -
 -                    compressed.limit(length);
 -                    if (compressed.getInt() != checksum)
 +                    uncompressed.position(0).limit(chunk.length);
 +                    if (channel.read(uncompressed, chunk.offset) != 
chunk.length)
                          throw new CorruptBlockException(channel.filePath(), 
chunk);
-                     maybeCheckCrc(chunk, uncompressed);
+ 
 -                    compressed.position(0).limit(chunk.length);
 -                }
++                    if (shouldCheckCrc)
++                    {
++                        uncompressed.flip();
++                        int checksum = (int) 
ChecksumType.CRC32.of(uncompressed);
+ 
 -                try
 -                {
 -                    metadata.compressor().uncompress(compressed, 
uncompressed);
 -                }
 -                catch (IOException e)
 -                {
 -                    throw new CorruptBlockException(channel.filePath(), 
chunk, e);
 -                }
 -                finally
 -                {
 -                    uncompressed.flip();
++                        ByteBuffer scratch = compressedHolder.get();
++                        scratch.clear().limit(Integer.BYTES);
++
++                        if (channel.read(scratch, chunk.offset + 
chunk.length) != Integer.BYTES
++                                || scratch.getInt(0) != checksum)
++                            throw new 
CorruptBlockException(channel.filePath(), chunk);
++                    }
                  }
 +                uncompressed.flip();
              }
              catch (CorruptBlockException e)
              {
@@@ -199,29 -198,29 +216,29 @@@
  
                  uncompressed.clear();
  
 -                if (shouldCheckCrc())
 +                try
                  {
 -                    int checksum = (int) 
metadata.checksumType.of(compressedChunk);
++                    if (shouldCheckCrc())
++                    {
++                        int checksum = (int) 
ChecksumType.CRC32.of(compressedChunk);
+ 
 -                    compressedChunk.limit(compressedChunk.capacity());
 -                    if (compressedChunk.getInt() != checksum)
 -                        throw new CorruptBlockException(channel.filePath(), 
chunk);
++                        compressedChunk.limit(compressedChunk.capacity());
++                        if (compressedChunk.getInt() != checksum)
++                            throw new 
CorruptBlockException(channel.filePath(), chunk);
+ 
 -                    compressedChunk.position(chunkOffset).limit(chunkOffset + 
chunk.length);
 -                }
++                        
compressedChunk.position(chunkOffset).limit(chunkOffset + chunk.length);
++                    }
+ 
 -                try
 -                {
 -                    metadata.compressor().uncompress(compressedChunk, 
uncompressed);
 +                    if (chunk.length < maxCompressedLength)
 +                        metadata.compressor().uncompress(compressedChunk, 
uncompressed);
 +                    else
 +                        uncompressed.put(compressedChunk);
                  }
                  catch (IOException e)
                  {
                      throw new CorruptBlockException(channel.filePath(), 
chunk, e);
                  }
 -                finally
 -                {
 -                    uncompressed.flip();
 -                }
 +                uncompressed.flip();
- 
-                 if (shouldCheckCrc())
-                 {
-                     compressedChunk.position(chunkOffset).limit(chunkOffset + 
chunk.length);
- 
-                     int checksum = (int) 
ChecksumType.CRC32.of(compressedChunk);
- 
-                     compressedChunk.limit(compressedChunk.capacity());
-                     if (compressedChunk.getInt() != checksum)
-                         throw new CorruptBlockException(channel.filePath(), 
chunk);
-                 }
              }
              catch (CorruptBlockException e)
              {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to