parquet-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From alexleven...@apache.org
Subject [1/2] parquet-mr git commit: PARQUET-251: Binary column statistics error when reuse byte[] among rows
Date Wed, 01 Jul 2015 01:34:27 GMT
Repository: parquet-mr
Updated Branches:
  refs/heads/master e6ee42e9b -> e3b95020f


http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordReader.java
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordReader.java
index 7b83770..beb7dd6 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordReader.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordReader.java
@@ -188,9 +188,8 @@ public class ParquetRecordReader<T> extends RecordReader<Void,
T> {
       }
     }
     MessageType fileSchema = footer.getFileMetaData().getSchema();
-    Map<String, String> fileMetaData = footer.getFileMetaData().getKeyValueMetaData();
     internalReader.initialize(
-        fileSchema, fileMetaData, path, filteredBlocks, configuration);
+        fileSchema, footer.getFileMetaData(), path, filteredBlocks, configuration);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestColumnChunkPageWriteStore.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestColumnChunkPageWriteStore.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestColumnChunkPageWriteStore.java
index c8b3778..2c644b6 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestColumnChunkPageWriteStore.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestColumnChunkPageWriteStore.java
@@ -116,7 +116,8 @@ public class TestColumnChunkPageWriteStore {
 
     {
       ParquetMetadata footer = ParquetFileReader.readFooter(conf, file, NO_FILTER);
-      ParquetFileReader reader = new ParquetFileReader(conf, file, footer.getBlocks(), schema.getColumns());
+      ParquetFileReader reader = new ParquetFileReader(
+          conf, footer.getFileMetaData(), file, footer.getBlocks(), schema.getColumns());
       PageReadStore rowGroup = reader.readNextRowGroup();
       PageReader pageReader = rowGroup.getPageReader(col);
       DataPageV2 page = (DataPageV2)pageReader.readPage();

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java
index 1f17209..6151f48 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java
@@ -172,7 +172,8 @@ public class TestParquetFileWriter {
     assertEquals(expectedEncoding,readFooter.getBlocks().get(0).getColumns().get(0).getEncodings());
 
     { // read first block of col #1
-      ParquetFileReader r = new ParquetFileReader(configuration, path, Arrays.asList(readFooter.getBlocks().get(0)),
Arrays.asList(SCHEMA.getColumnDescription(PATH1)));
+      ParquetFileReader r = new ParquetFileReader(configuration, readFooter.getFileMetaData(),
path,
+          Arrays.asList(readFooter.getBlocks().get(0)), Arrays.asList(SCHEMA.getColumnDescription(PATH1)));
       PageReadStore pages = r.readNextRowGroup();
       assertEquals(3, pages.getRowCount());
       validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1));
@@ -182,7 +183,8 @@ public class TestParquetFileWriter {
 
     { // read all blocks of col #1 and #2
 
-      ParquetFileReader r = new ParquetFileReader(configuration, path, readFooter.getBlocks(),
Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2)));
+      ParquetFileReader r = new ParquetFileReader(configuration, readFooter.getFileMetaData(),
path,
+          readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2)));
 
       PageReadStore pages = r.readNextRowGroup();
       assertEquals(3, pages.getRowCount());
@@ -275,7 +277,8 @@ public class TestParquetFileWriter {
         120, readFooter.getBlocks().get(1).getStartingPos());
 
     { // read first block of col #1
-      ParquetFileReader r = new ParquetFileReader(conf, path, Arrays.asList(readFooter.getBlocks().get(0)),
Arrays.asList(SCHEMA.getColumnDescription(PATH1)));
+      ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path,
+          Arrays.asList(readFooter.getBlocks().get(0)), Arrays.asList(SCHEMA.getColumnDescription(PATH1)));
       PageReadStore pages = r.readNextRowGroup();
       assertEquals(3, pages.getRowCount());
       validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1));
