You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by om...@apache.org on 2019/10/02 20:33:34 UTC

[orc] branch branch-1.6 updated (56e4019 -> 2f1cc76)

This is an automated email from the ASF dual-hosted git repository.

omalley pushed a change to branch branch-1.6
in repository https://gitbox.apache.org/repos/asf/orc.git.


    from 56e4019  ORC-504: Introduce reproducible java build.
     new 726be32  ORC-424: Enable findbugs check at test classes
     new 2f1cc76  ORC-554: Float to timestamp schema evolution should handle overflow.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 java/core/src/findbugs/exclude.xml                 |  23 ++
 .../apache/orc/impl/ConvertTreeReaderFactory.java  |  18 +-
 .../test/org/apache/orc/TestInMemoryKeystore.java  |  18 +-
 .../org/apache/orc/TestNewIntegerEncoding.java     |   8 -
 .../test/org/apache/orc/TestOrcDSTNoTimezone.java  |   3 -
 .../src/test/org/apache/orc/TestOrcNoTimezone.java |   3 -
 .../org/apache/orc/TestOrcNullOptimization.java    |   3 +-
 .../test/org/apache/orc/TestStringDictionary.java  |  14 +-
 .../src/test/org/apache/orc/TestVectorOrcFile.java |  87 +++--
 .../src/test/org/apache/orc/impl/TestInStream.java | 382 +++++++++++----------
 .../orc/impl/TestIntegerCompressionReader.java     |   8 +-
 .../test/org/apache/orc/impl/TestOutStream.java    | 197 +++++------
 .../orc/impl/TestPredicatePushDownBounds.java      |   6 -
 .../test/org/apache/orc/impl/TestReaderImpl.java   |  27 +-
 .../org/apache/orc/impl/TestRecordReaderImpl.java  |  17 +-
 .../orc/impl/TestRunLengthIntegerReader.java       |   8 +-
 .../org/apache/orc/impl/TestSchemaEvolution.java   | 113 +++++-
 .../apache/orc/impl/TestStringRedBlackTree.java    |  12 +-
 .../src/test/org/apache/orc/util/TestMurmur3.java  |  27 +-
 java/mapreduce/src/findbugs/exclude.xml            |   4 +
 .../org/apache/orc/mapred/TestOrcOutputFormat.java |   1 -
 java/pom.xml                                       |   3 +-
 .../src/test/org/apache/orc/impl/TestRLEv2.java    |  35 +-
 .../test/org/apache/orc/tools/TestFileDump.java    |  51 +--
 .../org/apache/orc/tools/TestJsonFileDump.java     |  21 +-
 .../apache/orc/tools/convert/TestCsvReader.java    |   7 -
 26 files changed, 615 insertions(+), 481 deletions(-)


[orc] 02/02: ORC-554: Float to timestamp schema evolution should handle overflow.

Posted by om...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

omalley pushed a commit to branch branch-1.6
in repository https://gitbox.apache.org/repos/asf/orc.git

commit 2f1cc766ee19e655bd7db16f36f6790f6ea93ccf
Author: Laszlo Bodor <bo...@gmail.com>
AuthorDate: Thu Sep 12 09:21:46 2019 +0200

    ORC-554: Float to timestamp schema evolution should handle overflow.
    
    Fixes #431
    
    Signed-off-by: Owen O'Malley <om...@apache.org>
---
 java/core/src/findbugs/exclude.xml                 |   6 --
 .../apache/orc/impl/ConvertTreeReaderFactory.java  |  18 +++-
 .../org/apache/orc/impl/TestSchemaEvolution.java   | 101 +++++++++++++++++++++
 3 files changed, 116 insertions(+), 9 deletions(-)

diff --git a/java/core/src/findbugs/exclude.xml b/java/core/src/findbugs/exclude.xml
index 76d395a..6112afd 100644
--- a/java/core/src/findbugs/exclude.xml
+++ b/java/core/src/findbugs/exclude.xml
@@ -60,11 +60,5 @@
   <Match>
     <Bug pattern="RCN_REDUNDANT_NULLCHECK_WOULD_HAVE_BEEN_A_NPE"/>
     <Class name="org.apache.orc.impl.TestSchemaEvolution"/>
-    <Method name="testEvolutionFromTimestamp"/>
-  </Match>
-  <Match>
-    <Bug pattern="RCN_REDUNDANT_NULLCHECK_WOULD_HAVE_BEEN_A_NPE"/>
-    <Class name="org.apache.orc.impl.TestSchemaEvolution"/>
-    <Method name="testEvolutionToTimestamp"/>
   </Match>
 </FindBugsFilter>
diff --git a/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java b/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
index a6c158b..1ea870a 100644
--- a/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
+++ b/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
@@ -1409,9 +1409,21 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
         seconds = SerializationUtils.convertFromUtc(local, seconds);
       }
       long wholeSec = (long) Math.floor(seconds);
-      timestampColVector.time[elementNum] = wholeSec * 1000;
-      timestampColVector.nanos[elementNum] =
-          1_000_000 * (int) Math.round((seconds - wholeSec) * 1000);
+
+      // overflow
+      double doubleMillis = seconds * 1000;
+      long millis = wholeSec * 1000;
+      if (doubleMillis > Long.MAX_VALUE || doubleMillis < Long.MIN_VALUE ||
+              ((millis >= 0) != (doubleMillis >= 0))) {
+        timestampColVector.time[elementNum] = 0L;
+        timestampColVector.nanos[elementNum] = 0;
+        timestampColVector.isNull[elementNum] = true;
+        timestampColVector.noNulls = false;
+      } else {
+        timestampColVector.time[elementNum] = wholeSec * 1000;
+        timestampColVector.nanos[elementNum] =
+            1_000_000 * (int) Math.round((seconds - wholeSec) * 1000);
+      }
     }
 
     @Override
diff --git a/java/core/src/test/org/apache/orc/impl/TestSchemaEvolution.java b/java/core/src/test/org/apache/orc/impl/TestSchemaEvolution.java
index 1dda07e..304ee4b 100644
--- a/java/core/src/test/org/apache/orc/impl/TestSchemaEvolution.java
+++ b/java/core/src/test/org/apache/orc/impl/TestSchemaEvolution.java
@@ -2263,4 +2263,105 @@ public class TestSchemaEvolution {
       TimeZone.setDefault(oldDefault);
     }
   }
+
+  @Test
+  public void doubleToTimeStampOverflow() throws Exception {
+    floatAndDoubleToTimeStampOverflow("double",
+        340282347000000000000000000000000000000000.0,
+        1e16,
+        9223372036854775.0,
+        9000000000000000.1,
+        10000000000.0,
+        10000000.123,
+        -1000000.123,
+        -10000000000.0,
+        -9000000000000000.1,
+        -9223372036854775.0,
+        -1e16,
+        -340282347000000000000000000000000000000000.0);
+  }
+
+  @Test
+  public void floatToTimeStampPositiveOverflow() throws Exception {
+    floatAndDoubleToTimeStampOverflow("float",
+        340282347000000000000000000000000000000000.0,
+        1e16,
+        9223372036854775.0,
+        9000000000000000.1,
+        10000000000.0,
+        10000000.123,
+        -1000000.123,
+        -10000000000.0,
+        -9000000000000000.1,
+        -9223372036854775.0,
+        -1e16,
+        -340282347000000000000000000000000000000000.0);
+  }
+
+  private void floatAndDoubleToTimeStampOverflow(String typeInFileSchema,
+                                                 double... values) throws Exception {
+    boolean isFloat = typeInFileSchema.equals("float");
+    TypeDescription fileSchema =
+        TypeDescription.fromString(String.format("struct<c1:%s>", typeInFileSchema));
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(fileSchema)
+            .stripeSize(10000)
+            .useUTCTimestamp(true));
+
+    VectorizedRowBatch batch = fileSchema.createRowBatchV2();
+    DoubleColumnVector fl1 = (DoubleColumnVector) batch.cols[0];
+
+    for (double v : values) {
+      int row = batch.size++;
+      fl1.vector[row] = v;
+
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size != 0) {
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+
+    TypeDescription readerSchema = TypeDescription.fromString("struct<c1:timestamp>");
+    VectorizedRowBatch batchTimeStamp = readerSchema.createRowBatchV2();
+    TimestampColumnVector t1 = (TimestampColumnVector) batchTimeStamp.cols[0];
+
+    OrcFile.ReaderOptions options = OrcFile
+                                        .readerOptions(conf)
+                                        .useUTCTimestamp(true);
+
+    try (Reader reader = OrcFile.createReader(testFilePath, options);
+         RecordReader rows = reader.rows(reader.options().schema(readerSchema))) {
+      int value = 0;
+      while (value < values.length) {
+        assertTrue("value " + value, rows.nextBatch(batchTimeStamp));
+        for(int row=0; row < batchTimeStamp.size; ++row) {
+          double expected = values[value + row];
+          String rowName = String.format("value %d", value + row);
+          boolean isPositive = ((long)Math.floor(expected) * 1000) >= 0;
+          if (expected * 1000 < Long.MIN_VALUE ||
+                  expected * 1000 > Long.MAX_VALUE ||
+                  ((expected >= 0) != isPositive)) {
+            assertFalse(rowName, t1.noNulls);
+            assertTrue(rowName, t1.isNull[row]);
+          } else {
+            double actual = t1.time[row] / 1000.0 + t1.nanos[row] / 1_000_000_000.0;
+            assertEquals(rowName, expected, actual,
+                Math.abs(expected * (isFloat ? 0.000001 : 0.0000000000000001)));
+            assertFalse(rowName, t1.isNull[row]);
+            assertTrue(String.format(
+                "%s nanos should be 0 to 1,000,000,000 instead it's: %d",
+                rowName, t1.nanos[row]),
+                t1.nanos[row] >= 0 && t1.nanos[row] < 1_000_000_000);
+          }
+        }
+        value += batchTimeStamp.size;
+      }
+      assertFalse(rows.nextBatch(batchTimeStamp));
+    }
+  }
 }


[orc] 01/02: ORC-424: Enable findbugs check at test classes

Posted by om...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

omalley pushed a commit to branch branch-1.6
in repository https://gitbox.apache.org/repos/asf/orc.git

commit 726be322beb257b084c1e81dacd35e99ad348ded
Author: Yukihiro Okada <ca...@gmail.com>
AuthorDate: Wed Sep 4 21:53:52 2019 +0900

    ORC-424: Enable findbugs check at test classes
    
    Fixes #428
    
    Signed-off-by: Owen O'Malley <om...@apache.org>
---
 java/core/src/findbugs/exclude.xml                 |  29 ++
 .../test/org/apache/orc/TestInMemoryKeystore.java  |  18 +-
 .../org/apache/orc/TestNewIntegerEncoding.java     |   8 -
 .../test/org/apache/orc/TestOrcDSTNoTimezone.java  |   3 -
 .../src/test/org/apache/orc/TestOrcNoTimezone.java |   3 -
 .../org/apache/orc/TestOrcNullOptimization.java    |   3 +-
 .../test/org/apache/orc/TestStringDictionary.java  |  14 +-
 .../src/test/org/apache/orc/TestVectorOrcFile.java |  87 +++--
 .../src/test/org/apache/orc/impl/TestInStream.java | 382 +++++++++++----------
 .../orc/impl/TestIntegerCompressionReader.java     |   8 +-
 .../test/org/apache/orc/impl/TestOutStream.java    | 197 +++++------
 .../orc/impl/TestPredicatePushDownBounds.java      |   6 -
 .../test/org/apache/orc/impl/TestReaderImpl.java   |  27 +-
 .../org/apache/orc/impl/TestRecordReaderImpl.java  |  17 +-
 .../orc/impl/TestRunLengthIntegerReader.java       |   8 +-
 .../org/apache/orc/impl/TestSchemaEvolution.java   |  12 +-
 .../apache/orc/impl/TestStringRedBlackTree.java    |  12 +-
 .../src/test/org/apache/orc/util/TestMurmur3.java  |  27 +-
 java/mapreduce/src/findbugs/exclude.xml            |   4 +
 .../org/apache/orc/mapred/TestOrcOutputFormat.java |   1 -
 java/pom.xml                                       |   3 +-
 .../src/test/org/apache/orc/impl/TestRLEv2.java    |  35 +-
 .../test/org/apache/orc/tools/TestFileDump.java    |  51 +--
 .../org/apache/orc/tools/TestJsonFileDump.java     |  21 +-
 .../apache/orc/tools/convert/TestCsvReader.java    |   7 -
 25 files changed, 505 insertions(+), 478 deletions(-)

diff --git a/java/core/src/findbugs/exclude.xml b/java/core/src/findbugs/exclude.xml
index a78dda7..76d395a 100644
--- a/java/core/src/findbugs/exclude.xml
+++ b/java/core/src/findbugs/exclude.xml
@@ -31,6 +31,15 @@
     <Bug pattern="MS_PKGPROTECT"/>
     <Class name="org.apache.orc.impl.RecordReaderImpl$SargApplier"/>
   </Match>
+  <Match>
+    <Bug pattern="CNT_ROUGH_CONSTANT_VALUE,IM_BAD_CHECK_FOR_ODD"/>
+    <Class name="org.apache.orc.TestVectorOrcFile"/>
+  </Match>
+  <Match>
+    <Bug pattern="RR_NOT_CHECKED"/>
+    <Class name="org.apache.orc.impl.TestInStream"/>
+    <Method name="testCorruptStream"/>
+  </Match>
   <!-- Java's try with resources causes a false positive.
        See https://github.com/SERG-Delft/jpacman/pull/27 . -->
   <Match>
@@ -38,4 +47,24 @@
     <Class name="org.apache.orc.impl.OrcAcidUtils"/>
     <Method name="getLastFlushLength"/>
   </Match>
