hive-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From na...@apache.org
Subject svn commit: r1451130 - in /hive/trunk: cli/src/java/org/apache/hadoop/hive/cli/RCFileCat.java ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java
Date Thu, 28 Feb 2013 07:52:00 GMT
Author: namit
Date: Thu Feb 28 07:52:00 2013
New Revision: 1451130

URL: http://svn.apache.org/r1451130
Log:
HIVE-4056 Extend rcfilecat to support (un)compressed size and no. of row
Gang Tim Liu 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

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=1451130&r1=1451129&r2=1451130&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 Thu Feb 28 07:52:00
2013
@@ -76,6 +76,7 @@ public class RCFileCat implements Tool{
     boolean verbose = false;
     boolean columnSizes = false;
     boolean pretty = false;
+    boolean fileSizes = false;
 
     //get options from arguments
     if (args.length < 1 || args.length > 3) {
@@ -95,6 +96,8 @@ public class RCFileCat implements Tool{
       } else if (arg.equals("--column-sizes-pretty")) {
         columnSizes = true;
         pretty = true;
+      } else if (arg.equals("--file-sizes")){
+        fileSizes = true;
       } else if (fileName == null){
         fileName = new Path(arg);
       } else {
@@ -119,10 +122,14 @@ public class RCFileCat implements Tool{
     FileSplit split = new FileSplit(fileName,start, length, new JobConf(conf));
     RCFileRecordReader recordReader = new RCFileRecordReader(conf, split);
 
-    if (columnSizes) {
+    if (columnSizes || fileSizes) {
       // Print out the un/compressed sizes of each column
       long[] compressedColumnSizes = null;
       long[] uncompressedColumnSizes = null;
+      // un/compressed sizes of file and no. of rows
+      long rowNo = 0;
+      long uncompressedFileSize = 0;
+      long compressedFileSize = 0;
       // Skip from block to block since we only need the header
       while (recordReader.nextBlock()) {
         // Get the sizes from the key buffer and aggregate
@@ -137,9 +144,10 @@ public class RCFileCat implements Tool{
           uncompressedColumnSizes[i] += keyBuffer.getEachColumnUncompressedValueLen()[i];
           compressedColumnSizes[i] += keyBuffer.getEachColumnValueLen()[i];
         }
+        rowNo += keyBuffer.getNumberRows();
       }
 
-      if (uncompressedColumnSizes != null && compressedColumnSizes != null) {
+      if (columnSizes && 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++) {
@@ -153,6 +161,18 @@ public class RCFileCat implements Tool{
         }
       }
 
+      if (fileSizes) {
+        if (uncompressedColumnSizes != null && compressedColumnSizes != null) {
+          for (int i = 0; i < uncompressedColumnSizes.length; i++) {
+            uncompressedFileSize += uncompressedColumnSizes[i];
+            compressedFileSize += compressedColumnSizes[i];
+          }
+        }
+        System.out.print("File size (uncompressed): " + uncompressedFileSize
+            + ". File size (compressed): " + compressedFileSize + ". Number of rows: " +
rowNo
+            + "." + NEWLINE);
+      }
+
       System.out.flush();
       return 0;
     }
@@ -219,7 +239,7 @@ public class RCFileCat implements Tool{
   }
 
   private static String Usage = "RCFileCat [--start=start_offet] [--length=len] [--verbose]
" +
-      "[--column-sizes | --column-sizes-pretty] fileName";
+      "[--column-sizes | --column-sizes-pretty] [--file-sizes] 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=1451130&r1=1451129&r2=1451130&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 Thu Feb 28 07:52:00 2013
@@ -325,6 +325,13 @@ public class RCFile {
     public int[] getEachColumnValueLen() {
       return eachColumnValueLen;
     }
+
+    /**
+     * @return the numberRows
+     */
+    public int getNumberRows() {
+      return numberRows;
+    }
   }
 
   /**



Mime
View raw message