You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2012/03/24 02:08:03 UTC

svn commit: r1304693 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/io/RCFile.java test/org/apache/hadoop/hive/ql/io/TestRCFile.java

Author: hashutosh
Date: Sat Mar 24 01:08:02 2012
New Revision: 1304693

URL: http://svn.apache.org/viewvc?rev=1304693&view=rev
Log:
HIVE-2894 [jira] RCFile Reader doesn't provide access to Metadata
(Owen O'Malley via Ashutosh Chauhan)

Summary:
hive-2894

Add an accessor for RCFile's metadata.

Currently the RCFile writer can add metadata to an RCFile, but the reader
doesn't provide an accessor. I'd like to add one.

Test Plan:
I added a call to test that the metadata that was passed in was available from
the reader.

Reviewers: JIRA, ashutoshc

Reviewed By: ashutoshc

Differential Revision: https://reviews.facebook.net/D2421

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=1304693&r1=1304692&r2=1304693&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 Sat Mar 24 01:08:02 2012
@@ -1187,6 +1187,14 @@ public class RCFile {
     }
 
     /**
+     * Return the metadata (string to string map) that was written into the
+     * file.
+     */
+    public Metadata getMetadata() {
+      return metadata;
+    }
+
+    /**
      * Override this method to specialize the type of
      * {@link FSDataInputStream} returned.
      */

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=1304693&r1=1304692&r2=1304693&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 Sat Mar 24 01:08:02 2012
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.serde2.obj
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile.Metadata;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.compress.CompressionCodec;
@@ -148,10 +149,11 @@ public class TestRCFile extends TestCase
         "123".getBytes("UTF-8"), "1000".getBytes("UTF-8"),
         "5.3".getBytes("UTF-8"), "hive and hadoop".getBytes("UTF-8"),
         new byte[0], "NULL".getBytes("UTF-8")};
-
+    Metadata metadata = new Metadata();
+    metadata.set(new Text("apple"), new Text("block"));
     RCFileOutputFormat.setColumnNumber(conf, expectedFieldsData.length);
     RCFile.Writer writer = new RCFile.Writer(fs, conf, file, null,
-        new DefaultCodec());
+                                             metadata, new DefaultCodec());
     BytesRefArrayWritable bytes = new BytesRefArrayWritable(record_1.length);
     for (int i = 0; i < record_1.length; i++) {
       BytesRefWritable cu = new BytesRefWritable(record_1[i], 0,
@@ -179,7 +181,8 @@ public class TestRCFile extends TestCase
         new Text("hive and hadoop"), null, null};
 
     RCFile.Reader reader = new RCFile.Reader(fs, file, conf);
-
+    assertEquals(new Text("block"),
+                 reader.getMetadata().get(new Text("apple")));
     LongWritable rowID = new LongWritable();
 
     for (int i = 0; i < 2; i++) {
@@ -518,10 +521,10 @@ public class TestRCFile extends TestCase
     }
     assertEquals("readCount should be equal to writeCount", readCount, writeCount);
   }
-  
+
 
   // adopted Hadoop-5476 (calling new SequenceFile.Reader(...) leaves an
-  // InputStream open, if the given sequence file is broken) to RCFile 
+  // InputStream open, if the given sequence file is broken) to RCFile
   private static class TestFSDataInputStream extends FSDataInputStream {
     private boolean closed = false;