+  <Match>
+    <Bug pattern="RCN_REDUNDANT_NULLCHECK_WOULD_HAVE_BEEN_A_NPE"/>
+    <Class name="org.apache.orc.impl.TestReaderImpl"/>
+    <Method name="testOptionSafety"/>
+  </Match>
+  <Match>
+    <Bug pattern="RCN_REDUNDANT_NULLCHECK_WOULD_HAVE_BEEN_A_NPE"/>
+    <Class name="org.apache.orc.TestVectorOrcFile"/>
+    <Method name="testZstd"/>
+  </Match>
+  <Match>
+    <Bug pattern="RCN_REDUNDANT_NULLCHECK_WOULD_HAVE_BEEN_A_NPE"/>
+    <Class name="org.apache.orc.impl.TestSchemaEvolution"/>
+    <Method name="testEvolutionFromTimestamp"/>
+  </Match>
+  <Match>
+    <Bug pattern="RCN_REDUNDANT_NULLCHECK_WOULD_HAVE_BEEN_A_NPE"/>
+    <Class name="org.apache.orc.impl.TestSchemaEvolution"/>
+    <Method name="testEvolutionToTimestamp"/>
+  </Match>
 </FindBugsFilter>
diff --git a/java/core/src/test/org/apache/orc/TestInMemoryKeystore.java b/java/core/src/test/org/apache/orc/TestInMemoryKeystore.java
index aabc4dd..3f9ca0b 100644
--- a/java/core/src/test/org/apache/orc/TestInMemoryKeystore.java
+++ b/java/core/src/test/org/apache/orc/TestInMemoryKeystore.java
@@ -17,6 +17,7 @@
  */
 package org.apache.orc;
 
+import java.nio.charset.StandardCharsets;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.orc.impl.HadoopShims;
 import org.apache.orc.impl.LocalKey;
@@ -46,9 +47,9 @@ public class TestInMemoryKeystore {
     Random random = new Random(2);
     memoryKeystore =
         new InMemoryKeystore(random)
-            .addKey("key128", EncryptionAlgorithm.AES_CTR_128, "123".getBytes())
-            .addKey("key256", EncryptionAlgorithm.AES_CTR_256, "secret123".getBytes())
-            .addKey("key256short", EncryptionAlgorithm.AES_CTR_256, "5".getBytes());
+            .addKey("key128", EncryptionAlgorithm.AES_CTR_128, "123".getBytes(StandardCharsets.UTF_8))
+            .addKey("key256", EncryptionAlgorithm.AES_CTR_256, "secret123".getBytes(StandardCharsets.UTF_8))
+            .addKey("key256short", EncryptionAlgorithm.AES_CTR_256, "5".getBytes(StandardCharsets.UTF_8));
 
   }
 
@@ -134,7 +135,7 @@ public class TestInMemoryKeystore {
 
     Assert.assertEquals(0,
         memoryKeystore.getCurrentKeyVersion("key128").getVersion());
-    memoryKeystore.addKey("key128", 1, EncryptionAlgorithm.AES_CTR_128, "NewSecret".getBytes());
+    memoryKeystore.addKey("key128", 1, EncryptionAlgorithm.AES_CTR_128, "NewSecret".getBytes(StandardCharsets.UTF_8));
     Assert.assertEquals(1,
         memoryKeystore.getCurrentKeyVersion("key128").getVersion());
   }
@@ -143,7 +144,7 @@ public class TestInMemoryKeystore {
   public void testDuplicateKeyNames() {
     try {
       memoryKeystore.addKey("key128", 0, EncryptionAlgorithm.AES_CTR_128,
-          "exception".getBytes());
+          "exception".getBytes(StandardCharsets.UTF_8));
       Assert.fail("Keys with same name cannot be added.");
     } catch (IOException e) {
       Assert.assertTrue(e.toString().contains("equal or higher version"));
@@ -162,20 +163,21 @@ public class TestInMemoryKeystore {
   public void testMultipleVersion() throws IOException {
     Assert.assertEquals(0,
         memoryKeystore.getCurrentKeyVersion("key256").getVersion());
-    memoryKeystore.addKey("key256", 1, EncryptionAlgorithm.AES_CTR_256, "NewSecret".getBytes());
+    memoryKeystore.addKey("key256", 1, EncryptionAlgorithm.AES_CTR_256,
+        "NewSecret".getBytes(StandardCharsets.UTF_8));
     Assert.assertEquals(1,
         memoryKeystore.getCurrentKeyVersion("key256").getVersion());
 
     try {
       memoryKeystore.addKey("key256", 1, EncryptionAlgorithm.AES_CTR_256,
-          "BadSecret".getBytes());
+          "BadSecret".getBytes(StandardCharsets.UTF_8));
       Assert.fail("Keys with smaller version should not be added.");
     } catch (final IOException e) {
       Assert.assertTrue(e.toString().contains("equal or higher version"));
     }
 
     memoryKeystore.addKey("key256", 2, EncryptionAlgorithm.AES_CTR_256,
-        "NewerSecret".getBytes());
+        "NewerSecret".getBytes(StandardCharsets.UTF_8));
     Assert.assertEquals(2,
         memoryKeystore.getCurrentKeyVersion("key256").getVersion());
 
diff --git a/java/core/src/test/org/apache/orc/TestNewIntegerEncoding.java b/java/core/src/test/org/apache/orc/TestNewIntegerEncoding.java
index 3dce13c..2fe058f 100644
--- a/java/core/src/test/org/apache/orc/TestNewIntegerEncoding.java
+++ b/java/core/src/test/org/apache/orc/TestNewIntegerEncoding.java
@@ -59,14 +59,6 @@ public class TestNewIntegerEncoding {
     return Arrays.asList(data);
   }
 
-  public static class TSRow {
-    Timestamp ts;
-
-    public TSRow(Timestamp ts) {
-      this.ts = ts;
-    }
-  }
-
   public static TypeDescription getRowSchema() {
     return TypeDescription.createStruct()
         .addField("int1", TypeDescription.createInt())
diff --git a/java/core/src/test/org/apache/orc/TestOrcDSTNoTimezone.java b/java/core/src/test/org/apache/orc/TestOrcDSTNoTimezone.java
index 7730aa5..e756670 100644
--- a/java/core/src/test/org/apache/orc/TestOrcDSTNoTimezone.java
+++ b/java/core/src/test/org/apache/orc/TestOrcDSTNoTimezone.java
@@ -68,9 +68,6 @@ public class TestOrcDSTNoTimezone {
     return result;
   }
 
-  @Rule
-  public TestName testCaseName = new TestName();
-
   @Before
   public void openFileSystem() throws Exception {
     conf = new Configuration();
diff --git a/java/core/src/test/org/apache/orc/TestOrcNoTimezone.java b/java/core/src/test/org/apache/orc/TestOrcNoTimezone.java
index 48208bb..c17ea28 100644
--- a/java/core/src/test/org/apache/orc/TestOrcNoTimezone.java
+++ b/java/core/src/test/org/apache/orc/TestOrcNoTimezone.java
@@ -67,9 +67,6 @@ public class TestOrcNoTimezone {
     return result;
   }
 
-  @Rule
-  public TestName testCaseName = new TestName();
-
   @Before
   public void openFileSystem() throws Exception {
     conf = new Configuration();
diff --git a/java/core/src/test/org/apache/orc/TestOrcNullOptimization.java b/java/core/src/test/org/apache/orc/TestOrcNullOptimization.java
index de22301..02e601f 100644
--- a/java/core/src/test/org/apache/orc/TestOrcNullOptimization.java
+++ b/java/core/src/test/org/apache/orc/TestOrcNullOptimization.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertArrayEquals;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.List;
 import java.util.Random;
 
@@ -80,7 +81,7 @@ public class TestOrcNullOptimization {
       bColumn.noNulls = false;
       bColumn.isNull[row] = true;
     } else {
-      bColumn.setVal(row, b.getBytes());
+      bColumn.setVal(row, b.getBytes(StandardCharsets.UTF_8));
     }
     if (c == null) {
       cColumn.noNulls = false;
diff --git a/java/core/src/test/org/apache/orc/TestStringDictionary.java b/java/core/src/test/org/apache/orc/TestStringDictionary.java
index b0d39a0..b301dd6 100644
--- a/java/core/src/test/org/apache/orc/TestStringDictionary.java
+++ b/java/core/src/test/org/apache/orc/TestStringDictionary.java
@@ -83,7 +83,7 @@ public class TestStringDictionary {
         writer.addRowBatch(batch);
         batch.reset();
       }
-      col.setVal(batch.size++, String.valueOf(i).getBytes());
+      col.setVal(batch.size++, String.valueOf(i).getBytes(StandardCharsets.UTF_8));
     }
     writer.addRowBatch(batch);
     writer.close();
@@ -132,7 +132,7 @@ public class TestStringDictionary {
         writer.addRowBatch(batch);
         batch.reset();
       }
-      col.setVal(batch.size++, String.valueOf(input[i]).getBytes());
+      col.setVal(batch.size++, String.valueOf(input[i]).getBytes(StandardCharsets.UTF_8));
     }
     writer.addRowBatch(batch);
     writer.close();
@@ -309,7 +309,7 @@ public class TestStringDictionary {
         writer.addRowBatch(batch);
         batch.reset();
       }
-      string.setVal(batch.size++, String.valueOf(i).getBytes());
+      string.setVal(batch.size++, String.valueOf(i).getBytes(StandardCharsets.UTF_8));
     }
     writer.addRowBatch(batch);
     writer.close();
@@ -360,7 +360,7 @@ public class TestStringDictionary {
         writer.addRowBatch(batch);
         batch.reset();
       }
-      string.setVal(batch.size++, String.valueOf(input[i]).getBytes());
+      string.setVal(batch.size++, String.valueOf(input[i]).getBytes(StandardCharsets.UTF_8));
     }
     writer.addRowBatch(batch);
     writer.close();
@@ -404,7 +404,7 @@ public class TestStringDictionary {
         writer.addRowBatch(batch);
         batch.reset();
       }
-      string.setVal(batch.size++, String.valueOf(i).getBytes());
+      string.setVal(batch.size++, String.valueOf(i).getBytes(StandardCharsets.UTF_8));
     }
     writer.addRowBatch(batch);
     writer.close();
@@ -462,8 +462,8 @@ public class TestStringDictionary {
         writer.addRowBatch(batch);
         batch.reset();
       }
-      shortStringColumnVector.setVal(batch.size, SHORT_STRING_VALUE.getBytes());
-      longStringColumnVector.setVal( batch.size, LONG_STRING_VALUE.getBytes());
+      shortStringColumnVector.setVal(batch.size, SHORT_STRING_VALUE.getBytes(StandardCharsets.UTF_8));
+      longStringColumnVector.setVal( batch.size, LONG_STRING_VALUE.getBytes(StandardCharsets.UTF_8));
       ++batch.size;
     }
     writer.addRowBatch(batch);
