Ver Fonte

HADOOP-2080. Fix the calculation of the checksum file sizes to use integer
math instead of floating point math. Contributed by omalley.


git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@587281 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley há 17 anos atrás
pai
commit
6992c43faa

+ 3 - 0
CHANGES.txt

@@ -335,6 +335,9 @@ Branch 0.15 (unreleased changes)
     that before waiting for the application to finish to ensure all buffered
     data is flushed. (Owen O'Malley via acmurthy)
 
+    HADOOP-2080.  Fixed calculation of the checksum file size when the values
+    are large. (omalley)
+
   IMPROVEMENTS
 
     HADOOP-1908. Restructure data node code so that block sending and 

+ 17 - 9
src/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.util.StringUtils;
  *****************************************************************/
 public abstract class ChecksumFileSystem extends FilterFileSystem {
   private static final byte[] CHECKSUM_VERSION = new byte[] {'c', 'r', 'c', 0};
+  private final int bytesPerChecksum;
 
   public static double getApproxChkSumLength(long size) {
     return ChecksumFSOutputSummer.CHKSUM_AS_FRACTION * size;
@@ -44,6 +45,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   
   public ChecksumFileSystem(FileSystem fs) {
     super(fs);
+    bytesPerChecksum = getConf().getInt("io.bytes.per.checksum", 512);
   }
 
   /** get the raw file system */
@@ -66,12 +68,12 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
    * actual file.
    **/
   public long getChecksumFileLength(Path file, long fileSize) {
-    return ChecksumFSOutputSummer.getChecksumLength(fileSize, getBytesPerSum());
+    return getChecksumLength(fileSize, getBytesPerSum());
   }
 
   /** Return the bytes Per Checksum */
   public int getBytesPerSum() {
-    return getConf().getInt("io.bytes.per.checksum", 512);
+    return bytesPerChecksum;
   }
 
   private int getSumBufferSize(int bytesPerSum, int bufferSize) {
@@ -267,6 +269,19 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
         new ChecksumFSInputChecker(this, f, bufferSize) );
   }
 
+  /**
+   * Calculated the length of the checksum file in bytes.
+   * @param size the length of the data file in bytes
+   * @param bytesPerSum the number of bytes in a checksum block
+   * @return the number of bytes in the checksum file
+   */
+  public static long getChecksumLength(long size, int bytesPerSum) {
+    //the checksum length is equal to size passed divided by bytesPerSum +
+    //bytes written in the beginning of the checksum file.  
+    return ((size + bytesPerSum - 1) / bytesPerSum) * 4 +
+             CHECKSUM_VERSION.length + 4;  
+  }
+
   /** This class provides an output stream for a checksummed file.
    * It generates checksums for data. */
   private static class ChecksumFSOutputSummer extends FSOutputSummer {
@@ -312,13 +327,6 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       datas.close();
     }
     
-    public static long getChecksumLength(long size, int bytesPerSum) {
-      //the checksum length is equal to size passed divided by bytesPerSum +
-      //bytes written in the beginning of the checksum file.  
-      return ((long)(Math.ceil((float)size/bytesPerSum)) + 1) * 4 + 
-        CHECKSUM_VERSION.length;  
-    }
-
     @Override
     protected void writeChunk(byte[] b, int offset, int len, byte[] checksum)
     throws IOException {

+ 35 - 0
src/test/org/apache/hadoop/fs/TestChecksumFileSystem.java

@@ -0,0 +1,35 @@
+/**
+ * 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.hadoop.fs;
+
+import junit.framework.TestCase;
+
+public class TestChecksumFileSystem extends TestCase {
+  public void testgetChecksumLength() throws Exception {
+    assertEquals(8, ChecksumFileSystem.getChecksumLength(0L, 512));
+    assertEquals(12, ChecksumFileSystem.getChecksumLength(1L, 512));
+    assertEquals(12, ChecksumFileSystem.getChecksumLength(512L, 512));
+    assertEquals(16, ChecksumFileSystem.getChecksumLength(513L, 512));
+    assertEquals(16, ChecksumFileSystem.getChecksumLength(1023L, 512));
+    assertEquals(16, ChecksumFileSystem.getChecksumLength(1024L, 512));
+    assertEquals(408, ChecksumFileSystem.getChecksumLength(100L, 1));
+    assertEquals(4000000000008L,
+                 ChecksumFileSystem.getChecksumLength(10000000000000L, 10));    
+  }    
+}