ffernandez92 commented on code in PR #28865:
URL: https://github.com/apache/beam/pull/28865#discussion_r1349484401
##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaReadSchemaTransformProvider.java:
##########
@@ -112,16 +113,60 @@ protected SchemaTransform
from(KafkaReadSchemaTransformConfiguration configurati
consumerConfigs.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 100);
consumerConfigs.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
autoOffsetReset);
+ String format = configuration.getFormat();
+
+ if (format == null || format.isEmpty()) {
+ format = "raw";
+ }
+
+ if (format.equalsIgnoreCase("raw")) {
+ if (inputSchema != null && !inputSchema.isEmpty()) {
+ throw new IllegalArgumentException(
+ "To read from Kafka in raw format, you can't provide a schema.");
+ }
+ Schema rawSchema = Schema.builder().addField("payload",
Schema.FieldType.BYTES).build();
+ SerializableFunction<byte[], Row> valueMapper =
getRawBytesToRowFunction(rawSchema);
+ return new SchemaTransform() {
+ @Override
+ public PCollectionRowTuple expand(PCollectionRowTuple input) {
+ KafkaIO.Read<byte[], byte[]> kafkaRead =
+ KafkaIO.readBytes()
+ .withConsumerConfigUpdates(consumerConfigs)
+ .withConsumerFactoryFn(new
ConsumerFactoryWithGcsTrustStores())
+ .withTopic(configuration.getTopic())
+ .withBootstrapServers(configuration.getBootstrapServers());
+ if (isTest) {
+ kafkaRead =
kafkaRead.withMaxReadTime(Duration.standardSeconds(testTimeoutSecs));
+ }
+
+ PCollection<byte[]> kafkaValues =
+
input.getPipeline().apply(kafkaRead.withoutMetadata()).apply(Values.create());
+
+ PCollectionTuple outputTuple =
+ kafkaValues.apply(
+ ParDo.of(new ErrorFn("Kafka-read-error-counter",
valueMapper))
+ .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG)));
+
+ return PCollectionRowTuple.of(
+ "output",
+ outputTuple.get(OUTPUT_TAG).setRowSchema(rawSchema),
+ "errors",
+ outputTuple.get(ERROR_TAG).setRowSchema(ERROR_SCHEMA));
+ }
+ };
+ }
+
if (inputSchema != null && !inputSchema.isEmpty()) {
assert
Strings.isNullOrEmpty(configuration.getConfluentSchemaRegistryUrl())
: "To read from Kafka, a schema must be provided directly or though
Confluent "
+ "Schema Registry, but not both.";
Review Comment:
Exactly, that was my idea here. Again, it's a bit uncertain to me if this
isn't used by someone somewhere, so I think we still want to support that
functionality. That said, though, it makes total sense to include other schema
registry technologies as we keep refactoring this class.
--
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]