You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pr...@apache.org on 2015/10/26 20:18:57 UTC

hive git commit: HIVE-11523: org.apache.hadoop.hive.ql.io.orc.FileDump should handle errors (Prasanth Jayachandran reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master a91e1471c -> d84e393ed


HIVE-11523: org.apache.hadoop.hive.ql.io.orc.FileDump should handle errors (Prasanth Jayachandran reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: d84e393ed66829fe0c8cc87254fef2a329b96163
Parents: a91e147
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Mon Oct 26 14:18:36 2015 -0500
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Mon Oct 26 14:18:36 2015 -0500

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/io/orc/FileDump.java  | 193 +++++++++--------
 .../hadoop/hive/ql/io/orc/JsonFileDump.java     | 210 ++++++++++---------
 .../hadoop/hive/ql/io/orc/TestFileDump.java     |  50 -----
 3 files changed, 213 insertions(+), 240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d84e393e/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
index a1c5058..9c6538f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
@@ -136,9 +136,16 @@ public final class FileDump {
   private static void printData(List<String> files, Configuration conf) throws IOException,
       JSONException {
     for (String file : files) {
-      printJsonData(conf, file);
-      if (files.size() > 1) {
-        System.out.println(Strings.repeat("=", 80) + "\n");
+      try {
+        printJsonData(conf, file);
+        if (files.size() > 1) {
+          System.out.println(Strings.repeat("=", 80) + "\n");
+        }
+      } catch (Exception e) {
+        System.err.println("Unable to dump data for file: " + file);
+        e.printStackTrace();
+        System.err.println(Strings.repeat("=", 80) + "\n");
+        continue;
       }
     }
   }
@@ -146,103 +153,111 @@ public final class FileDump {
   private static void printMetaData(List<String> files, Configuration conf,
       List<Integer> rowIndexCols, boolean printTimeZone) throws IOException {
     for (String filename : files) {
-      System.out.println("Structure for " + filename);
-      Path path = new Path(filename);
-      Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
-      System.out.println("File Version: " + reader.getFileVersion().getName() +
-          " with " + reader.getWriterVersion());
-      RecordReaderImpl rows = (RecordReaderImpl) reader.rows();
-      System.out.println("Rows: " + reader.getNumberOfRows());
-      System.out.println("Compression: " + reader.getCompression());
-      if (reader.getCompression() != CompressionKind.NONE) {
-        System.out.println("Compression size: " + reader.getCompressionSize());
-      }
-      System.out.println("Type: " + reader.getObjectInspector().getTypeName());
-      System.out.println("\nStripe Statistics:");
-      List<StripeStatistics> stripeStats = reader.getStripeStatistics();
-      for (int n = 0; n < stripeStats.size(); n++) {
-        System.out.println("  Stripe " + (n + 1) + ":");
-        StripeStatistics ss = stripeStats.get(n);
-        for (int i = 0; i < ss.getColumnStatistics().length; ++i) {
-          System.out.println("    Column " + i + ": " +
-              ss.getColumnStatistics()[i].toString());
+      try {
+        Path path = new Path(filename);
+        Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
+        System.out.println("Structure for " + filename);
+        System.out.println("File Version: " + reader.getFileVersion().getName() +
+            " with " + reader.getWriterVersion());
+        RecordReaderImpl rows = (RecordReaderImpl) reader.rows();
+        System.out.println("Rows: " + reader.getNumberOfRows());
+        System.out.println("Compression: " + reader.getCompression());
+        if (reader.getCompression() != CompressionKind.NONE) {
+          System.out.println("Compression size: " + reader.getCompressionSize());
         }
-      }
-      ColumnStatistics[] stats = reader.getStatistics();
-      int colCount = stats.length;
-      System.out.println("\nFile Statistics:");
-      for (int i = 0; i < stats.length; ++i) {
-        System.out.println("  Column " + i + ": " + stats[i].toString());
-      }
-      System.out.println("\nStripes:");
-      int stripeIx = -1;
-      for (StripeInformation stripe : reader.getStripes()) {
-        ++stripeIx;
-        long stripeStart = stripe.getOffset();
-        OrcProto.StripeFooter footer = rows.readStripeFooter(stripe);
-        if (printTimeZone) {
-          String tz = footer.getWriterTimezone();
-          if (tz == null || tz.isEmpty()) {
-            tz = UNKNOWN;
+        System.out.println("Type: " + reader.getObjectInspector().getTypeName());
+        System.out.println("\nStripe Statistics:");
+        List<StripeStatistics> stripeStats = reader.getStripeStatistics();
+        for (int n = 0; n < stripeStats.size(); n++) {
+          System.out.println("  Stripe " + (n + 1) + ":");
+          StripeStatistics ss = stripeStats.get(n);
+          for (int i = 0; i < ss.getColumnStatistics().length; ++i) {
+            System.out.println("    Column " + i + ": " +
+                ss.getColumnStatistics()[i].toString());
           }
-          System.out.println("  Stripe: " + stripe.toString() + " timezone: " + tz);
-        } else {
-          System.out.println("  Stripe: " + stripe.toString());
         }
-        long sectionStart = stripeStart;
-        for (OrcProto.Stream section : footer.getStreamsList()) {
-          String kind = section.hasKind() ? section.getKind().name() : UNKNOWN;
-          System.out.println("    Stream: column " + section.getColumn() +
-              " section " + kind + " start: " + sectionStart +
-              " length " + section.getLength());
-          sectionStart += section.getLength();
+        ColumnStatistics[] stats = reader.getStatistics();
+        int colCount = stats.length;
+        System.out.println("\nFile Statistics:");
+        for (int i = 0; i < stats.length; ++i) {
+          System.out.println("  Column " + i + ": " + stats[i].toString());
         }
-        for (int i = 0; i < footer.getColumnsCount(); ++i) {
-          OrcProto.ColumnEncoding encoding = footer.getColumns(i);
-          StringBuilder buf = new StringBuilder();
-          buf.append("    Encoding column ");
-          buf.append(i);
-          buf.append(": ");
-          buf.append(encoding.getKind());
-          if (encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
-              encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
-            buf.append("[");
-            buf.append(encoding.getDictionarySize());
-            buf.append("]");
+        System.out.println("\nStripes:");
+        int stripeIx = -1;
+        for (StripeInformation stripe : reader.getStripes()) {
+          ++stripeIx;
+          long stripeStart = stripe.getOffset();
+          OrcProto.StripeFooter footer = rows.readStripeFooter(stripe);
+          if (printTimeZone) {
+            String tz = footer.getWriterTimezone();
+            if (tz == null || tz.isEmpty()) {
+              tz = UNKNOWN;
+            }
+            System.out.println("  Stripe: " + stripe.toString() + " timezone: " + tz);
+          } else {
+            System.out.println("  Stripe: " + stripe.toString());
           }
-          System.out.println(buf);
-        }
-        if (rowIndexCols != null && !rowIndexCols.isEmpty()) {
-          // include the columns that are specified, only if the columns are included, bloom filter
-          // will be read
-          boolean[] sargColumns = new boolean[colCount];
-          for (int colIdx : rowIndexCols) {
-            sargColumns[colIdx] = true;
+          long sectionStart = stripeStart;
+          for (OrcProto.Stream section : footer.getStreamsList()) {
+            String kind = section.hasKind() ? section.getKind().name() : UNKNOWN;
+            System.out.println("    Stream: column " + section.getColumn() +
+                " section " + kind + " start: " + sectionStart +
+                " length " + section.getLength());
+            sectionStart += section.getLength();
           }
-          RecordReaderImpl.Index indices = rows.readRowIndex(stripeIx, null, null, null, sargColumns);
-          for (int col : rowIndexCols) {
+          for (int i = 0; i < footer.getColumnsCount(); ++i) {
+            OrcProto.ColumnEncoding encoding = footer.getColumns(i);
             StringBuilder buf = new StringBuilder();
-            String rowIdxString = getFormattedRowIndices(col, indices.getRowGroupIndex());
-            buf.append(rowIdxString);
-            String bloomFilString = getFormattedBloomFilters(col, indices.getBloomFilterIndex());
-            buf.append(bloomFilString);
+            buf.append("    Encoding column ");
+            buf.append(i);
+            buf.append(": ");
+            buf.append(encoding.getKind());
+            if (encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
+                encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
+              buf.append("[");
+              buf.append(encoding.getDictionarySize());
+              buf.append("]");
+            }
             System.out.println(buf);
           }
+          if (rowIndexCols != null && !rowIndexCols.isEmpty()) {
+            // include the columns that are specified, only if the columns are included, bloom filter
+            // will be read
+            boolean[] sargColumns = new boolean[colCount];
+            for (int colIdx : rowIndexCols) {
+              sargColumns[colIdx] = true;
+            }
+            RecordReaderImpl.Index indices = rows
+                .readRowIndex(stripeIx, null, null, null, sargColumns);
+            for (int col : rowIndexCols) {
+              StringBuilder buf = new StringBuilder();
+              String rowIdxString = getFormattedRowIndices(col, indices.getRowGroupIndex());
+              buf.append(rowIdxString);
+              String bloomFilString = getFormattedBloomFilters(col, indices.getBloomFilterIndex());
+              buf.append(bloomFilString);
+              System.out.println(buf);
+            }
+          }
         }
-      }
 
-      FileSystem fs = path.getFileSystem(conf);
-      long fileLen = fs.getContentSummary(path).getLength();
-      long paddedBytes = getTotalPaddingSize(reader);
-      // empty ORC file is ~45 bytes. Assumption here is file length always >0
-      double percentPadding = ((double) paddedBytes / (double) fileLen) * 100;
-      DecimalFormat format = new DecimalFormat("##.##");
-      System.out.println("\nFile length: " + fileLen + " bytes");
-      System.out.println("Padding length: " + paddedBytes + " bytes");
-      System.out.println("Padding ratio: " + format.format(percentPadding) + "%");
-      rows.close();
-      if (files.size() > 1) {
-        System.out.println(Strings.repeat("=", 80) + "\n");
+        FileSystem fs = path.getFileSystem(conf);
+        long fileLen = fs.getContentSummary(path).getLength();
+        long paddedBytes = getTotalPaddingSize(reader);
+        // empty ORC file is ~45 bytes. Assumption here is file length always >0
+        double percentPadding = ((double) paddedBytes / (double) fileLen) * 100;
+        DecimalFormat format = new DecimalFormat("##.##");
+        System.out.println("\nFile length: " + fileLen + " bytes");
+        System.out.println("Padding length: " + paddedBytes + " bytes");
+        System.out.println("Padding ratio: " + format.format(percentPadding) + "%");
+        rows.close();
+        if (files.size() > 1) {
+          System.out.println(Strings.repeat("=", 80) + "\n");
+        }
+      } catch (Exception e) {
+        System.err.println("Unable to dump metadata for file: " + filename);
+        e.printStackTrace();
+        System.err.println(Strings.repeat("=", 80) + "\n");
+        continue;
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/d84e393e/ql/src/java/org/apache/hadoop/hive/ql/io/orc/JsonFileDump.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/JsonFileDump.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/JsonFileDump.java
index a438855..02e01b4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/JsonFileDump.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/JsonFileDump.java
@@ -45,130 +45,138 @@ public class JsonFileDump {
       writer.object();
     }
     for (String filename : files) {
-      if (multiFile) {
-        writer.object();
-      }
-      writer.key("fileName").value(filename);
-      Path path = new Path(filename);
-      Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
-      writer.key("fileVersion").value(reader.getFileVersion().getName());
-      writer.key("writerVersion").value(reader.getWriterVersion());
-      RecordReaderImpl rows = (RecordReaderImpl) reader.rows();
-      writer.key("numberOfRows").value(reader.getNumberOfRows());
-      writer.key("compression").value(reader.getCompression());
-      if (reader.getCompression() != CompressionKind.NONE) {
-        writer.key("compressionBufferSize").value(reader.getCompressionSize());
-      }
-      writer.key("schemaString").value(reader.getObjectInspector().getTypeName());
-      writer.key("schema").array();
-      writeSchema(writer, reader.getTypes());
-      writer.endArray();
-
-      writer.key("stripeStatistics").array();
-      List<StripeStatistics> stripeStatistics = reader.getStripeStatistics();
-      for (int n = 0; n < stripeStatistics.size(); n++) {
-        writer.object();
-        writer.key("stripeNumber").value(n + 1);
-        StripeStatistics ss = stripeStatistics.get(n);
-        writer.key("columnStatistics").array();
-        for (int i = 0; i < ss.getColumnStatistics().length; i++) {
+      try {
+        if (multiFile) {
           writer.object();
-          writer.key("columnId").value(i);
-          writeColumnStatistics(writer, ss.getColumnStatistics()[i]);
-          writer.endObject();
         }
-        writer.endArray();
-        writer.endObject();
-      }
-      writer.endArray();
-
-      ColumnStatistics[] stats = reader.getStatistics();
-      int colCount = stats.length;
-      writer.key("fileStatistics").array();
-      for (int i = 0; i < stats.length; ++i) {
-        writer.object();
-        writer.key("columnId").value(i);
-        writeColumnStatistics(writer, stats[i]);
-        writer.endObject();
-      }
-      writer.endArray();
-
-      writer.key("stripes").array();
-      int stripeIx = -1;
-      for (StripeInformation stripe : reader.getStripes()) {
-        ++stripeIx;
-        long stripeStart = stripe.getOffset();
-        OrcProto.StripeFooter footer = rows.readStripeFooter(stripe);
-        writer.object(); // start of stripe information
-        writer.key("stripeNumber").value(stripeIx + 1);
-        writer.key("stripeInformation");
-        writeStripeInformation(writer, stripe);
-        if (printTimeZone) {
-          writer.key("writerTimezone").value(
-              footer.hasWriterTimezone() ? footer.getWriterTimezone() : FileDump.UNKNOWN);
+        writer.key("fileName").value(filename);
+        Path path = new Path(filename);
+        Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
+        writer.key("fileVersion").value(reader.getFileVersion().getName());
+        writer.key("writerVersion").value(reader.getWriterVersion());
+        RecordReaderImpl rows = (RecordReaderImpl) reader.rows();
+        writer.key("numberOfRows").value(reader.getNumberOfRows());
+        writer.key("compression").value(reader.getCompression());
+        if (reader.getCompression() != CompressionKind.NONE) {
+          writer.key("compressionBufferSize").value(reader.getCompressionSize());
         }
-        long sectionStart = stripeStart;
+        writer.key("schemaString").value(reader.getObjectInspector().getTypeName());
+        writer.key("schema").array();
+        writeSchema(writer, reader.getTypes());
+        writer.endArray();
 
-        writer.key("streams").array();
-        for (OrcProto.Stream section : footer.getStreamsList()) {
+        writer.key("stripeStatistics").array();
+        List<StripeStatistics> stripeStatistics = reader.getStripeStatistics();
+        for (int n = 0; n < stripeStatistics.size(); n++) {
           writer.object();
-          String kind = section.hasKind() ? section.getKind().name() : FileDump.UNKNOWN;
-          writer.key("columnId").value(section.getColumn());
-          writer.key("section").value(kind);
-          writer.key("startOffset").value(sectionStart);
-          writer.key("length").value(section.getLength());
-          sectionStart += section.getLength();
+          writer.key("stripeNumber").value(n + 1);
+          StripeStatistics ss = stripeStatistics.get(n);
+          writer.key("columnStatistics").array();
+          for (int i = 0; i < ss.getColumnStatistics().length; i++) {
+            writer.object();
+            writer.key("columnId").value(i);
+            writeColumnStatistics(writer, ss.getColumnStatistics()[i]);
+            writer.endObject();
+          }
+          writer.endArray();
           writer.endObject();
         }
         writer.endArray();
 
-        writer.key("encodings").array();
-        for (int i = 0; i < footer.getColumnsCount(); ++i) {
+        ColumnStatistics[] stats = reader.getStatistics();
+        int colCount = stats.length;
+        writer.key("fileStatistics").array();
+        for (int i = 0; i < stats.length; ++i) {
           writer.object();
-          OrcProto.ColumnEncoding encoding = footer.getColumns(i);
           writer.key("columnId").value(i);
-          writer.key("kind").value(encoding.getKind());
-          if (encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
-              encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
-            writer.key("dictionarySize").value(encoding.getDictionarySize());
-          }
+          writeColumnStatistics(writer, stats[i]);
           writer.endObject();
         }
         writer.endArray();
 
-        if (rowIndexCols != null && !rowIndexCols.isEmpty()) {
-          // include the columns that are specified, only if the columns are included, bloom filter
-          // will be read
-          boolean[] sargColumns = new boolean[colCount];
-          for (int colIdx : rowIndexCols) {
-            sargColumns[colIdx] = true;
+        writer.key("stripes").array();
+        int stripeIx = -1;
+        for (StripeInformation stripe : reader.getStripes()) {
+          ++stripeIx;
+          long stripeStart = stripe.getOffset();
+          OrcProto.StripeFooter footer = rows.readStripeFooter(stripe);
+          writer.object(); // start of stripe information
+          writer.key("stripeNumber").value(stripeIx + 1);
+          writer.key("stripeInformation");
+          writeStripeInformation(writer, stripe);
+          if (printTimeZone) {
+            writer.key("writerTimezone").value(
+                footer.hasWriterTimezone() ? footer.getWriterTimezone() : FileDump.UNKNOWN);
           }
-          RecordReaderImpl.Index indices = rows.readRowIndex(stripeIx, null, sargColumns);
-          writer.key("indexes").array();
-          for (int col : rowIndexCols) {
+          long sectionStart = stripeStart;
+
+          writer.key("streams").array();
+          for (OrcProto.Stream section : footer.getStreamsList()) {
             writer.object();
-            writer.key("columnId").value(col);
-            writeRowGroupIndexes(writer, col, indices.getRowGroupIndex());
-            writeBloomFilterIndexes(writer, col, indices.getBloomFilterIndex());
+            String kind = section.hasKind() ? section.getKind().name() : FileDump.UNKNOWN;
+            writer.key("columnId").value(section.getColumn());
+            writer.key("section").value(kind);
+            writer.key("startOffset").value(sectionStart);
+            writer.key("length").value(section.getLength());
+            sectionStart += section.getLength();
             writer.endObject();
           }
           writer.endArray();
+
+          writer.key("encodings").array();
+          for (int i = 0; i < footer.getColumnsCount(); ++i) {
+            writer.object();
+            OrcProto.ColumnEncoding encoding = footer.getColumns(i);
+            writer.key("columnId").value(i);
+            writer.key("kind").value(encoding.getKind());
+            if (encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
+                encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
+              writer.key("dictionarySize").value(encoding.getDictionarySize());
+            }
+            writer.endObject();
+          }
+          writer.endArray();
+
+          if (rowIndexCols != null && !rowIndexCols.isEmpty()) {
+            // include the columns that are specified, only if the columns are included, bloom filter
+            // will be read
+            boolean[] sargColumns = new boolean[colCount];
+            for (int colIdx : rowIndexCols) {
+              sargColumns[colIdx] = true;
+            }
+            RecordReaderImpl.Index indices = rows.readRowIndex(stripeIx, null, sargColumns);
+            writer.key("indexes").array();
+            for (int col : rowIndexCols) {
+              writer.object();
+              writer.key("columnId").value(col);
+              writeRowGroupIndexes(writer, col, indices.getRowGroupIndex());
+              writeBloomFilterIndexes(writer, col, indices.getBloomFilterIndex());
+              writer.endObject();
+            }
+            writer.endArray();
+          }
+          writer.endObject(); // end of stripe information
         }
-        writer.endObject(); // end of stripe information
-      }
-      writer.endArray();
+        writer.endArray();
 
-      FileSystem fs = path.getFileSystem(conf);
-      long fileLen = fs.getContentSummary(path).getLength();
-      long paddedBytes = FileDump.getTotalPaddingSize(reader);
-      // empty ORC file is ~45 bytes. Assumption here is file length always >0
-      double percentPadding = ((double) paddedBytes / (double) fileLen) * 100;
-      writer.key("fileLength").value(fileLen);
-      writer.key("paddingLength").value(paddedBytes);
-      writer.key("paddingRatio").value(percentPadding);
-      rows.close();
+        FileSystem fs = path.getFileSystem(conf);
+        long fileLen = fs.getContentSummary(path).getLength();
+        long paddedBytes = FileDump.getTotalPaddingSize(reader);
+        // empty ORC file is ~45 bytes. Assumption here is file length always >0
+        double percentPadding = ((double) paddedBytes / (double) fileLen) * 100;
+        writer.key("fileLength").value(fileLen);
+        writer.key("paddingLength").value(paddedBytes);
+        writer.key("paddingRatio").value(percentPadding);
+        writer.key("status").value("OK");
+        rows.close();
 
-      writer.endObject();
+        writer.endObject();
+      } catch (Exception e) {
+        writer.key("status").value("FAILED");
+        System.err.println("Unable to dump data for file: " + filename);
+        e.printStackTrace();
+        throw e;
+      }
     }
     if (multiFile) {
       writer.endArray();

http://git-wip-us.apache.org/repos/asf/hive/blob/d84e393e/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
index 68d503e..40674ea 100644
--- 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
@@ -26,8 +26,6 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileReader;
-import java.io.IOException;
-import java.io.OutputStream;
 import java.io.PrintStream;
 import java.sql.Date;
 import java.sql.Timestamp;
@@ -252,54 +250,6 @@ public class TestFileDump {
     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(expected = IOException.class)
-  public void testDataDumpThrowsIOException() throws Exception {
-    PrintStream origOut = System.out;
-    try {
-      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")));
-      
-      writer.close();
-      
-      OutputStream myOut = new OutputStream() {
-        @Override
-        public void write(int b) throws IOException {
-          throw new IOException();
-        }
-      };
-      
-      // replace stdout and run command
-      System.setOut(new PrintStream(myOut));
-      FileDump.main(new String[]{testFilePath.toString(), "-d"});
-    } finally {
-      System.setOut(origOut);
-    }
-  }
-
   // 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.