Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 6066af3

Browse files
authored
[YAML] - Kafka Proto String schema (#29835)
* [YAML] - Kafka Proto String schema
1 parent 8aa16df commit 6066af3

10 files changed

Lines changed: 373 additions & 125 deletions

File tree

sdks/java/extensions/protobuf/build.gradle

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,8 @@ dependencies {
3939
implementation library.java.slf4j_api
4040
implementation project(path: ":sdks:java:core", configuration: "shadow")
4141
implementation library.java.protobuf_java
42+
implementation("com.squareup.wire:wire-schema-jvm:4.9.3")
43+
implementation("io.apicurio:apicurio-registry-protobuf-schema-utilities:3.0.0.M2")
4244
testImplementation project(path: ":sdks:java:core", configuration: "shadowTest")
4345
testImplementation library.java.junit
4446
testRuntimeOnly library.java.slf4j_jdk14

sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtils.java

Lines changed: 85 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,10 @@
2424
import com.google.protobuf.Descriptors;
2525
import com.google.protobuf.DynamicMessage;
2626
import com.google.protobuf.InvalidProtocolBufferException;
27+
import com.squareup.wire.schema.Location;
28+
import com.squareup.wire.schema.internal.parser.ProtoFileElement;
29+
import com.squareup.wire.schema.internal.parser.ProtoParser;
30+
import io.apicurio.registry.utils.protobuf.schema.FileDescriptorUtils;
2731
import java.io.IOException;
2832
import java.io.InputStream;
2933
import java.io.Serializable;
@@ -55,6 +59,8 @@ public class ProtoByteUtils {
5559

5660
private static final Logger LOG = LoggerFactory.getLogger(ProtoByteUtils.class);
5761

62+
private static final Location LOCATION = Location.get("");
63+
5864
/**
5965
* Retrieves a Beam Schema from a Protocol Buffer message.
6066
*
@@ -68,6 +74,68 @@ public static Schema getBeamSchemaFromProto(String fileDescriptorPath, String me
6874
return ProtoDynamicMessageSchema.forDescriptor(protoDomain, messageName).getSchema();
6975
}
7076

77+
/**
78+
* Parses the given Protocol Buffers schema string, retrieves the Descriptor for the specified
79+
* message name, and constructs a Beam Schema from it.
80+
*
81+
* @param schemaString The Protocol Buffers schema string.
82+
* @param messageName The name of the message type for which the Beam Schema is desired.
83+
* @return The Beam Schema constructed from the specified Protocol Buffers schema.
84+
* @throws RuntimeException If there is an error during parsing, descriptor retrieval, or schema
85+
* construction.
86+
*/
87+
public static Schema getBeamSchemaFromProtoSchema(String schemaString, String messageName) {
88+
Descriptors.Descriptor descriptor = getDescriptorFromProtoSchema(schemaString, messageName);
89+
return ProtoDynamicMessageSchema.forDescriptor(ProtoDomain.buildFrom(descriptor), descriptor)
90+
.getSchema();
91+
}
92+
93+
/**
94+
* Parses the given Protocol Buffers schema string, retrieves the FileDescriptor, and returns the
95+
* Descriptor for the specified message name.
96+
*
97+
* @param schemaString The Protocol Buffers schema string.
98+
* @param messageName The name of the message type for which the descriptor is desired.
99+
* @return The Descriptor for the specified message name.
100+
* @throws RuntimeException If there is an error during parsing or descriptor validation.
101+
*/
102+
private static Descriptors.Descriptor getDescriptorFromProtoSchema(
103+
final String schemaString, final String messageName) {
104+
ProtoFileElement result = ProtoParser.Companion.parse(LOCATION, schemaString);
105+
try {
106+
Descriptors.FileDescriptor fileDescriptor =
107+
FileDescriptorUtils.protoFileToFileDescriptor(result);
108+
return fileDescriptor.findMessageTypeByName(messageName);
109+
} catch (Descriptors.DescriptorValidationException e) {
110+
throw new RuntimeException(e);
111+
}
112+
}
113+
114+
public static SerializableFunction<byte[], Row> getProtoBytesToRowFromSchemaFunction(
115+
String schemaString, String messageName) {
116+
117+
Descriptors.Descriptor descriptor = getDescriptorFromProtoSchema(schemaString, messageName);
118+
119+
ProtoDynamicMessageSchema<DynamicMessage> protoDynamicMessageSchema =
120+
ProtoDynamicMessageSchema.forDescriptor(ProtoDomain.buildFrom(descriptor), descriptor);
121+
return new SimpleFunction<byte[], Row>() {
122+
@Override
123+
public Row apply(byte[] input) {
124+
try {
125+
Descriptors.Descriptor descriptorFunction =
126+
getDescriptorFromProtoSchema(schemaString, messageName);
127+
DynamicMessage dynamicMessage = DynamicMessage.parseFrom(descriptorFunction, input);
128+
SerializableFunction<DynamicMessage, Row> res =
129+
protoDynamicMessageSchema.getToRowFunction();
130+
return res.apply(dynamicMessage);
131+
} catch (InvalidProtocolBufferException e) {
132+
LOG.error("Error parsing to DynamicMessage", e);
133+
throw new RuntimeException(e);
134+
}
135+
}
136+
};
137+
}
138+
71139
public static SerializableFunction<byte[], Row> getProtoBytesToRowFunction(
72140
String fileDescriptorPath, String messageName) {
73141

@@ -96,6 +164,23 @@ public Row apply(byte[] input) {
96164
};
97165
}
98166

167+
public static SerializableFunction<Row, byte[]> getRowToProtoBytesFromSchema(
168+
String schemaString, String messageName) {
169+
170+
Descriptors.Descriptor descriptor = getDescriptorFromProtoSchema(schemaString, messageName);
171+
172+
ProtoDynamicMessageSchema<DynamicMessage> protoDynamicMessageSchema =
173+
ProtoDynamicMessageSchema.forDescriptor(ProtoDomain.buildFrom(descriptor), descriptor);
174+
return new SimpleFunction<Row, byte[]>() {
175+
@Override
176+
public byte[] apply(Row input) {
177+
SerializableFunction<Row, DynamicMessage> res =
178+
protoDynamicMessageSchema.getFromRowFunction();
179+
return res.apply(input).toByteArray();
180+
}
181+
};
182+
}
183+
99184
public static SerializableFunction<Row, byte[]> getRowToProtoBytes(
100185
String fileDescriptorPath, String messageName) {
101186
ProtoSchemaInfo dynamicProtoDomain = getProtoDomain(fileDescriptorPath);

sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtilsTest.java

Lines changed: 49 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,25 @@
2929
@RunWith(JUnit4.class)
3030
public class ProtoByteUtilsTest {
3131

32+
private static final String PROTO_STRING_SCHEMA =
33+
"syntax = \"proto3\";\n"
34+
+ "\n"
35+
+ "message MyMessage {\n"
36+
+ " int32 id = 1;\n"
37+
+ " string name = 2;\n"
38+
+ " bool active = 3;\n"
39+
+ "\n"
40+
+ " // Nested field\n"
41+
+ " message Address {\n"
42+
+ " string street = 1;\n"
43+
+ " string city = 2;\n"
44+
+ " string state = 3;\n"
45+
+ " string zip_code = 4;\n"
46+
+ " }\n"
47+
+ "\n"
48+
+ " Address address = 4;\n"
49+
+ "}";
50+
3251
private static final String DESCRIPTOR_PATH =
3352
Objects.requireNonNull(
3453
ProtoByteUtilsTest.class.getResource(
@@ -59,13 +78,26 @@ public void testProtoSchemaToBeamSchema() {
5978
Assert.assertEquals(schema.getFieldNames(), SCHEMA.getFieldNames());
6079
}
6180

81+
@Test
82+
public void testProtoSchemaStringToBeamSchema() {
83+
Schema schema = ProtoByteUtils.getBeamSchemaFromProtoSchema(PROTO_STRING_SCHEMA, "MyMessage");
84+
Assert.assertEquals(schema.getFieldNames(), SCHEMA.getFieldNames());
85+
}
86+
6287
@Test
6388
public void testProtoBytesToRowFunctionGenerateSerializableFunction() {
6489
SerializableFunction<byte[], Row> protoBytesToRowFunction =
6590
ProtoByteUtils.getProtoBytesToRowFunction(DESCRIPTOR_PATH, MESSAGE_NAME);
6691
Assert.assertNotNull(protoBytesToRowFunction);
6792
}
6893

94+
@Test
95+
public void testProtoBytesToRowSchemaStringGenerateSerializableFunction() {
96+
SerializableFunction<byte[], Row> protoBytesToRowFunction =
97+
ProtoByteUtils.getProtoBytesToRowFromSchemaFunction(PROTO_STRING_SCHEMA, "MyMessage");
98+
Assert.assertNotNull(protoBytesToRowFunction);
99+
}
100+
69101
@Test(expected = java.lang.RuntimeException.class)
70102
public void testProtoBytesToRowFunctionReturnsRowFailure() {
71103
// Create a proto bytes to row function
@@ -95,4 +127,21 @@ public void testRowToProtoFunction() {
95127
Assert.assertNotNull(
96128
ProtoByteUtils.getRowToProtoBytes(DESCRIPTOR_PATH, MESSAGE_NAME).apply(row));
97129
}
130+
131+
@Test
132+
public void testRowToProtoSchemaFunction() {
133+
Row row =
134+
Row.withSchema(SCHEMA)
135+
.withFieldValue("id", 1234)
136+
.withFieldValue("name", "Doe")
137+
.withFieldValue("active", false)
138+
.withFieldValue("address.city", "seattle")
139+
.withFieldValue("address.street", "fake street")
140+
.withFieldValue("address.zip_code", "TO-1234")
141+
.withFieldValue("address.state", "wa")
142+
.build();
143+
144+
Assert.assertNotNull(
145+
ProtoByteUtils.getRowToProtoBytesFromSchema(PROTO_STRING_SCHEMA, "MyMessage").apply(row));
146+
}
98147
}

sdks/java/io/kafka/build.gradle

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -89,6 +89,7 @@ dependencies {
8989
testImplementation project(path: ":sdks:java:core", configuration: "shadowTest")
9090
testImplementation project(":sdks:java:io:synthetic")
9191
testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration")
92+
testImplementation project(path: ":sdks:java:extensions:protobuf", configuration: "testRuntimeMigration")
9293
testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration")
9394
testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration")
9495
// For testing Cross-language transforms

sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformConfiguration.java

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,29 @@ public void validate() {
5151
final String dataFormat = this.getFormat();
5252
assert dataFormat == null || VALID_DATA_FORMATS.contains(dataFormat)
5353
: "Valid data formats are " + VALID_DATA_FORMATS;
54+
55+
final String inputSchema = this.getSchema();
56+
final String messageName = this.getMessageName();
57+
final String fileDescriptorPath = this.getFileDescriptorPath();
58+
final String confluentSchemaRegUrl = this.getConfluentSchemaRegistryUrl();
59+
final String confluentSchemaRegSubject = this.getConfluentSchemaRegistrySubject();
60+
61+
if (confluentSchemaRegUrl != null) {
62+
assert confluentSchemaRegSubject != null
63+
: "To read from Kafka, a schema must be provided directly or though Confluent "
64+
+ "Schema Registry. Make sure you are providing one of these parameters.";
65+
} else if (dataFormat != null && dataFormat.equals("RAW")) {
66+
assert inputSchema == null : "To read from Kafka in RAW format, you can't provide a schema.";
67+
} else if (dataFormat != null && dataFormat.equals("JSON")) {
68+
assert inputSchema != null : "To read from Kafka in JSON format, you must provide a schema.";
69+
} else if (dataFormat != null && dataFormat.equals("PROTO")) {
70+
assert messageName != null
71+
: "To read from Kafka in PROTO format, messageName must be provided.";
72+
assert fileDescriptorPath != null || inputSchema != null
73+
: "To read from Kafka in PROTO format, fileDescriptorPath or schema must be provided.";
74+
} else {
75+
assert inputSchema != null : "To read from Kafka in AVRO format, you must provide a schema.";
76+
}
5477
}
5578

5679
/** Instantiates a {@link KafkaReadSchemaTransformConfiguration.Builder} instance. */

0 commit comments

Comments
 (0)