Index: ql/src/java/org/apache/hadoop/hive/ql/io/orc/OutStream.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/io/orc/OutStream.java (revision 1455674) +++ ql/src/java/org/apache/hadoop/hive/ql/io/orc/OutStream.java (working copy) @@ -45,17 +45,14 @@ this.bufferSize = bufferSize; this.codec = codec; this.receiver = receiver; - getNewInputBuffer(); } public void clear() throws IOException { - current.position(codec == null ? 0 : HEADER_SIZE); - if (compressed != null) { - compressed.clear(); - } - if (overflow != null) { - overflow.clear(); - } + uncompressedBytes = 0; + compressedBytes = 0; + compressed = null; + overflow = null; + current = null; } /** @@ -100,6 +97,9 @@ @Override public void write(int i) throws IOException { + if (current == null) { + getNewInputBuffer(); + } if (current.remaining() < 1) { spill(); } @@ -109,6 +109,9 @@ @Override public void write(byte[] bytes, int offset, int length) throws IOException { + if (current == null) { + getNewInputBuffer(); + } int remaining = Math.min(current.remaining(), length); current.put(bytes, offset, remaining); uncompressedBytes += remaining; @@ -125,7 +128,7 @@ private void spill() throws java.io.IOException { // if there isn't anything in the current buffer, don't spill - if (current.position() == (codec == null ? 0 : HEADER_SIZE)) { + if (current == null || current.position() == (codec == null ? 0 : HEADER_SIZE)) { return; } flip(); @@ -211,8 +214,7 @@ receiver.output(compressed); compressed = null; } - uncompressedBytes = 0; - compressedBytes = 0; + clear(); } @Override