@@ -285,7 +288,8 @@ public class TestParquetFileWriter {
 
     { // read all blocks of col #1 and #2
 
-      ParquetFileReader r = new ParquetFileReader(conf, path, readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1),
SCHEMA.getColumnDescription(PATH2)));
+      ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path,
+          readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2)));
 
       PageReadStore pages = r.readNextRowGroup();
       assertEquals(3, pages.getRowCount());
@@ -378,7 +382,8 @@ public class TestParquetFileWriter {
         109, readFooter.getBlocks().get(1).getStartingPos());
 
     { // read first block of col #1
-      ParquetFileReader r = new ParquetFileReader(conf, path, Arrays.asList(readFooter.getBlocks().get(0)),
Arrays.asList(SCHEMA.getColumnDescription(PATH1)));
+      ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path,
+          Arrays.asList(readFooter.getBlocks().get(0)), Arrays.asList(SCHEMA.getColumnDescription(PATH1)));
       PageReadStore pages = r.readNextRowGroup();
       assertEquals(3, pages.getRowCount());
       validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1));
@@ -388,7 +393,8 @@ public class TestParquetFileWriter {
 
     { // read all blocks of col #1 and #2
 
-      ParquetFileReader r = new ParquetFileReader(conf, path, readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1),
SCHEMA.getColumnDescription(PATH2)));
+      ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path,
+          readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2)));
 
       PageReadStore pages = r.readNextRowGroup();
       assertEquals(3, pages.getRowCount());
@@ -417,8 +423,13 @@ public class TestParquetFileWriter {
     for (long l: longArray) {
       parquetMRstats.updateStats(l);
     }
-    Statistics thriftStats = org.apache.parquet.format.converter.ParquetMetadataConverter.toParquetStatistics(parquetMRstats);
-    LongStatistics convertedBackStats = (LongStatistics) org.apache.parquet.format.converter.ParquetMetadataConverter.fromParquetStatistics(thriftStats,
PrimitiveTypeName.INT64);
+    final String createdBy =
+        "parquet-mr version 1.8.0 (build d4d5a07ec9bd262ca1e93c309f1d7d4a74ebda4c)";
+    Statistics thriftStats =
+        org.apache.parquet.format.converter.ParquetMetadataConverter.toParquetStatistics(parquetMRstats);
+    LongStatistics convertedBackStats =
+        (LongStatistics) org.apache.parquet.format.converter.ParquetMetadataConverter.fromParquetStatistics(
+            createdBy, thriftStats, PrimitiveTypeName.INT64);
 
     assertEquals(parquetMRstats.getMax(), convertedBackStats.getMax());
     assertEquals(parquetMRstats.getMin(), convertedBackStats.getMin());

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
index 2583278..3dcec30 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
@@ -37,7 +37,6 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
 
@@ -95,7 +94,7 @@ public class TestParquetWriter {
               .append("float_field", 1.0f)
               .append("double_field", 2.0d)
               .append("flba_field", "foo")
-              .append("int96_field", Binary.fromByteArray(new byte[12])));
+              .append("int96_field", Binary.fromConstantByteArray(new byte[12])));
         }
         writer.close();
         ParquetReader<Group> reader = ParquetReader.builder(new GroupReadSupport(),
file).withConf(conf).build();
@@ -108,7 +107,8 @@ public class TestParquetWriter {
           assertEquals(1.0f, group.getFloat("float_field", 0), 0.001);
           assertEquals(2.0d, group.getDouble("double_field", 0), 0.001);
           assertEquals("foo", group.getBinary("flba_field", 0).toStringUsingUTF8());
-          assertEquals(Binary.fromByteArray(new byte[12]), group.getInt96("int96_field",
0));
+          assertEquals(Binary.fromConstantByteArray(new byte[12]),
+              group.getInt96("int96_field",0));
         }
         reader.close();
         ParquetMetadata footer = readFooter(conf, file, NO_FILTER);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriterNewPage.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriterNewPage.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriterNewPage.java
