Skip to content
Open
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.io.OutputFile;
import org.apache.parquet.proto.ProtoWriteSupport;

/** Convenience builder for creating {@link ParquetWriterFactory} instances for Protobuf classes. */
public class ParquetProtoWriters {
Expand Down Expand Up @@ -62,7 +61,8 @@ protected ParquetProtoWriterBuilder<T> self() {

@Override
protected WriteSupport<T> getWriteSupport(Configuration conf) {
return new ProtoWriteSupport<>(clazz);
// Use patched implementation compatible with protobuf 4.x
return new PatchedProtoWriteSupport<>(clazz);
}
}

Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -0,0 +1,267 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.formats.parquet.protobuf;

import org.apache.hadoop.fs.Path;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.hadoop.util.HadoopOutputFile;
import org.apache.parquet.io.OutputFile;
import org.apache.parquet.proto.ProtoParquetReader;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;

import java.io.File;
import java.io.IOException;

import static org.apache.flink.formats.parquet.protobuf.SimpleRecord.SimpleProtoRecord;
import static org.apache.flink.formats.parquet.protobuf.TestProto2.TestProto2Record;
import static org.assertj.core.api.Assertions.assertThat;

/**
* Tests for {@link PatchedProtoWriteSupport} to verify protobuf 4.x compatibility.
*
* <p>This test validates that the patched string-based syntax detection correctly handles both
* proto2 and proto3 messages when using protobuf 4.x, where the enum-based Syntax API was removed.
*/
class PatchedProtoWriteSupportTest {

@TempDir File tempDir;

/**
* Tests that proto3 messages can be written and read correctly with the patched write support.
*/
@Test
void testProto3SyntaxDetection() throws IOException {
File outputFile = new File(tempDir, "proto3_test.parquet");
Path path = new Path(outputFile.toURI());

// Create a proto3 message
SimpleProtoRecord record =
SimpleProtoRecord.newBuilder()
.setFoo("test_foo")
.setBar("test_bar")
.setNum(42)
.build();

// Write using PatchedProtoWriteSupport directly
try (ParquetWriter<SimpleProtoRecord> writer =
new ParquetWriter<>(
path,
new PatchedProtoWriteSupport<>(SimpleProtoRecord.class),
CompressionCodecName.SNAPPY,
ParquetWriter.DEFAULT_BLOCK_SIZE,
ParquetWriter.DEFAULT_PAGE_SIZE)) {
writer.write(record);
}

// Read back and verify
try (ParquetReader<SimpleProtoRecord.Builder> reader =
ProtoParquetReader.<SimpleProtoRecord.Builder>builder(path).build()) {
SimpleProtoRecord.Builder readRecord = reader.read();
assertThat(readRecord).isNotNull();
assertThat(readRecord.build()).isEqualTo(record);
}
}

/**
* Tests that proto2 messages can be written and read correctly with the patched write support.
*/
@Test
void testProto2SyntaxDetection() throws IOException {
File outputFile = new File(tempDir, "proto2_test.parquet");
Path path = new Path(outputFile.toURI());

// Create a proto2 message with only some fields set
TestProto2Record record =
TestProto2Record.newBuilder().setName("test_name").setValue(123).build();

// Write using PatchedProtoWriteSupport directly
try (ParquetWriter<TestProto2Record> writer =
new ParquetWriter<>(
path,
new PatchedProtoWriteSupport<>(TestProto2Record.class),
CompressionCodecName.SNAPPY,
ParquetWriter.DEFAULT_BLOCK_SIZE,
ParquetWriter.DEFAULT_PAGE_SIZE)) {
writer.write(record);
}

// Read back and verify
try (ParquetReader<TestProto2Record.Builder> reader =
ProtoParquetReader.<TestProto2Record.Builder>builder(path).build()) {
TestProto2Record.Builder readRecord = reader.read();
assertThat(readRecord).isNotNull();
TestProto2Record result = readRecord.build();
assertThat(result.getName()).isEqualTo("test_name");
assertThat(result.getValue()).isEqualTo(123);
// flag field was not set, should be default
assertThat(result.hasFlag()).isFalse();
}
}

/**
* Tests that proto3 messages with default values are handled correctly.
*
* <p>In proto3, all fields are written including those with default values.
*/
@Test
void testProto3WithDefaults() throws IOException {
File outputFile = new File(tempDir, "proto3_defaults.parquet");
Path path = new Path(outputFile.toURI());

// Create a proto3 message with default values
SimpleProtoRecord record =
SimpleProtoRecord.newBuilder().setFoo("").setBar("").setNum(0).build();

// Write using PatchedProtoWriteSupport
try (ParquetWriter<SimpleProtoRecord> writer =
new ParquetWriter<>(
path,
new PatchedProtoWriteSupport<>(SimpleProtoRecord.class),
CompressionCodecName.SNAPPY,
ParquetWriter.DEFAULT_BLOCK_SIZE,
ParquetWriter.DEFAULT_PAGE_SIZE)) {
writer.write(record);
}

// Read back and verify - proto3 should read all fields even if default
try (ParquetReader<SimpleProtoRecord.Builder> reader =
ProtoParquetReader.<SimpleProtoRecord.Builder>builder(path).build()) {
SimpleProtoRecord.Builder readRecord = reader.read();
assertThat(readRecord).isNotNull();
assertThat(readRecord.build()).isEqualTo(record);
}
}

/**
* Tests that proto2 only writes fields that have been explicitly set.
*
* <p>In proto2, unset optional fields should not be written to the file.
*/
@Test
void testProto2OnlyWritesSetFields() throws IOException {
File outputFile = new File(tempDir, "proto2_partial.parquet");
Path path = new Path(outputFile.toURI());

// Create a proto2 message with only one field set
TestProto2Record record = TestProto2Record.newBuilder().setName("only_name").build();

// Write using PatchedProtoWriteSupport
try (ParquetWriter<TestProto2Record> writer =
new ParquetWriter<>(
path,
new PatchedProtoWriteSupport<>(TestProto2Record.class),
CompressionCodecName.SNAPPY,
ParquetWriter.DEFAULT_BLOCK_SIZE,
ParquetWriter.DEFAULT_PAGE_SIZE)) {
writer.write(record);
}

// Read back and verify
try (ParquetReader<TestProto2Record.Builder> reader =
ProtoParquetReader.<TestProto2Record.Builder>builder(path).build()) {
TestProto2Record.Builder readRecord = reader.read();
assertThat(readRecord).isNotNull();
TestProto2Record result = readRecord.build();
assertThat(result.getName()).isEqualTo("only_name");
// value and flag were not set
assertThat(result.hasValue()).isFalse();
assertThat(result.hasFlag()).isFalse();
}
}

/**
* Integration test using ParquetProtoWriters (Flink's production API).
*
* <p>This validates that PatchedProtoWriteSupport works correctly when used through Flink's
* ParquetProtoWriters factory, which is the actual production code path.
*/
@Test
void testViaParquetProtoWritersForProto3() throws IOException {
File outputFile = new File(tempDir, "proto3_via_writers.parquet");
Path hadoopPath = new Path(outputFile.toURI());
OutputFile outputFileObj =
HadoopOutputFile.fromPath(hadoopPath, new org.apache.hadoop.conf.Configuration());

// Create a proto3 message
SimpleProtoRecord record =
SimpleProtoRecord.newBuilder()
.setFoo("via_writers")
.setBar("test")
.setNum(99)
.build();

// Write using ParquetProtoWriters (production code path)
try (ParquetWriter<SimpleProtoRecord> writer =
new ParquetProtoWriters.ParquetProtoWriterBuilder<>(
outputFileObj, SimpleProtoRecord.class)
.withCompressionCodec(CompressionCodecName.SNAPPY)
.build()) {
writer.write(record);
}

// Read back and verify
try (ParquetReader<SimpleProtoRecord.Builder> reader =
ProtoParquetReader.<SimpleProtoRecord.Builder>builder(hadoopPath).build()) {
SimpleProtoRecord.Builder readRecord = reader.read();
assertThat(readRecord).isNotNull();
assertThat(readRecord.build()).isEqualTo(record);
}
}

/**
* Integration test using ParquetProtoWriters for proto2 messages.
*
* <p>Verifies that proto2 syntax detection works correctly through the production API.
*/
@Test
void testViaParquetProtoWritersForProto2() throws IOException {
File outputFile = new File(tempDir, "proto2_via_writers.parquet");
Path hadoopPath = new Path(outputFile.toURI());
OutputFile outputFileObj =
HadoopOutputFile.fromPath(hadoopPath, new org.apache.hadoop.conf.Configuration());

// Create a proto2 message with partial fields
TestProto2Record record =
TestProto2Record.newBuilder().setName("proto2_writer").setFlag(true).build();

// Write using ParquetProtoWriters (production code path)
try (ParquetWriter<TestProto2Record> writer =
new ParquetProtoWriters.ParquetProtoWriterBuilder<>(
outputFileObj, TestProto2Record.class)
.withCompressionCodec(CompressionCodecName.SNAPPY)
.build()) {
writer.write(record);
}

// Read back and verify
try (ParquetReader<TestProto2Record.Builder> reader =
ProtoParquetReader.<TestProto2Record.Builder>builder(hadoopPath).build()) {
TestProto2Record.Builder readRecord = reader.read();
assertThat(readRecord).isNotNull();
TestProto2Record result = readRecord.build();
assertThat(result.getName()).isEqualTo("proto2_writer");
assertThat(result.getFlag()).isTrue();
// value was not set
assertThat(result.hasValue()).isFalse();
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,9 @@
syntax = "proto2";

package org.apache.flink.formats.parquet.protobuf;

message TestProto2Record {
optional string name = 1;
optional int32 value = 2;
optional bool flag = 3;
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@

import com.google.protobuf.ByteString;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Descriptors.FileDescriptor.Syntax;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -69,7 +68,7 @@ public ProtoToRowConverter(RowType rowType, PbFormatConfig formatConfig)
Thread.currentThread().getContextClassLoader());
String fullMessageClassName = PbFormatUtils.getFullJavaName(descriptor);
boolean readDefaultValuesForPrimitiveTypes = formatConfig.isReadDefaultValues();
if (descriptor.getFile().getSyntax() == Syntax.PROTO3) {
if ("proto3".equals(descriptor.getFile().toProto().getSyntax())) {
// pb3 always read default values for primitive types
readDefaultValuesForPrimitiveTypes = true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,4 +7,4 @@ The Apache Software Foundation (http://www.apache.org/).
This project bundles the following dependencies under BSD-3 License (https://opensource.org/licenses/BSD-3-Clause).
See bundled license files for details.

- com.google.protobuf:protobuf-java:3.21.7
- com.google.protobuf:protobuf-java:4.32.1
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@
import org.apache.flink.streaming.api.runners.python.beam.BeamPythonFunctionRunner;
import org.apache.flink.util.Preconditions;

import com.google.protobuf.GeneratedMessageV3;
import com.google.protobuf.GeneratedMessage;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.runners.core.construction.graph.TimerReference;

Expand All @@ -50,14 +50,14 @@ public class BeamTablePythonFunctionRunner extends BeamPythonFunctionRunner {
/** The urn which represents the function kind to be executed. */
private final String functionUrn;

private final GeneratedMessageV3 userDefinedFunctionProto;
private final GeneratedMessage userDefinedFunctionProto;

public BeamTablePythonFunctionRunner(
Environment environment,
String taskName,
ProcessPythonEnvironmentManager environmentManager,
String functionUrn,
GeneratedMessageV3 userDefinedFunctionProto,
GeneratedMessage userDefinedFunctionProto,
FlinkMetricContainer flinkMetricContainer,
KeyedStateBackend<?> keyedStateBackend,
TypeSerializer<?> keySerializer,
Expand Down Expand Up @@ -124,7 +124,7 @@ public static BeamTablePythonFunctionRunner stateless(
String taskName,
ProcessPythonEnvironmentManager environmentManager,
String functionUrn,
GeneratedMessageV3 userDefinedFunctionProto,
GeneratedMessage userDefinedFunctionProto,
FlinkMetricContainer flinkMetricContainer,
MemoryManager memoryManager,
double managedMemoryFraction,
Expand All @@ -151,7 +151,7 @@ public static BeamTablePythonFunctionRunner stateful(
String taskName,
ProcessPythonEnvironmentManager environmentManager,
String functionUrn,
GeneratedMessageV3 userDefinedFunctionProto,
GeneratedMessage userDefinedFunctionProto,
FlinkMetricContainer flinkMetricContainer,
KeyedStateBackend<?> keyedStateBackend,
TypeSerializer<?> keySerializer,
Expand Down
2 changes: 1 addition & 1 deletion flink-python/src/main/resources/META-INF/NOTICE
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ This project bundles the following dependencies under the BSD license.
See bundled license files for details

- net.sf.py4j:py4j:0.10.9.7
- com.google.protobuf:protobuf-java:3.21.7
- com.google.protobuf:protobuf-java:4.32.1

This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT)
See bundled license files for details.
Expand Down
2 changes: 1 addition & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -162,7 +162,7 @@ under the License.
<assertj.version>3.27.3</assertj.version>
<py4j.version>0.10.9.7</py4j.version>
<beam.version>2.54.0</beam.version>
<protoc.version>3.21.7</protoc.version>
<protoc.version>4.32.1</protoc.version>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This upgrade sounds like a great idea. I think we should update the docs to draw users attention to this new support.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good call. I'll update the relevant docs.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated the protobuf format docs and add a section to the 2.1 release ntoes.

<okhttp.version>3.14.9</okhttp.version>
<testcontainers.version>1.20.2</testcontainers.version>
<lz4.version>1.8.0</lz4.version>
Expand Down