You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by om...@apache.org on 2016/05/20 21:22:47 UTC

[09/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
index 40cc86f..dad35e3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
@@ -38,7 +38,7 @@ import org.apache.orc.CompressionCodec;
 import org.apache.orc.DataReader;
 import org.apache.orc.OrcConf;
 import org.apache.orc.impl.OutStream;
-import org.apache.hadoop.hive.ql.io.orc.RecordReaderUtils;
+import org.apache.orc.impl.RecordReaderUtils;
 import org.apache.orc.impl.StreamName;
 import org.apache.orc.StripeInformation;
 import org.apache.orc.impl.BufferChunk;

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
index fe46446..b44da06 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.common.io.encoded.EncodedColumnBatch.ColumnStreamD
 import org.apache.orc.CompressionCodec;
 import org.apache.orc.impl.PositionProvider;
 import org.apache.orc.impl.SettableUncompressedStream;
-import org.apache.hadoop.hive.ql.io.orc.TreeReaderFactory;
+import org.apache.orc.impl.TreeReaderFactory;
 import org.apache.orc.OrcProto;
 
 public class EncodedTreeReaderFactory extends TreeReaderFactory {

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
index b20ce28..e4cbd5f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.hadoop.hive.ql.io.orc.FileDump;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.txn.AcidHouseKeeperService;
@@ -36,7 +35,6 @@ import org.junit.Test;
 import org.junit.rules.TestName;
 
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.util.ArrayList;
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
index 6c46257..2fa9ab2 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
@@ -20,14 +20,11 @@ package org.apache.hadoop.hive.ql.exec.vector;
 
 import org.junit.Test;
 
-import java.math.BigDecimal;
-import java.math.RoundingMode;
 import java.sql.Timestamp;
-import java.util.Date;
 import java.util.Random;
 
 import org.apache.hadoop.hive.common.type.RandomTypeUtil;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
 
 import static org.junit.Assert.*;
 
@@ -58,7 +55,7 @@ public class TestTimestampWritableAndColumnVector {
       if (!retrievedTimestamp.equals(randTimestamp)) {
         assertTrue(false);
       }
-      double randDouble = TimestampWritable.getDouble(randTimestamp);
+      double randDouble = TimestampUtils.getDouble(randTimestamp);
       double retrievedDouble = timestampColVector.getDouble(i);
       if (randDouble != retrievedDouble) {
         assertTrue(false);

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
index 1e41fce..e7a044e 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
@@ -33,7 +33,6 @@ import java.util.concurrent.TimeUnit;
 
 import junit.framework.Assert;
 
-import org.apache.hadoop.hive.common.type.Decimal128;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.RandomTypeUtil;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
@@ -44,6 +43,7 @@ import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.*;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
 import org.junit.Test;
@@ -91,8 +91,8 @@ public class TestVectorTypeCasts {
     b.cols[0].noNulls = true;
     VectorExpression expr = new CastDoubleToTimestamp(0, 1);
     expr.evaluate(b);
-    Assert.assertEquals(0.0, TimestampWritable.getDouble(resultV.asScratchTimestamp(3)));
-    Assert.assertEquals(0.5d, TimestampWritable.getDouble(resultV.asScratchTimestamp(4)));
+    Assert.assertEquals(0.0, TimestampUtils.getDouble(resultV.asScratchTimestamp(3)));
+    Assert.assertEquals(0.5d, TimestampUtils.getDouble(resultV.asScratchTimestamp(4)));
   }
 
   @Test
@@ -152,7 +152,7 @@ public class TestVectorTypeCasts {
     expr.evaluate(b);
     for (int i = 0; i < doubleValues.length; i++) {
       double actual = resultV.vector[i];
-      double doubleValue = TimestampWritable.getDouble(inV.asScratchTimestamp(i));
+      double doubleValue = TimestampUtils.getDouble(inV.asScratchTimestamp(i));
       assertEquals(actual, doubleValue, 0.000000001F);
     }
   }
@@ -382,7 +382,7 @@ public class TestVectorTypeCasts {
     TimestampColumnVector r = (TimestampColumnVector) b.cols[1];
     for (int i = 0; i < doubleValues.length; i++) {
       Timestamp timestamp = r.asScratchTimestamp(i);
-      double asDouble = TimestampWritable.getDouble(timestamp);
+      double asDouble = TimestampUtils.getDouble(timestamp);
       double expectedDouble = doubleValues[i];
       if (expectedDouble != asDouble) {
         assertTrue(false);

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
index a7567b7..b78c1f2 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
@@ -27,8 +27,6 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
-import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
-import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFArgDesc;
 import org.apache.hadoop.hive.ql.exec.vector.udf.generic.GenericUDFIsNull;
 import org.apache.hadoop.hive.ql.exec.vector.udf.legacy.ConcatTextLongDoubleUDF;
 import org.apache.hadoop.hive.ql.exec.vector.udf.legacy.LongUDF;

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java
deleted file mode 100644
index 5f0146f..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java
+++ /dev/null
@@ -1,352 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.io.orc;
-
-import static junit.framework.Assert.assertEquals;
-import static org.junit.Assume.assumeTrue;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.PrintStream;
-import java.sql.Timestamp;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.orc.ColumnStatistics;
-import org.apache.orc.impl.ColumnStatisticsImpl;
-import org.apache.orc.DateColumnStatistics;
-import org.apache.orc.DecimalColumnStatistics;
-import org.apache.orc.DoubleColumnStatistics;
-import org.apache.orc.IntegerColumnStatistics;
-import org.apache.orc.StringColumnStatistics;
-import org.apache.orc.StripeStatistics;
-import org.apache.orc.TimestampColumnStatistics;
-import org.apache.orc.TypeDescription;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-/**
- * Test ColumnStatisticsImpl for ORC.
- */
-public class TestColumnStatistics {
-
-  @Test
-  public void testLongMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createInt();
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateInteger(10, 2);
-    stats2.updateInteger(1, 1);
-    stats2.updateInteger(1000, 1);
-    stats1.merge(stats2);
-    IntegerColumnStatistics typed = (IntegerColumnStatistics) stats1;
-    assertEquals(1, typed.getMinimum());
-    assertEquals(1000, typed.getMaximum());
-    stats1.reset();
-    stats1.updateInteger(-10, 1);
-    stats1.updateInteger(10000, 1);
-    stats1.merge(stats2);
-    assertEquals(-10, typed.getMinimum());
-    assertEquals(10000, typed.getMaximum());
-  }
-
-  @Test
-  public void testDoubleMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createDouble();
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateDouble(10.0);
-    stats1.updateDouble(100.0);
-    stats2.updateDouble(1.0);
-    stats2.updateDouble(1000.0);
-    stats1.merge(stats2);
-    DoubleColumnStatistics typed = (DoubleColumnStatistics) stats1;
-    assertEquals(1.0, typed.getMinimum(), 0.001);
-    assertEquals(1000.0, typed.getMaximum(), 0.001);
-    stats1.reset();
-    stats1.updateDouble(-10);
-    stats1.updateDouble(10000);
-    stats1.merge(stats2);
-    assertEquals(-10, typed.getMinimum(), 0.001);
-    assertEquals(10000, typed.getMaximum(), 0.001);
-  }
-
-
-  @Test
-  public void testStringMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createString();
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateString(new Text("bob"));
-    stats1.updateString(new Text("david"));
-    stats1.updateString(new Text("charles"));
-    stats2.updateString(new Text("anne"));
-    byte[] erin = new byte[]{0, 1, 2, 3, 4, 5, 101, 114, 105, 110};
-    stats2.updateString(erin, 6, 4, 5);
-    assertEquals(24, ((StringColumnStatistics)stats2).getSum());
-    stats1.merge(stats2);
-    StringColumnStatistics typed = (StringColumnStatistics) stats1;
-    assertEquals("anne", typed.getMinimum());
-    assertEquals("erin", typed.getMaximum());
-    assertEquals(39, typed.getSum());
-    stats1.reset();
-    stats1.updateString(new Text("aaa"));
-    stats1.updateString(new Text("zzz"));
-    stats1.merge(stats2);
-    assertEquals("aaa", typed.getMinimum());
-    assertEquals("zzz", typed.getMaximum());
-  }
-
-  @Test
-  public void testDateMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createDate();
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateDate(new DateWritable(1000));
-    stats1.updateDate(new DateWritable(100));
-    stats2.updateDate(new DateWritable(10));
-    stats2.updateDate(new DateWritable(2000));
-    stats1.merge(stats2);
-    DateColumnStatistics typed = (DateColumnStatistics) stats1;
-    assertEquals(new DateWritable(10).get(), typed.getMinimum());
-    assertEquals(new DateWritable(2000).get(), typed.getMaximum());
-    stats1.reset();
-    stats1.updateDate(new DateWritable(-10));
-    stats1.updateDate(new DateWritable(10000));
-    stats1.merge(stats2);
-    assertEquals(new DateWritable(-10).get(), typed.getMinimum());
-    assertEquals(new DateWritable(10000).get(), typed.getMaximum());
-  }
-
-  @Test
-  public void testTimestampMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createTimestamp();
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateTimestamp(new Timestamp(10));
-    stats1.updateTimestamp(new Timestamp(100));
-    stats2.updateTimestamp(new Timestamp(1));
-    stats2.updateTimestamp(new Timestamp(1000));
-    stats1.merge(stats2);
-    TimestampColumnStatistics typed = (TimestampColumnStatistics) stats1;
-    assertEquals(1, typed.getMinimum().getTime());
-    assertEquals(1000, typed.getMaximum().getTime());
-    stats1.reset();
-    stats1.updateTimestamp(new Timestamp(-10));
-    stats1.updateTimestamp(new Timestamp(10000));
-    stats1.merge(stats2);
-    assertEquals(-10, typed.getMinimum().getTime());
-    assertEquals(10000, typed.getMaximum().getTime());
-  }
-
-  @Test
-  public void testDecimalMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createDecimal()
-        .withPrecision(38).withScale(16);
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateDecimal(HiveDecimal.create(10));
-    stats1.updateDecimal(HiveDecimal.create(100));
-    stats2.updateDecimal(HiveDecimal.create(1));
-    stats2.updateDecimal(HiveDecimal.create(1000));
-    stats1.merge(stats2);
-    DecimalColumnStatistics typed = (DecimalColumnStatistics) stats1;
-    assertEquals(1, typed.getMinimum().longValue());
-    assertEquals(1000, typed.getMaximum().longValue());
-    stats1.reset();
-    stats1.updateDecimal(HiveDecimal.create(-10));
-    stats1.updateDecimal(HiveDecimal.create(10000));
-    stats1.merge(stats2);
-    assertEquals(-10, typed.getMinimum().longValue());
-    assertEquals(10000, typed.getMaximum().longValue());
-  }
-
-
-  public static class SimpleStruct {
-    BytesWritable bytes1;
-    Text string1;
-
-    SimpleStruct(BytesWritable b1, String s1) {
-      this.bytes1 = b1;
-      if (s1 == null) {
-        this.string1 = null;
-      } else {
-        this.string1 = new Text(s1);
-      }
-    }
-  }
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir",
-      "target" + File.separator + "test" + File.separator + "tmp"));
-
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-
-  @Rule
-  public TestName testCaseName = new TestName();
-
-  @Before
-  public void openFileSystem() throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    fs.setWorkingDirectory(workDir);
-    testFilePath = new Path("TestOrcFile." + testCaseName.getMethodName() + ".orc");
-    fs.delete(testFilePath, false);
-  }
-
-  private static BytesWritable bytes(int... items) {
-    BytesWritable result = new BytesWritable();
-    result.setSize(items.length);
-    for (int i = 0; i < items.length; ++i) {
-      result.getBytes()[i] = (byte) items[i];
-    }
-    return result;
-  }
-
-  @Test
-  public void testHasNull() throws Exception {
-
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (SimpleStruct.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .inspector(inspector)
-            .rowIndexStride(1000)
-            .stripeSize(10000)
-            .bufferSize(10000));
-    // STRIPE 1
-    // RG1
-    for(int i=0; i<1000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), "RG1"));
-    }
-    // RG2
-    for(int i=0; i<1000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), null));
-    }
-    // RG3
-    for(int i=0; i<1000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), "RG3"));
-    }
-    // RG4
-    for(int i=0; i<1000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), null));
-    }
-    // RG5
-    for(int i=0; i<1000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), null));
-    }
-    // STRIPE 2
-    for(int i=0; i<5000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), null));
-    }
-    // STRIPE 3
-    for(int i=0; i<5000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), "STRIPE-3"));
-    }
-    // STRIPE 4
-    for(int i=0; i<5000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), null));
-    }
-    writer.close();
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-
-    // check the file level stats
-    ColumnStatistics[] stats = reader.getStatistics();
-    assertEquals(20000, stats[0].getNumberOfValues());
-    assertEquals(20000, stats[1].getNumberOfValues());
-    assertEquals(7000, stats[2].getNumberOfValues());
-    assertEquals(false, stats[0].hasNull());
-    assertEquals(false, stats[1].hasNull());
-    assertEquals(true, stats[2].hasNull());
-
-    // check the stripe level stats
-    List<StripeStatistics> stripeStats = reader.getStripeStatistics();
-    // stripe 1 stats
-    StripeStatistics ss1 = stripeStats.get(0);
-    ColumnStatistics ss1_cs1 = ss1.getColumnStatistics()[0];
-    ColumnStatistics ss1_cs2 = ss1.getColumnStatistics()[1];
-    ColumnStatistics ss1_cs3 = ss1.getColumnStatistics()[2];
-    assertEquals(false, ss1_cs1.hasNull());
-    assertEquals(false, ss1_cs2.hasNull());
-    assertEquals(true, ss1_cs3.hasNull());
-
-    // stripe 2 stats
-    StripeStatistics ss2 = stripeStats.get(1);
-    ColumnStatistics ss2_cs1 = ss2.getColumnStatistics()[0];
-    ColumnStatistics ss2_cs2 = ss2.getColumnStatistics()[1];
-    ColumnStatistics ss2_cs3 = ss2.getColumnStatistics()[2];
-    assertEquals(false, ss2_cs1.hasNull());
-    assertEquals(false, ss2_cs2.hasNull());
-    assertEquals(true, ss2_cs3.hasNull());
-
-    // stripe 3 stats
-    StripeStatistics ss3 = stripeStats.get(2);
-    ColumnStatistics ss3_cs1 = ss3.getColumnStatistics()[0];
-    ColumnStatistics ss3_cs2 = ss3.getColumnStatistics()[1];
-    ColumnStatistics ss3_cs3 = ss3.getColumnStatistics()[2];
-    assertEquals(false, ss3_cs1.hasNull());
-    assertEquals(false, ss3_cs2.hasNull());
-    assertEquals(false, ss3_cs3.hasNull());
-
-    // stripe 4 stats
-    StripeStatistics ss4 = stripeStats.get(3);
-    ColumnStatistics ss4_cs1 = ss4.getColumnStatistics()[0];
-    ColumnStatistics ss4_cs2 = ss4.getColumnStatistics()[1];
-    ColumnStatistics ss4_cs3 = ss4.getColumnStatistics()[2];
-    assertEquals(false, ss4_cs1.hasNull());
-    assertEquals(false, ss4_cs2.hasNull());
-    assertEquals(true, ss4_cs3.hasNull());
-
-    // Test file dump
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-has-null.out";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=2"});
-    System.out.flush();
-    System.setOut(origOut);
-    // If called with an expression evaluating to false, the test will halt
-    // and be ignored.
-    assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
-    TestFileDump.checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java
deleted file mode 100644
index 554033c..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java
+++ /dev/null
@@ -1,418 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.io.orc;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-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.sql.Date;
-import java.sql.Timestamp;
-import java.util.Arrays;
-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;
-import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hive.common.util.HiveTestUtils;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestFileDump {
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir"));
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-
-  @Before
-  public void openFileSystem () throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    fs.setWorkingDirectory(workDir);
-    testFilePath = new Path("TestFileDump.testDump.orc");
-    fs.delete(testFilePath, false);
-  }
-
-  static class MyRecord {
-    int i;
-    long l;
-    String s;
-    MyRecord(int i, long l, String s) {
-      this.i = i;
-      this.l = l;
-      this.s = s;
-    }
-  }
-
-  static class AllTypesRecord {
-    static class Struct {
-      int i;
-      String s;
-
-      Struct(int i, String s) {
-        this.i = i;
-        this.s = s;
-      }
-    }
-    boolean b;
-    byte bt;
-    short s;
-    int i;
-    long l;
-    float f;
-    double d;
-    HiveDecimal de;
-    Timestamp t;
-    Date dt;
-    String str;
-    HiveChar c;
-    HiveVarchar vc;
-    Map<String, String> m;
-    List<Integer> a;
-    Struct st;
-
-    AllTypesRecord(boolean b, byte bt, short s, int i, long l, float f, double d, HiveDecimal de,
-                   Timestamp t, Date dt, String str, HiveChar c, HiveVarchar vc, Map<String,
-                   String> m, List<Integer> a, Struct st) {
-      this.b = b;
-      this.bt = bt;
-      this.s = s;
-      this.i = i;
-      this.l = l;
-      this.f = f;
-      this.d = d;
-      this.de = de;
-      this.t = t;
-      this.dt = dt;
-      this.str = str;
-      this.c = c;
-      this.vc = vc;
-      this.m = m;
-      this.a = a;
-      this.st = st;
-    }
-  }
-
-  static void checkOutput(String expected,
-                                  String actual) throws Exception {
-    BufferedReader eStream =
-        new BufferedReader(new FileReader(HiveTestUtils.getFileFromClasspath(expected)));
-    BufferedReader aStream =
-        new BufferedReader(new FileReader(actual));
-    String expectedLine = eStream.readLine().trim();
-    while (expectedLine != null) {
-      String actualLine = aStream.readLine().trim();
-      System.out.println("actual:   " + actualLine);
-      System.out.println("expected: " + expectedLine);
-      assertEquals(expectedLine, actualLine);
-      expectedLine = eStream.readLine();
-      expectedLine = expectedLine == null ? null : expectedLine.trim();
-    }
-    assertNull(eStream.readLine());
-    assertNull(aStream.readLine());
-    eStream.close();
-    aStream.close();
-  }
-
-  @Test
-  public void testDump() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .fileSystem(fs)
-            .inspector(inspector)
-            .batchSize(1000)
-            .compress(CompressionKind.ZLIB)
-            .stripeSize(100000)
-            .rowIndexStride(1000));
-    Random r1 = new Random(1);
-    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
-        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
-        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
-        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
-        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
-        "it", "was", "the", "season", "of", "Darkness,", "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"};
-    for(int i=0; i < 21000; ++i) {
-      writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(),
-          words[r1.nextInt(words.length)]));
-    }
-    writer.close();
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-dump.out";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
-    System.out.flush();
-    System.setOut(origOut);
-
-
-    checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-
-  @Test
-  public void testDataDump() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (AllTypesRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
-        100000, CompressionKind.NONE, 10000, 1000);
-    Map<String, String> m = new HashMap<String, String>(2);
-    m.put("k1", "v1");
-    writer.addRow(new AllTypesRecord(
-        true,
-        (byte) 10,
-        (short) 100,
-        1000,
-        10000L,
-        4.0f,
-        20.0,
-        HiveDecimal.create("4.2222"),
-        new Timestamp(1416967764000L),
-        new Date(1416967764000L),
-        "string",
-        new HiveChar("hello", 5),
-        new HiveVarchar("hello", 10),
-        m,
-        Arrays.asList(100, 200),
-        new AllTypesRecord.Struct(10, "foo")));
-    m.clear();
-    m.put("k3", "v3");
-    writer.addRow(new AllTypesRecord(
-        false,
-        (byte)20,
-        (short)200,
-        2000,
-        20000L,
-        8.0f,
-        40.0,
-        HiveDecimal.create("2.2222"),
-        new Timestamp(1416967364000L),
-        new Date(1411967764000L),
-        "abcd",
-        new HiveChar("world", 5),
-        new HiveVarchar("world", 10),
-        m,
-        Arrays.asList(200, 300),
-        new AllTypesRecord.Struct(20, "bar")));
-
-    writer.close();
-    PrintStream origOut = System.out;
-    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "-d"});
-    System.out.flush();
-    System.setOut(origOut);
-
-    String[] lines = myOut.toString().split("\n");
-    // Don't be fooled by the big space in the middle, this line is quite long
-    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\",\"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]);
-    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\",\"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]);
-  }
-  
-  // Test that if the fraction of rows that have distinct strings is greater than the configured
-  // threshold dictionary encoding is turned off.  If dictionary encoding is turned off the length
-  // of the dictionary stream for the column will be 0 in the ORC file dump.
-  @Test
-  public void testDictionaryThreshold() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    Configuration conf = new Configuration();
-    conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
-    conf.setFloat(HiveConf.ConfVars.HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD.varname, 0.49f);
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .fileSystem(fs)
-            .batchSize(1000)
-            .inspector(inspector)
-            .stripeSize(100000)
-            .compress(CompressionKind.ZLIB)
-            .rowIndexStride(1000)
-            .bufferSize(10000));
-    Random r1 = new Random(1);
-    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
-        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
-        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
-        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
-        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
-        "it", "was", "the", "season", "of", "Darkness,", "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"};
-    int nextInt = 0;
-    for(int i=0; i < 21000; ++i) {
-      // Write out the same string twice, this guarantees the fraction of rows with
-      // distinct strings is 0.5
-      if (i % 2 == 0) {
-        nextInt = r1.nextInt(words.length);
-        // Append the value of i to the word, this guarantees when an index or word is repeated
-        // the actual string is unique.
-        words[nextInt] += "-" + i;
-      }
-      writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(),
-          words[nextInt]));
-    }
-    writer.close();
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-dump-dictionary-threshold.out";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
-    System.out.flush();
-    System.setOut(origOut);
-
-    checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-
-  @Test
-  public void testBloomFilter() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
-    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
-        .fileSystem(fs)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .compress(CompressionKind.ZLIB)
-        .bufferSize(10000)
-        .rowIndexStride(1000)
-        .batchSize(1000)
-        .bloomFilterColumns("S");
-    Writer writer = OrcFile.createWriter(testFilePath, options);
-    Random r1 = new Random(1);
-    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
-        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
-        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
-        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
-        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
-        "it", "was", "the", "season", "of", "Darkness,", "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"};
-    for(int i=0; i < 21000; ++i) {
-      writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(),
-          words[r1.nextInt(words.length)]));
-    }
-    writer.close();
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-dump-bloomfilter.out";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=3"});
-    System.out.flush();
-    System.setOut(origOut);
-
-
-    checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-
-  @Test
-  public void testBloomFilter2() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
-    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
-        .fileSystem(fs)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .compress(CompressionKind.ZLIB)
-        .bufferSize(10000)
-        .rowIndexStride(1000)
-        .bloomFilterColumns("l")
-        .bloomFilterFpp(0.01)
-        .batchSize(1000);
-    Writer writer = OrcFile.createWriter(testFilePath, options);
-    Random r1 = new Random(1);
-    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
-        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
-        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
-        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
-        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
-        "it", "was", "the", "season", "of", "Darkness,", "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"};
-    for(int i=0; i < 21000; ++i) {
-      writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(),
-          words[r1.nextInt(words.length)]));
-    }
-    writer.close();
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-dump-bloomfilter2.out";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=2"});
-    System.out.flush();
-    System.setOut(origOut);
-
-
-    checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestJsonFileDump.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestJsonFileDump.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestJsonFileDump.java
deleted file mode 100644
index acf232d..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestJsonFileDump.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.io.orc;
-
-import static org.junit.Assert.assertEquals;
-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.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.conf.HiveConf;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hive.common.util.HiveTestUtils;
-import org.apache.orc.CompressionKind;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestJsonFileDump {
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir"));
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-
-  @Before
-  public void openFileSystem () throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    fs.setWorkingDirectory(workDir);
-    testFilePath = new Path("TestFileDump.testDump.orc");
-    fs.delete(testFilePath, false);
-  }
-
-  static class MyRecord {
-    int i;
-    long l;
-    String s;
-    MyRecord(int i, long l, String s) {
-      this.i = i;
-      this.l = l;
-      this.s = s;
-    }
-  }
-
-  static void checkOutput(String expected,
-                                  String actual) throws Exception {
-    BufferedReader eStream =
-        new BufferedReader(new FileReader(HiveTestUtils.getFileFromClasspath(expected)));
-    BufferedReader aStream =
-        new BufferedReader(new FileReader(actual));
-    String expectedLine = eStream.readLine();
-    while (expectedLine != null) {
-      String actualLine = aStream.readLine();
-      System.out.println("actual:   " + actualLine);
-      System.out.println("expected: " + expectedLine);
-      assertEquals(expectedLine, actualLine);
-      expectedLine = eStream.readLine();
-    }
-    assertNull(eStream.readLine());
-    assertNull(aStream.readLine());
-  }
-
-  @Test
-  public void testJsonDump() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
-    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
-        .fileSystem(fs)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .compress(CompressionKind.ZLIB)
-        .bufferSize(10000)
-        .rowIndexStride(1000)
-        .bloomFilterColumns("s");
-    Writer writer = OrcFile.createWriter(testFilePath, options);
-    Random r1 = new Random(1);
-    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
-        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
-        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
-        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
-        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
-        "it", "was", "the", "season", "of", "Darkness,", "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"};
-    for(int i=0; i < 21000; ++i) {
-      if (i % 100 == 0) {
-        writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(), null));
-      } else {
-        writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(),
-            words[r1.nextInt(words.length)]));
-      }
-    }
-
-    writer.close();
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-dump.json";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "-j", "-p", "--rowindex=3"});
-    System.out.flush();
-    System.setOut(origOut);
-
-
-    checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-}