huaxingao commented on code in PR #14297:
URL: https://github.com/apache/iceberg/pull/14297#discussion_r3193246189
##########
parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java:
##########
@@ -331,4 +335,186 @@ public void testDataWriterWithVariantShredding() throws
IOException {
testDataWriter(
variantSchema, (id, name) ->
ParquetVariantUtil.toParquetSchema(variant.value()));
}
+
+ @Test
+ public void testShreddingWriteReturnsBufferedAppender() throws IOException {
+ Schema variantSchema =
+ new Schema(
+ Types.NestedField.required(1, "id", Types.LongType.get()),
+ Types.NestedField.optional(2, "v", Types.VariantType.get()));
+
+ VariantShreddingAnalyzer<Record, Void> testAnalyzer =
+ new VariantShreddingAnalyzer<Record, Void>() {
+ @Override
+ protected List<VariantValue> extractVariantValues(List<Record> rows,
int idx) {
+ return java.util.Collections.emptyList();
+ }
+
+ @Override
+ protected int resolveColumnIndex(Void engineSchema, String
columnName) {
+ return -1;
+ }
+ };
+
+ OutputFile outputFile = Files.localOutput(createTempFile(temp));
+
+ ParquetFormatModel<Record, Void, ParquetValueReader<?>> model =
+ ParquetFormatModel.create(
+ Record.class,
+ Void.class,
+ (icebergSchema, messageType, engineSchema) ->
+ GenericParquetWriter.create(icebergSchema, messageType),
+ (icebergSchema, fileSchema, engineSchema, idToConstant) ->
+ GenericParquetReaders.buildReader(icebergSchema, fileSchema),
+ testAnalyzer,
+ record -> record);
+
+ try (FileAppender<Record> appender =
+ model
+ .writeBuilder(EncryptedFiles.plainAsEncryptedOutput(outputFile))
+ .schema(variantSchema)
+ .setAll(ImmutableMap.of(TableProperties.PARQUET_SHRED_VARIANTS,
"true"))
+ .content(FileContent.DATA)
+ .build()) {
+ assertThat(appender).isInstanceOf(BufferedFileAppender.class);
+ }
+ }
+
+ @Test
+ public void testWriteBuilderReturnsDirectAppenderWithNullAnalyzer() throws
IOException {
+ Schema variantSchema =
+ new Schema(
+ Types.NestedField.required(1, "id", Types.LongType.get()),
+ Types.NestedField.optional(2, "v", Types.VariantType.get()));
+
+ OutputFile outputFile = Files.localOutput(createTempFile(temp));
+
+ ParquetFormatModel<Record, Void, ParquetValueReader<?>> model =
+ ParquetFormatModel.create(
+ Record.class,
+ Void.class,
+ (icebergSchema, messageType, engineSchema) ->
+ GenericParquetWriter.create(icebergSchema, messageType),
+ (icebergSchema, fileSchema, engineSchema, idToConstant) ->
+ GenericParquetReaders.buildReader(icebergSchema, fileSchema),
+ null,
+ null);
+
+ try (FileAppender<Record> appender =
+ model
+ .writeBuilder(EncryptedFiles.plainAsEncryptedOutput(outputFile))
+ .schema(variantSchema)
+ .setAll(ImmutableMap.of(TableProperties.PARQUET_SHRED_VARIANTS,
"true"))
+ .content(FileContent.DATA)
+ .build()) {
+ // Even with shredding property set, null variantAnalyzer means no
BufferedFileAppender
+ assertThat(appender).isNotInstanceOf(BufferedFileAppender.class);
+ }
+ }
+
+ @Test
+ public void testFormatModelVariantShreddingRoundTrip() throws IOException {
+ Schema variantSchema =
+ new Schema(
+ Types.NestedField.required(1, "id", Types.LongType.get()),
+ Types.NestedField.optional(2, "v", Types.VariantType.get()));
+
+ VariantShreddingAnalyzer<Record, Void> analyzer =
+ new VariantShreddingAnalyzer<Record, Void>() {
+ @Override
+ protected List<VariantValue> extractVariantValues(List<Record> rows,
int idx) {
+ List<VariantValue> values = Lists.newArrayList();
+ for (Record row : rows) {
+ Object obj = row.get(idx);
+ if (obj instanceof Variant) {
+ values.add(((Variant) obj).value());
+ }
+ }
+ return values;
+ }
+
+ @Override
+ protected int resolveColumnIndex(Void engineSchema, String
columnName) {
+ // GenericRecord uses schema column order
+ return
variantSchema.columns().indexOf(variantSchema.findField(columnName));
+ }
+ };
+
+ ByteBuffer metadataBuffer =
VariantTestUtil.createMetadata(ImmutableList.of("a", "b"), true);
+ VariantMetadata metadata = Variants.metadata(metadataBuffer);
+ ByteBuffer objectBuffer =
+ VariantTestUtil.createObject(
+ metadataBuffer,
+ ImmutableMap.of(
+ "a", Variants.of(42),
+ "b", Variants.of("hello")));
+ Variant variant = Variant.of(metadata, Variants.value(metadata,
objectBuffer));
+
+ GenericRecord record = GenericRecord.create(variantSchema);
+ List<Record> variantRecords =
+ ImmutableList.of(
+ record.copy(ImmutableMap.of("id", 1L, "v", variant)),
+ record.copy(ImmutableMap.of("id", 2L, "v", variant)),
+ record.copy(ImmutableMap.of("id", 3L, "v", variant)));
+
+ OutputFile outputFile = Files.localOutput(createTempFile(temp));
+
+ ParquetFormatModel<Record, Void, ParquetValueReader<?>> model =
+ ParquetFormatModel.create(
+ Record.class,
+ Void.class,
+ (icebergSchema, messageType, engineSchema) ->
+ GenericParquetWriter.create(icebergSchema, messageType),
+ (icebergSchema, fileSchema, engineSchema, idToConstant) ->
+ GenericParquetReaders.buildReader(icebergSchema, fileSchema),
+ analyzer,
+ record1 -> record1);
+
+ try (FileAppender<Record> appender =
+ model
+ .writeBuilder(EncryptedFiles.plainAsEncryptedOutput(outputFile))
+ .schema(variantSchema)
+ .setAll(
+ ImmutableMap.of(
+ TableProperties.PARQUET_SHRED_VARIANTS, "true",
+ TableProperties.PARQUET_VARIANT_BUFFER_SIZE, "2"))
+ .content(FileContent.DATA)
+ .build()) {
+ assertThat(appender).isInstanceOf(BufferedFileAppender.class);
+ for (Record rec : variantRecords) {
+ appender.add(rec);
+ }
+ }
+
+ // Verify shredded Parquet schema
+ try (ParquetFileReader reader =
+ ParquetFileReader.open(ParquetIO.file(outputFile.toInputFile()))) {
+ MessageType parquetSchema =
reader.getFooter().getFileMetaData().getSchema();
+ GroupType variantGroup = parquetSchema.getType("v").asGroupType();
+ assertThat(variantGroup.containsField("metadata")).isTrue();
+ assertThat(variantGroup.containsField("value")).isTrue();
+ assertThat(variantGroup.containsField("typed_value")).isTrue();
+
+ GroupType typedValue = variantGroup.getType("typed_value").asGroupType();
+ assertThat(typedValue.containsField("a")).isTrue();
+ assertThat(typedValue.containsField("b")).isTrue();
Review Comment:
The test verifies the shredded schema and the data round-trip. Should we
also verify the data is in the typed columns to prove the data is really
shredded?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]