Skip to content

Commit ca2100e

Browse files
committed
Test out Parquet 1.14.0
1 parent 10ffc60 commit ca2100e

File tree

11 files changed

+133
-95
lines changed

11 files changed

+133
-95
lines changed
Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,10 @@
1+
# Danger! Multiple jars contain identically named classes. This may cause different behaviour depending on classpath ordering.
2+
# Run ./gradlew checkClassUniqueness --write-locks to update this file
3+
4+
## runtimeClasspath
5+
[com.fasterxml.jackson.core:jackson-core, org.apache.parquet:parquet-jackson]
6+
- META-INF.versions.11.com.fasterxml.jackson.core.io.doubleparser.BigSignificand
7+
- META-INF.versions.11.com.fasterxml.jackson.core.io.doubleparser.FastDoubleSwar
8+
- META-INF.versions.11.com.fasterxml.jackson.core.io.doubleparser.FastIntegerMath
9+
- META-INF.versions.17.com.fasterxml.jackson.core.io.doubleparser.FastDoubleSwar
10+
- META-INF.versions.17.com.fasterxml.jackson.core.io.doubleparser.FastIntegerMath

build.gradle

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -128,6 +128,9 @@ allprojects {
128128
repositories {
129129
mavenCentral()
130130
mavenLocal()
131+
maven {
132+
url = uri("https://repository.apache.org/content/groups/staging")
133+
}
131134
}
132135
}
133136

gradle/libs.versions.toml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -70,7 +70,7 @@ netty-buffer = "4.1.109.Final"
7070
netty-buffer-compat = "4.1.109.Final"
7171
object-client-bundle = "3.3.2"
7272
orc = "1.9.3"
73-
parquet = "1.13.1"
73+
parquet = "1.14.0"
7474
pig = "0.17.0"
7575
roaringbitmap = "1.0.6"
7676
s3mock-junit5 = "2.11.0"

parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -103,6 +103,7 @@
103103
import org.apache.parquet.avro.AvroWriteSupport;
104104
import org.apache.parquet.column.ParquetProperties;
105105
import org.apache.parquet.column.ParquetProperties.WriterVersion;
106+
import org.apache.parquet.conf.ParquetConfiguration;
106107
import org.apache.parquet.crypto.FileDecryptionProperties;
107108
import org.apache.parquet.crypto.FileEncryptionProperties;
108109
import org.apache.parquet.hadoop.ParquetFileReader;
@@ -366,6 +367,7 @@ public <D> FileAppender<D> build() throws IOException {
366367
new ParquetWriteBuilder<D>(ParquetIO.file(file))
367368
.withWriterVersion(writerVersion)
368369
.setType(type)
370+
.withConf(conf)
369371
.setConfig(config)
370372
.setKeyValueMetadata(metadata)
371373
.setWriteSupport(getWriteSupport(type))
@@ -991,6 +993,14 @@ protected WriteSupport<T> getWriteSupport(Configuration configuration) {
991993
}
992994
return new ParquetWriteSupport<>(type, keyValueMetadata, writeSupport);
993995
}
996+
997+
@Override
998+
protected WriteSupport<T> getWriteSupport(ParquetConfiguration conf) {
999+
for (Map.Entry<String, String> entry : config.entrySet()) {
1000+
conf.set(entry.getKey(), entry.getValue());
1001+
}
1002+
return new ParquetWriteSupport<>(type, keyValueMetadata, writeSupport);
1003+
}
9941004
}
9951005

9961006
public static ReadBuilder read(InputFile file) {

parquet/src/main/java/org/apache/iceberg/parquet/ParquetCodecFactory.java

Lines changed: 0 additions & 92 deletions
This file was deleted.

parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java

Lines changed: 72 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@
1818
*/
1919
package org.apache.iceberg.parquet;
2020

21+
import static org.apache.parquet.avro.AvroReadSupport.AVRO_REQUESTED_PROJECTION;
22+
2123
import java.util.Map;
2224
import java.util.Set;
2325
import org.apache.hadoop.conf.Configuration;
@@ -27,6 +29,7 @@
2729
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
2830
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
2931
import org.apache.parquet.avro.AvroReadSupport;
32+
import org.apache.parquet.conf.ParquetConfiguration;
3033
import org.apache.parquet.hadoop.api.InitContext;
3134
import org.apache.parquet.hadoop.api.ReadSupport;
3235
import org.apache.parquet.io.api.RecordMaterializer;
@@ -104,6 +107,60 @@ public ReadContext init(
104107
projection, context != null ? context.getReadSupportMetadata() : ImmutableMap.of());
105108
}
106109

