hive-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From prasan...@apache.org
Subject svn commit: r1659739 - in /hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc: FileDump.java RecordReaderImpl.java
Date Sat, 14 Feb 2015 01:17:16 GMT
Author: prasanthj
Date: Sat Feb 14 01:17:15 2015
New Revision: 1659739

URL: http://svn.apache.org/r1659739
Log:
HIVE-9684: Incorrect disk range computation in ORC because of optional stream kind (Prasanth
Jayachandran reviewed by Gopal V)

Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java?rev=1659739&r1=1659738&r2=1659739&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java Sat Feb 14 01:17:15
2015
@@ -129,9 +129,10 @@ public final class FileDump {
         OrcProto.StripeFooter footer = rows.readStripeFooter(stripe);
         long sectionStart = stripeStart;
         for(OrcProto.Stream section: footer.getStreamsList()) {
+          String kind = section.hasKind() ? section.getKind().name() : "UNKNOWN";
           System.out.println("    Stream: column " + section.getColumn() +
-            " section " + section.getKind() + " start: " + sectionStart +
-            " length " + section.getLength());
+              " section " + kind + " start: " + sectionStart +
+              " length " + section.getLength());
           sectionStart += section.getLength();
         }
         for (int i = 0; i < footer.getColumnsCount(); ++i) {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java?rev=1659739&r1=1659738&r2=1659739&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java Sat Feb
14 01:17:15 2015
@@ -2989,7 +2989,7 @@ class RecordReaderImpl implements Record
     // figure out which columns have a present stream
     boolean[] hasNull = new boolean[types.size()];
     for(OrcProto.Stream stream: streamList) {
-      if (stream.getKind() == OrcProto.Stream.Kind.PRESENT) {
+      if (stream.hasKind() && (stream.getKind() == OrcProto.Stream.Kind.PRESENT))
{
         hasNull[stream.getColumn()] = true;
       }
     }
@@ -2997,7 +2997,9 @@ class RecordReaderImpl implements Record
       long length = stream.getLength();
       int column = stream.getColumn();
       OrcProto.Stream.Kind streamKind = stream.getKind();
-      if (StreamName.getArea(streamKind) == StreamName.Area.DATA &&
+      // since stream kind is optional, first check if it exists
+      if (stream.hasKind() &&
+          (StreamName.getArea(streamKind) == StreamName.Area.DATA) &&
           includedColumns[column]) {
         // if we aren't filtering or it is a dictionary, load it.
         if (includedRowGroups == null ||
@@ -3134,8 +3136,10 @@ class RecordReaderImpl implements Record
     long offset = 0;
     for(OrcProto.Stream streamDesc: streamDescriptions) {
       int column = streamDesc.getColumn();
+      // do not create stream if stream kind does not exist
       if ((includeColumn == null || includeColumn[column]) &&
-          StreamName.getArea(streamDesc.getKind()) == StreamName.Area.DATA) {
+          streamDesc.hasKind() &&
+          (StreamName.getArea(streamDesc.getKind()) == StreamName.Area.DATA)) {
         long length = streamDesc.getLength();
         int first = -1;
         int last = -2;
@@ -3381,7 +3385,7 @@ class RecordReaderImpl implements Record
       int len = (int) stream.getLength();
       // row index stream and bloom filter are interlaced, check if the sarg column contains
bloom
       // filter and combine the io to read row index and bloom filters for that column together
-      if (stream.getKind() == OrcProto.Stream.Kind.ROW_INDEX) {
+      if (stream.hasKind() && (stream.getKind() == OrcProto.Stream.Kind.ROW_INDEX))
{
         boolean readBloomFilter = false;
         if (sargColumns != null && sargColumns[col] &&
             nextStream.getKind() == OrcProto.Stream.Kind.BLOOM_FILTER) {



Mime
View raw message