Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[YAML] - Kafka write and RAW format #29160

Merged
merged 2 commits into from
Nov 1, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
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 @@ -41,6 +41,7 @@
import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollectionRowTuple;
import org.apache.beam.sdk.values.PCollectionTuple;
Expand All @@ -60,7 +61,7 @@ public class KafkaWriteSchemaTransformProvider
extends TypedSchemaTransformProvider<
KafkaWriteSchemaTransformProvider.KafkaWriteSchemaTransformConfiguration> {

public static final String SUPPORTED_FORMATS_STR = "JSON,AVRO";
public static final String SUPPORTED_FORMATS_STR = "RAW,JSON,AVRO";
public static final Set<String> SUPPORTED_FORMATS =
Sets.newHashSet(SUPPORTED_FORMATS_STR.split(","));
public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {};
Expand Down Expand Up @@ -131,10 +132,18 @@ public void finish() {
@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
Schema inputSchema = input.get("input").getSchema();
final SerializableFunction<Row, byte[]> toBytesFn =
configuration.getFormat().equals("JSON")
? JsonUtils.getRowToJsonBytesFunction(inputSchema)
: AvroUtils.getRowToAvroBytesFunction(inputSchema);
final SerializableFunction<Row, byte[]> toBytesFn;
if (configuration.getFormat().equals("RAW")) {
int numFields = inputSchema.getFields().size();
if (numFields != 1) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Perhaps check its type as well?

Copy link
Contributor

Choose a reason for hiding this comment

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

We can do this as a follow-up.

throw new IllegalArgumentException("Expecting exactly one field, found " + numFields);
}
toBytesFn = getRowToRawBytesFunction(inputSchema.getField(0).getName());
} else if (configuration.getFormat().equals("JSON")) {
toBytesFn = JsonUtils.getRowToJsonBytesFunction(inputSchema);
} else {
toBytesFn = AvroUtils.getRowToAvroBytesFunction(inputSchema);
}

final Map<String, String> configOverrides = configuration.getProducerConfigUpdates();
PCollectionTuple outputTuple =
Expand Down Expand Up @@ -163,6 +172,19 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) {
}
}

public static SerializableFunction<Row, byte[]> getRowToRawBytesFunction(String rowFieldName) {
return new SimpleFunction<Row, byte[]>() {
@Override
public byte[] apply(Row input) {
byte[] rawBytes = input.getBytes(rowFieldName);
if (rawBytes == null) {
throw new NullPointerException();
}
return rawBytes;
}
};
}

@Override
public @UnknownKeyFor @NonNull @Initialized String identifier() {
return "beam:schematransform:org.apache.beam:kafka_write:v1";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,9 @@
*/
package org.apache.beam.sdk.io.kafka;

import static org.apache.beam.sdk.io.kafka.KafkaWriteSchemaTransformProvider.getRowToRawBytesFunction;

import java.io.UnsupportedEncodingException;
import java.util.Arrays;
import java.util.List;
import org.apache.beam.sdk.io.kafka.KafkaWriteSchemaTransformProvider.KafkaWriteSchemaTransform.ErrorCounterFn;
Expand Down Expand Up @@ -47,6 +50,9 @@ public class KafkaWriteSchemaTransformProviderTest {

private static final Schema BEAMSCHEMA =
Schema.of(Schema.Field.of("name", Schema.FieldType.STRING));

private static final Schema BEAMRAWSCHEMA =
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: Why aren't these CAP_UNDERSCORE_CASE?

Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES));
private static final Schema ERRORSCHEMA = KafkaWriteSchemaTransformProvider.ERROR_SCHEMA;

private static final List<Row> ROWS =
Expand All @@ -55,9 +61,27 @@ public class KafkaWriteSchemaTransformProviderTest {
Row.withSchema(BEAMSCHEMA).withFieldValue("name", "b").build(),
Row.withSchema(BEAMSCHEMA).withFieldValue("name", "c").build());

private static final List<Row> RAW_ROWS;

static {
try {
RAW_ROWS =
Arrays.asList(
Row.withSchema(BEAMRAWSCHEMA).withFieldValue("payload", "a".getBytes("UTF8")).build(),
Row.withSchema(BEAMRAWSCHEMA).withFieldValue("payload", "b".getBytes("UTF8")).build(),
Row.withSchema(BEAMRAWSCHEMA)
.withFieldValue("payload", "c".getBytes("UTF8"))
.build());
} catch (UnsupportedEncodingException e) {
throw new RuntimeException(e);
}
}

final SerializableFunction<Row, byte[]> valueMapper =
JsonUtils.getRowToJsonBytesFunction(BEAMSCHEMA);

final SerializableFunction<Row, byte[]> valueRawMapper = getRowToRawBytesFunction("payload");

@Rule public transient TestPipeline p = TestPipeline.create();

@Test
Expand All @@ -79,4 +103,24 @@ public void testKafkaErrorFnSuccess() throws Exception {
PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(msg);
p.run().waitUntilFinish();
}

@Test
public void testKafkaErrorFnRawSuccess() throws Exception {
List<KV<byte[], byte[]>> msg =
Arrays.asList(
KV.of(new byte[1], "a".getBytes("UTF8")),
KV.of(new byte[1], "b".getBytes("UTF8")),
KV.of(new byte[1], "c".getBytes("UTF8")));

PCollection<Row> input = p.apply(Create.of(RAW_ROWS));
PCollectionTuple output =
input.apply(
ParDo.of(new ErrorCounterFn("Kafka-write-error-counter", valueRawMapper))
.withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG)));

output.get(ERROR_TAG).setRowSchema(ERRORSCHEMA);

PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(msg);
p.run().waitUntilFinish();
}
}
7 changes: 7 additions & 0 deletions sdks/python/apache_beam/yaml/standard_io.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
- type: renaming
transforms:
'ReadFromKafka': 'ReadFromKafka'
'WriteToKafka': 'WriteToKafka'
config:
mappings:
'ReadFromKafka':
Expand All @@ -60,10 +61,16 @@
'confluent_schema_registry_url': 'confluentSchemaRegistryUrl'
'confluent_schema_registry_subject': 'confluentSchemaRegistrySubject'
'auto_offset_reset_config': 'autoOffsetResetConfig'
'WriteToKafka':
'format': 'format'
'topic': 'topic'
'bootstrap_servers': 'bootstrapServers'
'producer_config_updates': 'ProducerConfigUpdates'
underlying_provider:
type: beamJar
transforms:
'ReadFromKafka': 'beam:schematransform:org.apache.beam:kafka_read:v1'
'WriteToKafka': 'beam:schematransform:org.apache.beam:kafka_write:v1'
config:
gradle_target: 'sdks:java:io:expansion-service:shadowJar'

Expand Down
Loading