nickuncaged1201 commented on code in PR #27039:
URL: https://github.com/apache/beam/pull/27039#discussion_r1231373423
##########
sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/AvroWriteSchemaTransformFormatProvider.java:
##########
@@ -47,22 +57,27 @@ public String identifier() {
* {@link PCollection} file names written using {@link AvroIO.Write}.
*/
@Override
- public PTransform<PCollection<Row>, PCollection<String>> buildTransform(
+ public PTransform<PCollection<Row>, PCollectionTuple> buildTransform(
FileWriteSchemaTransformConfiguration configuration, Schema schema) {
- return new PTransform<PCollection<Row>, PCollection<String>>() {
+ return new PTransform<PCollection<Row>, PCollectionTuple>() {
@Override
- public PCollection<String> expand(PCollection<Row> input) {
+ public PCollectionTuple expand(PCollection<Row> input) {
org.apache.avro.Schema avroSchema = AvroUtils.toAvroSchema(schema);
AvroGenericCoder coder = AvroGenericCoder.of(avroSchema);
- PCollection<GenericRecord> avro =
- input
- .apply(
- "Row To Avro Generic Record",
-
FileWriteSchemaTransformFormatProviders.mapRowsToGenericRecords(schema))
- .setCoder(coder);
+ PCollectionTuple tuple =
+ input.apply(
+ "Row To Avro Generic Record",
+ ParDo.of(
+ new ErrorCounterFn<GenericRecord>(
+ "Avro-write-error-counter",
+
AvroUtils.getRowToGenericRecordFunction(AvroUtils.toAvroSchema(schema)),
+ ERROR_FN_OUPUT_TAG))
+ .withOutputTags(ERROR_FN_OUPUT_TAG,
TupleTagList.of(ERROR_TAG)));
Review Comment:
r: @manavgarg The ERROR_FN_OUTPUT_TAG is for successful outputs. Let me know
if there's a better name for this.
r: @johnjcasey The AvroWriteIo doesn't expose any errors by itself, so I
won't be able to count any error there. I implement the error counting for this
conversion step to GenericRecords with the intention that if such a conversion
fails, then definitely some error happens with the record. If the conversion is
successful, I don't see a reason the write will fail anyway. This is a bit of
an unfortunate misnomer to call it write level error count
--
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]