You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/03/15 01:00:36 UTC

hive git commit: HIVE-13185 : orc.ReaderImp.ensureOrcFooter() method fails on small text files with IndexOutOfBoundsException (Illya Yalovyy, reviewed by Sergey Shelukhin)

Repository: hive
Updated Branches:
  refs/heads/master b6af0124b -> f07fdfbc7


HIVE-13185 : orc.ReaderImp.ensureOrcFooter() method fails on small text files with IndexOutOfBoundsException (Illya Yalovyy, reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f07fdfbc
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f07fdfbc
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f07fdfbc

Branch: refs/heads/master
Commit: f07fdfbc7226364cdb21784b308bd2adfe114309
Parents: b6af012
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Mar 14 16:57:57 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Mar 14 16:57:57 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |  15 +-
 .../hadoop/hive/ql/io/orc/TestReaderImpl.java   | 151 +++++++++++++++++++
 2 files changed, 159 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f07fdfbc/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
index 773c2b1..a031a92 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
@@ -256,21 +256,22 @@ public class ReaderImpl implements Reader {
                                       Path path,
                                       int psLen,
                                       ByteBuffer buffer) throws IOException {
-    int len = OrcFile.MAGIC.length();
-    if (psLen < len + 1) {
+    int magicLength = OrcFile.MAGIC.length();
+    int fullLength = magicLength + 1;
+    if (psLen < fullLength || buffer.remaining() < fullLength) {
       throw new FileFormatException("Malformed ORC file " + path +
           ". Invalid postscript length " + psLen);
     }
-    int offset = buffer.arrayOffset() + buffer.position() + buffer.limit() - 1 - len;
+    int offset = buffer.arrayOffset() + buffer.position() + buffer.limit() - fullLength;
     byte[] array = buffer.array();
     // now look for the magic string at the end of the postscript.
-    if (!Text.decode(array, offset, len).equals(OrcFile.MAGIC)) {
+    if (!Text.decode(array, offset, magicLength).equals(OrcFile.MAGIC)) {
       // If it isn't there, this may be the 0.11.0 version of ORC.
       // Read the first 3 bytes of the file to check for the header
-      byte[] header = new byte[len];
-      in.readFully(0, header, 0, len);
+      byte[] header = new byte[magicLength];
+      in.readFully(0, header, 0, magicLength);
       // if it isn't there, this isn't an ORC file
-      if (!Text.decode(header, 0 , len).equals(OrcFile.MAGIC)) {
+      if (!Text.decode(header, 0 , magicLength).equals(OrcFile.MAGIC)) {
         throw new FileFormatException("Malformed ORC file " + path +
             ". Invalid postscript.");
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/f07fdfbc/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestReaderImpl.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestReaderImpl.java
new file mode 100644
index 0000000..e0199d6
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestReaderImpl.java
@@ -0,0 +1,151 @@
+/*
+ * Copyright 2016 The Apache Software Foundation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.io.orc;
+
+import java.io.ByteArrayInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hive.ql.io.FileFormatException;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+
+public class TestReaderImpl {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private final Path path = new Path("test-file.orc");
+  private FSDataInputStream in;
+  private int psLen;
+  private ByteBuffer buffer;
+
+  @Before
+  public void setup() {
+    in = null;
+  }
+
+  @Test
+  public void testEnsureOrcFooterSmallTextFile() throws IOException {
+    prepareTestCase("1".getBytes());
+    thrown.expect(FileFormatException.class);
+    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
+  }
+
+  @Test
+  public void testEnsureOrcFooterLargeTextFile() throws IOException {
+    prepareTestCase("This is Some Text File".getBytes());
+    thrown.expect(FileFormatException.class);
+    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
+  }
+
+  @Test
+  public void testEnsureOrcFooter011ORCFile() throws IOException {
+    prepareTestCase(composeContent(OrcFile.MAGIC, "FOOTER"));
+    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
+  }
+
+  @Test
+  public void testEnsureOrcFooterCorrectORCFooter() throws IOException {
+    prepareTestCase(composeContent("",OrcFile.MAGIC));
+    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
+  }
+
+  private void prepareTestCase(byte[] bytes) {
+    buffer = ByteBuffer.wrap(bytes);
+    psLen = buffer.get(bytes.length - 1) & 0xff;
+    in = new FSDataInputStream(new SeekableByteArrayInputStream(bytes));
+  }
+
+  private byte[] composeContent(String headerStr, String footerStr) throws CharacterCodingException {
+    ByteBuffer header = Text.encode(headerStr);
+    ByteBuffer footer = Text.encode(footerStr);
+    int headerLen = header.remaining();
+    int footerLen = footer.remaining() + 1;
+
+    ByteBuffer buf = ByteBuffer.allocate(headerLen + footerLen);
+
+    buf.put(header);
+    buf.put(footer);
+    buf.put((byte) footerLen);
+    return buf.array();
+  }
+
+  private static final class SeekableByteArrayInputStream extends ByteArrayInputStream
+          implements Seekable, PositionedReadable {
+
+    public SeekableByteArrayInputStream(byte[] buf) {
+      super(buf);
+    }
+
+    @Override
+    public void seek(long pos) throws IOException {
+      this.reset();
+      this.skip(pos);
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return pos;
+    }
+
+    @Override
+    public boolean seekToNewSource(long targetPos) throws IOException {
+      return false;
+    }
+
+    @Override
+    public int read(long position, byte[] buffer, int offset, int length)
+            throws IOException {
+      long oldPos = getPos();
+      int nread = -1;
+      try {
+        seek(position);
+        nread = read(buffer, offset, length);
+      } finally {
+        seek(oldPos);
+      }
+      return nread;
+    }
+
+    @Override
+    public void readFully(long position, byte[] buffer, int offset, int length)
+            throws IOException {
+      int nread = 0;
+      while (nread < length) {
+        int nbytes = read(position + nread, buffer, offset + nread, length - nread);
+        if (nbytes < 0) {
+          throw new EOFException("End of file reached before reading fully.");
+        }
+        nread += nbytes;
+      }
+    }
+
+    @Override
+    public void readFully(long position, byte[] buffer)
+            throws IOException {
+      readFully(position, buffer, 0, buffer.length);
+    }
+  }
+}