diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java index 889a69d..9788c16 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java @@ -129,9 +129,10 @@ private static void printMetaData(List files, Configuration conf, 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) { diff --git ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java index 136d575..2745bee 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java @@ -2989,7 +2989,7 @@ static boolean isDictionary(OrcProto.Stream.Kind kind, // 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 @@ static boolean isDictionary(OrcProto.Stream.Kind kind, 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 @@ static void createStreams(List streamDescriptions, 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 @@ Index readRowIndex(int stripeIndex, boolean[] sargColumns) throws IOException { 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) {