robertwb commented on code in PR #29164: URL: https://github.com/apache/beam/pull/29164#discussion_r1402618822
########## sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java: ########## @@ -0,0 +1,140 @@ +/* + * 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.beam.sdk.transforms.errorhandling; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.PrintStream; +import java.io.Serializable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord.Failure; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord.Record; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public interface BadRecordRouter extends Serializable { + + BadRecordRouter THROWING_ROUTER = new ThrowingBadRecordRouter(); + + BadRecordRouter RECORDING_ROUTER = new RecordingBadRecordRouter(); + + TupleTag<BadRecord> BAD_RECORD_TAG = new TupleTag<>(); + + <RecordT> void route( + MultiOutputReceiver outputReceiver, + RecordT record, + @Nullable Coder<RecordT> coder, + @Nullable Exception exception, + String description, + String failingTransform) + throws Exception; + + class ThrowingBadRecordRouter implements BadRecordRouter { + + @Override + public <RecordT> void route( + MultiOutputReceiver outputReceiver, + RecordT record, + @Nullable Coder<RecordT> coder, + @Nullable Exception exception, + String description, + String failingTransform) + throws Exception { + if (exception != null) { + throw exception; + } + } + } + + class RecordingBadRecordRouter implements BadRecordRouter { + + private static final Logger LOG = LoggerFactory.getLogger(RecordingBadRecordRouter.class); + + @Override + public <RecordT> void route( + MultiOutputReceiver outputReceiver, + RecordT record, + @Nullable Coder<RecordT> coder, + @Nullable Exception exception, + String description, + String failingTransform) + throws Exception { + Preconditions.checkArgumentNotNull(record); + ObjectWriter objectWriter = new ObjectMapper().writer().withDefaultPrettyPrinter(); + + // Build up record information Review Comment: I wonder if this logic would be better placed as factory methods on BadRecord itself. ########## sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java: ########## @@ -0,0 +1,140 @@ +/* + * 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.beam.sdk.transforms.errorhandling; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.PrintStream; +import java.io.Serializable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord.Failure; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord.Record; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public interface BadRecordRouter extends Serializable { + + BadRecordRouter THROWING_ROUTER = new ThrowingBadRecordRouter(); + + BadRecordRouter RECORDING_ROUTER = new RecordingBadRecordRouter(); + + TupleTag<BadRecord> BAD_RECORD_TAG = new TupleTag<>(); + + <RecordT> void route( + MultiOutputReceiver outputReceiver, + RecordT record, + @Nullable Coder<RecordT> coder, + @Nullable Exception exception, + String description, + String failingTransform) + throws Exception; + + class ThrowingBadRecordRouter implements BadRecordRouter { + + @Override + public <RecordT> void route( + MultiOutputReceiver outputReceiver, + RecordT record, + @Nullable Coder<RecordT> coder, + @Nullable Exception exception, + String description, + String failingTransform) + throws Exception { + if (exception != null) { + throw exception; + } Review Comment: Presumably we should be throwing a generic exception if one is not provided? ########## sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java: ########## @@ -0,0 +1,138 @@ +/* + * 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.beam.sdk.transforms.errorhandling; + +import com.google.auto.value.AutoValue; +import java.io.Serializable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.checkerframework.checker.nullness.qual.Nullable; + +@AutoValue +@DefaultSchema(AutoValueSchema.class) +public abstract class BadRecord implements Serializable { + + /** Information about the record that failed. */ + public abstract Record getRecord(); + + /** Information about why the record failed. */ + public abstract Failure getFailure(); + + public static Builder builder() { + return new AutoValue_BadRecord.Builder(); + } + + public static Coder<BadRecord> getCoder(Pipeline pipeline) { + try { + SchemaRegistry schemaRegistry = pipeline.getSchemaRegistry(); + return SchemaCoder.of( + schemaRegistry.getSchema(BadRecord.class), + TypeDescriptor.of(BadRecord.class), + schemaRegistry.getToRowFunction(BadRecord.class), + schemaRegistry.getFromRowFunction(BadRecord.class)); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setRecord(Record record); + + public abstract Builder setFailure(Failure error); + + public abstract BadRecord build(); + } + + @AutoValue + @DefaultSchema(AutoValueSchema.class) + public abstract static class Record implements Serializable { + + /** The failing record, encoded as JSON. Will be null if serialization as JSON fails. */ + public abstract @Nullable String getJsonRecord(); Review Comment: So, thinking about this, we don't expect all records to be (faithfully) represented in json, but that's not the point. The point is to give the user some idea what the record is without having to scrutinize the bytes. Should we lengthen this to humanReadableJsonRecord to better capture its intent? ########## sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/BRHEnabledPTransform.java: ########## @@ -0,0 +1,104 @@ +/* + * 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.beam.sdk.transforms.errorhandling; + +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.NoOpErrorHandler; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.TypeDescriptor; + +/** Dummy PTransform that is configurable with a Bad Record Handler. */ +public class BRHEnabledPTransform extends PTransform<PCollection<Integer>, PCollection<Integer>> { + + private ErrorHandler<BadRecord, ?> errorHandler = new NoOpErrorHandler<>(); + + private BadRecordRouter badRecordRouter = BadRecordRouter.THROWING_ROUTER; + + private static final TupleTag<Integer> RECORDS = new TupleTag<>(); + + public BRHEnabledPTransform() {} + + public BRHEnabledPTransform withBadRecordHandler(ErrorHandler<BadRecord, ?> errorHandler) { + this.errorHandler = errorHandler; + this.badRecordRouter = BadRecordRouter.RECORDING_ROUTER; + return this; + } + + @Override + public PCollection<Integer> expand(PCollection<Integer> input) { + PCollectionTuple pCollectionTuple = + input.apply( + "NoOpDoFn", + ParDo.of(new OddIsBad(badRecordRouter)) + .withOutputTags(RECORDS, TupleTagList.of(BadRecordRouter.BAD_RECORD_TAG))); + + Coder<BadRecord> badRecordCoder; + + try { + SchemaRegistry schemaRegistry = input.getPipeline().getSchemaRegistry(); + badRecordCoder = + SchemaCoder.of( + schemaRegistry.getSchema(BadRecord.class), + TypeDescriptor.of(BadRecord.class), + schemaRegistry.getToRowFunction(BadRecord.class), + schemaRegistry.getFromRowFunction(BadRecord.class)); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + + errorHandler.addErrorCollection( + pCollectionTuple.get(BadRecordRouter.BAD_RECORD_TAG).setCoder(badRecordCoder)); + + return pCollectionTuple.get(RECORDS).setCoder(BigEndianIntegerCoder.of()); + } + + public static class OddIsBad extends DoFn<Integer, Integer> { + + private BadRecordRouter badRecordRouter; + + public OddIsBad(BadRecordRouter badRecordRouter) { + this.badRecordRouter = badRecordRouter; + } + + @ProcessElement + public void processElement(@Element Integer element, MultiOutputReceiver receiver) + throws Exception { + if (element % 2 == 0) { + receiver.get(RECORDS).output(element); + } else { + badRecordRouter.route( + receiver, + element, + BigEndianIntegerCoder.of(), + new RuntimeException(), + "Integer was odd", + "NoOpDoFn"); Review Comment: I think we'd have to extract this from the DoFn Context object. (Or, more hacky, leverage the same globals that Counters and logging do.) ########## sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.beam.sdk.transforms.errorhandling; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.POutput; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An Error Handler is a utility object used for plumbing error PCollections to a configured sink + * Error Handlers must be closed before a pipeline is run to properly pipe error collections to the + * sink, and the pipeline will be rejected if any handlers aren't closed. + * + * @param <ErrorT> The type of the error object. This will usually be a {@link BadRecord}, but can + * be any type + * @param <OutputT> The return type of the sink PTransform. + * <p>Usage of Error Handlers: + * <p>Simple usage with one DLQ + * <pre>{@code + * PCollection<?> records = ...; + * try (ErrorHandler<E,T> errorHandler = pipeline.registerErrorHandler(SomeSink.write())) { + * PCollection<?> results = records.apply(SomeIO.write().withDeadLetterQueue(errorHandler)); + * } + * results.apply(SomeOtherTransform); + * }</pre> + * Usage with multiple DLQ stages + * <pre>{@code + * PCollection<?> records = ...; + * try (ErrorHandler<E,T> errorHandler = pipeline.registerErrorHandler(SomeSink.write())) { + * PCollection<?> results = records.apply(SomeIO.write().withDeadLetterQueue(errorHandler)) + * .apply(OtherTransform.builder().withDeadLetterQueue(errorHandler)); + * } + * results.apply(SomeOtherTransform); + * }</pre> + */ +public interface ErrorHandler<ErrorT, OutputT extends POutput> extends AutoCloseable { + + void addErrorCollection(PCollection<ErrorT> errorCollection); + + boolean isClosed(); + + @Nullable + OutputT getOutput(); + + class PTransformErrorHandler<ErrorT, OutputT extends POutput> + implements ErrorHandler<ErrorT, OutputT> { + + private static final Logger LOG = LoggerFactory.getLogger(PTransformErrorHandler.class); + private final PTransform<PCollection<ErrorT>, OutputT> sinkTransform; + + private final Pipeline pipeline; + + private final Coder<ErrorT> coder; + + private final List<PCollection<ErrorT>> errorCollections = new ArrayList<>(); + + private @Nullable OutputT sinkOutput = null; + + private boolean closed = false; + + /** + * Constructs a new ErrorHandler, but should not be called directly. Instead, call + * pipeline.registerErrorHandler to ensure safe pipeline construction + */ + @Internal + public PTransformErrorHandler( + PTransform<PCollection<ErrorT>, OutputT> sinkTransform, + Pipeline pipeline, + Coder<ErrorT> coder) { + this.sinkTransform = sinkTransform; + this.pipeline = pipeline; + this.coder = coder; + } + + @Override + public void addErrorCollection(PCollection<ErrorT> errorCollection) { + errorCollections.add(errorCollection); + } + + @Override + public boolean isClosed() { + return closed; + } + + @Override + public @Nullable OutputT getOutput() { + if (!this.isClosed()) { + throw new IllegalStateException( + "ErrorHandler must be finalized before the output can be returned"); + } + return sinkOutput; + } + + @Override + public void close() { + closed = true; Review Comment: Should it be an error (or no-op?) to close this twice? ########## sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlerTest.java: ########## @@ -0,0 +1,85 @@ +/* + * 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.beam.sdk.transforms.errorhandling; + +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class ErrorHandlerTest { + @Rule public final TestPipeline pipeline = TestPipeline.create(); + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + @Category(NeedsRunner.class) + public void testGoodErrorHandlerUsage() throws Exception { + try (ErrorHandler<String, PCollection<String>> eh = + pipeline.registerErrorHandler(new DummySinkTransform<>())) {} + + pipeline.run(); + } + + @Test + public void testBadErrorHandlerUsage() { + + pipeline.registerErrorHandler(new DummySinkTransform<PCollection<String>>()); + + thrown.expect(IllegalStateException.class); + + pipeline.run(); + } + + @Test + public void testBRHEnabledPTransform() { + PCollection<Integer> record = pipeline.apply(Create.of(1, 2, 3, 4)); + record.apply(new BRHEnabledPTransform()); + + thrown.expect(RuntimeException.class); Review Comment: Maybe make the comment a bit clearer? E.g. "Exception correctly aborts the entire pipeline because no error handler was registered?" ########## sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.beam.sdk.transforms.errorhandling; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.POutput; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An Error Handler is a utility object used for plumbing error PCollections to a configured sink + * Error Handlers must be closed before a pipeline is run to properly pipe error collections to the + * sink, and the pipeline will be rejected if any handlers aren't closed. + * + * @param <ErrorT> The type of the error object. This will usually be a {@link BadRecord}, but can + * be any type + * @param <OutputT> The return type of the sink PTransform. + * <p>Usage of Error Handlers: + * <p>Simple usage with one DLQ + * <pre>{@code + * PCollection<?> records = ...; + * try (ErrorHandler<E,T> errorHandler = pipeline.registerErrorHandler(SomeSink.write())) { + * PCollection<?> results = records.apply(SomeIO.write().withDeadLetterQueue(errorHandler)); + * } + * results.apply(SomeOtherTransform); + * }</pre> + * Usage with multiple DLQ stages + * <pre>{@code + * PCollection<?> records = ...; + * try (ErrorHandler<E,T> errorHandler = pipeline.registerErrorHandler(SomeSink.write())) { + * PCollection<?> results = records.apply(SomeIO.write().withDeadLetterQueue(errorHandler)) + * .apply(OtherTransform.builder().withDeadLetterQueue(errorHandler)); + * } + * results.apply(SomeOtherTransform); + * }</pre> + */ +public interface ErrorHandler<ErrorT, OutputT extends POutput> extends AutoCloseable { + + void addErrorCollection(PCollection<ErrorT> errorCollection); + + boolean isClosed(); + + @Nullable + OutputT getOutput(); + + class PTransformErrorHandler<ErrorT, OutputT extends POutput> + implements ErrorHandler<ErrorT, OutputT> { + + private static final Logger LOG = LoggerFactory.getLogger(PTransformErrorHandler.class); + private final PTransform<PCollection<ErrorT>, OutputT> sinkTransform; + + private final Pipeline pipeline; + + private final Coder<ErrorT> coder; + + private final List<PCollection<ErrorT>> errorCollections = new ArrayList<>(); + + private @Nullable OutputT sinkOutput = null; + + private boolean closed = false; + + /** + * Constructs a new ErrorHandler, but should not be called directly. Instead, call + * pipeline.registerErrorHandler to ensure safe pipeline construction + */ + @Internal + public PTransformErrorHandler( + PTransform<PCollection<ErrorT>, OutputT> sinkTransform, + Pipeline pipeline, + Coder<ErrorT> coder) { + this.sinkTransform = sinkTransform; + this.pipeline = pipeline; + this.coder = coder; + } + + @Override + public void addErrorCollection(PCollection<ErrorT> errorCollection) { + errorCollections.add(errorCollection); + } + + @Override + public boolean isClosed() { + return closed; + } + + @Override + public @Nullable OutputT getOutput() { + if (!this.isClosed()) { + throw new IllegalStateException( + "ErrorHandler must be finalized before the output can be returned"); + } + return sinkOutput; + } + + @Override + public void close() { + closed = true; + PCollection<ErrorT> flattened; + if (errorCollections.isEmpty()) { + LOG.warn("Empty list of error pcollections passed to ErrorHandler."); Review Comment: Warn is probably too strong, and likely not very actionable. ########## sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.beam.sdk.transforms.errorhandling; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.POutput; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An Error Handler is a utility object used for plumbing error PCollections to a configured sink + * Error Handlers must be closed before a pipeline is run to properly pipe error collections to the + * sink, and the pipeline will be rejected if any handlers aren't closed. + * + * @param <ErrorT> The type of the error object. This will usually be a {@link BadRecord}, but can + * be any type + * @param <OutputT> The return type of the sink PTransform. + * <p>Usage of Error Handlers: + * <p>Simple usage with one DLQ + * <pre>{@code + * PCollection<?> records = ...; + * try (ErrorHandler<E,T> errorHandler = pipeline.registerErrorHandler(SomeSink.write())) { + * PCollection<?> results = records.apply(SomeIO.write().withDeadLetterQueue(errorHandler)); + * } + * results.apply(SomeOtherTransform); + * }</pre> + * Usage with multiple DLQ stages + * <pre>{@code + * PCollection<?> records = ...; + * try (ErrorHandler<E,T> errorHandler = pipeline.registerErrorHandler(SomeSink.write())) { + * PCollection<?> results = records.apply(SomeIO.write().withDeadLetterQueue(errorHandler)) + * .apply(OtherTransform.builder().withDeadLetterQueue(errorHandler)); + * } + * results.apply(SomeOtherTransform); + * }</pre> + */ +public interface ErrorHandler<ErrorT, OutputT extends POutput> extends AutoCloseable { + + void addErrorCollection(PCollection<ErrorT> errorCollection); + + boolean isClosed(); + + @Nullable + OutputT getOutput(); + + class PTransformErrorHandler<ErrorT, OutputT extends POutput> + implements ErrorHandler<ErrorT, OutputT> { + + private static final Logger LOG = LoggerFactory.getLogger(PTransformErrorHandler.class); + private final PTransform<PCollection<ErrorT>, OutputT> sinkTransform; + + private final Pipeline pipeline; + + private final Coder<ErrorT> coder; + + private final List<PCollection<ErrorT>> errorCollections = new ArrayList<>(); + + private @Nullable OutputT sinkOutput = null; + + private boolean closed = false; + + /** + * Constructs a new ErrorHandler, but should not be called directly. Instead, call + * pipeline.registerErrorHandler to ensure safe pipeline construction + */ + @Internal + public PTransformErrorHandler( + PTransform<PCollection<ErrorT>, OutputT> sinkTransform, + Pipeline pipeline, + Coder<ErrorT> coder) { + this.sinkTransform = sinkTransform; + this.pipeline = pipeline; + this.coder = coder; + } + + @Override + public void addErrorCollection(PCollection<ErrorT> errorCollection) { + errorCollections.add(errorCollection); + } + + @Override + public boolean isClosed() { + return closed; + } + + @Override + public @Nullable OutputT getOutput() { + if (!this.isClosed()) { + throw new IllegalStateException( + "ErrorHandler must be finalized before the output can be returned"); + } + return sinkOutput; + } + + @Override + public void close() { + closed = true; + PCollection<ErrorT> flattened; + if (errorCollections.isEmpty()) { + LOG.warn("Empty list of error pcollections passed to ErrorHandler."); + flattened = pipeline.apply(Create.empty(coder)); + } else { + flattened = PCollectionList.of(errorCollections).apply(Flatten.pCollections()); + } + LOG.debug( + "{} error collections are being sent to {}", + errorCollections.size(), + sinkTransform.getName()); + String sinkTransformName = sinkTransform.getName(); + sinkOutput = + flattened + .apply( + "Record Error Metrics to " + sinkTransformName, + new WriteErrorMetrics<ErrorT>(sinkTransformName)) + .apply( + "Write to error Sink", + sinkTransform.addAnnotation( + "FeatureMetric", "ErrorHandler".getBytes(StandardCharsets.UTF_8))); + } + + public static class WriteErrorMetrics<ErrorT> + extends PTransform<PCollection<ErrorT>, PCollection<ErrorT>> { + + private final Counter errorCounter; + + public WriteErrorMetrics(String sinkTransformName) { + errorCounter = Metrics.counter("ErrorMetrics", sinkTransformName + "-input"); + } + + @Override + public PCollection<ErrorT> expand(PCollection<ErrorT> input) { + return input.apply(ParDo.of(new CountErrors<ErrorT>(errorCounter))); + } + + public static class CountErrors<ErrorT> extends DoFn<ErrorT, ErrorT> { + + private final Counter errorCounter; + + public CountErrors(Counter errorCounter) { + this.errorCounter = errorCounter; + } + + @ProcessElement + public void processElement(@Element ErrorT error, OutputReceiver<ErrorT> receiver) { + errorCounter.inc(); + receiver.output(error); + } + } + } + } + + class BadRecordErrorHandler<OutputT extends POutput> + extends PTransformErrorHandler<BadRecord, OutputT> { + + /** Constructs a new ErrorHandler for handling BadRecords. */ + @Internal + public BadRecordErrorHandler( + PTransform<PCollection<BadRecord>, OutputT> sinkTransform, Pipeline pipeline) { + super(sinkTransform, pipeline, BadRecord.getCoder(pipeline)); + } + } + + /** + * A default, placeholder error handler that exists to allow usage of .addErrorCollection() + * without effects. This enables more simple codepaths without checking for whether the user + * configured an error handler or not. + */ + @Internal + class NoOpErrorHandler<ErrorT, OutputT extends POutput> implements ErrorHandler<ErrorT, OutputT> { Review Comment: NoOp seems to imply that we do nothing with the errors (like /dev/null). What we want here is something that we can call addErrorCollection on when we know that the passed error collection will be empty, but should never be used in as an actual error handler. In particular, if there happens to be something in those collections, that's a bug. (Should this be asserted? Actually, in this case it's fine to call close() and all. It's a NoneShallPassErrorHandler.) I just can't shake the feeling that this'd all be so much easier if we finish pushing error handling into ParDo itself... -- 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]
