hive-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From na...@apache.org
Subject svn commit: r1433559 - in /hive/trunk: cli/src/java/org/apache/hadoop/hive/cli/RCFileCat.java ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java ql/src/java/org/apache/hadoop/hive/ql/io/RCFileRecordReader.java
Date Tue, 15 Jan 2013 18:08:44 GMT
Author: namit
Date: Tue Jan 15 18:08:44 2013
New Revision: 1433559

URL: http://svn.apache.org/viewvc?rev=1433559&view=rev
Log:
HIVE-3897 Add a way to get the uncompressed/compressed sizes of columns
from an RC File (Kevin Wilfong via namit)


Modified:
    hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/RCFileCat.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileRecordReader.java

Modified: hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/RCFileCat.java
URL: http://svn.apache.org/viewvc/hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/RCFileCat.java?rev=1433559&r1=1433558&r2=1433559&view=diff
==============================================================================
--- hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/RCFileCat.java (original)
+++ hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/RCFileCat.java Tue Jan 15 18:08:44
2013
@@ -31,11 +31,11 @@ import java.nio.charset.CodingErrorActio
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.RCFile.KeyBuffer;
 import org.apache.hadoop.hive.ql.io.RCFileRecordReader;
 import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
 import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable;
 import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.Tool;
@@ -74,6 +74,8 @@ public class RCFileCat implements Tool{
     int recordCount = 0;
     long startT = System.currentTimeMillis();
     boolean verbose = false;
+    boolean columnSizes = false;
+    boolean pretty = false;
 
     //get options from arguments
     if (args.length < 1 || args.length > 3) {
@@ -88,6 +90,11 @@ public class RCFileCat implements Tool{
         length = Long.parseLong(arg.substring("--length=".length()));
       } else if (arg.equals("--verbose")) {
         verbose = true;
+      } else if (arg.equals("--column-sizes")) {
+        columnSizes = true;
+      } else if (arg.equals("--column-sizes-pretty")) {
+        columnSizes = true;
+        pretty = true;
       } else if (fileName == null){
         fileName = new Path(arg);
       } else {
@@ -111,6 +118,45 @@ public class RCFileCat implements Tool{
     //share the code with RecordReader.
     FileSplit split = new FileSplit(fileName,start, length, new JobConf(conf));
     RCFileRecordReader recordReader = new RCFileRecordReader(conf, split);
+
+    if (columnSizes) {
+      // Print out the un/compressed sizes of each column
+      int[] compressedColumnSizes = null;
+      int[] uncompressedColumnSizes = null;
+      // Skip from block to block since we only need the header
+      while (recordReader.nextBlock()) {
+        // Get the sizes from the key buffer and aggregate
+        KeyBuffer keyBuffer = recordReader.getKeyBuffer();
+        if (uncompressedColumnSizes == null) {
+          uncompressedColumnSizes = new int[keyBuffer.getColumnNumber()];
+        }
+        if (compressedColumnSizes == null) {
+          compressedColumnSizes = new int[keyBuffer.getColumnNumber()];
+        }
+        for (int i = 0; i < keyBuffer.getColumnNumber(); i++) {
+          uncompressedColumnSizes[i] += keyBuffer.getEachColumnUncompressedValueLen()[i];
+          compressedColumnSizes[i] += keyBuffer.getEachColumnValueLen()[i];
+        }
+      }
+
+      if (uncompressedColumnSizes != null && compressedColumnSizes != null) {
+        // Print out the sizes, if pretty is set, print it out in a human friendly format,
+        // otherwise print it out as if it were a row
+        for (int i = 0; i < uncompressedColumnSizes.length; i++) {
+          if (pretty) {
+            System.out.println("Column " + i + ": Uncompressed size: " +
+                uncompressedColumnSizes[i] + " Compressed size: " + compressedColumnSizes[i]);
+          } else {
+            System.out.print(i + TAB + uncompressedColumnSizes[i] + TAB +
+                compressedColumnSizes[i] + NEWLINE);
+          }
+        }
+      }
+
+      System.out.flush();
+      return 0;
+    }
+
     LongWritable key = new LongWritable();
     BytesRefArrayWritable value = new BytesRefArrayWritable();
     StringBuilder buf = new StringBuilder(STRING_BUFFER_SIZE); // extra capacity in case
we overrun, to avoid resizing
@@ -172,7 +218,8 @@ public class RCFileCat implements Tool{
     this.conf = conf;
   }
 
-  private static String Usage = "RCFileCat [--start=start_offet] [--length=len] [--verbose]
fileName";
+  private static String Usage = "RCFileCat [--start=start_offet] [--length=len] [--verbose]
" +
+      "[--column-sizes | --column-sizes-pretty] fileName";
 
   public static void main(String[] args) {
     try {

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=1433559&r1=1433558&r2=1433559&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 Tue Jan 15 18:08:44 2013
@@ -317,6 +317,14 @@ public class RCFile {
       throw new RuntimeException("compareTo not supported in class "
           + this.getClass().getName());
     }
+
+    public int[] getEachColumnUncompressedValueLen() {
+      return eachColumnUncompressedValueLen;
+    }
+
+    public int[] getEachColumnValueLen() {
+      return eachColumnValueLen;
+    }
   }
 
   /**

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileRecordReader.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileRecordReader.java?rev=1433559&r1=1433558&r2=1433559&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileRecordReader.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileRecordReader.java Tue Jan 15
18:08:44 2013
@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.RCFile.KeyBuffer;
 import org.apache.hadoop.hive.ql.io.RCFile.Reader;
 import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -78,6 +79,10 @@ public class RCFileRecordReader<K extend
         conf);
   }
 
+  public boolean nextBlock() throws IOException {
+    return in.nextBlock();
+  }
+
   @Override
   public boolean next(LongWritable key, BytesRefArrayWritable value)
       throws IOException {
@@ -125,6 +130,10 @@ public class RCFileRecordReader<K extend
     return in.getPosition();
   }
 
+  public KeyBuffer getKeyBuffer() {
+    return in.getCurrentKeyBufferObj();
+  }
+
   protected void seek(long pos) throws IOException {
     in.seek(pos);
   }



Mime
View raw message