This is an automated email from the ASF dual-hosted git repository.
gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-java.git
The following commit(s) were added to refs/heads/master by this push:
new ac6a5a005 GH-2962: Set dictionary_page_offset even when encoding_stats
are missing (#3012)
ac6a5a005 is described below
commit ac6a5a00525df4237f615972c99bda4359304a44
Author: RamaKrishna Mothukuri <[email protected]>
AuthorDate: Tue Sep 24 21:15:14 2024 +0530
GH-2962: Set dictionary_page_offset even when encoding_stats are missing
(#3012)
---
.../format/converter/ParquetMetadataConverter.java | 5 ++--
.../converter/TestParquetMetadataConverter.java | 33 ++++++++++++++++++----
2 files changed, 31 insertions(+), 7 deletions(-)
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 194670f2d..bb9e58b7a 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
@@ -557,8 +557,9 @@ public class ParquetMetadataConverter {
columnMetaData.getTotalUncompressedSize(),
columnMetaData.getTotalSize(),
columnMetaData.getFirstDataPageOffset());
- if (columnMetaData.getEncodingStats() != null
- && columnMetaData.getEncodingStats().hasDictionaryPages()) {
+ if ((columnMetaData.getEncodingStats() != null
+ && columnMetaData.getEncodingStats().hasDictionaryPages())
+ || columnMetaData.hasDictionaryPage()) {
metaData.setDictionary_page_offset(columnMetaData.getDictionaryPageOffset());
}
long bloomFilterOffset = columnMetaData.getBloomFilterOffset();
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 2cffb5186..6b3259070 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
@@ -206,7 +206,16 @@ public class TestParquetMetadataConverter {
@Test
public void testParquetMetadataConverterWithDictionary() throws IOException {
ParquetMetadata parquetMetaData =
createParquetMetaData(Encoding.PLAIN_DICTIONARY, Encoding.PLAIN);
+ testParquetMetadataConverterWithDictionary(parquetMetaData);
+ }
+ @Test
+ public void
testParquetMetadataConverterWithDictionaryAndWithoutEncodingStats() throws
IOException {
+ ParquetMetadata parquetMetaData =
createParquetMetaData(Encoding.PLAIN_DICTIONARY, Encoding.PLAIN, false);
+ testParquetMetadataConverterWithDictionary(parquetMetaData);
+ }
+
+ private void testParquetMetadataConverterWithDictionary(ParquetMetadata
parquetMetaData) throws IOException {
ParquetMetadataConverter converter = new ParquetMetadataConverter();
FileMetaData fmd1 = converter.toParquetMetadata(1, parquetMetaData);
@@ -1283,18 +1292,32 @@ public class TestParquetMetadataConverter {
}
private static ParquetMetadata createParquetMetaData(Encoding dicEncoding,
Encoding dataEncoding) {
+ return createParquetMetaData(dicEncoding, dataEncoding, true);
+ }
+
+ private static ParquetMetadata createParquetMetaData(
+ Encoding dicEncoding, Encoding dataEncoding, boolean
includeEncodingStats) {
MessageType schema = parseMessageType("message schema { optional int32 col
(INT_32); }");
org.apache.parquet.hadoop.metadata.FileMetaData fileMetaData =
new org.apache.parquet.hadoop.metadata.FileMetaData(schema, new
HashMap<String, String>(), null);
List<BlockMetaData> blockMetaDataList = new ArrayList<BlockMetaData>();
BlockMetaData blockMetaData = new BlockMetaData();
- EncodingStats.Builder builder = new EncodingStats.Builder();
- if (dicEncoding != null) {
- builder.addDictEncoding(dicEncoding).build();
+ EncodingStats es = null;
+ if (includeEncodingStats) {
+ EncodingStats.Builder builder = new EncodingStats.Builder();
+ if (dicEncoding != null) {
+ builder.addDictEncoding(dicEncoding).build();
+ }
+ builder.addDataEncoding(dataEncoding);
+ es = builder.build();
}
- builder.addDataEncoding(dataEncoding);
- EncodingStats es = builder.build();
Set<org.apache.parquet.column.Encoding> e = new
HashSet<org.apache.parquet.column.Encoding>();
+ if (!includeEncodingStats) {
+ if (dicEncoding != null) {
+ e.add(dicEncoding);
+ }
+ e.add(dataEncoding);
+ }
PrimitiveTypeName t = PrimitiveTypeName.INT32;
ColumnPath p = ColumnPath.get("col");
CompressionCodecName c = CompressionCodecName.UNCOMPRESSED;