index 9dd1323..b885a86 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriterNewPage.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriterNewPage.java
@@ -99,7 +99,7 @@ public class TestParquetWriterNewPage {
               .append("float_field", 1.0f)
               .append("double_field", 2.0d)
               .append("flba_field", "foo")
-              .append("int96_field", Binary.fromByteArray(new byte[12])));
+              .append("int96_field", Binary.fromConstantByteArray(new byte[12])));
         }
         writer.close();
 
@@ -113,7 +113,8 @@ public class TestParquetWriterNewPage {
           assertEquals(1.0f, group.getFloat("float_field", 0), 0.001);
           assertEquals(2.0d, group.getDouble("double_field", 0), 0.001);
           assertEquals("foo", group.getBinary("flba_field", 0).toStringUsingUTF8());
-          assertEquals(Binary.fromByteArray(new byte[12]), group.getInt96("int96_field",
0));
+          assertEquals(Binary.fromConstantByteArray(new byte[12]), group.getInt96("int96_field",
+              0));
           assertEquals(0, group.getFieldRepetitionCount("null_field"));
         }
         reader.close();

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-hive/parquet-hive-storage-handler/src/main/java/org/apache/hadoop/hive/ql/io/parquet/writable/BinaryWritable.java
----------------------------------------------------------------------
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/org/apache/hadoop/hive/ql/io/parquet/writable/BinaryWritable.java
b/parquet-hive/parquet-hive-storage-handler/src/main/java/org/apache/hadoop/hive/ql/io/parquet/writable/BinaryWritable.java
index 0779822..09744a5 100644
--- a/parquet-hive/parquet-hive-storage-handler/src/main/java/org/apache/hadoop/hive/ql/io/parquet/writable/BinaryWritable.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/org/apache/hadoop/hive/ql/io/parquet/writable/BinaryWritable.java
@@ -57,7 +57,7 @@ public class BinaryWritable implements Writable {
   public void readFields(DataInput input) throws IOException {
     byte[] bytes = new byte[input.readInt()];
     input.readFully(bytes);
-    binary = Binary.fromByteArray(bytes);
+    binary = Binary.fromConstantByteArray(bytes);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-hive/parquet-hive-storage-handler/src/main/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/ParquetStringInspector.java
----------------------------------------------------------------------
diff --git a/parquet-hive/parquet-hive-storage-handler/src/main/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/ParquetStringInspector.java
b/parquet-hive/parquet-hive-storage-handler/src/main/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/ParquetStringInspector.java
index b734f8a..1f056e2 100644
--- a/parquet-hive/parquet-hive-storage-handler/src/main/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/ParquetStringInspector.java
+++ b/parquet-hive/parquet-hive-storage-handler/src/main/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/ParquetStringInspector.java
@@ -77,7 +77,8 @@ public class ParquetStringInspector extends JavaStringObjectInspector implements
 
   @Override
   public Object set(final Object o, final Text text) {
-    return new BinaryWritable(text == null ? null : Binary.fromByteArray(text.getBytes()));
+    return new BinaryWritable(text == null ? null : Binary.fromReusedByteArray(text.getBytes
+        ()));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-pig/src/main/java/org/apache/parquet/pig/TupleWriteSupport.java
----------------------------------------------------------------------
diff --git a/parquet-pig/src/main/java/org/apache/parquet/pig/TupleWriteSupport.java b/parquet-pig/src/main/java/org/apache/parquet/pig/TupleWriteSupport.java
index 2cf79ee..829fe70 100644
--- a/parquet-pig/src/main/java/org/apache/parquet/pig/TupleWriteSupport.java
+++ b/parquet-pig/src/main/java/org/apache/parquet/pig/TupleWriteSupport.java
@@ -172,7 +172,7 @@ public class TupleWriteSupport extends WriteSupport<Tuple> {
           } else {
             throw new UnsupportedOperationException("can not convert from " + DataType.findTypeName(pigType.type)
+ " to BINARY ");
           }
-          recordConsumer.addBinary(Binary.fromByteArray(bytes));
+          recordConsumer.addBinary(Binary.fromReusedByteArray(bytes));
           break;
         case BOOLEAN:
           recordConsumer.addBoolean((Boolean)t.get(i));

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java
----------------------------------------------------------------------
diff --git a/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java
b/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java
index 180ce41..40e36d5 100644
--- a/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java
+++ b/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java
@@ -318,7 +318,7 @@ public class ProtoWriteSupport<T extends MessageOrBuilder> extends
WriteSupport<
     @Override
     final void writeRawValue(Object value) {
       ByteString byteString = (ByteString) value;
-      Binary binary = Binary.fromByteArray(byteString.toByteArray());
+      Binary binary = Binary.fromConstantByteArray(byteString.toByteArray());
       recordConsumer.addBinary(binary);
     }
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoWriteSupportTest.java
----------------------------------------------------------------------
diff --git a/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoWriteSupportTest.java
b/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoWriteSupportTest.java
index c7430c5..73f7734 100644
--- a/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoWriteSupportTest.java
+++ b/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoWriteSupportTest.java
@@ -50,7 +50,7 @@ public class ProtoWriteSupportTest {
 
     inOrder.verify(readConsumerMock).startMessage();
     inOrder.verify(readConsumerMock).startField("one", 0);
-    inOrder.verify(readConsumerMock).addBinary(Binary.fromByteArray("oneValue".getBytes()));
+    inOrder.verify(readConsumerMock).addBinary(Binary.fromString("oneValue"));
     inOrder.verify(readConsumerMock).endField("one", 0);
 
     inOrder.verify(readConsumerMock).endMessage();
@@ -95,10 +95,10 @@ public class ProtoWriteSupportTest {
     inOrder.verify(readConsumerMock).startField("inner", 0);
     inOrder.verify(readConsumerMock).startGroup();
     inOrder.verify(readConsumerMock).startField("one", 0);
-    inOrder.verify(readConsumerMock).addBinary(Binary.fromByteArray("one".getBytes()));
+    inOrder.verify(readConsumerMock).addBinary(Binary.fromConstantByteArray("one".getBytes()));
     inOrder.verify(readConsumerMock).endField("one", 0);
     inOrder.verify(readConsumerMock).startField("two", 1);
-    inOrder.verify(readConsumerMock).addBinary(Binary.fromByteArray("two".getBytes()));
+    inOrder.verify(readConsumerMock).addBinary(Binary.fromConstantByteArray("two".getBytes()));
     inOrder.verify(readConsumerMock).endField("two", 1);
     inOrder.verify(readConsumerMock).endGroup();
     inOrder.verify(readConsumerMock).endField("inner", 0);
@@ -124,14 +124,14 @@ public class ProtoWriteSupportTest {
     //first inner message
     inOrder.verify(readConsumerMock).startGroup();
     inOrder.verify(readConsumerMock).startField("one", 0);
-    inOrder.verify(readConsumerMock).addBinary(Binary.fromByteArray("one".getBytes()));
+    inOrder.verify(readConsumerMock).addBinary(Binary.fromConstantByteArray("one".getBytes()));
     inOrder.verify(readConsumerMock).endField("one", 0);
     inOrder.verify(readConsumerMock).endGroup();
 
     //second inner message
     inOrder.verify(readConsumerMock).startGroup();
     inOrder.verify(readConsumerMock).startField("two", 1);
-    inOrder.verify(readConsumerMock).addBinary(Binary.fromByteArray("two".getBytes()));
+    inOrder.verify(readConsumerMock).addBinary(Binary.fromConstantByteArray("two".getBytes()));
     inOrder.verify(readConsumerMock).endField("two", 1);
     inOrder.verify(readConsumerMock).endGroup();
 
@@ -157,7 +157,7 @@ public class ProtoWriteSupportTest {
 
     inOrder.verify(readConsumerMock).startGroup();
     inOrder.verify(readConsumerMock).startField("one", 0);
-    inOrder.verify(readConsumerMock).addBinary(Binary.fromByteArray("one".getBytes()));
+    inOrder.verify(readConsumerMock).addBinary(Binary.fromConstantByteArray("one".getBytes()));
     inOrder.verify(readConsumerMock).endField("one", 0);
     inOrder.verify(readConsumerMock).endGroup();
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-thrift/src/main/java/org/apache/parquet/thrift/ParquetWriteProtocol.java
----------------------------------------------------------------------
diff --git a/parquet-thrift/src/main/java/org/apache/parquet/thrift/ParquetWriteProtocol.java
b/parquet-thrift/src/main/java/org/apache/parquet/thrift/ParquetWriteProtocol.java
index 6530823..40984cc 100644
--- a/parquet-thrift/src/main/java/org/apache/parquet/thrift/ParquetWriteProtocol.java
+++ b/parquet-thrift/src/main/java/org/apache/parquet/thrift/ParquetWriteProtocol.java
@@ -645,7 +645,8 @@ public class ParquetWriteProtocol extends ParquetProtocol {
   }
 
   private void writeBinaryToRecordConsumer(ByteBuffer buf) {
-    recordConsumer.addBinary(Binary.fromByteArray(buf.array(), buf.position(), buf.limit()
- buf.position()));
+    recordConsumer.addBinary(Binary.fromReusedByteArray(buf.array(), buf.position(),
+        buf.limit() - buf.position()));
   }
 
   private void writeStringToRecordConsumer(String str) {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e3b95020/parquet-tools/src/main/java/org/apache/parquet/tools/command/DumpCommand.java
----------------------------------------------------------------------
diff --git a/parquet-tools/src/main/java/org/apache/parquet/tools/command/DumpCommand.java
b/parquet-tools/src/main/java/org/apache/parquet/tools/command/DumpCommand.java
index a4660bd..837a46a 100644
--- a/parquet-tools/src/main/java/org/apache/parquet/tools/command/DumpCommand.java
+++ b/parquet-tools/src/main/java/org/apache/parquet/tools/command/DumpCommand.java
@@ -178,7 +178,8 @@ public class DumpCommand extends ArgsOnlyCommand {
                     MetadataUtils.showDetails(out, ccmds);
 
                     List<BlockMetaData> rblocks = Collections.singletonList(block);
-                    freader = new ParquetFileReader(conf, inpath, rblocks, columns);
+                    freader = new ParquetFileReader(
+                        conf, meta.getFileMetaData(), inpath, rblocks, columns);
                     PageReadStore store = freader.readNextRowGroup();
                     while (store != null) {
                         out.incrementTabLevel();
@@ -211,7 +212,8 @@ public class DumpCommand extends ArgsOnlyCommand {
                     long page = 1;
                     long total = blocks.size();
                     long offset = 1;
-                    freader = new ParquetFileReader(conf, inpath, blocks, Collections.singletonList(column));
+                    freader = new ParquetFileReader(
+                        conf, meta.getFileMetaData(), inpath, blocks, Collections.singletonList(column));
                     PageReadStore store = freader.readNextRowGroup();
                     while (store != null) {
                         ColumnReadStoreImpl crstore = new ColumnReadStoreImpl(store, new
DumpGroupConverter(), schema);
@@ -307,7 +309,7 @@ public class DumpCommand extends ArgsOnlyCommand {
     }
 
     public static String binaryToString(Binary value) {
-        byte[] data = value.getBytes();
+        byte[] data = value.getBytesUnsafe();
         if (data == null) return null;
 
         try {
@@ -320,7 +322,7 @@ public class DumpCommand extends ArgsOnlyCommand {
     }
 
     public static BigInteger binaryToBigInteger(Binary value) {
-        byte[] data = value.getBytes();
+        byte[] data = value.getBytesUnsafe();
         if (data == null) return null;
 
         return new BigInteger(data);


Mime
View raw message