You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@commons.apache.org by bo...@apache.org on 2013/12/07 19:01:01 UTC

svn commit: r1548947 - in /commons/proper/compress/trunk/src: main/java/org/apache/commons/compress/compressors/snappy/ main/java/org/apache/commons/compress/utils/ test/java/org/apache/commons/compress/compressors/ test/java/org/apache/commons/compres...

Author: bodewig
Date: Sat Dec  7 18:01:01 2013
New Revision: 1548947

URL: http://svn.apache.org/r1548947
Log:
COMPRESS-147 checksum verification

Added:
    commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/ChecksumVerifyingInputStream.java
      - copied, changed from r1548937, commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/CRC32VerifyingInputStream.java
    commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/PureJavaCrc32C.java   (contents, props changed)
      - copied, changed from r1548942, hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java
    commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/snappy/
    commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/snappy/FramedSnappyCompressorInputStreamTest.java
      - copied, changed from r1548932, commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/FramedSnappyTestCase.java
Modified:
    commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/compressors/snappy/FramedSnappyCompressorInputStream.java
    commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/CRC32VerifyingInputStream.java
    commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/FramedSnappyTestCase.java
    commons/proper/compress/trunk/src/test/resources/mixed.txt.sz

Modified: commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/compressors/snappy/FramedSnappyCompressorInputStream.java
URL: http://svn.apache.org/viewvc/commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/compressors/snappy/FramedSnappyCompressorInputStream.java?rev=1548947&r1=1548946&r2=1548947&view=diff
==============================================================================
--- commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/compressors/snappy/FramedSnappyCompressorInputStream.java (original)
+++ commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/compressors/snappy/FramedSnappyCompressorInputStream.java Sat Dec  7 18:01:01 2013
@@ -26,6 +26,7 @@ import java.util.Arrays;
 import org.apache.commons.compress.compressors.CompressorInputStream;
 import org.apache.commons.compress.utils.BoundedInputStream;
 import org.apache.commons.compress.utils.IOUtils;
+import org.apache.commons.compress.utils.PureJavaCrc32C;
 
 /**
  * CompressorInputStream for the framing Snappy format.
@@ -36,6 +37,11 @@ import org.apache.commons.compress.utils
  * @since 1.7
  */
 public class FramedSnappyCompressorInputStream extends CompressorInputStream {
+    /**
+     * package private for tests only.
+     */
+    static final long MASK_OFFSET = 0xa282ead8L;
+
     private static final int STREAM_IDENTIFIER_TYPE = 0xff;
     private static final int COMPRESSED_CHUNK_TYPE = 0;
     private static final int UNCOMPRESSED_CHUNK_TYPE = 1;
@@ -61,6 +67,8 @@ public class FramedSnappyCompressorInput
     private boolean endReached, inUncompressedChunk;
 
     private int uncompressedBytesRemaining;
+    private long expectedChecksum = -1;
+    private PureJavaCrc32C checksum = new PureJavaCrc32C();
 
     /**
      * Constructs a new input stream that decompresses snappy-framed-compressed data
@@ -143,10 +151,14 @@ public class FramedSnappyCompressorInput
                 count(currentCompressedChunk.getBytesRead() - before);
             }
         }
+        if (read > 0) {
+            checksum.update(b, off, read);
+        }
         return read;
     }
 
     private void readNextBlock() throws IOException {
+        verifyLastChecksumAndReset();
         inUncompressedChunk = false;
         int type = readOneByte();
         if (type == -1) {
@@ -167,10 +179,10 @@ public class FramedSnappyCompressorInput
         } else if (type == UNCOMPRESSED_CHUNK_TYPE) {
             inUncompressedChunk = true;
             uncompressedBytesRemaining = readSize() - 4 /* CRC */;
-            readCrc();
+            expectedChecksum = unmask(readCrc());
         } else if (type == COMPRESSED_CHUNK_TYPE) {
-            int size = readSize() - 4 /* CRC */;
-            readCrc();
+            long size = readSize() - 4 /* CRC */;
+            expectedChecksum = unmask(readCrc());
             currentCompressedChunk =
                 new SnappyCompressorInputStream(new BoundedInputStream(in, size));
             // constructor reads uncompressed size
@@ -182,13 +194,26 @@ public class FramedSnappyCompressorInput
         }
     }
 