diff --git a/java/core/src/test/org/apache/orc/TestVectorOrcFile.java b/java/core/src/test/org/apache/orc/TestVectorOrcFile.java
index d1bb0d9..6226719 100644
--- a/java/core/src/test/org/apache/orc/TestVectorOrcFile.java
+++ b/java/core/src/test/org/apache/orc/TestVectorOrcFile.java
@@ -453,11 +453,11 @@ public class TestVectorOrcFile {
     field1.setVal(2, bytesArray(0, 1, 2, 3, 4, 5));
     field1.noNulls = false;
     field1.isNull[3] = true;
-    field2.setVal(0, "foo".getBytes());
-    field2.setVal(1, "bar".getBytes());
+    field2.setVal(0, "foo".getBytes(StandardCharsets.UTF_8));
+    field2.setVal(1, "bar".getBytes(StandardCharsets.UTF_8));
     field2.noNulls = false;
     field2.isNull[2] = true;
-    field2.setVal(3, "hi".getBytes());
+    field2.setVal(3, "hi".getBytes(StandardCharsets.UTF_8));
     writer.addRowBatch(batch);
     writer.close();
     schema = writer.getSchema();
@@ -577,14 +577,14 @@ public class TestVectorOrcFile {
       if (b >= 5) {
         if (b >= 10) {
           field1.vector[0] = 3;
-          field2.setVal(0, "three".getBytes());
+          field2.setVal(0, "three".getBytes(StandardCharsets.UTF_8));
         } else {
           field1.vector[0] = 2;
-          field2.setVal(0, "two".getBytes());
+          field2.setVal(0, "two".getBytes(StandardCharsets.UTF_8));
         }
       } else {
         field1.vector[0] = 1;
-        field2.setVal(0, "one".getBytes());
+        field2.setVal(0, "one".getBytes(StandardCharsets.UTF_8));
       }
       writer.addRowBatch(batch);
     }
@@ -658,7 +658,7 @@ public class TestVectorOrcFile {
                                int i, String value) {
     ((LongColumnVector) inner.fields[0]).vector[rowId] = i;
     if (value != null) {
-      ((BytesColumnVector) inner.fields[1]).setVal(rowId, value.getBytes());
+      ((BytesColumnVector) inner.fields[1]).setVal(rowId, value.getBytes(StandardCharsets.UTF_8));
     } else {
       inner.fields[1].isNull[rowId] = true;
       inner.fields[1].noNulls = false;
@@ -727,7 +727,7 @@ public class TestVectorOrcFile {
       map.offsets[rowId] = offset;
 
       for (Map.Entry<String, InnerStruct> entry : value.entrySet()) {
-        ((BytesColumnVector) map.keys).setVal(offset, entry.getKey().getBytes());
+        ((BytesColumnVector) map.keys).setVal(offset, entry.getKey().getBytes(StandardCharsets.UTF_8));
         InnerStruct inner = entry.getValue();
         setInner((StructColumnVector) map.values, offset, inner.int1,
             inner.string1.toString());
@@ -800,7 +800,7 @@ public class TestVectorOrcFile {
       batch.cols[7].noNulls = false;
     }
     if (s2 != null) {
-      ((BytesColumnVector) batch.cols[8]).setVal(rowId, s2.getBytes());
+      ((BytesColumnVector) batch.cols[8]).setVal(rowId, s2.getBytes(StandardCharsets.UTF_8));
     } else {
       batch.cols[8].isNull[rowId] = true;
       batch.cols[8].noNulls = false;
@@ -1260,7 +1260,7 @@ public class TestVectorOrcFile {
         }
         first = false;
         ((LongColumnVector) batch.cols[0]).vector[r] = x;
-        ((BytesColumnVector) batch.cols[1]).setVal(r, y.getBytes());
+        ((BytesColumnVector) batch.cols[1]).setVal(r, y.getBytes(StandardCharsets.UTF_8));
       }
       writer.addRowBatch(batch);
     }
@@ -1500,7 +1500,7 @@ public class TestVectorOrcFile {
         }
       } else if (tag == 1) {
         if (s != null) {
-          ((BytesColumnVector) union.fields[tag]).setVal(rowId, s.getBytes());
+          ((BytesColumnVector) union.fields[tag]).setVal(rowId, s.getBytes(StandardCharsets.UTF_8));
         } else {
           union.fields[tag].isNull[rowId] = true;
           union.fields[tag].noNulls = false;
@@ -2232,7 +2232,7 @@ public class TestVectorOrcFile {
       for (int r=0; r < size; ++r) {
         ((LongColumnVector) batch.cols[0]).vector[r] = rand.nextInt();
         ((BytesColumnVector) batch.cols[1]).setVal(r,
-            Integer.toHexString(rand.nextInt()).getBytes());
+            Integer.toHexString(rand.nextInt()).getBytes(StandardCharsets.UTF_8));
       }
       writer.addRowBatch(batch);
     }
@@ -2262,7 +2262,7 @@ public class TestVectorOrcFile {
     for(int i=0; i < 10000; ++i) {
       ((LongColumnVector) batch.cols[0]).vector[0] = rand.nextInt();
       ((BytesColumnVector) batch.cols[1])
-          .setVal(0, Integer.toBinaryString(rand.nextInt()).getBytes());
+          .setVal(0, Integer.toBinaryString(rand.nextInt()).getBytes(StandardCharsets.UTF_8));
       writer.addRowBatch(batch);
     }
     writer.close();
@@ -2452,7 +2452,6 @@ public class TestVectorOrcFile {
   private static class MyMemoryManager implements MemoryManager {
     double rate;
     Path path = null;
-    long lastAllocation = 0;
     int rows = 0;
     Callback callback;
 
@@ -2464,14 +2463,12 @@ public class TestVectorOrcFile {
     public void addWriter(Path path, long requestedAllocation,
                    Callback callback) {
       this.path = path;
-      this.lastAllocation = requestedAllocation;
       this.callback = callback;
     }
 
     @Override
     public synchronized void removeWriter(Path path) {
       this.path = null;
-      this.lastAllocation = 0;
     }
 
 
@@ -2505,7 +2502,7 @@ public class TestVectorOrcFile {
     for(int i=0; i < 2500; ++i) {
       ((LongColumnVector) batch.cols[0]).vector[0] = i * 300;
       ((BytesColumnVector) batch.cols[1]).setVal(0,
-          Integer.toHexString(10*i).getBytes());
+          Integer.toHexString(10*i).getBytes(StandardCharsets.UTF_8));
       writer.addRowBatch(batch);
     }
     writer.close();
@@ -2542,7 +2539,7 @@ public class TestVectorOrcFile {
     for(int i=0; i < 2500; ++i) {
       ((LongColumnVector) batch.cols[0]).vector[0] = i * 300;
       ((BytesColumnVector) batch.cols[1]).setVal(0,
-          Integer.toHexString(10*i).getBytes());
+          Integer.toHexString(10*i).getBytes(StandardCharsets.UTF_8));
       writer.addRowBatch(batch);
     }
     writer.close();
@@ -2579,7 +2576,7 @@ public class TestVectorOrcFile {
     for(int i=0; i < 3500; ++i) {
       ((LongColumnVector) batch.cols[0]).vector[i] = i * 300;
       ((BytesColumnVector) batch.cols[1]).setVal(i,
-          Integer.toHexString(10*i).getBytes());
+          Integer.toHexString(10*i).getBytes(StandardCharsets.UTF_8));
     }
     writer.addRowBatch(batch);
     writer.close();
@@ -2713,7 +2710,7 @@ public class TestVectorOrcFile {
     for(int c =0; c < batch.cols.length; ++c) {
       batch.cols[c].isNull[0] = false;
     }
-    ((BytesColumnVector) batch.cols[0]).setVal(0, "Horton".getBytes());
+    ((BytesColumnVector) batch.cols[0]).setVal(0, "Horton".getBytes(StandardCharsets.UTF_8));
     ((LongColumnVector) batch.cols[1]).vector[0] = 1;
     ((LongColumnVector) batch.cols[2]).vector[0] = 130;
     ((LongColumnVector) batch.cols[3]).vector[0] = 0x123456789abcdef0L;
@@ -2726,9 +2723,9 @@ public class TestVectorOrcFile {
             999999999));
     ((DecimalColumnVector) batch.cols[8]).vector[0] =
         new HiveDecimalWritable("1.234567");
-    ((BytesColumnVector) batch.cols[9]).setVal(0, "Echelon".getBytes());
-    ((BytesColumnVector) batch.cols[10]).setVal(0, "Juggernaut".getBytes());
-    ((BytesColumnVector) batch.cols[11]).setVal(0, "Dreadnaught".getBytes());
+    ((BytesColumnVector) batch.cols[9]).setVal(0, "Echelon".getBytes(StandardCharsets.UTF_8));
+    ((BytesColumnVector) batch.cols[10]).setVal(0, "Juggernaut".getBytes(StandardCharsets.UTF_8));
+    ((BytesColumnVector) batch.cols[11]).setVal(0, "Dreadnaught".getBytes(StandardCharsets.UTF_8));
     ((LongColumnVector) ((StructColumnVector) batch.cols[12]).fields[0])
         .vector[0] = 123;
     ((UnionColumnVector) batch.cols[13]).tags[0] = 1;
@@ -2743,13 +2740,13 @@ public class TestVectorOrcFile {
     ((MapColumnVector) batch.cols[15]).lengths[0] = 3;
     ((MapColumnVector) batch.cols[15]).values.isRepeating = true;
     ((BytesColumnVector) ((MapColumnVector) batch.cols[15]).keys)
-        .setVal(0, "ORC".getBytes());
+        .setVal(0, "ORC".getBytes(StandardCharsets.UTF_8));
     ((BytesColumnVector) ((MapColumnVector) batch.cols[15]).keys)
-        .setVal(1, "Hive".getBytes());
+        .setVal(1, "Hive".getBytes(StandardCharsets.UTF_8));
     ((BytesColumnVector) ((MapColumnVector) batch.cols[15]).keys)
-        .setVal(2, "LLAP".getBytes());
+        .setVal(2, "LLAP".getBytes(StandardCharsets.UTF_8));
     ((BytesColumnVector) ((MapColumnVector) batch.cols[15]).values)
-        .setVal(0, "fast".getBytes());
+        .setVal(0, "fast".getBytes(StandardCharsets.UTF_8));
     writer.addRowBatch(batch);
 
     // write 1024 null without repeat
@@ -2768,7 +2765,7 @@ public class TestVectorOrcFile {
     ((MapColumnVector) batch.cols[15]).values.ensureSize(3 * 1024, false);
     for(int r=0; r < 1024; ++r) {
       ((BytesColumnVector) batch.cols[0]).setVal(r,
-          Integer.toHexString(r).getBytes());
+          Integer.toHexString(r).getBytes(StandardCharsets.UTF_8));
       ((LongColumnVector) batch.cols[1]).vector[r] = r % 2;
       ((LongColumnVector) batch.cols[2]).vector[r] = (r % 255);
       ((LongColumnVector) batch.cols[3]).vector[r] = 31415L * r;
@@ -2782,11 +2779,11 @@ public class TestVectorOrcFile {
       ((DecimalColumnVector) batch.cols[8]).vector[r] =
           new HiveDecimalWritable("1.234567");
       ((BytesColumnVector) batch.cols[9]).setVal(r,
-          Integer.toString(r).getBytes());
+          Integer.toString(r).getBytes(StandardCharsets.UTF_8));
       ((BytesColumnVector) batch.cols[10]).setVal(r,
-          Integer.toHexString(r).getBytes());
+          Integer.toHexString(r).getBytes(StandardCharsets.UTF_8));
       ((BytesColumnVector) batch.cols[11]).setVal(r,
-          Integer.toHexString(r * 128).getBytes());
+          Integer.toHexString(r * 128).getBytes(StandardCharsets.UTF_8));
       ((LongColumnVector) ((StructColumnVector) batch.cols[12]).fields[0])
           .vector[r] = r + 13;
       ((UnionColumnVector) batch.cols[13]).tags[r] = 1;
@@ -2802,9 +2799,9 @@ public class TestVectorOrcFile {
       ((MapColumnVector) batch.cols[15]).lengths[r] = 3;
       for(int i=0; i < 3; ++i) {
         ((BytesColumnVector) ((MapColumnVector) batch.cols[15]).keys)
-            .setVal(3 * r + i, Integer.toHexString(3 * r + i).getBytes());
+            .setVal(3 * r + i, Integer.toHexString(3 * r + i).getBytes(StandardCharsets.UTF_8));
         ((BytesColumnVector) ((MapColumnVector) batch.cols[15]).values)
-            .setVal(3 * r + i, Integer.toString(3 * r + i).getBytes());
+            .setVal(3 * r + i, Integer.toString(3 * r + i).getBytes(StandardCharsets.UTF_8));
       }
     }
     writer.addRowBatch(batch);
@@ -2974,7 +2971,7 @@ public class TestVectorOrcFile {
     }
     if (vector.noNulls || !vector.isNull[row]) {
       return new String(vector.vector[row], vector.start[row],
-          vector.length[row]);
+          vector.length[row], StandardCharsets.UTF_8);
     } else {
       return null;
     }
@@ -2994,11 +2991,11 @@ public class TestVectorOrcFile {
     VectorizedRowBatch batch = schema.createRowBatch();
     batch.size = 4;
     for(int c=0; c < batch.cols.length; ++c) {
-      ((BytesColumnVector) batch.cols[c]).setVal(0, "".getBytes());
-      ((BytesColumnVector) batch.cols[c]).setVal(1, "xyz".getBytes());
-      ((BytesColumnVector) batch.cols[c]).setVal(2, "0123456789".getBytes());
+      ((BytesColumnVector) batch.cols[c]).setVal(0, "".getBytes(StandardCharsets.UTF_8));
+      ((BytesColumnVector) batch.cols[c]).setVal(1, "xyz".getBytes(StandardCharsets.UTF_8));
+      ((BytesColumnVector) batch.cols[c]).setVal(2, "0123456789".getBytes(StandardCharsets.UTF_8));
       ((BytesColumnVector) batch.cols[c]).setVal(3,
-          "0123456789abcdef".getBytes());
+          "0123456789abcdef".getBytes(StandardCharsets.UTF_8));
     }
     writer.addRowBatch(batch);
     writer.close();
@@ -3047,11 +3044,11 @@ public class TestVectorOrcFile {
     batch.size = 1024;
     for(int r=0; r < batch.size; ++r) {
       ((BytesColumnVector) batch.cols[0]).setVal(r,
-          Integer.toString(r * 10001).getBytes());
+          Integer.toString(r * 10001).getBytes(StandardCharsets.UTF_8));
     }
     writer.addRowBatch(batch);
     batch.cols[0].isRepeating = true;
-    ((BytesColumnVector) batch.cols[0]).setVal(0, "Halloween".getBytes());
+    ((BytesColumnVector) batch.cols[0]).setVal(0, "Halloween".getBytes(StandardCharsets.UTF_8));
     writer.addRowBatch(batch);
     writer.close();
 
@@ -3360,7 +3357,7 @@ public class TestVectorOrcFile {
     list2.offsets[1] = 2000;
     list2.lengths[1] = 3000;
     for(int v=0; v < 5000; ++v) {
-      byte[] bytes = Long.toHexString(v).getBytes();
+      byte[] bytes = Long.toHexString(v).getBytes(StandardCharsets.UTF_8);
       str.setVal(v, bytes);
       bin.setVal(v, bytes);
     }
@@ -3920,8 +3917,8 @@ public class TestVectorOrcFile {
     Assume.assumeTrue(fileFormat == OrcFile.Version.V_0_11);
     Path zeroFile = new Path(exampleDir, "version1999.orc");
     try {
-      Reader reader = OrcFile.createReader(zeroFile, OrcFile.readerOptions(conf));
-      assertTrue("no exception for bad verion", false);
+      OrcFile.createReader(zeroFile, OrcFile.readerOptions(conf));
+      assertTrue("no exception for bad version", false);
     } catch (UnknownFormatException uf) {
       assertEquals("path is correct", "version1999.orc", uf.getPath().getName());
       assertEquals("19.99", uf.getVersionString());
@@ -3984,7 +3981,7 @@ public class TestVectorOrcFile {
       ((LongColumnVector) batch.cols[0]).vector[i] = i;
       ((LongColumnVector)((StructColumnVector) batch.cols[1]).fields[0]).vector[i] = i * 300;
       ((BytesColumnVector)((StructColumnVector) batch.cols[1]).fields[1]).setVal(i,
-              Integer.toHexString(10*i).getBytes());
+              Integer.toHexString(10*i).getBytes(StandardCharsets.UTF_8));
     }
     writer.addRowBatch(batch);
     writer.close();
@@ -4315,7 +4312,7 @@ public class TestVectorOrcFile {
             expected = Long.toString(b % 2);
             actual = row -> Long.toString(((LongColumnVector) batch.cols[4]).vector[row]);
             break;
-          case 5:
+          default:
             expected = Integer.toString(b);
             actual = row -> ((BytesColumnVector) batch.cols[5]).toString(row);
             break;
diff --git a/java/core/src/test/org/apache/orc/impl/TestInStream.java b/java/core/src/test/org/apache/orc/impl/TestInStream.java
index f09169d..9c6490c 100644
--- a/java/core/src/test/org/apache/orc/impl/TestInStream.java
+++ b/java/core/src/test/org/apache/orc/impl/TestInStream.java
@@ -18,6 +18,9 @@
 
 package org.apache.orc.impl;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
@@ -27,7 +30,7 @@ import java.security.Key;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-
+import javax.crypto.spec.SecretKeySpec;
 import org.apache.hadoop.hive.common.io.DiskRangeList;
 import org.apache.orc.CompressionCodec;
 import org.apache.orc.EncryptionAlgorithm;
@@ -36,11 +39,6 @@ import org.apache.orc.PhysicalWriter;
 import org.apache.orc.impl.writer.StreamOptions;
 import org.junit.Test;
 
-import javax.crypto.spec.SecretKeySpec;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 public class TestInStream {
 
   public static class OutputCollector implements PhysicalWriter.OutputReceiver {
@@ -92,13 +90,14 @@ public class TestInStream {
 
   static byte[] getUncompressed(PositionCollector[] positions) throws IOException {
     OutputCollector collect = new OutputCollector();
-    OutStream out = new OutStream("test", new StreamOptions(100), collect);
-    for(int i=0; i < 1024; ++i) {
-      positions[i] = new PositionCollector();
-      out.getPosition(positions[i]);
-      out.write(i);
+    try (OutStream out = new OutStream("test", new StreamOptions(100), collect)) {
+      for (int i = 0; i < 1024; ++i) {
+        positions[i] = new PositionCollector();
+        out.getPosition(positions[i]);
+        out.write(i);
+      }
+      out.flush();
     }
-    out.flush();
     assertEquals(1024, collect.buffer.size());
     for(int i=0; i < 1024; ++i) {
       assertEquals((byte) i, collect.buffer.get(i));
@@ -168,16 +167,17 @@ public class TestInStream {
     writerOptions.modifyIv(CryptoUtils.modifyIvForStream(0,
         OrcProto.Stream.Kind.DATA, 1));
     System.arraycopy(writerOptions.getIv(), 0, iv, 0, iv.length);
-    OutStream out = new OutStream("test", writerOptions, collect);
-    DataOutputStream outStream = new DataOutputStream(out);
-    for(int i=0; i < ROW_COUNT; ++i) {
-      positions[i] = new PositionCollector();
-      out.getPosition(positions[i]);
-      outStream.writeLong(i * DATA_CONST);
-    }
-    out.flush();
+    try (OutStream out = new OutStream("test", writerOptions, collect);
+         DataOutputStream outStream = new DataOutputStream(out)) {
+      for (int i = 0; i < ROW_COUNT; ++i) {
+        positions[i] = new PositionCollector();
+        out.getPosition(positions[i]);
+        outStream.writeLong(i * DATA_CONST);
+      }
+      out.flush();
+    }
     byte[] result = collect.buffer.get();
-    assertEquals(ROW_COUNT * 8, result.length);
+    assertEquals(ROW_COUNT * 8L, result.length);
     return result;
   }
 
@@ -205,19 +205,20 @@ public class TestInStream {
       offset += size;
     }
 
-    InStream in = InStream.create("test", list.get(), 0, bytes.length,
-        InStream.options().withEncryption(EncryptionAlgorithm.AES_CTR_128,
-            new SecretKeySpec(rawKey, algorithm.getAlgorithm()), iv));
-    assertEquals("encrypted uncompressed stream test position: 0 length: 8192" +
-            " range: 0 offset: 0 position: 0 limit: 1965",
-        in.toString());
-    DataInputStream inputStream = new DataInputStream(in);
-    for(int i=0; i < ROW_COUNT; ++i) {
-      assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
-    }
-    for(int i=ROW_COUNT - 1; i >= 0; --i) {
-      in.seek(positions[i]);
-      assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+    try (InStream in = InStream.create("test", list.get(), 0, bytes.length,
+            InStream.options().withEncryption(EncryptionAlgorithm.AES_CTR_128,
+                new SecretKeySpec(rawKey, algorithm.getAlgorithm()), iv));
+         DataInputStream inputStream = new DataInputStream(in)) {
+      assertEquals("encrypted uncompressed stream test position: 0 length: 8192" +
+                       " range: 0 offset: 0 position: 0 limit: 1965",
+          in.toString());
+      for (int i = 0; i < ROW_COUNT; ++i) {
+        assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+      }
+      for (int i = ROW_COUNT - 1; i >= 0; --i) {
+        in.seek(positions[i]);
+        assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+      }
     }
   }
 
@@ -252,19 +253,20 @@ public class TestInStream {
     buffer.clear();
     list.add(new BufferChunk(buffer, 2000 + SECOND_SIZE));
 
-    InStream in = InStream.create("test", list.get(), 35, bytes.length,
-        InStream.options().withEncryption(EncryptionAlgorithm.AES_CTR_128,
-            new SecretKeySpec(rawKey, algorithm.getAlgorithm()), iv));
-    assertEquals("encrypted uncompressed stream test position: 0 length: 8192" +
-                     " range: 0 offset: 0 position: 0 limit: 1965",
-        in.toString());
-    DataInputStream inputStream = new DataInputStream(in);
-    for(int i=0; i < ROW_COUNT; ++i) {
-      assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
-    }
-    for(int i=ROW_COUNT - 1; i >= 0; --i) {
-      in.seek(positions[i]);
-      assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+    try (InStream in = InStream.create("test", list.get(), 35, bytes.length,
+            InStream.options().withEncryption(EncryptionAlgorithm.AES_CTR_128,
+                new SecretKeySpec(rawKey, algorithm.getAlgorithm()), iv));
+         DataInputStream inputStream = new DataInputStream(in)) {
+      assertEquals("encrypted uncompressed stream test position: 0 length: 8192" +
+                       " range: 0 offset: 0 position: 0 limit: 1965",
+          in.toString());
+      for (int i = 0; i < ROW_COUNT; ++i) {
+        assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+      }
+      for (int i = ROW_COUNT - 1; i >= 0; --i) {
+        in.seek(positions[i]);
+        assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+      }
     }
   }
 
@@ -286,14 +288,15 @@ public class TestInStream {
     writerOptions.modifyIv(CryptoUtils.modifyIvForStream(0,
         OrcProto.Stream.Kind.DATA, 1));
     System.arraycopy(writerOptions.getIv(), 0, iv, 0, iv.length);
-    OutStream out = new OutStream("test", writerOptions, collect);
-    DataOutputStream outStream = new DataOutputStream(out);
-    for(int i=0; i < ROW_COUNT; ++i) {
-      positions[i] = new PositionCollector();
-      out.getPosition(positions[i]);
-      outStream.writeLong(i * DATA_CONST);
-    }
-    out.flush();
+    try (OutStream out = new OutStream("test", writerOptions, collect);
+         DataOutputStream outStream = new DataOutputStream(out)) {
+      for (int i = 0; i < ROW_COUNT; ++i) {
+        positions[i] = new PositionCollector();
+        out.getPosition(positions[i]);
+        outStream.writeLong(i * DATA_CONST);
+      }
+      out.flush();
+    }
     return collect.buffer.get();
   }
 
@@ -323,23 +326,26 @@ public class TestInStream {
       offset += size;
     }
 
-    InStream in = InStream.create("test", list.get(), 0, bytes.length,
-        InStream.options()
-            .withCodec(new ZlibCodec()).withBufferSize(500)
-            .withEncryption(algorithm, new SecretKeySpec(key, algorithm.getAlgorithm()), iv));
-    assertEquals("encrypted compressed stream test position: 0 length: " +
-            bytes.length + " range: 0 offset: 0 limit: 1998 range 0 = 0 to" +
-            " 1998;  range 1 = 1998 to " + (bytes.length - 15) +
-            ";  range 2 = " +
-            (bytes.length - 15) + " to " + bytes.length,
-        in.toString());
-    DataInputStream inputStream = new DataInputStream(in);
-    for(int i=0; i < ROW_COUNT; ++i) {
-      assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
-    }
-    for(int i=ROW_COUNT - 1; i >= 0; --i) {
-      in.seek(positions[i]);
-      assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+    try (InStream in = InStream.create("test", list.get(), 0, bytes.length,
+            InStream.options()
+              .withCodec(new ZlibCodec()).withBufferSize(500)
+              .withEncryption(algorithm, new SecretKeySpec(key,
+                  algorithm.getAlgorithm()), iv));
+         DataInputStream inputStream = new DataInputStream(in)) {
+      assertEquals("encrypted compressed stream test position: 0 length: " +
+                       bytes.length +
+                       " range: 0 offset: 0 limit: 1998 range 0 = 0 to" +
+                       " 1998;  range 1 = 1998 to " + (bytes.length - 15) +
+                       ";  range 2 = " +
+                       (bytes.length - 15) + " to " + bytes.length,
+          in.toString());
+      for (int i = 0; i < ROW_COUNT; ++i) {
+        assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+      }
+      for (int i = ROW_COUNT - 1; i >= 0; --i) {
+        in.seek(positions[i]);
+        assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+      }
     }
   }
 
@@ -376,21 +382,24 @@ public class TestInStream {
     buffer.clear();
     list.add(new BufferChunk(buffer, 2100 + SECOND_SIZE));
 
-    InStream in = InStream.create("test", list.get(), 102, bytes.length,
-        InStream.options()
-            .withCodec(new ZlibCodec()).withBufferSize(500)
-            .withEncryption(algorithm, new SecretKeySpec(key, algorithm.getAlgorithm()), iv));
-    assertEquals("encrypted compressed stream test position: 0 length: " +
-                     bytes.length + " range: 0 offset: 0 limit: 1998 range 0 = 100 to 2100;" +
-                     "  range 1 = 2100 to 4044;  range 2 = 4044 to 5044",
-        in.toString());
-    DataInputStream inputStream = new DataInputStream(in);
-    for(int i=0; i < ROW_COUNT; ++i) {
-      assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
-    }
-    for(int i=ROW_COUNT - 1; i >= 0; --i) {
-      in.seek(positions[i]);
-      assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+    try (InStream in = InStream.create("test", list.get(), 102, bytes.length,
+            InStream.options()
+               .withCodec(new ZlibCodec()).withBufferSize(500)
+                .withEncryption(algorithm, new SecretKeySpec(key,
+                    algorithm.getAlgorithm()), iv));
+         DataInputStream inputStream = new DataInputStream(in)) {
+      assertEquals("encrypted compressed stream test position: 0 length: " +
+                       bytes.length +
+                       " range: 0 offset: 0 limit: 1998 range 0 = 100 to 2100;" +
+                       "  range 1 = 2100 to 4044;  range 2 = 4044 to 5044",
+          in.toString());
+      for (int i = 0; i < ROW_COUNT; ++i) {
+        assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+      }
+      for (int i = ROW_COUNT - 1; i >= 0; --i) {
+        in.seek(positions[i]);
+        assertEquals("row " + i, i * DATA_CONST, inputStream.readLong());
+      }
     }
   }
 
@@ -399,14 +408,15 @@ public class TestInStream {
     StreamOptions options = new StreamOptions(300)
                                 .withCodec(codec, codec.getDefaultOptions());
     OutputCollector collect = new OutputCollector();
-    OutStream out = new OutStream("test", options, collect);
-    for(int i=0; i < 1024; ++i) {
-      positions[i] = new PositionCollector();
-      out.getPosition(positions[i]);
-      out.write(i);
+    try (OutStream out = new OutStream("test", options, collect)) {
+      for (int i = 0; i < 1024; ++i) {
+        positions[i] = new PositionCollector();
+        out.getPosition(positions[i]);
+        out.write(i);
+      }
+      out.flush();
+      assertEquals("test", out.toString());
     }
-    out.flush();
-    assertEquals("test", out.toString());
     return collect.buffer.get();
   }
 
@@ -465,14 +475,12 @@ public class TestInStream {
     CompressionCodec codec = new ZlibCodec();
     StreamOptions options = new StreamOptions(500)
                                 .withCodec(codec, codec.getDefaultOptions());
-    OutStream out = new OutStream("test", options, collect);
-    PositionCollector[] positions = new PositionCollector[1024];
-    for(int i=0; i < 1024; ++i) {
-      positions[i] = new PositionCollector();
-      out.getPosition(positions[i]);
-      out.write(i);
+    try (OutStream out = new OutStream("test", options, collect)) {
+      for (int i = 0; i < 1024; ++i) {
+        out.write(i);
+      }
+      out.flush();
     }
-    out.flush();
 
     // now try to read the stream with a buffer that is too small
     ByteBuffer inBuf = ByteBuffer.allocate(collect.buffer.size());
@@ -511,15 +519,17 @@ public class TestInStream {
     CompressionCodec codec = new ZlibCodec();
     StreamOptions options = new StreamOptions(400)
                                 .withCodec(codec, codec.getDefaultOptions());
-    OutStream out = new OutStream("test", options, collect);
     PositionCollector[] positions = new PositionCollector[1024];
-    DataOutput stream = new DataOutputStream(out);
-    for(int i=0; i < 1024; ++i) {
-      positions[i] = new PositionCollector();
-      out.getPosition(positions[i]);
-      stream.writeInt(i);
+    try (OutStream out = new OutStream("test", options, collect);
+         DataOutputStream stream = new DataOutputStream(out)) {
+
+      for (int i = 0; i < 1024; ++i) {
+        positions[i] = new PositionCollector();
+        out.getPosition(positions[i]);
+        stream.writeInt(i);
+      }
+      out.flush();
     }
-    out.flush();
     assertEquals(1674, collect.buffer.size());
     ByteBuffer[] inBuf = new ByteBuffer[3];
     inBuf[0] = ByteBuffer.allocate(500);
@@ -538,60 +548,62 @@ public class TestInStream {
     }
     InStream.StreamOptions inOptions = InStream.options()
         .withCodec(codec).withBufferSize(400);
-    InStream in = InStream.create("test", buffers.get(), 0, 1674, inOptions);
-    assertEquals("compressed stream test position: 0 length: 1674 range: 0" +
-                 " offset: 0 limit: 483 range 0 = 0 to 483;" +
-                 "  range 1 = 483 to 1625;  range 2 = 1625 to 1674",
-                 in.toString());
-    DataInputStream inStream = new DataInputStream(in);
-    for(int i=0; i < 1024; ++i) {
-      int x = inStream.readInt();
-      assertEquals(i, x);
-    }
-    assertEquals(0, in.available());
-    for(int i=1023; i >= 0; --i) {
-      in.seek(positions[i]);
-      assertEquals(i, inStream.readInt());
+    try (InStream in = InStream.create("test", buffers.get(), 0, 1674, inOptions);
+         DataInputStream inStream = new DataInputStream(in)) {
+      assertEquals("compressed stream test position: 0 length: 1674 range: 0" +
+                       " offset: 0 limit: 483 range 0 = 0 to 483;" +
+                       "  range 1 = 483 to 1625;  range 2 = 1625 to 1674",
+          in.toString());
+      for (int i = 0; i < 1024; ++i) {
+        int x = inStream.readInt();
+        assertEquals(i, x);
+      }
+      assertEquals(0, in.available());
+      for (int i = 1023; i >= 0; --i) {
+        in.seek(positions[i]);
+        assertEquals(i, inStream.readInt());
+      }
     }
-
     buffers.clear();
     buffers.add(new BufferChunk(inBuf[1], 483));
     buffers.add(new BufferChunk(inBuf[2], 1625));
-    in = InStream.create("test", buffers.get(), 0, 1674, inOptions);
-    inStream = new DataInputStream(in);
-    positions[303].reset();
-    in.seek(positions[303]);
-    for(int i=303; i < 1024; ++i) {
-      assertEquals(i, inStream.readInt());
+    try (InStream in = InStream.create("test", buffers.get(), 0, 1674, inOptions);
+         DataInputStream inStream = new DataInputStream(in)) {
+      positions[303].reset();
+      in.seek(positions[303]);
+      for (int i = 303; i < 1024; ++i) {
+        assertEquals(i, inStream.readInt());
+      }
     }
-
     buffers.clear();
     buffers.add(new BufferChunk(inBuf[0], 0));
     buffers.add(new BufferChunk(inBuf[2], 1625));
-    in = InStream.create("test", buffers.get(), 0, 1674, inOptions);
-    inStream = new DataInputStream(in);
-    positions[1001].reset();
-    for(int i=0; i < 300; ++i) {
-      assertEquals(i, inStream.readInt());
-    }
-    in.seek(positions[1001]);
-    for(int i=1001; i < 1024; ++i) {
-      assertEquals(i, inStream.readInt());
+    try (InStream in = InStream.create("test", buffers.get(), 0, 1674, inOptions);
+         DataInputStream inStream = new DataInputStream(in)) {
+      positions[1001].reset();
+      for (int i = 0; i < 300; ++i) {
+        assertEquals(i, inStream.readInt());
+      }
+      in.seek(positions[1001]);
+      for (int i = 1001; i < 1024; ++i) {
+        assertEquals(i, inStream.readInt());
+      }
     }
   }
 
   @Test
   public void testUncompressedDisjointBuffers() throws Exception {
     OutputCollector collect = new OutputCollector();
-    OutStream out = new OutStream("test", new StreamOptions(400), collect);
     PositionCollector[] positions = new PositionCollector[1024];
-    DataOutput stream = new DataOutputStream(out);
-    for(int i=0; i < 1024; ++i) {
-      positions[i] = new PositionCollector();
-      out.getPosition(positions[i]);
-      stream.writeInt(i);
+    try (OutStream out = new OutStream("test", new StreamOptions(400), collect);
+         DataOutputStream stream = new DataOutputStream(out)) {
+      for (int i = 0; i < 1024; ++i) {
+        positions[i] = new PositionCollector();
+        out.getPosition(positions[i]);
+        stream.writeInt(i);
+      }
+      out.flush();
     }
-    out.flush();
     assertEquals(4096, collect.buffer.size());
     ByteBuffer[] inBuf = new ByteBuffer[3];
     inBuf[0] = ByteBuffer.allocate(1100);
@@ -608,59 +620,63 @@ public class TestInStream {
     buffers.add(new BufferChunk(inBuf[0], 0));
     buffers.add(new BufferChunk(inBuf[1], 1024));
     buffers.add(new BufferChunk(inBuf[2], 3072));
-    InStream in = InStream.create("test", buffers.get(), 0, 4096);
-    assertEquals("uncompressed stream test position: 0 length: 4096" +
-                 " range: 0 offset: 0 position: 0 limit: 1024",
-                 in.toString());
-    DataInputStream inStream = new DataInputStream(in);
-    for(int i=0; i < 1024; ++i) {
-      int x = inStream.readInt();
-      assertEquals(i, x);
-    }
-    assertEquals(0, in.available());
-    for(int i=1023; i >= 0; --i) {
-      in.seek(positions[i]);
-      assertEquals(i, inStream.readInt());
+    try (InStream in = InStream.create("test", buffers.get(), 0, 4096);
+         DataInputStream inStream = new DataInputStream(in)) {
+      assertEquals("uncompressed stream test position: 0 length: 4096" +
+                       " range: 0 offset: 0 position: 0 limit: 1024",
+          in.toString());
+      for (int i = 0; i < 1024; ++i) {
+        int x = inStream.readInt();
+        assertEquals(i, x);
+      }
+      assertEquals(0, in.available());
+      for (int i = 1023; i >= 0; --i) {
+        in.seek(positions[i]);
+        assertEquals(i, inStream.readInt());
+      }
     }
 
     buffers.clear();
     buffers.add(new BufferChunk(inBuf[1], 1024));
     buffers.add(new BufferChunk(inBuf[2], 3072));
-    in = InStream.create("test", buffers.get(), 0, 4096);
-    inStream = new DataInputStream(in);
-    positions[256].reset();
-    in.seek(positions[256]);
-    for(int i=256; i < 1024; ++i) {
-      assertEquals(i, inStream.readInt());
+    try (InStream in = InStream.create("test", buffers.get(), 0, 4096);
+         DataInputStream inStream = new DataInputStream(in)) {
+      positions[256].reset();
+      in.seek(positions[256]);
+      for (int i = 256; i < 1024; ++i) {
+        assertEquals(i, inStream.readInt());
+      }
     }
 
     buffers.clear();
     buffers.add(new BufferChunk(inBuf[0], 0));
     buffers.add(new BufferChunk(inBuf[2], 3072));
-    in = InStream.create("test", buffers.get(), 0, 4096);
-    inStream = new DataInputStream(in);
-    positions[768].reset();
-    for(int i=0; i < 256; ++i) {
-      assertEquals(i, inStream.readInt());
-    }
-    in.seek(positions[768]);
-    for(int i=768; i < 1024; ++i) {
-      assertEquals(i, inStream.readInt());
+    try (InStream in = InStream.create("test", buffers.get(), 0, 4096);
+         DataInputStream inStream = new DataInputStream(in)) {
+      positions[768].reset();
+      for (int i = 0; i < 256; ++i) {
+        assertEquals(i, inStream.readInt());
+      }
+      in.seek(positions[768]);
+      for (int i = 768; i < 1024; ++i) {
+        assertEquals(i, inStream.readInt());
+      }
     }
   }
 
   @Test
   public void testEmptyDiskRange() throws IOException {
     DiskRangeList range = new BufferChunk(ByteBuffer.allocate(0), 0);
-    InStream stream = new InStream.UncompressedStream("test", range, 0, 0);
-    assertEquals(0, stream.available());
-    stream.seek(new PositionProvider() {
-      @Override
-      public long getNext() {
-        return 0;
-      }
-    });
-    assertEquals(0, stream.available());
+    try (InStream stream = new InStream.UncompressedStream("test", range, 0, 0)) {
+      assertEquals(0, stream.available());
+      stream.seek(new PositionProvider() {
+        @Override
+        public long getNext() {
+          return 0;
+        }
+      });
+      assertEquals(0, stream.available());
+    }
   }
 
   private static byte[] input(int... data) {
diff --git a/java/core/src/test/org/apache/orc/impl/TestIntegerCompressionReader.java b/java/core/src/test/org/apache/orc/impl/TestIntegerCompressionReader.java
index 3f66ace..6bf4e1a 100644
--- a/java/core/src/test/org/apache/orc/impl/TestIntegerCompressionReader.java
+++ b/java/core/src/test/org/apache/orc/impl/TestIntegerCompressionReader.java
@@ -67,10 +67,8 @@ public class TestIntegerCompressionReader {
       int x = (int) in.next();
       if (i < 1024) {
         assertEquals(i/4, x);
-      } else if (i < 2048) {
-        assertEquals(2*i, x);
       } else {
-        assertEquals(junk[i-2048], x);
+        assertEquals(2*i, x);
       }
     }
     for(int i=2047; i >= 0; --i) {
@@ -78,10 +76,8 @@ public class TestIntegerCompressionReader {
       int x = (int) in.next();
       if (i < 1024) {
         assertEquals(i/4, x);
-      } else if (i < 2048) {
-        assertEquals(2*i, x);
       } else {
-        assertEquals(junk[i-2048], x);
+        assertEquals(2*i, x);
       }
     }
   }
diff --git a/java/core/src/test/org/apache/orc/impl/TestOutStream.java b/java/core/src/test/org/apache/orc/impl/TestOutStream.java
index 95dedf6..41eef50 100644
--- a/java/core/src/test/org/apache/orc/impl/TestOutStream.java
+++ b/java/core/src/test/org/apache/orc/impl/TestOutStream.java
@@ -18,6 +18,17 @@
 
 package org.apache.orc.impl;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.Key;
+import javax.crypto.Cipher;
+import javax.crypto.spec.IvParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
 import org.apache.orc.CompressionCodec;
 import org.apache.orc.EncryptionAlgorithm;
 import org.apache.orc.InMemoryKeystore;
@@ -28,18 +39,6 @@ import org.junit.Assume;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import javax.crypto.Cipher;
-import javax.crypto.spec.IvParameterSpec;
-import javax.crypto.spec.SecretKeySpec;
-import java.io.BufferedReader;
-import java.io.InputStreamReader;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.security.Key;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 public class TestOutStream {
 
   @Test
@@ -49,12 +48,13 @@ public class TestOutStream {
     CompressionCodec codec = new ZlibCodec();
     StreamOptions options = new StreamOptions(128 * 1024)
         .withCodec(codec, codec.getDefaultOptions());
-    OutStream stream = new OutStream("test", options, receiver);
-    assertEquals(0L, stream.getBufferSize());
-    stream.write(new byte[]{0, 1, 2});
-    stream.flush();
-    Mockito.verify(receiver).output(Mockito.any(ByteBuffer.class));
-    assertEquals(0L, stream.getBufferSize());
+    try (OutStream stream = new OutStream("test", options, receiver)) {
+      assertEquals(0L, stream.getBufferSize());
+      stream.write(new byte[]{0, 1, 2});
+      stream.flush();
+      Mockito.verify(receiver).output(Mockito.any(ByteBuffer.class));
+      assertEquals(0L, stream.getBufferSize());
+    }
   }
 
   @Test
@@ -84,87 +84,88 @@ public class TestOutStream {
         .withEncryption(aes128, material);
     options.modifyIv(CryptoUtils.modifyIvForStream(0x34,
         OrcProto.Stream.Kind.DATA, 18));
-    OutStream stream = new OutStream("test", options, receiver);
-    byte[] data = new byte[210];
-    for(int i=0; i < data.length; ++i) {
-      data[i] = (byte) (i+3);
-    }
-
-    // make 17 empty stripes for the stream
-    for(int i=0; i < 18; ++i) {
+    try (OutStream stream = new OutStream("test", options, receiver)) {
+      byte[] data = new byte[210];
+      for (int i = 0; i < data.length; ++i) {
+        data[i] = (byte) (i + 3);
+      }
+
+      // make 17 empty stripes for the stream
+      for (int i = 0; i < 18; ++i) {
+        stream.flush();
+      }
+
+      stream.write(data);
       stream.flush();
-    }
+      byte[] output = receiver.buffer.get();
 
-    stream.write(data);
-    stream.flush();
-    byte[] output = receiver.buffer.get();
+      // These are the outputs of aes256 with the key and incrementing ivs.
+      // I included these hardcoded values to make sure that we are getting
+      // AES128 encryption.
+      //
+      // I used http://extranet.cryptomathic.com/aescalc/index to compute these:
+      // key: 000102030405060708090a0b0c0d0e0f
+      // input: 00003400010000120000000000000000
+      // ecb encrypt output: 822252A81CC7E7FE3E51F50E0E9B64B1
+      int[] generated = new int[]{
+          0x82, 0x22, 0x52, 0xA8, 0x1C, 0xC7, 0xE7, 0xFE, // block 0
+          0x3E, 0x51, 0xF5, 0x0E, 0x0E, 0x9B, 0x64, 0xB1,
 
-    // These are the outputs of aes256 with the key and incrementing ivs.
-    // I included these hardcoded values to make sure that we are getting
-    // AES128 encryption.
-    //
-    // I used http://extranet.cryptomathic.com/aescalc/index to compute these:
-    // key: 000102030405060708090a0b0c0d0e0f
-    // input: 00003400010000120000000000000000
-    // ecb encrypt output: 822252A81CC7E7FE3E51F50E0E9B64B1
-    int[] generated = new int[]{
-        0x82, 0x22, 0x52, 0xA8, 0x1C, 0xC7, 0xE7, 0xFE, // block 0
-        0x3E, 0x51, 0xF5, 0x0E, 0x0E, 0x9B, 0x64, 0xB1,
+          0xF6, 0x4D, 0x36, 0xA9, 0xD9, 0xD7, 0x55, 0xDE, // block 1
+          0xCB, 0xD5, 0x62, 0x0E, 0x6D, 0xA6, 0x6B, 0x16,
 
-        0xF6, 0x4D, 0x36, 0xA9, 0xD9, 0xD7, 0x55, 0xDE, // block 1
-        0xCB, 0xD5, 0x62, 0x0E, 0x6D, 0xA6, 0x6B, 0x16,
+          0x00, 0x0B, 0xE8, 0xBA, 0x9D, 0xDE, 0x78, 0xEC, // block 2
+          0x73, 0x05, 0xF6, 0x1E, 0x76, 0xD7, 0x9B, 0x7A,
 
-        0x00, 0x0B, 0xE8, 0xBA, 0x9D, 0xDE, 0x78, 0xEC, // block 2
-        0x73, 0x05, 0xF6, 0x1E, 0x76, 0xD7, 0x9B, 0x7A,
+          0x47, 0xE9, 0x61, 0x90, 0x65, 0x8B, 0x54, 0xAC, // block 3
+          0xF2, 0x3F, 0x67, 0xAE, 0x25, 0x63, 0x1D, 0x4B,
 
-        0x47, 0xE9, 0x61, 0x90, 0x65, 0x8B, 0x54, 0xAC, // block 3
-        0xF2, 0x3F, 0x67, 0xAE, 0x25, 0x63, 0x1D, 0x4B,
+          0x41, 0x48, 0xC4, 0x15, 0x5F, 0x2A, 0x7F, 0x91, // block 4
+          0x9A, 0x87, 0xA1, 0x09, 0xFF, 0x68, 0x68, 0xCC,
 
-        0x41, 0x48, 0xC4, 0x15, 0x5F, 0x2A, 0x7F, 0x91, // block 4
-        0x9A, 0x87, 0xA1, 0x09, 0xFF, 0x68, 0x68, 0xCC,
+          0xC0, 0x80, 0x52, 0xD4, 0xA5, 0x07, 0x4B, 0x79, // block 5
+          0xC7, 0x08, 0x46, 0x46, 0x8C, 0x74, 0x2C, 0x0D,
 
-        0xC0, 0x80, 0x52, 0xD4, 0xA5, 0x07, 0x4B, 0x79, // block 5
-        0xC7, 0x08, 0x46, 0x46, 0x8C, 0x74, 0x2C, 0x0D,
+          0x9F, 0x55, 0x7E, 0xA7, 0x17, 0x47, 0x91, 0xFD, // block 6
+          0x01, 0xD4, 0x24, 0x1F, 0x76, 0xA1, 0xDC, 0xC3,
 
-        0x9F, 0x55, 0x7E, 0xA7, 0x17, 0x47, 0x91, 0xFD, // block 6
-        0x01, 0xD4, 0x24, 0x1F, 0x76, 0xA1, 0xDC, 0xC3,
+          0xEA, 0x13, 0x4C, 0x29, 0xCA, 0x68, 0x1E, 0x4F, // block 7
+          0x0D, 0x19, 0xE5, 0x09, 0xF9, 0xC5, 0xF4, 0x15,
 
-        0xEA, 0x13, 0x4C, 0x29, 0xCA, 0x68, 0x1E, 0x4F, // block 7
-        0x0D, 0x19, 0xE5, 0x09, 0xF9, 0xC5, 0xF4, 0x15,
+          0x9A, 0xAD, 0xC4, 0xA1, 0x0F, 0x28, 0xD4, 0x3D, // block 8
+          0x59, 0xF0, 0x68, 0xD3, 0xC4, 0x98, 0x74, 0x68,
 
-        0x9A, 0xAD, 0xC4, 0xA1, 0x0F, 0x28, 0xD4, 0x3D, // block 8
-        0x59, 0xF0, 0x68, 0xD3, 0xC4, 0x98, 0x74, 0x68,
+          0x37, 0xA4, 0xF4, 0x7C, 0x02, 0xCE, 0xC6, 0xCA, // block 9
+          0xA1, 0xF8, 0xC3, 0x8C, 0x7B, 0x72, 0x38, 0xD1,
 
-        0x37, 0xA4, 0xF4, 0x7C, 0x02, 0xCE, 0xC6, 0xCA, // block 9
-        0xA1, 0xF8, 0xC3, 0x8C, 0x7B, 0x72, 0x38, 0xD1,
+          0xAA, 0x52, 0x90, 0xDE, 0x28, 0xA1, 0x53, 0x6E, // block a
+          0xA6, 0x5C, 0xC0, 0x89, 0xC4, 0x21, 0x76, 0xC0,
 
-        0xAA, 0x52, 0x90, 0xDE, 0x28, 0xA1, 0x53, 0x6E, // block a
-        0xA6, 0x5C, 0xC0, 0x89, 0xC4, 0x21, 0x76, 0xC0,
+          0x1F, 0xED, 0x0A, 0xF9, 0xA2, 0xA7, 0xC1, 0x8D, // block b
+          0xA0, 0x92, 0x44, 0x4F, 0x60, 0x51, 0x7F, 0xD8,
 
-        0x1F, 0xED, 0x0A, 0xF9, 0xA2, 0xA7, 0xC1, 0x8D, // block b
-        0xA0, 0x92, 0x44, 0x4F, 0x60, 0x51, 0x7F, 0xD8,
+          0x6D, 0x16, 0xAF, 0x46, 0x1C, 0x27, 0x20, 0x1C, // block c
+          0x01, 0xBD, 0xC5, 0x0B, 0x62, 0x3F, 0xEF, 0xEE,
 
-        0x6D, 0x16, 0xAF, 0x46, 0x1C, 0x27, 0x20, 0x1C, // block c
-        0x01, 0xBD, 0xC5, 0x0B, 0x62, 0x3F, 0xEF, 0xEE,
+          0x37, 0xae                                      // block d
+      };
+      assertEquals(generated.length, output.length);
+      for (int i = 0; i < generated.length; ++i) {
+        assertEquals("i = " + i, (byte) (generated[i] ^ data[i]), output[i]);
+      }
 
-        0x37, 0xae                                      // block d
-    };
-    assertEquals(generated.length, output.length);
-    for(int i=0; i < generated.length; ++i) {
-      assertEquals("i = " + i, (byte) (generated[i] ^ data[i]), output[i]);
-    }
+      receiver.buffer.clear();
+      stream.changeIv(CryptoUtils.modifyIvForStripe(19));
 
-    receiver.buffer.clear();
-    stream.changeIv(CryptoUtils.modifyIvForStripe(19));
-
-    data = new byte[]{0x47, 0x77, 0x65, 0x6e};
-    stream.write(data);
-    stream.flush();
-    output = receiver.buffer.get();
-    generated = new int[]{0x16, 0x03, 0xE6, 0xC3};
-    assertEquals(generated.length, output.length);
-    for(int i=0; i < generated.length; ++i) {
-      assertEquals("i = " + i, (byte) (generated[i] ^ data[i]), output[i]);
+      data = new byte[]{0x47, 0x77, 0x65, 0x6e};
+      stream.write(data);
+      stream.flush();
+      output = receiver.buffer.get();
+      generated = new int[]{0x16, 0x03, 0xE6, 0xC3};
+      assertEquals(generated.length, output.length);
+      for (int i = 0; i < generated.length; ++i) {
+        assertEquals("i = " + i, (byte) (generated[i] ^ data[i]), output[i]);
+      }
     }
   }
 
@@ -184,12 +185,13 @@ public class TestOutStream {
         .withCodec(codec, codec.getDefaultOptions())
         .withEncryption(aes256, material)
         .modifyIv(CryptoUtils.modifyIvForStream(0x1, OrcProto.Stream.Kind.DATA, 1));
-    OutStream stream = new OutStream("test", options, receiver);
-    for(int i=0; i < 10000; ++i) {
-      stream.write(("The Cheesy Poofs " + i + "\n")
-          .getBytes(StandardCharsets.UTF_8));
+    try (OutStream stream = new OutStream("test", options, receiver)) {
+      for (int i = 0; i < 10000; ++i) {
+        stream.write(("The Cheesy Poofs " + i + "\n")
+                         .getBytes(StandardCharsets.UTF_8));
+      }
+      stream.flush();
     }
-    stream.flush();
     // get the compressed, encrypted data
     byte[] encrypted = receiver.buffer.get();
 
@@ -202,16 +204,17 @@ public class TestOutStream {
     // use InStream to decompress it
     BufferChunkList ranges = new BufferChunkList();
     ranges.add(new BufferChunk(ByteBuffer.wrap(compressed), 0));
-    InStream decompressedStream = InStream.create("test", ranges.get(), 0,
-        compressed.length,
-        InStream.options().withCodec(new ZlibCodec()).withBufferSize(1024));
-
-    // check the contents of the decompressed stream
-    BufferedReader reader
-        = new BufferedReader(new InputStreamReader(decompressedStream));
-    for(int i=0; i < 10000; ++i) {
-      assertEquals("i = " + i, "The Cheesy Poofs " + i, reader.readLine());
+    try (InStream decompressedStream = InStream.create("test", ranges.get(), 0,
+             compressed.length,
+             InStream.options().withCodec(new ZlibCodec()).withBufferSize(1024));
+         BufferedReader reader
+             = new BufferedReader(new InputStreamReader(decompressedStream,
+             StandardCharsets.UTF_8))) {
+      // check the contents of the decompressed stream
+      for (int i = 0; i < 10000; ++i) {
+        assertEquals("i = " + i, "The Cheesy Poofs " + i, reader.readLine());
+      }
+      assertEquals(null, reader.readLine());
     }
-    assertEquals(null, reader.readLine());
   }
 }
diff --git a/java/core/src/test/org/apache/orc/impl/TestPredicatePushDownBounds.java b/java/core/src/test/org/apache/orc/impl/TestPredicatePushDownBounds.java
index d018efa..36d589a 100644
--- a/java/core/src/test/org/apache/orc/impl/TestPredicatePushDownBounds.java
+++ b/java/core/src/test/org/apache/orc/impl/TestPredicatePushDownBounds.java
@@ -276,12 +276,6 @@ public class TestPredicatePushDownBounds {
     assertEquals(SearchArgument.TruthValue.NO,
         RecordReaderImpl.evaluatePredicate(stat, predicateLowerBoundSet, null));
 
-    /* Test for a case LESS_THAN_EQUALS where only upperbound is set */
-    final PredicateLeaf predicateUpperBoundLessThanEquals = TestRecordReaderImpl
-        .createPredicateLeaf(PredicateLeaf.Operator.IN,
-            PredicateLeaf.Type.STRING, "x", null, args);
-
-
     /* Test the case were both upper and lower bounds are set */
     args.clear();
     args.add(lowerboundString);
diff --git a/java/core/src/test/org/apache/orc/impl/TestReaderImpl.java b/java/core/src/test/org/apache/orc/impl/TestReaderImpl.java
index 8471a1e..465d52e 100644
--- a/java/core/src/test/org/apache/orc/impl/TestReaderImpl.java
+++ b/java/core/src/test/org/apache/orc/impl/TestReaderImpl.java
@@ -15,18 +15,19 @@
  */
 package org.apache.orc.impl;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.ByteArrayInputStream;
 import java.io.EOFException;
-import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.InputStream;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -36,20 +37,19 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PositionedReadable;
 import org.apache.hadoop.fs.Seekable;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.Progressable;
 import org.apache.orc.FileFormatException;
-import org.apache.hadoop.io.Text;
 import org.apache.orc.OrcFile;
 import org.apache.orc.Reader;
 import org.apache.orc.RecordReader;
 import org.apache.orc.TestVectorOrcFile;
-import org.junit.Test;
 import org.junit.Before;
 import org.junit.Rule;
+import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
-import static org.junit.Assert.assertEquals;
-
 public class TestReaderImpl {
 
   @Rule
@@ -67,14 +67,14 @@ public class TestReaderImpl {
 
   @Test
   public void testEnsureOrcFooterSmallTextFile() throws IOException {
-    prepareTestCase("1".getBytes());
+    prepareTestCase("1".getBytes(StandardCharsets.UTF_8));
     thrown.expect(FileFormatException.class);
     ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
   }
 
   @Test
   public void testEnsureOrcFooterLargeTextFile() throws IOException {
-    prepareTestCase("This is Some Text File".getBytes());
+    prepareTestCase("This is Some Text File".getBytes(StandardCharsets.UTF_8));
     thrown.expect(FileFormatException.class);
     ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
   }
@@ -98,8 +98,13 @@ public class TestReaderImpl {
     Configuration conf = new Configuration();
     Path path = new Path(TestVectorOrcFile.getFileFromClasspath
         ("orc-file-11-format.orc"));
-    Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
-    RecordReader rows = reader.rows(options);
+    try (Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
+         RecordReader rows = reader.rows(options)) {
+      VectorizedRowBatch batch = reader.getSchema().createRowBatchV2();
+      while (rows.nextBatch(batch)) {
+        assertTrue(batch.size > 0);
+      }
+    }
     assertEquals(expected, options.toString());
   }
 
diff --git a/java/core/src/test/org/apache/orc/impl/TestRecordReaderImpl.java b/java/core/src/test/org/apache/orc/impl/TestRecordReaderImpl.java
index 3f3abdd..d150750 100644
--- a/java/core/src/test/org/apache/orc/impl/TestRecordReaderImpl.java
+++ b/java/core/src/test/org/apache/orc/impl/TestRecordReaderImpl.java
@@ -1893,11 +1893,12 @@ public class TestRecordReaderImpl {
           new PhysicalFsWriter.BufferedStream();
       StreamOptions writerOptions = new StreamOptions(options.getBufferSize())
           .withCodec(codec, codec.getDefaultOptions());
-      OutStream out = new OutStream("row index", writerOptions, buffer);
-      out.write(uncompressed.array(),
-          uncompressed.arrayOffset() + uncompressed.position(),
-          uncompressed.remaining());
-      out.flush();
+      try (OutStream out = new OutStream("row index", writerOptions, buffer)) {
+        out.write(uncompressed.array(),
+            uncompressed.arrayOffset() + uncompressed.position(),
+            uncompressed.remaining());
+        out.flush();
+      }
       return buffer.getByteBuffer();
     } else {
       return uncompressed;
@@ -1962,7 +1963,7 @@ public class TestRecordReaderImpl {
   }
 
   @Test
-  public void TestOldBloomFilters() throws Exception {
+  public void testOldBloomFilters() throws Exception {
     TypeDescription schema = TypeDescription.fromString("struct<x:int,y:decimal(10,2),z:string>");
     MockDataReader dataReader = createOldBlooms(schema);
     MockStripe stripe = dataReader.getStripe(0);
@@ -2035,7 +2036,7 @@ public class TestRecordReaderImpl {
   }
 
   @Test
-  public void TestCompatibleBloomFilters() throws Exception {
+  public void testCompatibleBloomFilters() throws Exception {
     TypeDescription schema = TypeDescription.fromString("struct<x:int,y:decimal(10,2),z:string>");
     MockDataReader dataReader = createMixedBlooms(schema);
     MockStripe stripe = dataReader.getStripe(0);
@@ -2080,7 +2081,7 @@ public class TestRecordReaderImpl {
   }
 
   @Test
-  public void TestNewBloomFilters() throws Exception {
+  public void testNewBloomFilters() throws Exception {
     TypeDescription schema = TypeDescription.fromString("struct<x:int,y:decimal(10,2),z:string>");
     MockDataReader dataReader = createNewBlooms(schema);
     MockStripe stripe = dataReader.getStripe(0);
diff --git a/java/core/src/test/org/apache/orc/impl/TestRunLengthIntegerReader.java b/java/core/src/test/org/apache/orc/impl/TestRunLengthIntegerReader.java
index ac41e6e..e57a1ba 100644
--- a/java/core/src/test/org/apache/orc/impl/TestRunLengthIntegerReader.java
+++ b/java/core/src/test/org/apache/orc/impl/TestRunLengthIntegerReader.java
@@ -66,10 +66,8 @@ public class TestRunLengthIntegerReader {
       int x = (int) in.next();
       if (i < 1024) {
         assertEquals(i/4, x);
-      } else if (i < 2048) {
-        assertEquals(2*i, x);
       } else {
-        assertEquals(junk[i-2048], x);
+        assertEquals(2*i, x);
       }
     }
     for(int i=2047; i >= 0; --i) {
@@ -77,10 +75,8 @@ public class TestRunLengthIntegerReader {
       int x = (int) in.next();
       if (i < 1024) {
         assertEquals(i/4, x);
-      } else if (i < 2048) {
-        assertEquals(2*i, x);
       } else {
-        assertEquals(junk[i-2048], x);
+        assertEquals(2*i, x);
       }
     }
   }
diff --git a/java/core/src/test/org/apache/orc/impl/TestSchemaEvolution.java b/java/core/src/test/org/apache/orc/impl/TestSchemaEvolution.java
index 00a3383..1dda07e 100644
--- a/java/core/src/test/org/apache/orc/impl/TestSchemaEvolution.java
+++ b/java/core/src/test/org/apache/orc/impl/TestSchemaEvolution.java
@@ -34,7 +34,6 @@ import java.time.format.DateTimeFormatter;
 import java.time.temporal.ChronoField;
 import java.util.Arrays;
 import java.util.TimeZone;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -590,8 +589,8 @@ public class TestSchemaEvolution {
     batch.cols[0] = bcv;
     batch.reset();
     batch.size = 1;
-    bcv.vector[0] = "74.19".getBytes();
-    bcv.length[0] = "74.19".getBytes().length;
+    bcv.vector[0] = "74.19".getBytes(StandardCharsets.UTF_8);
+    bcv.length[0] = "74.19".getBytes(StandardCharsets.UTF_8).length;
     writer.addRowBatch(batch);
     writer.close();
 
@@ -618,8 +617,8 @@ public class TestSchemaEvolution {
     batch.cols[0] = bcv;
     batch.reset();
     batch.size = 1;
-    bcv.vector[0] = "74.19".getBytes();
-    bcv.length[0] = "74.19".getBytes().length;
+    bcv.vector[0] = "74.19".getBytes(StandardCharsets.UTF_8);
+    bcv.length[0] = "74.19".getBytes(StandardCharsets.UTF_8).length;
     writer.addRowBatch(batch);
     writer.close();
 
@@ -1400,8 +1399,7 @@ public class TestSchemaEvolution {
     TypeDescription readerType = TypeDescription.fromString("struct<a:date>");
     boolean[] included = includeAll(readerType);
     options.tolerateMissingSchema(false);
-    SchemaEvolution transition =
-        new SchemaEvolution(fileType, readerType, options.include(included));
+    new SchemaEvolution(fileType, readerType, options.include(included));
   }
 
   @Test
diff --git a/java/core/src/test/org/apache/orc/impl/TestStringRedBlackTree.java b/java/core/src/test/org/apache/orc/impl/TestStringRedBlackTree.java
index 3d4612c..bc61dcf 100644
--- a/java/core/src/test/org/apache/orc/impl/TestStringRedBlackTree.java
+++ b/java/core/src/test/org/apache/orc/impl/TestStringRedBlackTree.java
@@ -18,16 +18,14 @@
 
 package org.apache.orc.impl;
 
+import static junit.framework.Assert.assertEquals;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.IntWritable;
-import org.apache.orc.impl.RedBlackTree;
-import org.apache.orc.impl.StringRedBlackTree;
 import org.junit.Test;
 
-import java.io.IOException;
-
-import static junit.framework.Assert.assertEquals;
-
 /**
  * Test the red-black tree with string keys.
  */
@@ -130,7 +128,7 @@ public class TestStringRedBlackTree {
         context.getOriginalPosition());
       buffer.reset();
       context.writeBytes(buffer);
-      assertEquals(word, new String(buffer.getData(),0,buffer.getLength()));
+      assertEquals(word, new String(buffer.getData(),0,buffer.getLength(), StandardCharsets.UTF_8));
       current += 1;
     }
   }
diff --git a/java/core/src/test/org/apache/orc/util/TestMurmur3.java b/java/core/src/test/org/apache/orc/util/TestMurmur3.java
index 575e250..13ddb9f 100644
--- a/java/core/src/test/org/apache/orc/util/TestMurmur3.java
+++ b/java/core/src/test/org/apache/orc/util/TestMurmur3.java
@@ -22,14 +22,12 @@ import static org.junit.Assert.assertEquals;
 
 import com.google.common.hash.HashFunction;
 import com.google.common.hash.Hashing;
-
-import org.apache.orc.util.Murmur3;
-import org.junit.Test;
-
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
+import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.Random;
+import org.junit.Test;
 
 /**
  * Tests for Murmur3 variants.
@@ -41,13 +39,15 @@ public class TestMurmur3 {
     String key = "test";
     int seed = 123;
     HashFunction hf = Hashing.murmur3_32(seed);
-    int hc1 = hf.hashBytes(key.getBytes()).asInt();
-    int hc2 = Murmur3.hash32(key.getBytes(), key.getBytes().length, seed);
+    int hc1 = hf.hashBytes(key.getBytes(StandardCharsets.UTF_8)).asInt();
+    int hc2 = Murmur3.hash32(key.getBytes(StandardCharsets.UTF_8),
+        key.getBytes(StandardCharsets.UTF_8).length, seed);
     assertEquals(hc1, hc2);
 
     key = "testkey";
-    hc1 = hf.hashBytes(key.getBytes()).asInt();
-    hc2 = Murmur3.hash32(key.getBytes(), key.getBytes().length, seed);
+    hc1 = hf.hashBytes(key.getBytes(StandardCharsets.UTF_8)).asInt();
+    hc2 = Murmur3.hash32(key.getBytes(StandardCharsets.UTF_8),
+        key.getBytes(StandardCharsets.UTF_8).length, seed);
     assertEquals(hc1, hc2);
   }
 
@@ -100,11 +100,12 @@ public class TestMurmur3 {
     HashFunction hf = Hashing.murmur3_128(seed);
     // guava stores the hashcodes in little endian order
     ByteBuffer buf = ByteBuffer.allocate(16).order(ByteOrder.LITTLE_ENDIAN);
-    buf.put(hf.hashBytes(key.getBytes()).asBytes());
+    buf.put(hf.hashBytes(key.getBytes(StandardCharsets.UTF_8)).asBytes());
     buf.flip();
     long gl1 = buf.getLong();
     long gl2 = buf.getLong(8);
-    long[] hc = Murmur3.hash128(key.getBytes(), 0, key.getBytes().length, seed);
+    long[] hc = Murmur3.hash128(key.getBytes(StandardCharsets.UTF_8), 0,
+        key.getBytes(StandardCharsets.UTF_8).length, seed);
     long m1 = hc[0];
     long m2 = hc[1];
     assertEquals(gl1, m1);
@@ -112,11 +113,11 @@ public class TestMurmur3 {
 
     key = "testkey128_testkey128";
     buf = ByteBuffer.allocate(16).order(ByteOrder.LITTLE_ENDIAN);
-    buf.put(hf.hashBytes(key.getBytes()).asBytes());
+    buf.put(hf.hashBytes(key.getBytes(StandardCharsets.UTF_8)).asBytes());
     buf.flip();
     gl1 = buf.getLong();
     gl2 = buf.getLong(8);
-    byte[] keyBytes = key.getBytes();
+    byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8);
     hc = Murmur3.hash128(keyBytes, 0, keyBytes.length, seed);
     m1 = hc[0];
     m2 = hc[1];
@@ -140,7 +141,7 @@ public class TestMurmur3 {
         " it was the spring of hope, it was the winter of despair," +
         " we had everything before us, we had nothing before us," +
         " we were all going direct to Heaven," +
-        " we were all going direct the other way.").getBytes();
+        " we were all going direct the other way.").getBytes(StandardCharsets.UTF_8);
     long hash = Murmur3.hash64(origin, 0, origin.length);
     assertEquals(305830725663368540L, hash);
 
diff --git a/java/mapreduce/src/findbugs/exclude.xml b/java/mapreduce/src/findbugs/exclude.xml
index 4a6e846..56b707e 100644
--- a/java/mapreduce/src/findbugs/exclude.xml
+++ b/java/mapreduce/src/findbugs/exclude.xml
@@ -13,4 +13,8 @@
   limitations under the License.
 -->
 <FindBugsFilter>
+  <Match>
+    <Bug pattern="CNT_ROUGH_CONSTANT_VALUE"/>
+    <Class name="org.apache.orc.mapred.TestOrcFileEvolution"/>
+  </Match>
 </FindBugsFilter>
diff --git a/java/mapreduce/src/test/org/apache/orc/mapred/TestOrcOutputFormat.java b/java/mapreduce/src/test/org/apache/orc/mapred/TestOrcOutputFormat.java
index a915ed3..4004681 100644
--- a/java/mapreduce/src/test/org/apache/orc/mapred/TestOrcOutputFormat.java
+++ b/java/mapreduce/src/test/org/apache/orc/mapred/TestOrcOutputFormat.java
@@ -207,7 +207,6 @@ public class TestOrcOutputFormat {
     final String typeStr = "bigint";
     OrcConf.MAPRED_OUTPUT_SCHEMA.setString(conf, typeStr);
     FileOutputFormat.setOutputPath(conf, workDir);
-    TypeDescription type = TypeDescription.fromString(typeStr);
     LongWritable value = new LongWritable();
     NullWritable nada = NullWritable.get();
     RecordWriter<NullWritable, LongWritable> writer =
diff --git a/java/pom.xml b/java/pom.xml
index 4b09d68..0252f67 100644
--- a/java/pom.xml
+++ b/java/pom.xml
@@ -195,11 +195,12 @@
             <excludeFilterFile>${basedir}/src/findbugs/exclude.xml</excludeFilterFile>
             <xmlOutput>true</xmlOutput>
             <findbugsXmlOutputDirectory>${project.build.directory}/findbugs</findbugsXmlOutputDirectory>
+            <includeTests>true</includeTests>
           </configuration>
           <executions>
             <execution>
               <id>analyze-compile</id>
-              <phase>compile</phase>
+              <phase>test</phase>
               <goals>
                 <goal>check</goal>
               </goals>
diff --git a/java/tools/src/test/org/apache/orc/impl/TestRLEv2.java b/java/tools/src/test/org/apache/orc/impl/TestRLEv2.java
index 441cb02..4d5fc03 100644
--- a/java/tools/src/test/org/apache/orc/impl/TestRLEv2.java
+++ b/java/tools/src/test/org/apache/orc/impl/TestRLEv2.java
@@ -24,16 +24,15 @@ import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.orc.CompressionCodec;
 import org.apache.orc.CompressionKind;
 import org.apache.orc.OrcFile;
 import org.apache.orc.PhysicalWriter;
@@ -89,10 +88,10 @@ public class TestRLEv2 {
 
     PrintStream origOut = System.out;
     ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toUri().toString()});
     System.out.flush();
-    String outDump = new String(myOut.toByteArray());
+    String outDump = new String(myOut.toByteArray(), StandardCharsets.UTF_8);
     // 10 runs of 512 elements. Each run has 2 bytes header, 2 bytes base (base = 123,
     // zigzag encoded varint) and 1 byte delta (delta = 0). In total, 5 bytes per run.
     assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 50"));
@@ -119,10 +118,10 @@ public class TestRLEv2 {
 
     PrintStream origOut = System.out;
     ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toUri().toString()});
     System.out.flush();
-    String outDump = new String(myOut.toByteArray());
+    String outDump = new String(myOut.toByteArray(), StandardCharsets.UTF_8);
     // 10 runs of 512 elements. Each run has 2 bytes header, 1 byte base (base = 0)
     // and 1 byte delta (delta = 1). In total, 4 bytes per run.
     assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 40"));
@@ -149,10 +148,10 @@ public class TestRLEv2 {
 
     PrintStream origOut = System.out;
     ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toUri().toString()});
     System.out.flush();
-    String outDump = new String(myOut.toByteArray());
+    String outDump = new String(myOut.toByteArray(), StandardCharsets.UTF_8);
     // 10 runs of 512 elements. Each run has 2 bytes header, 2 byte base (base = 512, zigzag + varint)
     // and 1 byte delta (delta = 1). In total, 5 bytes per run.
     assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 50"));
@@ -179,10 +178,10 @@ public class TestRLEv2 {
 
     PrintStream origOut = System.out;
     ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toUri().toString()});
     System.out.flush();
-    String outDump = new String(myOut.toByteArray());
+    String outDump = new String(myOut.toByteArray(), StandardCharsets.UTF_8);
     // 10 runs of 512 elements. Each run has 2 bytes header, 1 byte base (base = 0)
     // and 2 bytes delta (delta = 100, zigzag encoded varint). In total, 5 bytes per run.
     assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 50"));
@@ -209,10 +208,10 @@ public class TestRLEv2 {
 
     PrintStream origOut = System.out;
     ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toUri().toString()});
     System.out.flush();
-    String outDump = new String(myOut.toByteArray());
+    String outDump = new String(myOut.toByteArray(), StandardCharsets.UTF_8);
     // 10 runs of 512 elements. Each run has 2 bytes header, 2 byte base (base = 512, zigzag + varint)
     // and 2 bytes delta (delta = 100, zigzag encoded varint). In total, 6 bytes per run.
     assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 60"));
@@ -239,10 +238,10 @@ public class TestRLEv2 {
 
     PrintStream origOut = System.out;
     ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toUri().toString()});
     System.out.flush();
-    String outDump = new String(myOut.toByteArray());
+    String outDump = new String(myOut.toByteArray(), StandardCharsets.UTF_8);
     // 1 byte header + 1 byte value
     assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 2"));
     System.setOut(origOut);
@@ -269,10 +268,10 @@ public class TestRLEv2 {
 
     PrintStream origOut = System.out;
     ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toUri().toString()});
     System.out.flush();
-    String outDump = new String(myOut.toByteArray());
+    String outDump = new String(myOut.toByteArray(), StandardCharsets.UTF_8);
     // monotonicity will be undetermined for this sequence 0,0,1,2,3,...510. Hence DIRECT encoding
     // will be used. 2 bytes for header and 640 bytes for data (512 values with fixed bit of 10 bits
     // each, 5120/8 = 640). Total bytes 642
@@ -303,10 +302,10 @@ public class TestRLEv2 {
 
     PrintStream origOut = System.out;
     ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toUri().toString()});
     System.out.flush();
-    String outDump = new String(myOut.toByteArray());
+    String outDump = new String(myOut.toByteArray(), StandardCharsets.UTF_8);
     // use PATCHED_BASE encoding
     assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 583"));
     System.setOut(origOut);
diff --git a/java/tools/src/test/org/apache/orc/tools/TestFileDump.java b/java/tools/src/test/org/apache/orc/tools/TestFileDump.java
index 040a49c..daa9ba4 100644
--- a/java/tools/src/test/org/apache/orc/tools/TestFileDump.java
+++ b/java/tools/src/test/org/apache/orc/tools/TestFileDump.java
@@ -25,9 +25,10 @@ import java.io.BufferedReader;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
-import java.io.FileReader;
 import java.io.PrintStream;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.sql.Timestamp;
 import java.text.SimpleDateFormat;
 import java.util.Arrays;
@@ -35,7 +36,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -97,7 +97,7 @@ public class TestFileDump {
       batch.cols[2].isNull[batch.size] = true;
     } else {
       ((BytesColumnVector) batch.cols[2]).setVal(batch.size,
-          str.getBytes());
+          str.getBytes(StandardCharsets.UTF_8));
     }
     batch.size += 1;
   }
@@ -155,18 +155,18 @@ public class TestFileDump {
     ((DecimalColumnVector) batch.cols[7]).vector[row].set(de);
     ((TimestampColumnVector) batch.cols[8]).set(row, t);
     ((LongColumnVector) batch.cols[9]).vector[row] = dt.getDays();
-    ((BytesColumnVector) batch.cols[10]).setVal(row, str.getBytes());
-    ((BytesColumnVector) batch.cols[11]).setVal(row, c.getBytes());
-    ((BytesColumnVector) batch.cols[12]).setVal(row, vc.getBytes());
+    ((BytesColumnVector) batch.cols[10]).setVal(row, str.getBytes(StandardCharsets.UTF_8));
+    ((BytesColumnVector) batch.cols[11]).setVal(row, c.getBytes(StandardCharsets.UTF_8));
+    ((BytesColumnVector) batch.cols[12]).setVal(row, vc.getBytes(StandardCharsets.UTF_8));
     MapColumnVector map = (MapColumnVector) batch.cols[13];
     int offset = map.childCount;
     map.offsets[row] = offset;
     map.lengths[row] = m.size();
     map.childCount += map.lengths[row];
     for(Map.Entry<String, String> entry: m.entrySet()) {
-      ((BytesColumnVector) map.keys).setVal(offset, entry.getKey().getBytes());
+      ((BytesColumnVector) map.keys).setVal(offset, entry.getKey().getBytes(StandardCharsets.UTF_8));
       ((BytesColumnVector) map.values).setVal(offset++,
-          entry.getValue().getBytes());
+          entry.getValue().getBytes(StandardCharsets.UTF_8));
     }
     ListColumnVector list = (ListColumnVector) batch.cols[14];
     offset = list.childCount;
@@ -178,19 +178,22 @@ public class TestFileDump {
     }
     StructColumnVector struct = (StructColumnVector) batch.cols[15];
     ((LongColumnVector) struct.fields[0]).vector[row] = sti;
-    ((BytesColumnVector) struct.fields[1]).setVal(row, sts.getBytes());
+    ((BytesColumnVector) struct.fields[1]).setVal(row, sts.getBytes(StandardCharsets.UTF_8));
   }
 
   public static void checkOutput(String expected,
                                  String actual) throws Exception {
-    BufferedReader eStream =
-        new BufferedReader(new FileReader
-            (TestJsonFileDump.getFileFromClasspath(expected)));
-    BufferedReader aStream =
-        new BufferedReader(new FileReader(actual));
-    String expectedLine = eStream.readLine().trim();
+    BufferedReader eStream = Files.newBufferedReader(Paths.get(TestJsonFileDump.getFileFromClasspath(expected)), StandardCharsets.UTF_8);
+    BufferedReader aStream = Files.newBufferedReader(Paths.get(actual), StandardCharsets.UTF_8);
+    String expectedLine = eStream.readLine();
+    if (expectedLine != null) {
+      expectedLine = expectedLine.trim();
+    }
     while (expectedLine != null) {
-      String actualLine = aStream.readLine().trim();
+      String actualLine = aStream.readLine();
+      if (actualLine != null) {
+        actualLine = actualLine.trim();
+      }
       Assert.assertEquals(expectedLine, actualLine);
       expectedLine = eStream.readLine();
       expectedLine = expectedLine == null ? null : expectedLine.trim();
@@ -248,7 +251,7 @@ public class TestFileDump {
     FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
 
     // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
     System.out.flush();
     System.setOut(origOut);
@@ -321,11 +324,11 @@ public class TestFileDump {
     ByteArrayOutputStream myOut = new ByteArrayOutputStream();
 
     // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, "UTF-8"));
     FileDump.main(new String[]{testFilePath.toString(), "-d"});
     System.out.flush();
     System.setOut(origOut);
-    String[] lines = myOut.toString().split("\n");
+    String[] lines = myOut.toString(StandardCharsets.UTF_8.toString()).split("\n");
     Assert.assertEquals("{\"b\":true,\"bt\":10,\"s\":100,\"i\":1000,\"l\":10000,\"f\":4,\"d\":20,\"de\":\"4.2222\",\"t\":\"2014-11-25 18:09:24.0\",\"dt\":\"2014-11-25\",\"str\":\"string\",\"c\":\"hello\",\"vc\":\"hello\",\"m\":[{\"_key\":\"k1\",\"_value\":\"v1\"}],\"a\":[100,200],\"st\":{\"i\":10,\"s\":\"foo\"}}", lines[0]);
     Assert.assertEquals("{\"b\":false,\"bt\":20,\"s\":200,\"i\":2000,\"l\":20000,\"f\":8,\"d\":40,\"de\":\"2.2222\",\"t\":\"2014-11-25 18:02:44.0\",\"dt\":\"2014-09-28\",\"str\":\"abcd\",\"c\":\"world\",\"vc\":\"world\",\"m\":[{\"_key\":\"k3\",\"_value\":\"v3\"}],\"a\":[200,300],\"st\":{\"i\":20,\"s\":\"bar\"}}", lines[1]);
   }
@@ -385,7 +388,7 @@ public class TestFileDump {
     FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
 
     // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
     System.out.flush();
     System.setOut(origOut);
@@ -440,7 +443,7 @@ public class TestFileDump {
     FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
 
     // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toString(), "--rowindex=3"});
     System.out.flush();
     System.setOut(origOut);
@@ -494,7 +497,7 @@ public class TestFileDump {
     FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
 
     // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toString(), "--rowindex=2"});
     System.out.flush();
     System.setOut(origOut);
@@ -525,7 +528,7 @@ public class TestFileDump {
       batch.cols[1].noNulls = false;
       batch.cols[1].isNull[row] = true;
     } else {
-      ((BytesColumnVector) batch.cols[1]).setVal(row, str.getBytes());
+      ((BytesColumnVector) batch.cols[1]).setVal(row, str.getBytes(StandardCharsets.UTF_8));
     }
   }
 
@@ -648,7 +651,7 @@ public class TestFileDump {
     FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
 
     // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, false, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toString(), "--rowindex=2"});
     System.out.flush();
     System.setOut(origOut);
diff --git a/java/tools/src/test/org/apache/orc/tools/TestJsonFileDump.java b/java/tools/src/test/org/apache/orc/tools/TestJsonFileDump.java
index 66bb4fd..2ce733f 100644
--- a/java/tools/src/test/org/apache/orc/tools/TestJsonFileDump.java
+++ b/java/tools/src/test/org/apache/orc/tools/TestJsonFileDump.java
@@ -24,11 +24,12 @@ import static org.junit.Assert.assertNull;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileOutputStream;
-import java.io.FileReader;
 import java.io.PrintStream;
 import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.Random;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -69,10 +70,14 @@ public class TestJsonFileDump {
 
   static void checkOutput(String expected,
                                   String actual) throws Exception {
-    BufferedReader eStream =
-        new BufferedReader(new FileReader(getFileFromClasspath(expected)));
-    BufferedReader aStream =
-        new BufferedReader(new FileReader(actual));
+//    BufferedReader eStream =
+//        new BufferedReader(new FileReader(getFileFromClasspath(expected)));
+//    new BufferedReader(new FileReader(getFileFromClasspath(expected)));
+    BufferedReader eStream  = Files
+        .newBufferedReader(Paths.get(getFileFromClasspath(expected)), StandardCharsets.UTF_8);
+//    BufferedReader aStream =
+//        new BufferedReader(new FileReader(actual));
+    BufferedReader aStream = Files.newBufferedReader(Paths.get(actual), StandardCharsets.UTF_8);
     String expectedLine = eStream.readLine();
     while (expectedLine != null) {
       String actualLine = aStream.readLine();
@@ -121,7 +126,7 @@ public class TestJsonFileDump {
         batch.cols[2].isNull[batch.size] = true;
       } else {
         ((BytesColumnVector) batch.cols[2]).setVal(batch.size,
-            words[r1.nextInt(words.length)].getBytes());
+            words[r1.nextInt(words.length)].getBytes(StandardCharsets.UTF_8));
       }
       batch.size += 1;
       if (batch.size == batch.getMaxSize()) {
@@ -139,7 +144,7 @@ public class TestJsonFileDump {
     FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
 
     // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
+    System.setOut(new PrintStream(myOut, true, StandardCharsets.UTF_8.toString()));
     FileDump.main(new String[]{testFilePath.toString(), "-j", "-p", "--rowindex=3"});
     System.out.flush();
     System.setOut(origOut);
diff --git a/java/tools/src/test/org/apache/orc/tools/convert/TestCsvReader.java b/java/tools/src/test/org/apache/orc/tools/convert/TestCsvReader.java
index 12a72dc..6752de6 100644
--- a/java/tools/src/test/org/apache/orc/tools/convert/TestCsvReader.java
+++ b/java/tools/src/test/org/apache/orc/tools/convert/TestCsvReader.java
@@ -18,7 +18,6 @@
 
 package org.apache.orc.tools.convert;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
@@ -40,15 +39,9 @@ import static org.junit.Assert.assertEquals;
 
 public class TestCsvReader {
 
-  Configuration conf;
   Locale defaultLocale;
 
   @Before
-  public void openFileSystem () throws Exception {
-    conf = new Configuration();
-  }
-
-  @Before
   public void storeDefaultLocale() {
     defaultLocale = Locale.getDefault();
     Locale.setDefault(Locale.US);