hive-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From hashut...@apache.org
Subject svn commit: r1517547 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/io/RCFile.java test/org/apache/hadoop/hive/ql/io/TestRCFile.java
Date Mon, 26 Aug 2013 14:34:34 GMT
Author: hashutosh
Date: Mon Aug 26 14:34:33 2013
New Revision: 1517547

URL: http://svn.apache.org/r1517547
Log:
HIVE-5100 :  RCFile::sync(long)  missing 1 byte in System.arraycopy() (Gopal V via Ashutosh
Chauhan)

Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java?rev=1517547&r1=1517546&r2=1517547&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java Mon Aug 26 14:34:33 2013
@@ -1601,7 +1601,7 @@ public class RCFile {
             }
           }
           /* move the last 16 bytes to the prefix area */
-          System.arraycopy(buffer, buffer.length - prefix - 1, buffer, 0, prefix);
+          System.arraycopy(buffer, buffer.length - prefix, buffer, 0, prefix);
           n = (int)Math.min(n, end - in.getPos());
         }
       } catch (ChecksumException e) { // checksum failure

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java?rev=1517547&r1=1517546&r2=1517547&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java Mon Aug 26 14:34:33
2013
@@ -62,6 +62,7 @@ import org.apache.hadoop.mapred.InputSpl
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.FileSplit;
 
 /**
  * TestRCFile.
@@ -573,6 +574,50 @@ public class TestRCFile extends TestCase
     splitAfterSync();
   }
 
+  public void testSync() throws IOException {
+    Path testDir = new Path(System.getProperty("test.data.dir", ".")
+        + "/mapred/testsync");
+    Path testFile = new Path(testDir, "test_rcfile");
+    fs.delete(testFile, true);
+    int intervalRecordCount = 500;
+    CompressionCodec codec = null;
+    int writeCount = 2500;
+    Configuration cloneConf = new Configuration(conf);
+    RCFileOutputFormat.setColumnNumber(cloneConf, bytesArray.length);
+    cloneConf.setInt(RCFile.RECORD_INTERVAL_CONF_STR, intervalRecordCount);
+    RCFile.Writer writer = new RCFile.Writer(fs, cloneConf, testFile, null, codec);    
+
+    BytesRefArrayWritable bytes = new BytesRefArrayWritable(bytesArray.length);
+    for (int i = 0; i < bytesArray.length; i++) {
+      BytesRefWritable cu = null;
+      cu = new BytesRefWritable(bytesArray[i], 0, bytesArray[i].length);
+      bytes.set(i, cu);
+    }
+    for (int i = 0; i < writeCount; i++) {
+      writer.append(bytes);
+    }
+    writer.close();
+    long fileLen = fs.getFileStatus(testFile).getLen();
+
+    RCFileInputFormat inputFormat = new RCFileInputFormat();
+    JobConf jobconf = new JobConf(cloneConf);
+    jobconf.set("mapred.input.dir", testDir.toString());
+    jobconf.setLong("mapred.min.split.size", fileLen);
+    InputSplit[] splits = inputFormat.getSplits(jobconf, 1);
+    RCFileRecordReader rr = new RCFileRecordReader(jobconf, (FileSplit)splits[0]);
+    long lastSync = 0; 
+    for(int i = 0; i < 2500; i++) {
+      rr.sync(i);
+      if(rr.getPos() < lastSync) {
+        String reason = String.format("Sync at offset %d skipped sync block at location %d
(returned %d instead)", i-1, rr.getPos(), lastSync);
+        System.out.println(reason);
+        fail(reason);
+      }
+      lastSync = rr.getPos();
+    }
+    rr.close();
+  }
+
   private void splitBeforeSync() throws IOException {
     writeThenReadByRecordReader(600, 1000, 2, 1, null);
   }



Mime
View raw message