-    private void readCrc() throws IOException {
+    private long readCrc() throws IOException {
         byte[] b = new byte[4];
         int read = IOUtils.readFully(in, b);
         count(read);
         if (read != 4) {
             throw new IOException("premature end of stream");
         }
+        long crc = 0;
+        for (int i = 0; i < 4; i++) {
+            crc |= (b[i] & 0xFFL) << (8 * i);
+        }
+        return crc;
+    }
+
+    static long unmask(long x) {
+        // ugly, maybe we should just have used ints and deal with the
+        // overflow
+        x -= MASK_OFFSET;
+        x &= 0xffffFFFFL;
+        return ((x >> 17) | (x << 15)) & 0xffffFFFFL;
     }
 
     private int readSize() throws IOException {
@@ -231,6 +256,14 @@ public class FramedSnappyCompressorInput
         return -1;
     }
 
+    private void verifyLastChecksumAndReset() throws IOException {
+        if (expectedChecksum >= 0 && expectedChecksum != checksum.getValue()) {
+            throw new IOException("Checksum verification failed");
+        }
+        expectedChecksum = -1;
+        checksum.reset();
+    }
+
     /**
      * Checks if the signature matches what is expected for a .sz file.
      *

Modified: commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/CRC32VerifyingInputStream.java
URL: http://svn.apache.org/viewvc/commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/CRC32VerifyingInputStream.java?rev=1548947&r1=1548946&r2=1548947&view=diff
==============================================================================
--- commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/CRC32VerifyingInputStream.java (original)
+++ commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/CRC32VerifyingInputStream.java Sat Dec  7 18:01:01 2013
@@ -17,7 +17,6 @@
  */
 package org.apache.commons.compress.utils;
 
-import java.io.IOException;
 import java.io.InputStream;
 import java.util.zip.CRC32;
 
@@ -27,82 +26,10 @@ import java.util.zip.CRC32;
  * @NotThreadSafe
  * @since 1.6
  */
-public class CRC32VerifyingInputStream extends InputStream {
-    private final InputStream in;
-    private long bytesRemaining;
-    private final long expectedCrc32;
-    private final CRC32 crc32 = new CRC32();
+public class CRC32VerifyingInputStream extends ChecksumVerifyingInputStream {
     
     public CRC32VerifyingInputStream(final InputStream in, final long size, final long expectedCrc32) {
-        this.in = in;
-        this.expectedCrc32 = expectedCrc32;
-        this.bytesRemaining = size;
+        super(new CRC32(), in, size, expectedCrc32);
     }
 
-    /**
-     * Reads a single byte from the stream
-     * @throws IOException if the underlying stream throws or the
-     * stream is exhausted and the CRC doesn't match the expected
-     * value
-     */
-    @Override
-    public int read() throws IOException {
-        if (bytesRemaining <= 0) {
-            return -1;
-        }
-        int ret = in.read();
-        if (ret >= 0) {
-            crc32.update(ret);
-            --bytesRemaining;
-        }
-        if (bytesRemaining == 0 && expectedCrc32 != crc32.getValue()) {
-            throw new IOException("CRC32 verification failed");
-        }
-        return ret;
-    }
-
-    /**
-     * Reads a byte array from the stream
-     * @throws IOException if the underlying stream throws or the
-     * stream is exhausted and the CRC doesn't match the expected
-     * value
-     */
-    @Override
-    public int read(byte[] b) throws IOException {
-        return read(b, 0, b.length);
-    }
-
-    /**
-     * Reads from the stream into a byte array.
-     * @throws IOException if the underlying stream throws or the
-     * stream is exhausted and the CRC doesn't match the expected
-     * value
-     */
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        int ret = in.read(b, off, len);
-        if (ret >= 0) {
-            crc32.update(b, off, ret);
-            bytesRemaining -= ret;
-        }
-        if (bytesRemaining <= 0 && expectedCrc32 != crc32.getValue()) {
-            throw new IOException("CRC32 verification failed");
-        }
-        return ret;
-    }
-
-    @Override
-    public long skip(long n) throws IOException {
-        // Can't really skip, we have to hash everything to verify the checksum
-        if (read() >= 0) {
-            return 1;
-        } else {
-            return 0;
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        in.close();
-    }
 }

Copied: commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/ChecksumVerifyingInputStream.java (from r1548937, commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/CRC32VerifyingInputStream.java)
URL: http://svn.apache.org/viewvc/commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/ChecksumVerifyingInputStream.java?p2=commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/ChecksumVerifyingInputStream.java&p1=commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/CRC32VerifyingInputStream.java&r1=1548937&r2=1548947&rev=1548947&view=diff
==============================================================================
--- commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/CRC32VerifyingInputStream.java (original)
+++ commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/ChecksumVerifyingInputStream.java Sat Dec  7 18:01:01 2013
@@ -19,30 +19,32 @@ package org.apache.commons.compress.util
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.zip.CRC32;
+import java.util.zip.Checksum;
 
 /**
- * A stream that verifies the CRC of the data read once the stream is
+ * A stream that verifies the checksum of the data read once the stream is
  * exhausted.
  * @NotThreadSafe
- * @since 1.6
+ * @since 1.7
  */
-public class CRC32VerifyingInputStream extends InputStream {
+public class ChecksumVerifyingInputStream extends InputStream {
     private final InputStream in;
     private long bytesRemaining;
-    private final long expectedCrc32;
-    private final CRC32 crc32 = new CRC32();
+    private final long expectedChecksum;
+    private final Checksum checksum;
     
-    public CRC32VerifyingInputStream(final InputStream in, final long size, final long expectedCrc32) {
+    public ChecksumVerifyingInputStream(final Checksum checksum, final InputStream in,
+                                        final long size, final long expectedChecksum) {
+        this.checksum = checksum;
         this.in = in;
-        this.expectedCrc32 = expectedCrc32;
+        this.expectedChecksum = expectedChecksum;
         this.bytesRemaining = size;
     }
 
     /**
      * Reads a single byte from the stream
      * @throws IOException if the underlying stream throws or the
-     * stream is exhausted and the CRC doesn't match the expected
+     * stream is exhausted and the Checksum doesn't match the expected
      * value
      */
     @Override
@@ -52,11 +54,11 @@ public class CRC32VerifyingInputStream e
         }
         int ret = in.read();
         if (ret >= 0) {
-            crc32.update(ret);
+            checksum.update(ret);
             --bytesRemaining;
         }
-        if (bytesRemaining == 0 && expectedCrc32 != crc32.getValue()) {
-            throw new IOException("CRC32 verification failed");
+        if (bytesRemaining == 0 && expectedChecksum != checksum.getValue()) {
+            throw new IOException("Checksum verification failed");
         }
         return ret;
     }
@@ -64,7 +66,7 @@ public class CRC32VerifyingInputStream e
     /**
      * Reads a byte array from the stream
      * @throws IOException if the underlying stream throws or the
-     * stream is exhausted and the CRC doesn't match the expected
+     * stream is exhausted and the Checksum doesn't match the expected
      * value
      */
     @Override
@@ -75,18 +77,18 @@ public class CRC32VerifyingInputStream e
     /**
      * Reads from the stream into a byte array.
      * @throws IOException if the underlying stream throws or the
-     * stream is exhausted and the CRC doesn't match the expected
+     * stream is exhausted and the Checksum doesn't match the expected
      * value
      */
     @Override
     public int read(byte[] b, int off, int len) throws IOException {
         int ret = in.read(b, off, len);
         if (ret >= 0) {
-            crc32.update(b, off, ret);
+            checksum.update(b, off, ret);
             bytesRemaining -= ret;
         }
-        if (bytesRemaining <= 0 && expectedCrc32 != crc32.getValue()) {
-            throw new IOException("CRC32 verification failed");
+        if (bytesRemaining <= 0 && expectedChecksum != checksum.getValue()) {
+            throw new IOException("Checksum verification failed");
         }
         return ret;
     }

Copied: commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/PureJavaCrc32C.java (from r1548942, hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java)
URL: http://svn.apache.org/viewvc/commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/PureJavaCrc32C.java?p2=commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/PureJavaCrc32C.java&p1=hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java&r1=1548942&r2=1548947&rev=1548947&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java (original)
+++ commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/PureJavaCrc32C.java Sat Dec  7 18:01:01 2013
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,20 +18,20 @@
  * Some portions of this file Copyright (c) 2004-2006 Intel Corportation
  * and licensed under the BSD license.
  */
-package org.apache.hadoop.util;
+package org.apache.commons.compress.utils;
 
 import java.util.zip.Checksum;
 
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
 /**
  * A pure-java implementation of the CRC32 checksum that uses
  * the CRC32-C polynomial, the same polynomial used by iSCSI
  * and implemented on many Intel chipsets supporting SSE4.2.
+ *
+ * <p>This file is a copy of the implementation at the Apache Hadoop project.</p>
+ * @see "http://svn.apache.org/repos/asf/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java"
+ * @NotThreadSafe
+ * @since 1.7
  */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
 public class PureJavaCrc32C implements Checksum {
 
   /** the current CRC value, bit-flipped */

Propchange: commons/proper/compress/trunk/src/main/java/org/apache/commons/compress/utils/PureJavaCrc32C.java
------------------------------------------------------------------------------
--- svn:mergeinfo (added)
+++ svn:mergeinfo Sat Dec  7 18:01:01 2013
@@ -0,0 +1,9 @@
+/hadoop/common/branches/HDFS-1623/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java:1152502-1296519
+/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java:1360400-1480829
+/hadoop/common/branches/HDFS-3042/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java:1306184-1342109
+/hadoop/common/branches/HDFS-3077/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java:1363593-1396941
+/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java:1430995-1467533
+/hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java:1509426-1536569
+/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PureJavaCrc32C.java:1526848,1541667
+/hadoop/core/branches/branch-0.19/core/src/java/org/apache/hadoop/util/PureJavaCrc32C.java:713112
+/hadoop/core/trunk/src/core/org/apache/hadoop/util/PureJavaCrc32C.java:776175-785643,785929-786278

Modified: commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/FramedSnappyTestCase.java
URL: http://svn.apache.org/viewvc/commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/FramedSnappyTestCase.java?rev=1548947&r1=1548946&r2=1548947&view=diff
==============================================================================
--- commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/FramedSnappyTestCase.java (original)
+++ commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/FramedSnappyTestCase.java Sat Dec  7 18:01:01 2013
@@ -21,38 +21,19 @@ package org.apache.commons.compress.comp
 import static org.junit.Assert.assertArrayEquals;
 
 import java.io.BufferedInputStream;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.Arrays;
 
 import org.apache.commons.compress.AbstractTestCase;
-import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
 import org.apache.commons.compress.compressors.snappy.FramedSnappyCompressorInputStream;
 import org.apache.commons.compress.utils.IOUtils;
 
 public final class FramedSnappyTestCase
     extends AbstractTestCase {
 
-    public void testMatches() throws IOException {
-        assertFalse(FramedSnappyCompressorInputStream.matches(new byte[10], 10));
-        byte[] b = new byte[12];
-        final File input = getFile("bla.tar.sz");
-        FileInputStream in = new FileInputStream(input);
-        try {
-            IOUtils.readFully(in, b);
-        } finally {
-            in.close();
-        }
-        assertFalse(FramedSnappyCompressorInputStream.matches(b, 9));
-        assertTrue(FramedSnappyCompressorInputStream.matches(b, 10));
-        assertTrue(FramedSnappyCompressorInputStream.matches(b, 12));
-    }
-
     public void testDefaultExtraction() throws Exception {
         testUnarchive(new StreamWrapper<CompressorInputStream>() {
             public CompressorInputStream wrap(InputStream is) throws IOException {
@@ -79,114 +60,6 @@ public final class FramedSnappyTestCase
         });
     }
 
-    /**
-     * Something big enough to make buffers slide.
-     */
-    public void testLoremIpsum() throws Exception {
-        final FileInputStream isSz = new FileInputStream(getFile("lorem-ipsum.txt.sz"));
-        final File outputSz = new File(dir, "lorem-ipsum.1");
-        final File outputGz = new File(dir, "lorem-ipsum.2");
-        try {
-            CompressorInputStream in = new FramedSnappyCompressorInputStream(isSz);
-            FileOutputStream out = null;
-            try {
-                out = new FileOutputStream(outputSz);
-                IOUtils.copy(in, out);
-            } finally {
-                if (out != null) {
-                    out.close();
-                }
-                in.close();
-            }
-            final FileInputStream isGz = new FileInputStream(getFile("lorem-ipsum.txt.gz"));
-            try {
-                in = new GzipCompressorInputStream(isGz);
-                try {
-                    out = new FileOutputStream(outputGz);
-                    IOUtils.copy(in, out);
-                } finally {
-                    if (out != null) {
-                        out.close();
-                    }
-                    in.close();
-                }
-            } finally {
-                isGz.close();
-            }
-        } finally {
-            isSz.close();
-        }
-
-        final FileInputStream sz = new FileInputStream(outputSz);
-        try {
-            FileInputStream gz = new FileInputStream(outputGz);
-            try {
-                assertArrayEquals(IOUtils.toByteArray(sz),
-                                  IOUtils.toByteArray(gz));
-            } finally {
-                gz.close();
-            }
-        } finally {
-            sz.close();
-        }
-    }
-
-    public void testRemainingChunkTypes() throws Exception {
-        final FileInputStream isSz = new FileInputStream(getFile("mixed.txt.sz"));
-        ByteArrayOutputStream out = new ByteArrayOutputStream();
-        try {
-            CompressorInputStream in = new FramedSnappyCompressorInputStream(isSz);
-            IOUtils.copy(in, out);
-            out.close();
-        } finally {
-            isSz.close();
-        }
-
-        assertArrayEquals(new byte[] { '1', '2', '3', '4',
-                                       '5', '6', '7', '8', '9',
-                                       '5', '6', '7', '8', '9',
-                                       '5', '6', '7', '8', '9',
-                                       '5', '6', '7', '8', '9',
-                                       '5', '6', '7', '8', '9', 10,
-                                       '1', '2', '3', '4',
-                                       '1', '2', '3', '4',
-            }, out.toByteArray());
-    }
-
-    public void testAvailable() throws Exception {
-        final FileInputStream isSz = new FileInputStream(getFile("mixed.txt.sz"));
-        try {
-            CompressorInputStream in = new FramedSnappyCompressorInputStream(isSz);
-            assertEquals(0, in.available()); // no chunk read so far
-            assertEquals('1', in.read());
-            assertEquals(3, in.available()); // remainder of first uncompressed block
-            assertEquals(3, in.read(new byte[5], 0, 3));
-            assertEquals('5', in.read());
-            assertEquals(4, in.available()); // remainder of literal
-            assertEquals(4, in.read(new byte[5], 0, 4));
-            assertEquals('5', in.read());
-            assertEquals(19, in.available()); // remainder of copy
-            in.close();
-        } finally {
-            isSz.close();
-        }
-    }
-
-    public void testUnskippableChunk() {
-        byte[] input = new byte[] {
-            (byte) 0xff, 6, 0, 0, 's', 'N', 'a', 'P', 'p', 'Y',
-            2, 2, 0, 0, 1, 1
-        };
-        try {
-            CompressorInputStream in =
-                new FramedSnappyCompressorInputStream(new ByteArrayInputStream(input));
-            in.read();
-            fail("expected an exception");
-        } catch (IOException ex) {
-            assertTrue(ex.getMessage().indexOf("unskippable chunk") > -1);
-        }
-    }
-
     private void testUnarchive(StreamWrapper<CompressorInputStream> wrapper) throws Exception {
         final File input = getFile("bla.tar.sz");
         final File output = new File(dir, "bla.tar");
@@ -223,4 +96,5 @@ public final class FramedSnappyTestCase
             written.close();
         }
     }
+
 }

Copied: commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/snappy/FramedSnappyCompressorInputStreamTest.java (from r1548932, commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/FramedSnappyTestCase.java)
URL: http://svn.apache.org/viewvc/commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/snappy/FramedSnappyCompressorInputStreamTest.java?p2=commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/snappy/FramedSnappyCompressorInputStreamTest.java&p1=commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/FramedSnappyTestCase.java&r1=1548932&r2=1548947&rev=1548947&view=diff
==============================================================================
--- commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/FramedSnappyTestCase.java (original)
+++ commons/proper/compress/trunk/src/test/java/org/apache/commons/compress/compressors/snappy/FramedSnappyCompressorInputStreamTest.java Sat Dec  7 18:01:01 2013
@@ -16,11 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.commons.compress.compressors;
+package org.apache.commons.compress.compressors.snappy;
 
 import static org.junit.Assert.assertArrayEquals;
 
-import java.io.BufferedInputStream;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
@@ -32,10 +31,9 @@ import java.util.Arrays;
 
 import org.apache.commons.compress.AbstractTestCase;
 import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
-import org.apache.commons.compress.compressors.snappy.FramedSnappyCompressorInputStream;
 import org.apache.commons.compress.utils.IOUtils;
 
-public final class FramedSnappyTestCase
+public final class FramedSnappyCompressorInputStreamTest
     extends AbstractTestCase {
 
     public void testMatches() throws IOException {
@@ -53,32 +51,6 @@ public final class FramedSnappyTestCase
         assertTrue(FramedSnappyCompressorInputStream.matches(b, 12));
     }
 
-    public void testDefaultExtraction() throws Exception {
-        testUnarchive(new StreamWrapper<CompressorInputStream>() {
-            public CompressorInputStream wrap(InputStream is) throws IOException {
-                return new FramedSnappyCompressorInputStream(is);
-            }
-        });
-    }
-
-    public void testDefaultExtractionViaFactory() throws Exception {
-        testUnarchive(new StreamWrapper<CompressorInputStream>() {
-            public CompressorInputStream wrap(InputStream is) throws Exception {
-                return new CompressorStreamFactory()
-                    .createCompressorInputStream(CompressorStreamFactory.SNAPPY_FRAMED,
-                                                 is);
-            }
-        });
-    }
-
-    public void testDefaultExtractionViaFactoryAutodetection() throws Exception {
-        testUnarchive(new StreamWrapper<CompressorInputStream>() {
-            public CompressorInputStream wrap(InputStream is) throws Exception {
-                return new CompressorStreamFactory().createCompressorInputStream(is);
-            }
-        });
-    }
-
     /**
      * Something big enough to make buffers slide.
      */
@@ -87,7 +59,7 @@ public final class FramedSnappyTestCase
         final File outputSz = new File(dir, "lorem-ipsum.1");
         final File outputGz = new File(dir, "lorem-ipsum.2");
         try {
-            CompressorInputStream in = new FramedSnappyCompressorInputStream(isSz);
+            InputStream in = new FramedSnappyCompressorInputStream(isSz);
             FileOutputStream out = null;
             try {
                 out = new FileOutputStream(outputSz);
@@ -135,7 +107,7 @@ public final class FramedSnappyTestCase
         final FileInputStream isSz = new FileInputStream(getFile("mixed.txt.sz"));
         ByteArrayOutputStream out = new ByteArrayOutputStream();
         try {
-            CompressorInputStream in = new FramedSnappyCompressorInputStream(isSz);
+            FramedSnappyCompressorInputStream in = new FramedSnappyCompressorInputStream(isSz);
             IOUtils.copy(in, out);
             out.close();
         } finally {
@@ -156,7 +128,7 @@ public final class FramedSnappyTestCase
     public void testAvailable() throws Exception {
         final FileInputStream isSz = new FileInputStream(getFile("mixed.txt.sz"));
         try {
-            CompressorInputStream in = new FramedSnappyCompressorInputStream(isSz);
+            FramedSnappyCompressorInputStream in = new FramedSnappyCompressorInputStream(isSz);
             assertEquals(0, in.available()); // no chunk read so far
             assertEquals('1', in.read());
             assertEquals(3, in.available()); // remainder of first uncompressed block
@@ -178,7 +150,7 @@ public final class FramedSnappyTestCase
             2, 2, 0, 0, 1, 1
         };
         try {
-            CompressorInputStream in =
+            FramedSnappyCompressorInputStream in =
                 new FramedSnappyCompressorInputStream(new ByteArrayInputStream(input));
             in.read();
             fail("expected an exception");
@@ -187,40 +159,21 @@ public final class FramedSnappyTestCase
         }
     }
 
-    private void testUnarchive(StreamWrapper<CompressorInputStream> wrapper) throws Exception {
-        final File input = getFile("bla.tar.sz");
-        final File output = new File(dir, "bla.tar");
-        final FileInputStream is = new FileInputStream(input);
-        try {
-            // the intermediate BufferedInputStream is there for mark
-            // support in the autodetection test
-            final CompressorInputStream in = wrapper.wrap(new BufferedInputStream(is));
-            FileOutputStream out = null;
-            try {
-                out = new FileOutputStream(output);
-                IOUtils.copy(in, out);
-                assertEquals(995, in.getBytesRead());
-            } finally {
-                if (out != null) {
-                    out.close();
-                }
-                in.close();
-            }
-        } finally {
-            is.close();
-        }
-        final File original = getFile("bla.tar");
-        final FileInputStream written = new FileInputStream(output);
-        try {
-            FileInputStream orig = new FileInputStream(original);
-            try {
-                assertArrayEquals(IOUtils.toByteArray(written),
-                                  IOUtils.toByteArray(orig));
-            } finally {
-                orig.close();
-            }
-        } finally {
-            written.close();
-        }
+    public void testChecksumUnmasking() {
+        testChecksumUnmasking(0xc757l);
+        testChecksumUnmasking(0xffffc757l);
+    }
+
+    public void testChecksumUnmasking(long x) {
+        assertEquals(Long.toHexString(x),
+                     Long.toHexString(FramedSnappyCompressorInputStream
+                                      .unmask(mask(x))));
+    }
+
+    private long mask(long x) {
+        return (((x >>> 15) | (x << 17))
+                + FramedSnappyCompressorInputStream.MASK_OFFSET)
+             & 0xffffFFFFL;
     }
+
 }

Modified: commons/proper/compress/trunk/src/test/resources/mixed.txt.sz
URL: http://svn.apache.org/viewvc/commons/proper/compress/trunk/src/test/resources/mixed.txt.sz?rev=1548947&r1=1548946&r2=1548947&view=diff
==============================================================================
Binary files - no diff available.