110+
@Override
111+
@SuppressWarnings("deprecation")
112+
public ReadContext init(
113+
ParquetConfiguration configuration,
114+
Map<String, String> keyValueMetaData,
115+
MessageType fileSchema) {
116+
// Columns are selected from the Parquet file by taking the read context's message type and
117+
// matching to the file's columns by full path, so this must select columns by using the path
118+
// in the file's schema.
119+
120+
MessageType projection;
121+
if (ParquetSchemaUtil.hasIds(fileSchema)) {
122+
projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema);
123+
} else if (nameMapping != null) {
124+
MessageType typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping);
125+
projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema);
126+
} else {
127+
projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema);
128+
}
129+
130+
// override some known backward-compatibility options
131+
configuration.set("parquet.strict.typing", "false");
132+
configuration.set("parquet.avro.add-list-element-records", "false");
133+
configuration.set("parquet.avro.write-old-list-structure", "false");
134+
135+
// set Avro schemas in case the reader is Avro
136+
configuration.set(
137+
AVRO_REQUESTED_PROJECTION,
138+
AvroSchemaUtil.convert(expectedSchema, projection.getName()).toString());
139+
org.apache.avro.Schema avroReadSchema =
140+
AvroSchemaUtil.buildAvroProjection(
141+
AvroSchemaUtil.convert(ParquetSchemaUtil.convert(projection), projection.getName()),
142+
expectedSchema,
143+
ImmutableMap.of());
144+
configuration.set(
145+
"parquet.avro.read.schema", ParquetAvro.parquetAvroSchema(avroReadSchema).toString());
146+
147+
// let the context set up read support metadata, but always use the correct projection
148+
ReadContext context = null;
149+
if (callInit) {
150+
try {
151+
context = wrapped.init(configuration, keyValueMetaData, projection);
152+
} catch (UnsupportedOperationException e) {
153+
// try the InitContext version
154+
context =
155+
wrapped.init(
156+
new InitContext(configuration, makeMultimap(keyValueMetaData), projection));
157+
}
158+
}
159+
160+
return new ReadContext(
161+
projection, context != null ? context.getReadSupportMetadata() : ImmutableMap.of());
162+
}
163+
107164
@Override
108165
public RecordMaterializer<T> prepareForRead(
109166
Configuration configuration,
@@ -119,6 +176,21 @@ public RecordMaterializer<T> prepareForRead(
119176
return wrapped.prepareForRead(configuration, fileMetadata, readSchema, readContext);
120177
}
121178

179+
@Override
180+
public RecordMaterializer<T> prepareForRead(
181+
ParquetConfiguration configuration,
182+
Map<String, String> fileMetadata,
183+
MessageType fileMessageType,
184+
ReadContext readContext) {
185+
// This is the type created in init that was based on the file's schema. The schema that this
186+
// will pass to the wrapped ReadSupport needs to match the expected schema's names. Rather than
187+
// renaming the file's schema, convert the expected schema to Parquet. This relies on writing
188+
// files with the correct schema.
189+
// TODO: this breaks when columns are reordered.
190+
MessageType readSchema = ParquetSchemaUtil.convert(expectedSchema, fileMessageType.getName());
191+
return wrapped.prepareForRead(configuration, fileMetadata, readSchema, readContext);
192+
}
193+
122194
private Map<String, Set<String>> makeMultimap(Map<String, String> map) {
123195
ImmutableMap.Builder<String, Set<String>> builder = ImmutableMap.builder();
124196
for (Map.Entry<String, String> entry : map.entrySet()) {

parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteSupport.java

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import java.util.Map;
2222
import org.apache.hadoop.conf.Configuration;
2323
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
24+
import org.apache.parquet.conf.ParquetConfiguration;
2425
import org.apache.parquet.hadoop.api.WriteSupport;
2526
import org.apache.parquet.io.api.RecordConsumer;
2627
import org.apache.parquet.schema.MessageType;
@@ -48,6 +49,17 @@ public WriteContext init(Configuration configuration) {
4849
return new WriteContext(type, metadata);
4950
}
5051

52+
@Override
53+
public WriteContext init(ParquetConfiguration configuration) {
54+
WriteContext wrappedContext = wrapped.init(configuration);
55+
Map<String, String> metadata =
56+
ImmutableMap.<String, String>builder()
57+
.putAll(keyValueMetadata)
58+
.putAll(wrappedContext.getExtraMetaData())
59+
.buildOrThrow();
60+
return new WriteContext(type, metadata);
61+
}
62+
5163
@Override
5264
public String getName() {
5365
return "Iceberg/" + wrapped.getName();

parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -86,8 +86,7 @@ class ParquetWriter<T> implements FileAppender<T>, Closeable {
8686
this.targetRowGroupSize = rowGroupSize;
8787
this.props = properties;
8888
this.metadata = ImmutableMap.copyOf(metadata);
89-
this.compressor =
90-
new ParquetCodecFactory(conf, props.getPageSizeThreshold()).getCompressor(codec);
89+
this.compressor = new CodecFactory(conf, props.getPageSizeThreshold()).getCompressor(codec);
9190
this.parquetSchema = ParquetSchemaUtil.convert(schema, "table");
9291
this.model = (ParquetValueWriter<T>) createWriterFunc.apply(parquetSchema);
9392
this.metricsConfig = metricsConfig;

settings-gradle.lockfile

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,4 @@
1+
# This is a Gradle generated file for dependency locking.
2+
# Manual edits can break the build and are not advised.
3+
# This file is expected to be part of source control.
4+
empty=incomingCatalogForLibs0
Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,10 @@
1+
# Danger! Multiple jars contain identically named classes. This may cause different behaviour depending on classpath ordering.
2+
# Run ./gradlew checkClassUniqueness --write-locks to update this file
3+
4+
## runtimeClasspath
5+
[com.fasterxml.jackson.core:jackson-core, org.apache.parquet:parquet-jackson]
6+
- META-INF.versions.11.com.fasterxml.jackson.core.io.doubleparser.BigSignificand
7+
- META-INF.versions.11.com.fasterxml.jackson.core.io.doubleparser.FastDoubleSwar
8+
- META-INF.versions.11.com.fasterxml.jackson.core.io.doubleparser.FastIntegerMath
9+
- META-INF.versions.17.com.fasterxml.jackson.core.io.doubleparser.FastDoubleSwar
10+
- META-INF.versions.17.com.fasterxml.jackson.core.io.doubleparser.FastIntegerMath

0 commit comments

Comments
 (0)