Repository: parquet-mr Updated Branches: refs/heads/master 2d3203b10 -> 352b90699
PARQUET-1026: allow unsigned binary stats when min == max When min equals max this is a special case where unsigned stats would actually be the same as signed stats since there is only one value. This is useful when the data is partitioned by that column and there's only one value in the file. Drill for example takes advantage of this. Author: Julien Le Dem <[email protected]> Closes #416 from julienledem/min_eq_max and squashes the following commits: 1d71624 [Julien Le Dem] revert package import ordering change 47d89fc [Julien Le Dem] allow unsigned binary stats when min == max Project: http://git-wip-us.apache.org/repos/asf/parquet-mr/repo Commit: http://git-wip-us.apache.org/repos/asf/parquet-mr/commit/352b9069 Tree: http://git-wip-us.apache.org/repos/asf/parquet-mr/tree/352b9069 Diff: http://git-wip-us.apache.org/repos/asf/parquet-mr/diff/352b9069 Branch: refs/heads/master Commit: 352b906996f392030bfd53b93e3cf4adb78d1a55 Parents: 2d3203b Author: Julien Le Dem <[email protected]> Authored: Fri Jun 9 14:31:14 2017 -0700 Committer: Julien Le Dem <[email protected]> Committed: Fri Jun 9 14:31:14 2017 -0700 ---------------------------------------------------------------------- .../converter/ParquetMetadataConverter.java | 7 +++++-- .../converter/TestParquetMetadataConverter.java | 22 +++++++++++++++++++- .../parquet/hadoop/TestParquetFileWriter.java | 11 +++++----- 3 files changed, 32 insertions(+), 8 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/352b9069/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java ---------------------------------------------------------------------- diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java index bf22b61..aeb6152 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java @@ -340,13 +340,16 @@ public class ParquetMetadataConverter { org.apache.parquet.column.statistics.Statistics stats = org.apache.parquet.column.statistics.Statistics.getStatsBasedOnType(type); // If there was no statistics written to the footer, create an empty Statistics object and return + boolean isSet = statistics != null && statistics.isSetMax() && statistics.isSetMin(); + boolean maxEqualsMin = isSet ? Arrays.equals(statistics.getMin(), statistics.getMax()) : false; + boolean sortOrdersMatch = SortOrder.SIGNED == typeSortOrder; // NOTE: See docs in CorruptStatistics for explanation of why this check is needed // The sort order is checked to avoid returning min/max stats that are not // valid with the type's sort order. Currently, all stats are aggregated // using a signed ordering, which isn't valid for strings or unsigned ints. if (statistics != null && !CorruptStatistics.shouldIgnoreStatistics(createdBy, type) && - SortOrder.SIGNED == typeSortOrder) { - if (statistics.isSetMax() && statistics.isSetMin()) { + ( sortOrdersMatch || maxEqualsMin)) { + if (isSet) { stats.setMinMaxFromBytes(statistics.min.array(), statistics.max.array()); } stats.setNumNulls(statistics.null_count); http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/352b9069/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java ---------------------------------------------------------------------- diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java b/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java index 35c35c1..4df45dd 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java @@ -534,7 +534,27 @@ public class TestParquetMetadataConverter { Types.required(PrimitiveTypeName.BINARY) .as(OriginalType.UTF8).named("b")); - Assert.assertTrue("Stats should be empty", convertedStats.isEmpty()); + Assert.assertTrue("Stats should be empty: " + convertedStats, convertedStats.isEmpty()); + } + + @Test + public void testStillUseStatsWithSignedSortOrderIfSingleValue() { + ParquetMetadataConverter converter = new ParquetMetadataConverter(); + BinaryStatistics stats = new BinaryStatistics(); + stats.incrementNumNulls(); + stats.updateStats(Binary.fromString("A")); + stats.incrementNumNulls(); + stats.updateStats(Binary.fromString("A")); + stats.incrementNumNulls(); + + Statistics convertedStats = converter.fromParquetStatistics( + Version.FULL_VERSION, + ParquetMetadataConverter.toParquetStatistics(stats), + Types.required(PrimitiveTypeName.BINARY) + .as(OriginalType.UTF8).named("b")); + + Assert.assertFalse("Stats should not be empty: " + convertedStats, convertedStats.isEmpty()); + Assert.assertArrayEquals("min == max: " + convertedStats, convertedStats.getMaxBytes(), convertedStats.getMinBytes()); } @Test http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/352b9069/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 00e0c68..1442e04 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 @@ -595,17 +595,18 @@ public class TestParquetFileWriter { GroupWriteSupport.setSchema(schema, configuration); ParquetWriter<Group> writer = new ParquetWriter<Group>(path, configuration, new GroupWriteSupport()); - + Group r1 = new SimpleGroup(schema); writer.write(r1); writer.close(); - + ParquetMetadata readFooter = ParquetFileReader.readFooter(configuration, path); - + // assert the statistics object is not empty - assertTrue((readFooter.getBlocks().get(0).getColumns().get(0).getStatistics().isEmpty()) == false); + org.apache.parquet.column.statistics.Statistics stats = readFooter.getBlocks().get(0).getColumns().get(0).getStatistics(); + assertFalse("is empty: " + stats, stats.isEmpty()); // assert the number of nulls are correct for the first block - assertEquals(1, (readFooter.getBlocks().get(0).getColumns().get(0).getStatistics().getNumNulls())); + assertEquals("nulls: " + stats, 1, stats.getNumNulls()); } private void validateFooters(final List<Footer> metadata) {
