From c6ecbd4e9b1432ef3553234be0a4e1c53875bbe1 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Thu, 7 Dec 2023 14:38:49 -0500 Subject: [PATCH 01/21] first pass of wiring error handling into write files and adding tests --- .../java/org/apache/beam/sdk/io/FileIO.java | 20 ++ .../org/apache/beam/sdk/io/WriteFiles.java | 199 +++++++++++++++--- .../apache/beam/sdk/io/WriteFilesTest.java | 126 +++++++++++ .../errorhandling/ErrorHandlingTestUtils.java | 26 +++ .../apache/beam/sdk/io/kafka/KafkaIOIT.java | 2 +- .../apache/beam/sdk/io/kafka/KafkaIOTest.java | 20 +- 6 files changed, 338 insertions(+), 55 deletions(-) create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java index 76fc1a70b78c..d47e14571c83 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java @@ -61,6 +61,8 @@ import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; @@ -1016,6 +1018,10 @@ public static FileNaming relativeFileNaming( abstract boolean getNoSpilling(); + abstract @Nullable ErrorHandler getBadRecordErrorHandler(); + + abstract @Nullable SerializableFunction getBadRecordMatcher(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -1062,6 +1068,12 @@ abstract Builder setSharding( abstract Builder setNoSpilling(boolean noSpilling); + abstract Builder setBadRecordErrorHandler( + @Nullable ErrorHandler badRecordErrorHandler); + + abstract Builder setBadRecordMatcher( + @Nullable SerializableFunction badRecordMatcher); + abstract Write build(); } @@ -1288,6 +1300,11 @@ public Write withNoSpilling() { return toBuilder().setNoSpilling(true).build(); } + /** See {@link WriteFiles#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. */ + public Write withBadRecordErrorHandler(ErrorHandler errorHandler, SerializableFunction badRecordMatcher) { + return toBuilder().setBadRecordErrorHandler(errorHandler).setBadRecordMatcher(badRecordMatcher).build(); + } + @VisibleForTesting Contextful> resolveFileNamingFn() { if (getDynamic()) { @@ -1391,6 +1408,9 @@ public WriteFilesResult expand(PCollection input) { if (getNoSpilling()) { writeFiles = writeFiles.withNoSpilling(); } + if (getBadRecordErrorHandler() != null) { + writeFiles = writeFiles.withBadRecordErrorHandler(getBadRecordErrorHandler(),getBadRecordMatcher()); + } return input.apply(writeFiles); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java index 91d6082eede4..c33051bdd8dc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io; +import static org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter.BAD_RECORD_TAG; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; @@ -49,6 +50,7 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.GroupIntoBatches; @@ -57,11 +59,16 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Reify; import org.apache.beam.sdk.transforms.Reshuffle; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.DefaultErrorHandler; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -166,6 +173,9 @@ public static WriteFiles()) + .setBadRecordRouter(BadRecordRouter.THROWING_ROUTER) + .setBadRecordMatcher((e) -> true) .build(); } @@ -189,6 +199,12 @@ public static WriteFiles getShardingFunction(); + public abstract ErrorHandler getBadRecordErrorHandler(); + + public abstract BadRecordRouter getBadRecordRouter(); + + public abstract SerializableFunction getBadRecordMatcher(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -215,6 +231,15 @@ abstract Builder setSideInputs( abstract Builder setShardingFunction( @Nullable ShardingFunction shardingFunction); + abstract Builder setBadRecordErrorHandler( + ErrorHandler badRecordErrorHandler); + + abstract Builder setBadRecordRouter( + BadRecordRouter badRecordRouter); + + abstract Builder setBadRecordMatcher( + SerializableFunction badRecordMatcher); + abstract WriteFiles build(); } @@ -330,6 +355,10 @@ public WriteFiles withSkipIfEmpty() { return toBuilder().setSkipIfEmpty(true).build(); } + public WriteFiles withBadRecordErrorHandler(ErrorHandler errorHandler, SerializableFunction badRecordMatcher) { + return toBuilder().setBadRecordErrorHandler(errorHandler).setBadRecordRouter(BadRecordRouter.RECORDING_ROUTER).setBadRecordMatcher(badRecordMatcher).build(); + } + @Override public void validate(PipelineOptions options) { getSink().validate(options); @@ -495,28 +524,33 @@ private WriteUnshardedBundlesToTempFiles( @Override public PCollection> expand(PCollection input) { + TupleTag> writtenRecordsTag = new TupleTag<>("writtenRecords"); + TupleTag, UserT>> unwrittenRecordsTag = new TupleTag<>("unwrittenRecords"); + Coder inputCoder = input.getCoder(); if (getMaxNumWritersPerBundle() < 0) { - return input + PCollectionTuple writeTuple = input .apply( "WritedUnshardedBundles", - ParDo.of(new WriteUnshardedTempFilesFn(null, destinationCoder)) - .withSideInputs(getSideInputs())) - .setCoder(fileResultCoder); + ParDo.of(new WriteUnshardedTempFilesFn(null, destinationCoder, inputCoder)) + .withSideInputs(getSideInputs()) + .withOutputTags(writtenRecordsTag, TupleTagList.of(ImmutableList.of(unwrittenRecordsTag, BAD_RECORD_TAG)))); + addErrorCollection(writeTuple); + return writeTuple.get(writtenRecordsTag).setCoder(fileResultCoder); } - TupleTag> writtenRecordsTag = new TupleTag<>("writtenRecords"); - TupleTag, UserT>> unwrittenRecordsTag = - new TupleTag<>("unwrittenRecords"); + PCollectionTuple writeTuple = input.apply( "WriteUnshardedBundles", - ParDo.of(new WriteUnshardedTempFilesFn(unwrittenRecordsTag, destinationCoder)) + ParDo.of(new WriteUnshardedTempFilesFn(unwrittenRecordsTag, destinationCoder, inputCoder)) .withSideInputs(getSideInputs()) - .withOutputTags(writtenRecordsTag, TupleTagList.of(unwrittenRecordsTag))); + .withOutputTags(writtenRecordsTag, TupleTagList.of(ImmutableList.of(unwrittenRecordsTag, BAD_RECORD_TAG)))); + addErrorCollection(writeTuple); + PCollection> writtenBundleFiles = writeTuple.get(writtenRecordsTag).setCoder(fileResultCoder); // Any "spilled" elements are written using WriteShardedBundles. Assign shard numbers in // finalize to stay consistent with what WriteWindowedBundles does. - PCollection> writtenSpilledFiles = + PCollectionTuple spilledWriteTuple = writeTuple .get(unwrittenRecordsTag) .setCoder(KvCoder.of(ShardedKeyCoder.of(VarIntCoder.of()), input.getCoder())) @@ -529,7 +563,11 @@ public PCollection> expand(PCollection input) { .apply("GroupUnwritten", GroupByKey.create()) .apply( "WriteUnwritten", - ParDo.of(new WriteShardsIntoTempFilesFn()).withSideInputs(getSideInputs())) + ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())).withSideInputs(getSideInputs()).withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); + + addErrorCollection(spilledWriteTuple); + + PCollection> writtenSpilledFiles = spilledWriteTuple.get(writtenRecordsTag) .setCoder(fileResultCoder) .apply( "DropShardNum", @@ -556,6 +594,8 @@ private class WriteUnshardedTempFilesFn extends DoFn, UserT>> unwrittenRecordsTag; private final Coder destinationCoder; + private final Coder inputCoder; + // Initialized in startBundle() private @Nullable Map, Writer> writers; @@ -563,9 +603,11 @@ private class WriteUnshardedTempFilesFn extends DoFn, UserT>> unwrittenRecordsTag, - Coder destinationCoder) { + Coder destinationCoder, + Coder inputCoder) { this.unwrittenRecordsTag = unwrittenRecordsTag; this.destinationCoder = destinationCoder; + this.inputCoder = inputCoder; } @StartBundle @@ -575,7 +617,7 @@ public void startBundle(StartBundleContext c) { } @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) throws Exception { + public void processElement(ProcessContext c, BoundedWindow window, MultiOutputReceiver outputReceiver) throws Exception { getDynamicDestinations().setSideInputAccessorFromProcessContext(c); PaneInfo paneInfo = c.pane(); // If we are doing windowed writes, we need to ensure that we have separate files for @@ -583,7 +625,10 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except // destinations go to different writers. // In the case of unwindowed writes, the window and the pane will always be the same, and // the map will only have a single element. - DestinationT destination = getDynamicDestinations().getDestination(c.element()); + DestinationT destination = getDestinationWithErrorHandling(c.element(), outputReceiver, inputCoder); + if (destination == null) { + return; + } WriterKey key = new WriterKey<>(window, c.pane(), destination); Writer writer = writers.get(key); if (writer == null) { @@ -607,15 +652,18 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except } else { spilledShardNum = (spilledShardNum + 1) % SPILLED_RECORD_SHARDING_FACTOR; } - c.output( - unwrittenRecordsTag, + outputReceiver.get(unwrittenRecordsTag).output( KV.of( ShardedKey.of(hashDestination(destination, destinationCoder), spilledShardNum), c.element())); return; } } - writeOrClose(writer, getDynamicDestinations().formatRecord(c.element())); + OutputT formattedRecord = formatRecordWithErrorHandling(c.element(),outputReceiver,inputCoder); + if (formattedRecord == null){ + return; + } + writeOrClose(writer, formattedRecord); } @FinishBundle @@ -701,6 +749,45 @@ private static int hashDestination( .asInt(); } + // Utility method to get the dynamic destination based on a record. If the operation fails, and is + // output to the bad record router, this returns null + private @Nullable DestinationT getDestinationWithErrorHandling(UserT input, MultiOutputReceiver outputReceiver, Coder inputCoder) throws Exception{ + try { + return getDynamicDestinations().getDestination(input); + } catch (Exception e) { + if (getBadRecordMatcher().apply(e)){ + getBadRecordRouter().route(outputReceiver,input,inputCoder,e,"Unable to get dynamic destination for record"); + return null; + } else { + throw e; + } + } + } + + private void addErrorCollection(PCollectionTuple sourceTuple){ + getBadRecordErrorHandler() + .addErrorCollection( + sourceTuple + .get(BAD_RECORD_TAG) + .setCoder(BadRecord.getCoder(sourceTuple.getPipeline()))); + + } + + // Utility method to format a record based on the dynamic destination. If the operation fails, and + // is output to the bad record router, this returns null + private @Nullable OutputT formatRecordWithErrorHandling(UserT input, MultiOutputReceiver outputReceiver, Coder inputCoder) throws Exception{ + try { + return getDynamicDestinations().formatRecord(input); + } catch (Exception e) { + if (getBadRecordMatcher().apply(e)){ + getBadRecordRouter().route(outputReceiver,input,inputCoder,e,"Unable to format record for Dynamic Destination"); + return null; + } else { + throw e; + } + } + } + private class WriteShardedBundlesToTempFiles extends PTransform, PCollection>> { private final Coder destinationCoder; @@ -728,17 +815,25 @@ public PCollection> expand(PCollection input) { ? new RandomShardingFunction(destinationCoder) : getShardingFunction(); - return input + TupleTag, UserT>> shardedRecords = new TupleTag<>("shardedRecords"); + TupleTag> writtenRecordsTag = new TupleTag<>("writtenRecords"); + + PCollectionTuple shardedFiles = input .apply( "ApplyShardingKey", - ParDo.of(new ApplyShardingFunctionFn(shardingFunction, numShardsView)) - .withSideInputs(shardingSideInputs)) + ParDo.of(new ApplyShardingFunctionFn(shardingFunction, numShardsView, input.getCoder())) + .withSideInputs(shardingSideInputs).withOutputTags(shardedRecords, TupleTagList.of(BAD_RECORD_TAG))); + addErrorCollection(shardedFiles); + + PCollectionTuple writtenFiles = shardedFiles.get(shardedRecords) .setCoder(KvCoder.of(ShardedKeyCoder.of(VarIntCoder.of()), input.getCoder())) .apply("GroupIntoShards", GroupByKey.create()) .apply( "WriteShardsIntoTempFiles", - ParDo.of(new WriteShardsIntoTempFilesFn()).withSideInputs(getSideInputs())) - .setCoder(fileResultCoder); + ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())).withSideInputs(getSideInputs()).withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); + addErrorCollection(writtenFiles); + + return writtenFiles.get(writtenRecordsTag).setCoder(fileResultCoder); } } @@ -763,22 +858,30 @@ public PCollection>> expand(PCollection inp // // TODO(https://github.com/apache/beam/issues/20928): The implementation doesn't currently // work with merging windows. - PCollection, Iterable>> shardedInput = + TupleTag> shardTag = new TupleTag<>("shardTag"); + + PCollectionTuple shardedElements = input .apply( "KeyedByDestinationHash", ParDo.of( new DoFn>() { @ProcessElement - public void processElement(@Element UserT element, ProcessContext context) + public void processElement(@Element UserT element, ProcessContext context, MultiOutputReceiver outputReceiver) throws Exception { getDynamicDestinations().setSideInputAccessorFromProcessContext(context); - DestinationT destination = - getDynamicDestinations().getDestination(context.element()); + DestinationT destination = getDestinationWithErrorHandling(context.element(), outputReceiver, input.getCoder()); + if (destination == null) { + return; + } context.output( KV.of(hashDestination(destination, destinationCoder), element)); } - })) + }).withOutputTags(shardTag, TupleTagList.of(BAD_RECORD_TAG))); + addErrorCollection(shardedElements); + + PCollection, Iterable>> shardedInput = + shardedElements.get(shardTag) .setCoder(KvCoder.of(VarIntCoder.of(), input.getCoder())) .apply( "ShardAndBatch", @@ -791,8 +894,9 @@ public void processElement(@Element UserT element, ProcessContext context) org.apache.beam.sdk.util.ShardedKey.Coder.of(VarIntCoder.of()), IterableCoder.of(input.getCoder()))); + TupleTag> writtenRecordsTag = new TupleTag<>("writtenRecords"); // Write grouped elements to temp files. - PCollection> tempFiles = + PCollectionTuple writtenFiles = shardedInput .apply( "AddDummyShard", @@ -816,7 +920,11 @@ public KV, Iterable> apply( ShardedKeyCoder.of(VarIntCoder.of()), IterableCoder.of(input.getCoder()))) .apply( "WriteShardsIntoTempFiles", - ParDo.of(new WriteShardsIntoTempFilesFn()).withSideInputs(getSideInputs())) + ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())).withSideInputs(getSideInputs()).withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); + + addErrorCollection(writtenFiles); + + PCollection> tempFiles = writtenFiles.get(writtenRecordsTag) .setCoder(fileResultCoder) .apply( "DropShardNum", @@ -903,15 +1011,19 @@ private class ApplyShardingFunctionFn extends DoFn private final ShardingFunction shardingFn; private final @Nullable PCollectionView numShardsView; + private final Coder inputCoder; + ApplyShardingFunctionFn( ShardingFunction shardingFn, - @Nullable PCollectionView numShardsView) { + @Nullable PCollectionView numShardsView, + Coder inputCoder) { this.numShardsView = numShardsView; this.shardingFn = shardingFn; + this.inputCoder = inputCoder; } @ProcessElement - public void processElement(ProcessContext context) throws Exception { + public void processElement(ProcessContext context, MultiOutputReceiver outputReceiver) throws Exception { getDynamicDestinations().setSideInputAccessorFromProcessContext(context); final int shardCount; if (numShardsView != null) { @@ -927,7 +1039,10 @@ public void processElement(ProcessContext context) throws Exception { + " Got %s", shardCount); - DestinationT destination = getDynamicDestinations().getDestination(context.element()); + DestinationT destination = getDestinationWithErrorHandling(context.element(), outputReceiver, inputCoder); + if (destination == null) { + return; + } ShardedKey shardKey = shardingFn.assignShardKey(destination, context.element(), shardCount); context.output(KV.of(shardKey, context.element())); @@ -936,6 +1051,12 @@ public void processElement(ProcessContext context) throws Exception { private class WriteShardsIntoTempFilesFn extends DoFn, Iterable>, FileResult> { + + private final Coder inputCoder; + + public WriteShardsIntoTempFilesFn(Coder inputCoder) { + this.inputCoder = inputCoder; + } private transient List> closeFutures = new ArrayList<>(); private transient List>> deferredOutput = new ArrayList<>(); @@ -949,14 +1070,17 @@ private void readObject(java.io.ObjectInputStream in) } @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) throws Exception { + public void processElement(ProcessContext c, BoundedWindow window, MultiOutputReceiver outputReceiver) throws Exception { getDynamicDestinations().setSideInputAccessorFromProcessContext(c); // Since we key by a 32-bit hash of the destination, there might be multiple destinations // in this iterable. The number of destinations is generally very small (1000s or less), so // there will rarely be hash collisions. Map> writers = Maps.newHashMap(); for (UserT input : c.element().getValue()) { - DestinationT destination = getDynamicDestinations().getDestination(input); + DestinationT destination = getDestinationWithErrorHandling(input, outputReceiver, inputCoder); + if (destination == null) { + continue; + } Writer writer = writers.get(destination); if (writer == null) { String uuid = UUID.randomUUID().toString(); @@ -971,7 +1095,12 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except writer.open(uuid); writers.put(destination, writer); } - writeOrClose(writer, getDynamicDestinations().formatRecord(input)); + + OutputT formattedRecord = formatRecordWithErrorHandling(input,outputReceiver,inputCoder); + if (formattedRecord == null){ + return; + } + writeOrClose(writer, formattedRecord); } // Ensure that we clean-up any prior writers that were being closed as part of this bundle diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java index 39cb612f2d89..a46ad52550ef 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java @@ -61,6 +61,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactoryTest.TestPipelineOptions; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.testing.UsesTestStream; @@ -78,6 +79,10 @@ import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandlingTestUtils; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandlingTestUtils.ErrorSinkTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; @@ -634,6 +639,127 @@ private void testDynamicDestinationsHelper(boolean bounded, boolean emptyShards) } } + + // Test FailingDynamicDestinations class. Expects user values to be string-encoded integers. + // Throws exceptions when trying to format records or get destinations based on the mod + // of the element + static class FailingTestDestinations extends DynamicDestinations { + private ResourceId baseOutputDirectory; + + FailingTestDestinations(ResourceId baseOutputDirectory) { + this.baseOutputDirectory = baseOutputDirectory; + } + + @Override + public String formatRecord(String record) { + int value = Integer.valueOf(record); + if (value % 2 == 0) { + throw new RuntimeException("Failed To Format Record"); + } + return "record_" + record; + } + + @Override + public Integer getDestination(String element) { + int value = Integer.valueOf(element); + if (value % 3 == 0) { + throw new RuntimeException("Failed To Get Destination"); + } + return value % 5; + } + + @Override + public Integer getDefaultDestination() { + return 0; + } + + @Override + public FilenamePolicy getFilenamePolicy(Integer destination) { + return new PerWindowFiles( + baseOutputDirectory.resolve("file_" + destination, StandardResolveOptions.RESOLVE_FILE), + "simple"); + } + } + + @Test + @Category(NeedsRunner.class) + public void testFailingDynamicDestinationsBounded() throws Exception { + testFailingDynamicDestinationsHelper(true,false); + } + + @Test + @Category({NeedsRunner.class, UsesUnboundedPCollections.class}) + public void testFailingDynamicDestinationsUnbounded() throws Exception { + testFailingDynamicDestinationsHelper(false,false); + } + + @Test + @Category({NeedsRunner.class, UsesUnboundedPCollections.class}) + public void testFailingDynamicDestinationsAutosharding() throws Exception { + testFailingDynamicDestinationsHelper(false,true); + } + + private void testFailingDynamicDestinationsHelper(boolean bounded, boolean autosharding) + throws IOException { + FailingTestDestinations dynamicDestinations = new FailingTestDestinations(getBaseOutputDirectory()); + SimpleSink sink = + new SimpleSink<>(getBaseOutputDirectory(), dynamicDestinations, Compression.UNCOMPRESSED); + + // Flag to validate that the pipeline options are passed to the Sink. + WriteOptions options = TestPipeline.testingPipelineOptions().as(WriteOptions.class); + options.setTestFlag("test_value"); + Pipeline p = TestPipeline.create(options); + + final int numInputs = 100; + long expectedFailures = 0; + List inputs = Lists.newArrayList(); + for (int i = 0; i < numInputs; ++i) { + inputs.add(Integer.toString(i)); + if(i % 2 != 0 && i % 3 != 0){ + expectedFailures++; + } + } + // Prepare timestamps for the elements. + List timestamps = new ArrayList<>(); + for (long i = 0; i < inputs.size(); i++) { + timestamps.add(i + 1); + } + + BadRecordErrorHandler> errorHandler = p.registerBadRecordErrorHandler(new ErrorSinkTransform()); + int numShards = autosharding ? 0 : 2; + WriteFiles writeFiles = WriteFiles.to(sink).withNumShards(numShards).withBadRecordErrorHandler(errorHandler, (e) -> true); + errorHandler.close(); + + PAssert.thatSingleton(errorHandler.getOutput()).isEqualTo(expectedFailures); + + PCollection input = p.apply(Create.timestamped(inputs, timestamps)); + WriteFilesResult res; + if (!bounded) { + input.setIsBoundedInternal(IsBounded.UNBOUNDED); + input = input.apply(Window.into(FixedWindows.of(Duration.standardDays(1)))); + res = input.apply(writeFiles.withWindowedWrites()); + } else { + res = input.apply(writeFiles); + } + res.getPerDestinationOutputFilenames().apply(new VerifyFilesExist<>()); + p.run(); + + for (int i = 0; i < 5; ++i) { + ResourceId base = + getBaseOutputDirectory().resolve("file_" + i, StandardResolveOptions.RESOLVE_FILE); + List expected = Lists.newArrayList(); + for (int j = i; j < numInputs; j += 5) { + expected.add("record_" + j); + } + checkFileContents( + base.toString(), + expected, + Optional.of(numShards), + bounded /* expectRemovedTempDirectory */); + } + + } + @Test public void testShardedDisplayData() { DynamicDestinations dynamicDestinations = diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java new file mode 100644 index 000000000000..ce01ec881d6f --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java @@ -0,0 +1,26 @@ +package org.apache.beam.sdk.transforms.errorhandling; + +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.windowing.CalendarWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.PCollection; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; + +public class ErrorHandlingTestUtils { + public static class ErrorSinkTransform + extends PTransform, PCollection> { + + @Override + public @UnknownKeyFor @NonNull @Initialized PCollection expand( + PCollection input) { + return input + .apply("Window", Window.into(CalendarWindows.years(1))) + .apply("Combine", Combine.globally(Count.combineFn()).withoutDefaults()); + } + } + +} diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index 5b976687f2c1..ab6ac52e318d 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -44,7 +44,6 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.common.IOITHelper; import org.apache.beam.sdk.io.common.IOTestPipelineOptions; -import org.apache.beam.sdk.io.kafka.KafkaIOTest.ErrorSinkTransform; import org.apache.beam.sdk.io.kafka.KafkaIOTest.FailingLongSerializer; import org.apache.beam.sdk.io.kafka.ReadFromKafkaDoFnTest.FailingDeserializer; import org.apache.beam.sdk.io.synthetic.SyntheticBoundedSource; @@ -77,6 +76,7 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandlingTestUtils.ErrorSinkTransform; import org.apache.beam.sdk.transforms.windowing.CalendarWindows; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index b0df82bcdc19..9b15b86051f5 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -88,7 +88,6 @@ import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Distinct; @@ -97,15 +96,13 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.Max; import org.apache.beam.sdk.transforms.Min; -import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.errorhandling.BadRecord; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandlingTestUtils.ErrorSinkTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.CalendarWindows; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; @@ -145,10 +142,7 @@ import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Utils; -import org.checkerframework.checker.initialization.qual.Initialized; -import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; -import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.hamcrest.collection.IsIterableContainingInAnyOrder; import org.hamcrest.collection.IsIterableWithSize; import org.joda.time.Duration; @@ -1472,18 +1466,6 @@ public void testSinkWithSerializationErrors() throws Exception { } } - public static class ErrorSinkTransform - extends PTransform, PCollection> { - - @Override - public @UnknownKeyFor @NonNull @Initialized PCollection expand( - PCollection input) { - return input - .apply("Window", Window.into(CalendarWindows.years(1))) - .apply("Combine", Combine.globally(Count.combineFn()).withoutDefaults()); - } - } - @Test public void testValuesSink() throws Exception { // similar to testSink(), but use values()' interface. From ab68b1ece19e6459c07c537a5a81921670fc6570 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Fri, 8 Dec 2023 12:19:25 -0500 Subject: [PATCH 02/21] fix error handling to solve constant filenaming policy returning a null destination --- .../java/org/apache/beam/sdk/io/FileIO.java | 18 +- .../org/apache/beam/sdk/io/WriteFiles.java | 203 ++++++++++++------ .../apache/beam/sdk/io/WriteFilesTest.java | 23 +- .../errorhandling/ErrorHandlingTestUtils.java | 18 +- 4 files changed, 180 insertions(+), 82 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java index d47e14571c83..c48e13a7ade4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java @@ -1018,7 +1018,7 @@ public static FileNaming relativeFileNaming( abstract boolean getNoSpilling(); - abstract @Nullable ErrorHandler getBadRecordErrorHandler(); + abstract @Nullable ErrorHandler getBadRecordErrorHandler(); abstract @Nullable SerializableFunction getBadRecordMatcher(); @@ -1069,10 +1069,10 @@ abstract Builder setSharding( abstract Builder setNoSpilling(boolean noSpilling); abstract Builder setBadRecordErrorHandler( - @Nullable ErrorHandler badRecordErrorHandler); + @Nullable ErrorHandler badRecordErrorHandler); abstract Builder setBadRecordMatcher( - @Nullable SerializableFunction badRecordMatcher); + @Nullable SerializableFunction badRecordMatcher); abstract Write build(); } @@ -1301,8 +1301,13 @@ public Write withNoSpilling() { } /** See {@link WriteFiles#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. */ - public Write withBadRecordErrorHandler(ErrorHandler errorHandler, SerializableFunction badRecordMatcher) { - return toBuilder().setBadRecordErrorHandler(errorHandler).setBadRecordMatcher(badRecordMatcher).build(); + public Write withBadRecordErrorHandler( + ErrorHandler errorHandler, + SerializableFunction badRecordMatcher) { + return toBuilder() + .setBadRecordErrorHandler(errorHandler) + .setBadRecordMatcher(badRecordMatcher) + .build(); } @VisibleForTesting @@ -1409,7 +1414,8 @@ public WriteFilesResult expand(PCollection input) { writeFiles = writeFiles.withNoSpilling(); } if (getBadRecordErrorHandler() != null) { - writeFiles = writeFiles.withBadRecordErrorHandler(getBadRecordErrorHandler(),getBadRecordMatcher()); + writeFiles = + writeFiles.withBadRecordErrorHandler(getBadRecordErrorHandler(), getBadRecordMatcher()); } return input.apply(writeFiles); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java index c33051bdd8dc..6a133f9b8855 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java @@ -199,7 +199,7 @@ public static WriteFiles getShardingFunction(); - public abstract ErrorHandler getBadRecordErrorHandler(); + public abstract ErrorHandler getBadRecordErrorHandler(); public abstract BadRecordRouter getBadRecordRouter(); @@ -232,13 +232,13 @@ abstract Builder setShardingFunction( @Nullable ShardingFunction shardingFunction); abstract Builder setBadRecordErrorHandler( - ErrorHandler badRecordErrorHandler); + ErrorHandler badRecordErrorHandler); abstract Builder setBadRecordRouter( BadRecordRouter badRecordRouter); abstract Builder setBadRecordMatcher( - SerializableFunction badRecordMatcher); + SerializableFunction badRecordMatcher); abstract WriteFiles build(); } @@ -355,8 +355,14 @@ public WriteFiles withSkipIfEmpty() { return toBuilder().setSkipIfEmpty(true).build(); } - public WriteFiles withBadRecordErrorHandler(ErrorHandler errorHandler, SerializableFunction badRecordMatcher) { - return toBuilder().setBadRecordErrorHandler(errorHandler).setBadRecordRouter(BadRecordRouter.RECORDING_ROUTER).setBadRecordMatcher(badRecordMatcher).build(); + public WriteFiles withBadRecordErrorHandler( + ErrorHandler errorHandler, + SerializableFunction badRecordMatcher) { + return toBuilder() + .setBadRecordErrorHandler(errorHandler) + .setBadRecordRouter(BadRecordRouter.RECORDING_ROUTER) + .setBadRecordMatcher(badRecordMatcher) + .build(); } @Override @@ -525,15 +531,18 @@ private WriteUnshardedBundlesToTempFiles( @Override public PCollection> expand(PCollection input) { TupleTag> writtenRecordsTag = new TupleTag<>("writtenRecords"); - TupleTag, UserT>> unwrittenRecordsTag = new TupleTag<>("unwrittenRecords"); + TupleTag, UserT>> unwrittenRecordsTag = + new TupleTag<>("unwrittenRecords"); Coder inputCoder = input.getCoder(); if (getMaxNumWritersPerBundle() < 0) { - PCollectionTuple writeTuple = input - .apply( + PCollectionTuple writeTuple = + input.apply( "WritedUnshardedBundles", ParDo.of(new WriteUnshardedTempFilesFn(null, destinationCoder, inputCoder)) .withSideInputs(getSideInputs()) - .withOutputTags(writtenRecordsTag, TupleTagList.of(ImmutableList.of(unwrittenRecordsTag, BAD_RECORD_TAG)))); + .withOutputTags( + writtenRecordsTag, + TupleTagList.of(ImmutableList.of(unwrittenRecordsTag, BAD_RECORD_TAG)))); addErrorCollection(writeTuple); return writeTuple.get(writtenRecordsTag).setCoder(fileResultCoder); } @@ -541,9 +550,13 @@ public PCollection> expand(PCollection input) { PCollectionTuple writeTuple = input.apply( "WriteUnshardedBundles", - ParDo.of(new WriteUnshardedTempFilesFn(unwrittenRecordsTag, destinationCoder, inputCoder)) + ParDo.of( + new WriteUnshardedTempFilesFn( + unwrittenRecordsTag, destinationCoder, inputCoder)) .withSideInputs(getSideInputs()) - .withOutputTags(writtenRecordsTag, TupleTagList.of(ImmutableList.of(unwrittenRecordsTag, BAD_RECORD_TAG)))); + .withOutputTags( + writtenRecordsTag, + TupleTagList.of(ImmutableList.of(unwrittenRecordsTag, BAD_RECORD_TAG)))); addErrorCollection(writeTuple); PCollection> writtenBundleFiles = @@ -563,11 +576,15 @@ public PCollection> expand(PCollection input) { .apply("GroupUnwritten", GroupByKey.create()) .apply( "WriteUnwritten", - ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())).withSideInputs(getSideInputs()).withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); + ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())) + .withSideInputs(getSideInputs()) + .withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); addErrorCollection(spilledWriteTuple); - PCollection> writtenSpilledFiles = spilledWriteTuple.get(writtenRecordsTag) + PCollection> writtenSpilledFiles = + spilledWriteTuple + .get(writtenRecordsTag) .setCoder(fileResultCoder) .apply( "DropShardNum", @@ -617,7 +634,9 @@ public void startBundle(StartBundleContext c) { } @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window, MultiOutputReceiver outputReceiver) throws Exception { + public void processElement( + ProcessContext c, BoundedWindow window, MultiOutputReceiver outputReceiver) + throws Exception { getDynamicDestinations().setSideInputAccessorFromProcessContext(c); PaneInfo paneInfo = c.pane(); // If we are doing windowed writes, we need to ensure that we have separate files for @@ -625,10 +644,12 @@ public void processElement(ProcessContext c, BoundedWindow window, MultiOutputRe // destinations go to different writers. // In the case of unwindowed writes, the window and the pane will always be the same, and // the map will only have a single element. - DestinationT destination = getDestinationWithErrorHandling(c.element(), outputReceiver, inputCoder); - if (destination == null) { + MaybeDestination maybeDestination = + getDestinationWithErrorHandling(c.element(), outputReceiver, inputCoder); + if (!maybeDestination.isValid) { return; } + DestinationT destination = maybeDestination.destination; WriterKey key = new WriterKey<>(window, c.pane(), destination); Writer writer = writers.get(key); if (writer == null) { @@ -652,15 +673,19 @@ public void processElement(ProcessContext c, BoundedWindow window, MultiOutputRe } else { spilledShardNum = (spilledShardNum + 1) % SPILLED_RECORD_SHARDING_FACTOR; } - outputReceiver.get(unwrittenRecordsTag).output( - KV.of( - ShardedKey.of(hashDestination(destination, destinationCoder), spilledShardNum), - c.element())); + outputReceiver + .get(unwrittenRecordsTag) + .output( + KV.of( + ShardedKey.of( + hashDestination(destination, destinationCoder), spilledShardNum), + c.element())); return; } } - OutputT formattedRecord = formatRecordWithErrorHandling(c.element(),outputReceiver,inputCoder); - if (formattedRecord == null){ + OutputT formattedRecord = + formatRecordWithErrorHandling(c.element(), outputReceiver, inputCoder); + if (formattedRecord == null) { return; } writeOrClose(writer, formattedRecord); @@ -749,38 +774,62 @@ private static int hashDestination( .asInt(); } + private static class MaybeDestination { + final DestinationT destination; + final boolean isValid; + + MaybeDestination(DestinationT destination, boolean isValid) { + this.destination = destination; + this.isValid = true; + } + } // Utility method to get the dynamic destination based on a record. If the operation fails, and is - // output to the bad record router, this returns null - private @Nullable DestinationT getDestinationWithErrorHandling(UserT input, MultiOutputReceiver outputReceiver, Coder inputCoder) throws Exception{ + // output to the bad record router, this returns null. Returns a MaybeDestination because some + // implementations of dynamic destinations return null, despite this being prohibited by the + // interface + private MaybeDestination getDestinationWithErrorHandling( + UserT input, MultiOutputReceiver outputReceiver, Coder inputCoder) throws Exception { try { - return getDynamicDestinations().getDestination(input); + return new MaybeDestination<>(getDynamicDestinations().getDestination(input), true); } catch (Exception e) { - if (getBadRecordMatcher().apply(e)){ - getBadRecordRouter().route(outputReceiver,input,inputCoder,e,"Unable to get dynamic destination for record"); - return null; + if (getBadRecordMatcher().apply(e)) { + getBadRecordRouter() + .route( + outputReceiver, + input, + inputCoder, + e, + "Unable to get dynamic destination for record"); + return new MaybeDestination<>(null, false); } else { throw e; } } } - private void addErrorCollection(PCollectionTuple sourceTuple){ + private void addErrorCollection(PCollectionTuple sourceTuple) { getBadRecordErrorHandler() .addErrorCollection( sourceTuple .get(BAD_RECORD_TAG) .setCoder(BadRecord.getCoder(sourceTuple.getPipeline()))); - } // Utility method to format a record based on the dynamic destination. If the operation fails, and // is output to the bad record router, this returns null - private @Nullable OutputT formatRecordWithErrorHandling(UserT input, MultiOutputReceiver outputReceiver, Coder inputCoder) throws Exception{ + private @Nullable OutputT formatRecordWithErrorHandling( + UserT input, MultiOutputReceiver outputReceiver, Coder inputCoder) throws Exception { try { return getDynamicDestinations().formatRecord(input); } catch (Exception e) { - if (getBadRecordMatcher().apply(e)){ - getBadRecordRouter().route(outputReceiver,input,inputCoder,e,"Unable to format record for Dynamic Destination"); + if (getBadRecordMatcher().apply(e)) { + getBadRecordRouter() + .route( + outputReceiver, + input, + inputCoder, + e, + "Unable to format record for Dynamic Destination"); return null; } else { throw e; @@ -818,19 +867,26 @@ public PCollection> expand(PCollection input) { TupleTag, UserT>> shardedRecords = new TupleTag<>("shardedRecords"); TupleTag> writtenRecordsTag = new TupleTag<>("writtenRecords"); - PCollectionTuple shardedFiles = input - .apply( + PCollectionTuple shardedFiles = + input.apply( "ApplyShardingKey", - ParDo.of(new ApplyShardingFunctionFn(shardingFunction, numShardsView, input.getCoder())) - .withSideInputs(shardingSideInputs).withOutputTags(shardedRecords, TupleTagList.of(BAD_RECORD_TAG))); + ParDo.of( + new ApplyShardingFunctionFn( + shardingFunction, numShardsView, input.getCoder())) + .withSideInputs(shardingSideInputs) + .withOutputTags(shardedRecords, TupleTagList.of(BAD_RECORD_TAG))); addErrorCollection(shardedFiles); - PCollectionTuple writtenFiles = shardedFiles.get(shardedRecords) - .setCoder(KvCoder.of(ShardedKeyCoder.of(VarIntCoder.of()), input.getCoder())) - .apply("GroupIntoShards", GroupByKey.create()) - .apply( - "WriteShardsIntoTempFiles", - ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())).withSideInputs(getSideInputs()).withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); + PCollectionTuple writtenFiles = + shardedFiles + .get(shardedRecords) + .setCoder(KvCoder.of(ShardedKeyCoder.of(VarIntCoder.of()), input.getCoder())) + .apply("GroupIntoShards", GroupByKey.create()) + .apply( + "WriteShardsIntoTempFiles", + ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())) + .withSideInputs(getSideInputs()) + .withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); addErrorCollection(writtenFiles); return writtenFiles.get(writtenRecordsTag).setCoder(fileResultCoder); @@ -858,30 +914,37 @@ public PCollection>> expand(PCollection inp // // TODO(https://github.com/apache/beam/issues/20928): The implementation doesn't currently // work with merging windows. - TupleTag> shardTag = new TupleTag<>("shardTag"); + TupleTag> shardTag = new TupleTag<>("shardTag"); PCollectionTuple shardedElements = - input - .apply( - "KeyedByDestinationHash", - ParDo.of( + input.apply( + "KeyedByDestinationHash", + ParDo.of( new DoFn>() { @ProcessElement - public void processElement(@Element UserT element, ProcessContext context, MultiOutputReceiver outputReceiver) + public void processElement( + @Element UserT element, + ProcessContext context, + MultiOutputReceiver outputReceiver) throws Exception { getDynamicDestinations().setSideInputAccessorFromProcessContext(context); - DestinationT destination = getDestinationWithErrorHandling(context.element(), outputReceiver, input.getCoder()); - if (destination == null) { + MaybeDestination maybeDestination = + getDestinationWithErrorHandling( + context.element(), outputReceiver, input.getCoder()); + if (!maybeDestination.isValid) { return; } + DestinationT destination = maybeDestination.destination; context.output( KV.of(hashDestination(destination, destinationCoder), element)); } - }).withOutputTags(shardTag, TupleTagList.of(BAD_RECORD_TAG))); + }) + .withOutputTags(shardTag, TupleTagList.of(BAD_RECORD_TAG))); addErrorCollection(shardedElements); PCollection, Iterable>> shardedInput = - shardedElements.get(shardTag) + shardedElements + .get(shardTag) .setCoder(KvCoder.of(VarIntCoder.of(), input.getCoder())) .apply( "ShardAndBatch", @@ -920,11 +983,15 @@ public KV, Iterable> apply( ShardedKeyCoder.of(VarIntCoder.of()), IterableCoder.of(input.getCoder()))) .apply( "WriteShardsIntoTempFiles", - ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())).withSideInputs(getSideInputs()).withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); + ParDo.of(new WriteShardsIntoTempFilesFn(input.getCoder())) + .withSideInputs(getSideInputs()) + .withOutputTags(writtenRecordsTag, TupleTagList.of(BAD_RECORD_TAG))); addErrorCollection(writtenFiles); - PCollection> tempFiles = writtenFiles.get(writtenRecordsTag) + PCollection> tempFiles = + writtenFiles + .get(writtenRecordsTag) .setCoder(fileResultCoder) .apply( "DropShardNum", @@ -1023,7 +1090,8 @@ private class ApplyShardingFunctionFn extends DoFn } @ProcessElement - public void processElement(ProcessContext context, MultiOutputReceiver outputReceiver) throws Exception { + public void processElement(ProcessContext context, MultiOutputReceiver outputReceiver) + throws Exception { getDynamicDestinations().setSideInputAccessorFromProcessContext(context); final int shardCount; if (numShardsView != null) { @@ -1039,10 +1107,12 @@ public void processElement(ProcessContext context, MultiOutputReceiver outputRec + " Got %s", shardCount); - DestinationT destination = getDestinationWithErrorHandling(context.element(), outputReceiver, inputCoder); - if (destination == null) { + MaybeDestination maybeDestination = + getDestinationWithErrorHandling(context.element(), outputReceiver, inputCoder); + if (!maybeDestination.isValid) { return; } + DestinationT destination = maybeDestination.destination; ShardedKey shardKey = shardingFn.assignShardKey(destination, context.element(), shardCount); context.output(KV.of(shardKey, context.element())); @@ -1057,6 +1127,7 @@ private class WriteShardsIntoTempFilesFn public WriteShardsIntoTempFilesFn(Coder inputCoder) { this.inputCoder = inputCoder; } + private transient List> closeFutures = new ArrayList<>(); private transient List>> deferredOutput = new ArrayList<>(); @@ -1070,17 +1141,21 @@ private void readObject(java.io.ObjectInputStream in) } @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window, MultiOutputReceiver outputReceiver) throws Exception { + public void processElement( + ProcessContext c, BoundedWindow window, MultiOutputReceiver outputReceiver) + throws Exception { getDynamicDestinations().setSideInputAccessorFromProcessContext(c); // Since we key by a 32-bit hash of the destination, there might be multiple destinations // in this iterable. The number of destinations is generally very small (1000s or less), so // there will rarely be hash collisions. Map> writers = Maps.newHashMap(); for (UserT input : c.element().getValue()) { - DestinationT destination = getDestinationWithErrorHandling(input, outputReceiver, inputCoder); - if (destination == null) { - continue; + MaybeDestination maybeDestination = + getDestinationWithErrorHandling(input, outputReceiver, inputCoder); + if (!maybeDestination.isValid) { + return; } + DestinationT destination = maybeDestination.destination; Writer writer = writers.get(destination); if (writer == null) { String uuid = UUID.randomUUID().toString(); @@ -1096,8 +1171,8 @@ public void processElement(ProcessContext c, BoundedWindow window, MultiOutputRe writers.put(destination, writer); } - OutputT formattedRecord = formatRecordWithErrorHandling(input,outputReceiver,inputCoder); - if (formattedRecord == null){ + OutputT formattedRecord = formatRecordWithErrorHandling(input, outputReceiver, inputCoder); + if (formattedRecord == null) { return; } writeOrClose(writer, formattedRecord); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java index a46ad52550ef..ff1330cb98b4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java @@ -79,9 +79,7 @@ import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.errorhandling.BadRecord; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.BadRecordErrorHandler; -import org.apache.beam.sdk.transforms.errorhandling.ErrorHandlingTestUtils; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandlingTestUtils.ErrorSinkTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; @@ -639,7 +637,6 @@ private void testDynamicDestinationsHelper(boolean bounded, boolean emptyShards) } } - // Test FailingDynamicDestinations class. Expects user values to be string-encoded integers. // Throws exceptions when trying to format records or get destinations based on the mod // of the element @@ -684,24 +681,25 @@ public FilenamePolicy getFilenamePolicy(Integer destination) { @Test @Category(NeedsRunner.class) public void testFailingDynamicDestinationsBounded() throws Exception { - testFailingDynamicDestinationsHelper(true,false); + testFailingDynamicDestinationsHelper(true, false); } @Test @Category({NeedsRunner.class, UsesUnboundedPCollections.class}) public void testFailingDynamicDestinationsUnbounded() throws Exception { - testFailingDynamicDestinationsHelper(false,false); + testFailingDynamicDestinationsHelper(false, false); } @Test @Category({NeedsRunner.class, UsesUnboundedPCollections.class}) public void testFailingDynamicDestinationsAutosharding() throws Exception { - testFailingDynamicDestinationsHelper(false,true); + testFailingDynamicDestinationsHelper(false, true); } private void testFailingDynamicDestinationsHelper(boolean bounded, boolean autosharding) throws IOException { - FailingTestDestinations dynamicDestinations = new FailingTestDestinations(getBaseOutputDirectory()); + FailingTestDestinations dynamicDestinations = + new FailingTestDestinations(getBaseOutputDirectory()); SimpleSink sink = new SimpleSink<>(getBaseOutputDirectory(), dynamicDestinations, Compression.UNCOMPRESSED); @@ -715,7 +713,7 @@ private void testFailingDynamicDestinationsHelper(boolean bounded, boolean autos List inputs = Lists.newArrayList(); for (int i = 0; i < numInputs; ++i) { inputs.add(Integer.toString(i)); - if(i % 2 != 0 && i % 3 != 0){ + if (i % 2 != 0 && i % 3 != 0) { expectedFailures++; } } @@ -725,9 +723,13 @@ private void testFailingDynamicDestinationsHelper(boolean bounded, boolean autos timestamps.add(i + 1); } - BadRecordErrorHandler> errorHandler = p.registerBadRecordErrorHandler(new ErrorSinkTransform()); + BadRecordErrorHandler> errorHandler = + p.registerBadRecordErrorHandler(new ErrorSinkTransform()); int numShards = autosharding ? 0 : 2; - WriteFiles writeFiles = WriteFiles.to(sink).withNumShards(numShards).withBadRecordErrorHandler(errorHandler, (e) -> true); + WriteFiles writeFiles = + WriteFiles.to(sink) + .withNumShards(numShards) + .withBadRecordErrorHandler(errorHandler, (e) -> true); errorHandler.close(); PAssert.thatSingleton(errorHandler.getOutput()).isEqualTo(expectedFailures); @@ -757,7 +759,6 @@ private void testFailingDynamicDestinationsHelper(boolean bounded, boolean autos Optional.of(numShards), bounded /* expectRemovedTempDirectory */); } - } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java index ce01ec881d6f..1de2891ccb86 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java @@ -1,3 +1,20 @@ +/* + * 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.transforms.Combine; @@ -22,5 +39,4 @@ public static class ErrorSinkTransform .apply("Combine", Combine.globally(Count.combineFn()).withoutDefaults()); } } - } From 3d76ccddb34721d2df22db2b8e36a0b5ce7dd5d9 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Mon, 11 Dec 2023 12:11:19 -0500 Subject: [PATCH 03/21] fix tests, add a safety check to the error handler --- .../org/apache/beam/sdk/io/WriteFiles.java | 83 +++++++++++-------- .../errorhandling/ErrorHandler.java | 3 + .../apache/beam/sdk/io/WriteFilesTest.java | 22 +++-- .../errorhandling/ErrorHandlingTestUtils.java | 14 +++- 4 files changed, 75 insertions(+), 47 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java index 6a133f9b8855..7c0ce22638b2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java @@ -544,6 +544,8 @@ public PCollection> expand(PCollection input) { writtenRecordsTag, TupleTagList.of(ImmutableList.of(unwrittenRecordsTag, BAD_RECORD_TAG)))); addErrorCollection(writeTuple); + writeTuple.get(unwrittenRecordsTag) + .setCoder(KvCoder.of(ShardedKeyCoder.of(VarIntCoder.of()), input.getCoder())); return writeTuple.get(writtenRecordsTag).setCoder(fileResultCoder); } @@ -780,13 +782,12 @@ private static class MaybeDestination { MaybeDestination(DestinationT destination, boolean isValid) { this.destination = destination; - this.isValid = true; + this.isValid = isValid; } } - // Utility method to get the dynamic destination based on a record. If the operation fails, and is - // output to the bad record router, this returns null. Returns a MaybeDestination because some - // implementations of dynamic destinations return null, despite this being prohibited by the - // interface + // Utility method to get the dynamic destination based on a record. Returns a MaybeDestination + // because some implementations of dynamic destinations return null, despite this being prohibited + // by the interface private MaybeDestination getDestinationWithErrorHandling( UserT input, MultiOutputReceiver outputReceiver, Coder inputCoder) throws Exception { try { @@ -807,14 +808,6 @@ private MaybeDestination getDestinationWithErrorHandling( } } - private void addErrorCollection(PCollectionTuple sourceTuple) { - getBadRecordErrorHandler() - .addErrorCollection( - sourceTuple - .get(BAD_RECORD_TAG) - .setCoder(BadRecord.getCoder(sourceTuple.getPipeline()))); - } - // Utility method to format a record based on the dynamic destination. If the operation fails, and // is output to the bad record router, this returns null private @Nullable OutputT formatRecordWithErrorHandling( @@ -837,6 +830,14 @@ private void addErrorCollection(PCollectionTuple sourceTuple) { } } + private void addErrorCollection(PCollectionTuple sourceTuple) { + getBadRecordErrorHandler() + .addErrorCollection( + sourceTuple + .get(BAD_RECORD_TAG) + .setCoder(BadRecord.getCoder(sourceTuple.getPipeline()))); + } + private class WriteShardedBundlesToTempFiles extends PTransform, PCollection>> { private final Coder destinationCoder; @@ -919,26 +920,7 @@ public PCollection>> expand(PCollection inp PCollectionTuple shardedElements = input.apply( "KeyedByDestinationHash", - ParDo.of( - new DoFn>() { - @ProcessElement - public void processElement( - @Element UserT element, - ProcessContext context, - MultiOutputReceiver outputReceiver) - throws Exception { - getDynamicDestinations().setSideInputAccessorFromProcessContext(context); - MaybeDestination maybeDestination = - getDestinationWithErrorHandling( - context.element(), outputReceiver, input.getCoder()); - if (!maybeDestination.isValid) { - return; - } - DestinationT destination = maybeDestination.destination; - context.output( - KV.of(hashDestination(destination, destinationCoder), element)); - } - }) + ParDo.of(new KeyByDestinationHash(input.getCoder(), destinationCoder)) .withOutputTags(shardTag, TupleTagList.of(BAD_RECORD_TAG))); addErrorCollection(shardedElements); @@ -1040,6 +1022,37 @@ public void processElement( } } + private class KeyByDestinationHash extends DoFn> { + + private final Coder inputCoder; + + private final Coder destinationCoder; + + public KeyByDestinationHash(Coder inputCoder, Coder destinationCoder) { + this.inputCoder = inputCoder; + this.destinationCoder = destinationCoder; + } + + @ProcessElement + public void processElement( + @Element UserT element, + ProcessContext context, + MultiOutputReceiver outputReceiver) + throws Exception { + getDynamicDestinations().setSideInputAccessorFromProcessContext(context); + MaybeDestination maybeDestination = + getDestinationWithErrorHandling( + context.element(), outputReceiver, inputCoder); + if (!maybeDestination.isValid) { + return; + } + DestinationT destination = maybeDestination.destination; + context.output( + KV.of(hashDestination(destination, destinationCoder), element)); + } + } + + private class RandomShardingFunction implements ShardingFunction { private final Coder destinationCoder; @@ -1153,7 +1166,7 @@ public void processElement( MaybeDestination maybeDestination = getDestinationWithErrorHandling(input, outputReceiver, inputCoder); if (!maybeDestination.isValid) { - return; + continue; } DestinationT destination = maybeDestination.destination; Writer writer = writers.get(destination); @@ -1173,7 +1186,7 @@ public void processElement( OutputT formattedRecord = formatRecordWithErrorHandling(input, outputReceiver, inputCoder); if (formattedRecord == null) { - return; + continue; } writeOrClose(writer, formattedRecord); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java index e02965b72022..f93a409993b9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java @@ -119,6 +119,9 @@ private void readObject(ObjectInputStream aInputStream) @Override public void addErrorCollection(PCollection errorCollection) { + if (isClosed()) { + throw new IllegalStateException("Error collections cannot be added after Error Handler is closed"); + } errorCollections.add(errorCollection); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java index ff1330cb98b4..1b44e799a91a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java @@ -650,7 +650,8 @@ static class FailingTestDestinations extends DynamicDestinations inputs = Lists.newArrayList(); for (int i = 0; i < numInputs; ++i) { inputs.add(Integer.toString(i)); - if (i % 2 != 0 && i % 3 != 0) { + if (i % 7 == 0 || i % 3 == 0) { expectedFailures++; } } @@ -730,9 +732,6 @@ private void testFailingDynamicDestinationsHelper(boolean bounded, boolean autos WriteFiles.to(sink) .withNumShards(numShards) .withBadRecordErrorHandler(errorHandler, (e) -> true); - errorHandler.close(); - - PAssert.thatSingleton(errorHandler.getOutput()).isEqualTo(expectedFailures); PCollection input = p.apply(Create.timestamped(inputs, timestamps)); WriteFilesResult res; @@ -743,6 +742,11 @@ private void testFailingDynamicDestinationsHelper(boolean bounded, boolean autos } else { res = input.apply(writeFiles); } + + errorHandler.close(); + + PAssert.thatSingleton(errorHandler.getOutput()).isEqualTo(expectedFailures); + res.getPerDestinationOutputFilenames().apply(new VerifyFilesExist<>()); p.run(); @@ -751,12 +755,14 @@ private void testFailingDynamicDestinationsHelper(boolean bounded, boolean autos getBaseOutputDirectory().resolve("file_" + i, StandardResolveOptions.RESOLVE_FILE); List expected = Lists.newArrayList(); for (int j = i; j < numInputs; j += 5) { - expected.add("record_" + j); + if (j % 3 != 0 && j % 7 != 0) { + expected.add("record_" + j); + } } checkFileContents( base.toString(), expected, - Optional.of(numShards), + Optional.fromNullable(autosharding ? null : numShards), bounded /* expectRemovedTempDirectory */); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java index 1de2891ccb86..41367765b920 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandlingTestUtils.java @@ -20,12 +20,14 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.windowing.CalendarWindows; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; +import org.joda.time.Duration; public class ErrorHandlingTestUtils { public static class ErrorSinkTransform @@ -34,9 +36,13 @@ public static class ErrorSinkTransform @Override public @UnknownKeyFor @NonNull @Initialized PCollection expand( PCollection input) { - return input - .apply("Window", Window.into(CalendarWindows.years(1))) - .apply("Combine", Combine.globally(Count.combineFn()).withoutDefaults()); + if (input.isBounded() == IsBounded.BOUNDED) { + return input.apply("Combine", Combine.globally(Count.combineFn())); + } else { + return input + .apply("Window", Window.into(FixedWindows.of(Duration.standardDays(1)))) + .apply("Combine", Combine.globally(Count.combineFn()).withoutDefaults()); + } } } } From abde5a340da2ae0c2157f99d3de2af923e1e2773 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Mon, 11 Dec 2023 12:34:33 -0500 Subject: [PATCH 04/21] spotless --- .../org/apache/beam/sdk/io/WriteFiles.java | 30 ++++++++----------- .../errorhandling/ErrorHandler.java | 3 +- .../apache/beam/sdk/io/WriteFilesTest.java | 4 +-- 3 files changed, 17 insertions(+), 20 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java index 7c0ce22638b2..b33b66c541a2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java @@ -544,7 +544,8 @@ public PCollection> expand(PCollection input) { writtenRecordsTag, TupleTagList.of(ImmutableList.of(unwrittenRecordsTag, BAD_RECORD_TAG)))); addErrorCollection(writeTuple); - writeTuple.get(unwrittenRecordsTag) + writeTuple + .get(unwrittenRecordsTag) .setCoder(KvCoder.of(ShardedKeyCoder.of(VarIntCoder.of()), input.getCoder())); return writeTuple.get(writtenRecordsTag).setCoder(fileResultCoder); } @@ -1034,25 +1035,20 @@ public KeyByDestinationHash(Coder inputCoder, Coder destina } @ProcessElement - public void processElement( - @Element UserT element, - ProcessContext context, - MultiOutputReceiver outputReceiver) - throws Exception { - getDynamicDestinations().setSideInputAccessorFromProcessContext(context); - MaybeDestination maybeDestination = - getDestinationWithErrorHandling( - context.element(), outputReceiver, inputCoder); - if (!maybeDestination.isValid) { - return; - } - DestinationT destination = maybeDestination.destination; - context.output( - KV.of(hashDestination(destination, destinationCoder), element)); + public void processElement( + @Element UserT element, ProcessContext context, MultiOutputReceiver outputReceiver) + throws Exception { + getDynamicDestinations().setSideInputAccessorFromProcessContext(context); + MaybeDestination maybeDestination = + getDestinationWithErrorHandling(context.element(), outputReceiver, inputCoder); + if (!maybeDestination.isValid) { + return; } + DestinationT destination = maybeDestination.destination; + context.output(KV.of(hashDestination(destination, destinationCoder), element)); + } } - private class RandomShardingFunction implements ShardingFunction { private final Coder destinationCoder; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java index f93a409993b9..cf040470d608 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/ErrorHandler.java @@ -120,7 +120,8 @@ private void readObject(ObjectInputStream aInputStream) @Override public void addErrorCollection(PCollection errorCollection) { if (isClosed()) { - throw new IllegalStateException("Error collections cannot be added after Error Handler is closed"); + throw new IllegalStateException( + "Error collections cannot be added after Error Handler is closed"); } errorCollections.add(errorCollection); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java index 1b44e799a91a..8437ac030910 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java @@ -650,7 +650,7 @@ static class FailingTestDestinations extends DynamicDestinations Date: Tue, 12 Dec 2023 11:49:34 -0500 Subject: [PATCH 05/21] add documentation --- .../java/org/apache/beam/sdk/io/FileIO.java | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java index c48e13a7ade4..2aef300f43d2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java @@ -238,6 +238,29 @@ * destination-dependent: every window/pane for every destination will use the same number of shards * specified via {@link Write#withNumShards} or {@link Write#withSharding}. * + *

Handling Errors

+ * + *

When using dynamic destinations, or when using a formatting function to format a record for + * writing, it's possible for an individual record to be malformed, causing an exception. By default, + * these exceptions are propagated to the runner, and are usually retried, though this depends on + * the runner. Alternately, these errors can be routed to another {@link PTransform} by using + * {@link Write#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. The ErrorHandler + * is registered with the pipeline (see below), and the SerializableFunction lets you filter which + * exceptions should be sent to the error handler, and which should be handled by the runner. See + * {@link ErrorHandler} for more documentation. Of note, this error handling only handles errors + * related to specific records. It does not handle errors related to connectivity, authorization, + * etc. as those should be retried by the runner.

+ * + *
{@code
+ * PCollection<> records = ...;
+ * PTransform,?> alternateSink = ...;
+ * try (BadRecordErrorHandler handler = pipeline.registerBadRecordErrorHandler(alternateSink) {
+ *    records.apply("Write", FileIO.writeDynamic().otherConfigs()
+ *        .withBadRecordErrorHandler(handler, (exception) -> true));
+ * }
+ * }
+ * + * *

Writing custom types to sinks

* *

Normally, when writing a collection of a custom type using a {@link Sink} that takes a @@ -1300,6 +1323,12 @@ public Write withNoSpilling() { return toBuilder().setNoSpilling(true).build(); } + /** See {@link WriteFiles#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. */ + public Write withBadRecordErrorHandler( + ErrorHandler errorHandler) { + return withBadRecordErrorHandler(errorHandler, (e) -> true); + } + /** See {@link WriteFiles#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. */ public Write withBadRecordErrorHandler( ErrorHandler errorHandler, From c61243d4153712b448c1cb049cac8336a7145d0e Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Tue, 12 Dec 2023 15:30:58 -0500 Subject: [PATCH 06/21] add textio error handler pass-through --- .../java/org/apache/beam/sdk/io/FileIO.java | 19 +++++----- .../java/org/apache/beam/sdk/io/TextIO.java | 35 +++++++++++++++++++ 2 files changed, 44 insertions(+), 10 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java index 2aef300f43d2..88b0dcadf1b8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java @@ -241,15 +241,15 @@ *

Handling Errors

* *

When using dynamic destinations, or when using a formatting function to format a record for - * writing, it's possible for an individual record to be malformed, causing an exception. By default, - * these exceptions are propagated to the runner, and are usually retried, though this depends on - * the runner. Alternately, these errors can be routed to another {@link PTransform} by using - * {@link Write#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. The ErrorHandler - * is registered with the pipeline (see below), and the SerializableFunction lets you filter which - * exceptions should be sent to the error handler, and which should be handled by the runner. See - * {@link ErrorHandler} for more documentation. Of note, this error handling only handles errors - * related to specific records. It does not handle errors related to connectivity, authorization, - * etc. as those should be retried by the runner.

+ * writing, it's possible for an individual record to be malformed, causing an exception. By + * default, these exceptions are propagated to the runner, and are usually retried, though this + * depends on the runner. Alternately, these errors can be routed to another {@link PTransform} by + * using {@link Write#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. The + * ErrorHandler is registered with the pipeline (see below), and the SerializableFunction lets you + * filter which exceptions should be sent to the error handler, and which should be handled by the + * runner. See {@link ErrorHandler} for more documentation. Of note, this error handling only + * handles errors related to specific records. It does not handle errors related to connectivity, + * authorization, etc. as those should be retried by the runner. * *
{@code
  * PCollection<> records = ...;
@@ -260,7 +260,6 @@
  * }
  * }
* - * *

Writing custom types to sinks

* *

Normally, when writing a collection of a custom type using a {@link Sink} that takes a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index 2c7a4fc5d4f5..fb3b746b87a7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -51,6 +51,8 @@ import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -176,6 +178,10 @@ * *

For backwards compatibility, {@link TextIO} also supports the legacy {@link * DynamicDestinations} interface for advanced features via {@link Write#to(DynamicDestinations)}. + * + *

Error handling for records that are malformed can be handled by using {@link + * TypedWrite#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. See documentation in + * {@link FileIO} for details on usage */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) @@ -708,6 +714,10 @@ public abstract static class TypedWrite */ abstract WritableByteChannelFactory getWritableByteChannelFactory(); + abstract @Nullable ErrorHandler getBadRecordErrorHandler(); + + abstract @Nullable SerializableFunction getBadRecordMatcher(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -754,6 +764,12 @@ abstract Builder setNumShards( abstract Builder setWritableByteChannelFactory( WritableByteChannelFactory writableByteChannelFactory); + abstract Builder setBadRecordErrorHandler( + @Nullable ErrorHandler badRecordErrorHandler); + + abstract Builder setBadRecordMatcher( + @Nullable SerializableFunction badRecordMatcher); + abstract TypedWrite build(); } @@ -993,6 +1009,22 @@ public TypedWrite withNoSpilling() { return toBuilder().setNoSpilling(true).build(); } + /** See {@link WriteFiles#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. */ + public TypedWrite withBadRecordErrorHandler( + ErrorHandler errorHandler) { + return withBadRecordErrorHandler(errorHandler, (e) -> true); + } + + /** See {@link WriteFiles#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. */ + public TypedWrite withBadRecordErrorHandler( + ErrorHandler errorHandler, + SerializableFunction badRecordMatcher) { + return toBuilder() + .setBadRecordErrorHandler(errorHandler) + .setBadRecordMatcher(badRecordMatcher) + .build(); + } + /** Don't write any output files if the PCollection is empty. */ public TypedWrite skipIfEmpty() { return toBuilder().setSkipIfEmpty(true).build(); @@ -1083,6 +1115,9 @@ public WriteFilesResult expand(PCollection input) { if (getNoSpilling()) { write = write.withNoSpilling(); } + if (getBadRecordErrorHandler() != null) { + write = write.withBadRecordErrorHandler(getBadRecordErrorHandler(), getBadRecordMatcher()); + } if (getSkipIfEmpty()) { write = write.withSkipIfEmpty(); } From de5bd4c39abb3e70958723e1d9f975b3cdf1716e Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Mon, 18 Dec 2023 11:18:46 -0500 Subject: [PATCH 07/21] add avroio error handler pass-through --- .../beam/sdk/extensions/avro/io/AvroIO.java | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java index a65db5a90bad..d01cd8cb84bc 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java @@ -69,6 +69,8 @@ import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -1427,6 +1429,10 @@ public abstract static class TypedWrite abstract AvroSink.@Nullable DatumWriterFactory getDatumWriterFactory(); + abstract @Nullable ErrorHandler getBadRecordErrorHandler(); + + abstract @Nullable SerializableFunction getBadRecordMatcher(); + /** * The codec used to encode the blocks in the Avro file. String value drawn from those in * https://avro.apache.org/docs/1.7.7/api/java/org/apache/avro/file/CodecFactory.html @@ -1489,6 +1495,12 @@ abstract Builder setDynamicDestinations( abstract Builder setDatumWriterFactory( AvroSink.DatumWriterFactory datumWriterFactory); + abstract Builder setBadRecordErrorHandler( + @Nullable ErrorHandler badRecordErrorHandler); + + abstract Builder setBadRecordMatcher( + @Nullable SerializableFunction badRecordMatcher); + abstract TypedWrite build(); } @@ -1713,6 +1725,22 @@ public TypedWrite withMetadata(Map return toBuilder().setMetadata(ImmutableMap.copyOf(metadata)).build(); } + /** See {@link WriteFiles#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. */ + public TypedWrite withBadRecordErrorHandler( + ErrorHandler errorHandler) { + return withBadRecordErrorHandler(errorHandler, (e) -> true); + } + + /** See {@link WriteFiles#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. */ + public TypedWrite withBadRecordErrorHandler( + ErrorHandler errorHandler, + SerializableFunction badRecordMatcher) { + return toBuilder() + .setBadRecordErrorHandler(errorHandler) + .setBadRecordMatcher(badRecordMatcher) + .build(); + } + DynamicAvroDestinations resolveDynamicDestinations() { DynamicAvroDestinations dynamicDestinations = getDynamicDestinations(); @@ -1782,6 +1810,9 @@ public WriteFilesResult expand(PCollection input) { if (getNoSpilling()) { write = write.withNoSpilling(); } + if (getBadRecordErrorHandler() != null && getBadRecordMatcher() != null) { + write = write.withBadRecordErrorHandler(getBadRecordErrorHandler(), getBadRecordMatcher()); + } return input.apply("Write", write); } From 92948aa1da3b40def5173d2faed9ad41b1d33caa Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Mon, 18 Dec 2023 12:02:16 -0500 Subject: [PATCH 08/21] add documentation to avroio --- .../java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java index d01cd8cb84bc..3c69a4ae033a 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java @@ -339,6 +339,10 @@ * events.apply("WriteAvros", AvroIO.writeCustomTypeToGenericRecords() * .to(new UserDynamicAvroDestinations(userToSchemaMap))); * } + * + *

Error handling for writing records that are malformed can be handled by using {@link + * TypedWrite#withBadRecordErrorHandler(ErrorHandler, SerializableFunction)}. See documentation in + * {@link FileIO} for details on usage */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) From 3e0cae5ae24fb66c5733a40830f8244edf4f3801 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Mon, 18 Dec 2023 15:51:39 -0500 Subject: [PATCH 09/21] wire error handling into bigtable io --- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 81 +++++++++++++++++-- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 64 +++++++++++++++ 2 files changed, 138 insertions(+), 7 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 82c2d3142482..6f266b64870c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -19,10 +19,12 @@ import static org.apache.beam.sdk.io.gcp.bigtable.BigtableServiceFactory.BigtableServiceEntry; import static org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import static org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter.BAD_RECORD_TAG; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; +import com.google.api.gax.rpc.NotFoundException; import com.google.auto.value.AutoValue; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; @@ -69,11 +71,17 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects.ToStringHelper; @@ -773,6 +781,10 @@ public abstract static class Write @VisibleForTesting abstract BigtableServiceFactory getServiceFactory(); + abstract ErrorHandler getBadRecordErrorHandler(); + + abstract BadRecordRouter getBadRecordRouter(); + /** * Returns the Google Cloud Bigtable instance being written to, and other parameters. * @@ -796,6 +808,8 @@ static Write create() { .setBigtableConfig(config) .setBigtableWriteOptions(writeOptions) .setServiceFactory(new BigtableServiceFactory()) + .setBadRecordErrorHandler(new ErrorHandler.DefaultErrorHandler<>()) + .setBadRecordRouter(BadRecordRouter.THROWING_ROUTER) .build(); } @@ -808,6 +822,10 @@ abstract static class Builder { abstract Builder setServiceFactory(BigtableServiceFactory factory); + abstract Builder setBadRecordErrorHandler(ErrorHandler badRecordErrorHandler); + + abstract Builder setBadRecordRouter(BadRecordRouter badRecordRouter); + abstract Write build(); } @@ -1093,6 +1111,11 @@ public Write withFlowControl(boolean enableFlowControl) { .build(); } + public Write withErrorHandler(ErrorHandler badRecordErrorHandler){ + return toBuilder().setBadRecordErrorHandler(badRecordErrorHandler) + .setBadRecordRouter(BadRecordRouter.RECORDING_ROUTER).build(); + } + @VisibleForTesting Write withServiceFactory(BigtableServiceFactory factory) { return toBuilder().setServiceFactory(factory).build(); @@ -1104,7 +1127,7 @@ Write withServiceFactory(BigtableServiceFactory factory) { */ public WriteWithResults withWriteResults() { return new WriteWithResults( - getBigtableConfig(), getBigtableWriteOptions(), getServiceFactory()); + getBigtableConfig(), getBigtableWriteOptions(), getServiceFactory(), getBadRecordErrorHandler(), getBadRecordRouter()); } @Override @@ -1142,18 +1165,28 @@ public static class WriteWithResults private static final String BIGTABLE_WRITER_WAIT_TIMEOUT_MS = "bigtable_writer_wait_timeout_ms"; + private static final TupleTag WRITE_RESULTS = new TupleTag<>("writeResults"); + private final BigtableConfig bigtableConfig; private final BigtableWriteOptions bigtableWriteOptions; private final BigtableServiceFactory factory; + private final ErrorHandler badRecordErrorHandler; + + private final BadRecordRouter badRecordRouter; + WriteWithResults( BigtableConfig bigtableConfig, BigtableWriteOptions bigtableWriteOptions, - BigtableServiceFactory factory) { + BigtableServiceFactory factory, + ErrorHandler badRecordErrorHandler, + BadRecordRouter badRecordRouter) { this.bigtableConfig = bigtableConfig; this.bigtableWriteOptions = bigtableWriteOptions; this.factory = factory; + this.badRecordErrorHandler = badRecordErrorHandler; + this.badRecordRouter = badRecordRouter; } @Override @@ -1173,12 +1206,20 @@ public PCollection expand( closeWaitTimeout = Duration.millis(closeWaitTimeoutMs); } - return input.apply( + PCollectionTuple results = input.apply( ParDo.of( new BigtableWriterFn( factory, bigtableConfig, - bigtableWriteOptions.toBuilder().setCloseWaitTimeout(closeWaitTimeout).build()))); + bigtableWriteOptions.toBuilder().setCloseWaitTimeout(closeWaitTimeout).build(), + input.getCoder(), + badRecordRouter)) + .withOutputTags(WRITE_RESULTS, TupleTagList.of(BAD_RECORD_TAG))); + + badRecordErrorHandler.addErrorCollection(results.get(BAD_RECORD_TAG) + .setCoder(BadRecord.getCoder(input.getPipeline()))); + + return results.get(WRITE_RESULTS); } @Override @@ -1221,6 +1262,8 @@ private static class BigtableWriterFn private final BigtableServiceFactory factory; private final BigtableServiceFactory.ConfigId id; + private final Coder>> inputCoder; + private final BadRecordRouter badRecordRouter; // Assign serviceEntry in startBundle and clear it in tearDown. @Nullable private BigtableServiceEntry serviceEntry; @@ -1228,10 +1271,14 @@ private static class BigtableWriterFn BigtableWriterFn( BigtableServiceFactory factory, BigtableConfig bigtableConfig, - BigtableWriteOptions writeOptions) { + BigtableWriteOptions writeOptions, + Coder>> inputCoder, + BadRecordRouter badRecordRouter) { this.factory = factory; this.config = bigtableConfig; this.writeOptions = writeOptions; + this.inputCoder = inputCoder; + this.badRecordRouter = badRecordRouter; this.failures = new ConcurrentLinkedQueue<>(); this.id = factory.newId(); LOG.debug("Created Bigtable Write Fn with writeOptions {} ", writeOptions); @@ -1250,7 +1297,7 @@ public void startBundle(StartBundleContext c) throws IOException { } @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) throws Exception { + public void processElement(ProcessContext c, BoundedWindow window, MultiOutputReceiver outputReceiver) throws Exception { checkForFailures(); KV> record = c.element(); bigtableWriter @@ -1258,7 +1305,18 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except .whenComplete( (mutationResult, exception) -> { if (exception != null) { - failures.add(new BigtableWriteException(record, exception)); + if (exception instanceof NotFoundException && !((NotFoundException) exception).isRetryable()){ + //This case, of being an InvalidArgumentException and not retryable, + // indicates an issue with the data, so is sent to the Error Handler + try { + badRecordRouter.route(outputReceiver, record, inputCoder, + (Exception) exception, "Failed writing data to Bigtable"); + } catch (Exception e) { + failures.add(new BigtableWriteException(record, exception)); + } + } else { + failures.add(new BigtableWriteException(record, exception)); + } } }); ++recordsWritten; @@ -1861,13 +1919,22 @@ public final long getSplitPointsConsumed() { /** An exception that puts information about the failed record being written in its message. */ static class BigtableWriteException extends IOException { + + private final KV> record; + public BigtableWriteException(KV> record, Throwable cause) { super( String.format( "Error mutating row %s with mutations %s", record.getKey().toStringUtf8(), record.getValue()), cause); + this.record = record; + } + + public KV> getRecord() { + return record; } + } /** * Overwrite options to determine what to do if change stream name is being reused and there diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index 3e3f24fdd54d..64fda8e81660 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -35,16 +35,22 @@ import java.util.Date; import java.util.Iterator; import java.util.List; +import java.util.Objects; import java.util.stream.Collectors; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.errorhandling.BadRecord; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; +import org.apache.beam.sdk.transforms.errorhandling.ErrorHandlingTestUtils.ErrorSinkTransform; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.Matchers; import org.junit.After; @@ -69,6 +75,8 @@ public class BigtableWriteIT implements Serializable { private static BigtableTableAdminClient tableAdminClient; private final String tableId = String.format("BigtableWriteIT-%tF-%> testData = generateTableData(numRows); + + createEmptyTable(failureTableId); + + Pipeline p = Pipeline.create(options); + PCollection>> mutations = p.apply(GenerateSequence.from(0).to(numRows)) + .apply( + ParDo.of( + new DoFn>>() { + @ProcessElement + public void processElement(ProcessContext c) { + int index = c.element().intValue(); + + String familyName = COLUMN_FAMILY_NAME; + if (index % 1700 == 0 ) { + familyName = "malformed"; + } + Iterable mutations = ImmutableList.of( + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(testData.get(index).getValue()) + .setFamilyName(familyName)) + .build()); + c.output(KV.of(testData.get(index).getKey(), mutations)); + } + })); + ErrorHandler> errorHandler = p.registerBadRecordErrorHandler(new ErrorSinkTransform()); + mutations.apply( + BigtableIO.write() + .withProjectId(project) + .withInstanceId(options.getInstanceId()) + .withTableId(failureTableId) + .withErrorHandler(errorHandler)); + + errorHandler.close(); + PAssert.thatSingleton(Objects.requireNonNull(errorHandler.getOutput())).isEqualTo(10L); + + p.run(); + + // Test number of column families and column family name equality + Table table = getTable(failureTableId); + assertThat(table.getColumnFamilies(), Matchers.hasSize(1)); + assertThat( + table.getColumnFamilies().stream().map((c) -> c.getId()).collect(Collectors.toList()), + Matchers.contains(COLUMN_FAMILY_NAME)); + + // Test table data equality + List> tableData = getTableData(failureTableId); + assertThat(tableData, Matchers.containsInAnyOrder(testData.toArray())); + } + @After public void tearDown() throws Exception { deleteTable(tableId); + deleteTable(failureTableId); if (tableAdminClient != null) { tableAdminClient.close(); } From 1f0b7f08cde681800a316eef261a84d17fd88b47 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Tue, 19 Dec 2023 15:32:39 -0500 Subject: [PATCH 10/21] catch batching error when errors are detected --- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 6f266b64870c..d07157ca469c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -24,6 +24,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; +import com.google.api.gax.batching.BatchingException; import com.google.api.gax.rpc.NotFoundException; import com.google.auto.value.AutoValue; import com.google.bigtable.v2.Mutation; @@ -1326,12 +1327,23 @@ public void processElement(ProcessContext c, BoundedWindow window, MultiOutputRe @FinishBundle public void finishBundle(FinishBundleContext c) throws Exception { try { + checkForFailures(); + if (bigtableWriter != null) { - bigtableWriter.close(); + try { + bigtableWriter.close(); + } catch (IOException e){ + //If the writer fails due to a batching exception, but no failures were detected + //it means that error handling was enabled, and that errors were detected and routed + //to the error queue. Bigtable will successfully write other failures in the batch, + //so this exception should be ignored + if (!(e.getCause() instanceof BatchingException)){ + throw e; + } + } bigtableWriter = null; } - checkForFailures(); LOG.debug("Wrote {} records", recordsWritten); for (Map.Entry entry : seenWindows.entrySet()) { From 35c7124e50f706066614215522ec2ba298dfc31c Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Wed, 20 Dec 2023 13:44:37 -0500 Subject: [PATCH 11/21] continue work on bigtable error handling --- .../java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java | 2 +- .../org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index d07157ca469c..540c9563152b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1307,7 +1307,7 @@ public void processElement(ProcessContext c, BoundedWindow window, MultiOutputRe (mutationResult, exception) -> { if (exception != null) { if (exception instanceof NotFoundException && !((NotFoundException) exception).isRetryable()){ - //This case, of being an InvalidArgumentException and not retryable, + //This case, of being an NotFoundException and not retryable, // indicates an issue with the data, so is sent to the Error Handler try { badRecordRouter.route(outputReceiver, record, inputCoder, diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index 64fda8e81660..5f38588f10fd 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -194,7 +194,7 @@ public void processElement(ProcessContext c) { .withErrorHandler(errorHandler)); errorHandler.close(); - PAssert.thatSingleton(Objects.requireNonNull(errorHandler.getOutput())).isEqualTo(10L); + // PAssert.thatSingleton(Objects.requireNonNull(errorHandler.getOutput())).isEqualTo(10L); p.run(); From cba5bbd310d34e15da57961c696c4637fc85e6de Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Wed, 20 Dec 2023 17:38:37 -0500 Subject: [PATCH 12/21] break apart and retry batches when they fail in bigquery --- .../errorhandling/BadRecordRouter.java | 48 +++++ .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 171 +++++++++++++----- .../sdk/io/gcp/bigtable/BigtableService.java | 4 + .../io/gcp/bigtable/BigtableServiceImpl.java | 91 +++++++--- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 15 ++ .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 72 ++++---- 6 files changed, 292 insertions(+), 109 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java index 7670baf5c6ff..ec866ea2a616 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java @@ -19,7 +19,9 @@ import java.io.Serializable; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.TupleTag; import org.checkerframework.checker.nullness.qual.Nullable; @@ -40,6 +42,15 @@ void route( String description) throws Exception; + void route( + DoFn.FinishBundleContext c, + RecordT record, + @Nullable Coder coder, + @Nullable Exception exception, + String description, + BoundedWindow window) + throws Exception; + class ThrowingBadRecordRouter implements BadRecordRouter { @Override @@ -65,6 +76,31 @@ public void route( throw new RuntimeException("Encountered Bad Record: " + encodedRecord); } } + + @Override + public void route( + DoFn.FinishBundleContext c, + RecordT record, + @Nullable Coder coder, + @Nullable Exception exception, + String description, + BoundedWindow window) + throws Exception { + if (exception != null) { + throw exception; + } else { + Preconditions.checkArgumentNotNull(record); + String encodedRecord = + BadRecord.Record.builder() + .addHumanReadableJson(record) + .build() + .getHumanReadableJsonRecord(); + if (encodedRecord == null) { + encodedRecord = "Unable to serialize bad record"; + } + throw new RuntimeException("Encountered Bad Record: " + encodedRecord); + } + } } class RecordingBadRecordRouter implements BadRecordRouter { @@ -81,5 +117,17 @@ public void route( .get(BAD_RECORD_TAG) .output(BadRecord.fromExceptionInformation(record, coder, exception, description)); } + + @Override + public void route( + DoFn.FinishBundleContext c, + RecordT record, + @Nullable Coder coder, + @Nullable Exception exception, + String description, + BoundedWindow window) + throws Exception { + c.output(BAD_RECORD_TAG,BadRecord.fromExceptionInformation(record, coder, exception, description),window.maxTimestamp(),window); + } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 540c9563152b..81d539d009fb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -27,6 +27,7 @@ import com.google.api.gax.batching.BatchingException; import com.google.api.gax.rpc.NotFoundException; import com.google.auto.value.AutoValue; +import com.google.bigtable.v2.MutateRowResponse; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; import com.google.bigtable.v2.RowFilter; @@ -38,11 +39,15 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.CompletionStage; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.function.BiConsumer; import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; @@ -782,7 +787,7 @@ public abstract static class Write @VisibleForTesting abstract BigtableServiceFactory getServiceFactory(); - abstract ErrorHandler getBadRecordErrorHandler(); + abstract ErrorHandler getBadRecordErrorHandler(); abstract BadRecordRouter getBadRecordRouter(); @@ -1112,9 +1117,11 @@ public Write withFlowControl(boolean enableFlowControl) { .build(); } - public Write withErrorHandler(ErrorHandler badRecordErrorHandler){ - return toBuilder().setBadRecordErrorHandler(badRecordErrorHandler) - .setBadRecordRouter(BadRecordRouter.RECORDING_ROUTER).build(); + public Write withErrorHandler(ErrorHandler badRecordErrorHandler) { + return toBuilder() + .setBadRecordErrorHandler(badRecordErrorHandler) + .setBadRecordRouter(BadRecordRouter.RECORDING_ROUTER) + .build(); } @VisibleForTesting @@ -1128,7 +1135,11 @@ Write withServiceFactory(BigtableServiceFactory factory) { */ public WriteWithResults withWriteResults() { return new WriteWithResults( - getBigtableConfig(), getBigtableWriteOptions(), getServiceFactory(), getBadRecordErrorHandler(), getBadRecordRouter()); + getBigtableConfig(), + getBigtableWriteOptions(), + getServiceFactory(), + getBadRecordErrorHandler(), + getBadRecordRouter()); } @Override @@ -1166,7 +1177,8 @@ public static class WriteWithResults private static final String BIGTABLE_WRITER_WAIT_TIMEOUT_MS = "bigtable_writer_wait_timeout_ms"; - private static final TupleTag WRITE_RESULTS = new TupleTag<>("writeResults"); + private static final TupleTag WRITE_RESULTS = + new TupleTag<>("writeResults"); private final BigtableConfig bigtableConfig; private final BigtableWriteOptions bigtableWriteOptions; @@ -1207,18 +1219,22 @@ public PCollection expand( closeWaitTimeout = Duration.millis(closeWaitTimeoutMs); } - PCollectionTuple results = input.apply( - ParDo.of( - new BigtableWriterFn( - factory, - bigtableConfig, - bigtableWriteOptions.toBuilder().setCloseWaitTimeout(closeWaitTimeout).build(), - input.getCoder(), - badRecordRouter)) - .withOutputTags(WRITE_RESULTS, TupleTagList.of(BAD_RECORD_TAG))); - - badRecordErrorHandler.addErrorCollection(results.get(BAD_RECORD_TAG) - .setCoder(BadRecord.getCoder(input.getPipeline()))); + PCollectionTuple results = + input.apply( + ParDo.of( + new BigtableWriterFn( + factory, + bigtableConfig, + bigtableWriteOptions + .toBuilder() + .setCloseWaitTimeout(closeWaitTimeout) + .build(), + input.getCoder(), + badRecordRouter)) + .withOutputTags(WRITE_RESULTS, TupleTagList.of(BAD_RECORD_TAG))); + + badRecordErrorHandler.addErrorCollection( + results.get(BAD_RECORD_TAG).setCoder(BadRecord.getCoder(input.getPipeline()))); return results.get(WRITE_RESULTS); } @@ -1263,9 +1279,13 @@ private static class BigtableWriterFn private final BigtableServiceFactory factory; private final BigtableServiceFactory.ConfigId id; - private final Coder>> inputCoder; + private final Coder>> inputCoder; private final BadRecordRouter badRecordRouter; + private Set> badRecords = new HashSet<>(); + + private Set> unbatchedWrites = new HashSet<>(); + // Assign serviceEntry in startBundle and clear it in tearDown. @Nullable private BigtableServiceEntry serviceEntry; @@ -1273,7 +1293,7 @@ private static class BigtableWriterFn BigtableServiceFactory factory, BigtableConfig bigtableConfig, BigtableWriteOptions writeOptions, - Coder>> inputCoder, + Coder>> inputCoder, BadRecordRouter badRecordRouter) { this.factory = factory; this.config = bigtableConfig; @@ -1295,55 +1315,111 @@ public void startBundle(StartBundleContext c) throws IOException { factory.getServiceForWriting(id, config, writeOptions, c.getPipelineOptions()); bigtableWriter = serviceEntry.getService().openForWriting(writeOptions); } + + badRecords = new HashSet<>(); + unbatchedWrites = new HashSet<>(); } @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window, MultiOutputReceiver outputReceiver) throws Exception { + public void processElement(ProcessContext c, BoundedWindow window) throws Exception { checkForFailures(); KV> record = c.element(); - bigtableWriter - .writeRecord(record) - .whenComplete( - (mutationResult, exception) -> { - if (exception != null) { - if (exception instanceof NotFoundException && !((NotFoundException) exception).isRetryable()){ - //This case, of being an NotFoundException and not retryable, - // indicates an issue with the data, so is sent to the Error Handler - try { - badRecordRouter.route(outputReceiver, record, inputCoder, - (Exception) exception, "Failed writing data to Bigtable"); - } catch (Exception e) { - failures.add(new BigtableWriteException(record, exception)); - } - } else { - failures.add(new BigtableWriteException(record, exception)); - } - } - }); + bigtableWriter.writeRecord(record).whenComplete(handleMutationException(record, window)); ++recordsWritten; seenWindows.compute(window, (key, count) -> (count != null ? count : 0) + 1); } + public BiConsumer handleMutationException( + KV> record, BoundedWindow window) { + return (MutateRowResponse result, Throwable exception) -> { + if (exception != null) { + if (exception instanceof IllegalStateException) { + // This case indicates that an individual mutation is invalid, and should not poison the + // batch. It can safely be handled without retrying. + badRecords.add(KV.of(new BigtableWriteException(record, exception), window)); + } else if (exception instanceof NotFoundException + && !((NotFoundException) exception).isRetryable()) { + // This case, of being an NotFoundException and not retryable, + // indicates an issue with the data. However, we can't know if this record + // failed, or if it was another record in the batch. Thus, we retry each record + // individually. + retryIndividualRecord(record, window); + } else { + failures.add(new BigtableWriteException(record, exception)); + } + } + }; + } + + public void retryIndividualRecord( + KV> record, BoundedWindow window) { + try { + unbatchedWrites.add( + bigtableWriter + .writeRecordWithoutBatching(record) + .whenComplete( + (singleMutationResult, singleException) -> { + if (singleException != null) { + if (singleException instanceof NotFoundException + && !((NotFoundException) singleException).isRetryable()) { + // if we get another NotFoundException, we know this is the bad record. + badRecords.add( + KV.of(new BigtableWriteException(record, singleException), window)); + } else { + failures.add(new BigtableWriteException(record, singleException)); + } + } + })); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + @FinishBundle public void finishBundle(FinishBundleContext c) throws Exception { try { - checkForFailures(); if (bigtableWriter != null) { try { bigtableWriter.close(); - } catch (IOException e){ - //If the writer fails due to a batching exception, but no failures were detected - //it means that error handling was enabled, and that errors were detected and routed - //to the error queue. Bigtable will successfully write other failures in the batch, - //so this exception should be ignored - if (!(e.getCause() instanceof BatchingException)){ + } catch (IOException e) { + // If the writer fails due to a batching exception, but no failures were detected + // it means that error handling was enabled, and that errors were detected and routed + // to the error queue. Bigtable will successfully write other failures in the batch, + // so this exception should be ignored + if (!(e.getCause() instanceof BatchingException)) { throw e; } } bigtableWriter = null; } + for (CompletionStage unbatchedWrite : unbatchedWrites) { + try { + unbatchedWrite.toCompletableFuture().get(); + } catch (Exception e) { + // ignore exceptions here, they are handled by the .whenComplete as part of the + // completion stage this .get exists purely to make sure all unbatched writes are + // complete. + } + } + + for (KV badRecord : badRecords) { + try { + badRecordRouter.route( + c, + badRecord.getKey().getRecord(), + inputCoder, + (Exception) badRecord.getKey().getCause(), + "Failed to write malformed mutation to Bigtable", + badRecord.getValue()); + } catch (Exception e) { + failures.add(badRecord.getKey()); + } + } + + checkForFailures(); + LOG.debug("Wrote {} records", recordsWritten); for (Map.Entry entry : seenWindows.entrySet()) { @@ -1946,7 +2022,6 @@ public BigtableWriteException(KV> record, Throwab public KV> getRecord() { return record; } - } /** * Overwrite options to determine what to do if change stream name is being reused and there diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index b529d6530ef2..f39a039e76a2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -45,6 +45,10 @@ interface Writer { CompletionStage writeRecord(KV> record) throws IOException; + /** Like above, but will not batch the record. Useful for single record retries */ + CompletionStage writeRecordWithoutBatching( + KV> record) throws IOException; + /** * Closes the writer. * diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 344229b383fd..a55eca97898a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -44,6 +44,7 @@ import com.google.cloud.bigtable.data.v2.models.KeyOffset; import com.google.cloud.bigtable.data.v2.models.Query; import com.google.cloud.bigtable.data.v2.models.RowAdapter; +import com.google.cloud.bigtable.data.v2.models.RowMutation; import com.google.cloud.bigtable.data.v2.models.RowMutationEntry; import com.google.protobuf.ByteString; import io.grpc.CallOptions; @@ -473,6 +474,8 @@ public Duration getOperationTimeout() { @VisibleForTesting static class BigtableWriterImpl implements Writer { private Batcher bulkMutation; + + private BigtableDataClient client; private Integer outstandingMutations = 0; private Stopwatch stopwatch = Stopwatch.createUnstarted(); private String projectId; @@ -494,6 +497,7 @@ static class BigtableWriterImpl implements Writer { this.tableId = tableId; this.closeWaitTimeout = closeWaitTimeout; this.bulkMutation = client.newBulkMutationBatcher(tableId); + this.client = client; } @Override @@ -542,6 +546,39 @@ public CompletionStage writeRecord(KV result = new CompletableFuture<>(); + + outstandingMutations += 1; + Futures.addCallback( + new VendoredListenableFutureAdapter<>(bulkMutation.add(entry)), + new WriteMutationCallback(result, serviceCallMetric), + directExecutor()); + return result; + } + + @Override + public CompletionStage writeRecordWithoutBatching( + KV> record) { + com.google.cloud.bigtable.data.v2.models.Mutation mutation = + com.google.cloud.bigtable.data.v2.models.Mutation.fromProtoUnsafe(record.getValue()); + + RowMutation rowMutation = RowMutation.create(tableId, record.getKey(), mutation); + + ServiceCallMetric serviceCallMetric = createServiceCallMetric(); + + CompletableFuture result = new CompletableFuture<>(); + + outstandingMutations += 1; + Futures.addCallback( + new VendoredListenableFutureAdapter<>(client.mutateRowAsync(rowMutation)), + new WriteMutationCallback(result, serviceCallMetric), + directExecutor()); + return result; + } + + private ServiceCallMetric createServiceCallMetric() { // Populate metrics HashMap baseLabels = new HashMap<>(); baseLabels.put(MonitoringInfoConstants.Labels.PTRANSFORM, ""); @@ -555,34 +592,36 @@ public CompletionStage writeRecord(KV result = new CompletableFuture<>(); + private static class WriteMutationCallback implements FutureCallback { + private final CompletableFuture result; - outstandingMutations += 1; - Futures.addCallback( - new VendoredListenableFutureAdapter<>(bulkMutation.add(entry)), - new FutureCallback() { - @Override - public void onSuccess(MutateRowResponse mutateRowResponse) { - result.complete(mutateRowResponse); - serviceCallMetric.call("ok"); - } - - @Override - public void onFailure(Throwable throwable) { - if (throwable instanceof StatusRuntimeException) { - serviceCallMetric.call( - ((StatusRuntimeException) throwable).getStatus().getCode().value()); - } else { - serviceCallMetric.call("unknown"); - } - result.completeExceptionally(throwable); - } - }, - directExecutor()); - return result; + private final ServiceCallMetric serviceCallMetric; + + public WriteMutationCallback( + CompletableFuture result, ServiceCallMetric serviceCallMetric) { + this.result = result; + this.serviceCallMetric = serviceCallMetric; + } + + @Override + public void onSuccess(MutateRowResponse mutateRowResponse) { + result.complete(mutateRowResponse); + serviceCallMetric.call("ok"); + } + + @Override + public void onFailure(Throwable throwable) { + if (throwable instanceof StatusRuntimeException) { + serviceCallMetric.call( + ((StatusRuntimeException) throwable).getStatus().getCode().value()); + } else { + serviceCallMetric.call("unknown"); + } + result.completeExceptionally(throwable); + } } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 6cda518eb124..9a12729706a9 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -1929,6 +1929,12 @@ public CompletionStage writeRecord(KV writeRecordWithoutBatching( + KV> record) throws IOException { + return writeRecord(record); + } + @Override public void close() {} } @@ -1950,6 +1956,15 @@ public CompletionStage writeRecord(KV writeRecordWithoutBatching( + KV> record) throws IOException { + if (failureOptions.getFailAtWriteRecord()) { + throw new IOException("Fake IOException in writeRecord()"); + } + return super.writeRecordWithoutBatching(record); + } + private final FailureOptions failureOptions; } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index 5f38588f10fd..9e6a98fd7630 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.bigtable; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; import com.google.api.gax.rpc.ServerStream; import com.google.bigtable.v2.Mutation; @@ -76,7 +77,6 @@ public class BigtableWriteIT implements Serializable { private final String tableId = String.format("BigtableWriteIT-%tF-%> testData = generateTableData(numRows); - createEmptyTable(failureTableId); + createEmptyTable(tableId); Pipeline p = Pipeline.create(options); - PCollection>> mutations = p.apply(GenerateSequence.from(0).to(numRows)) - .apply( - ParDo.of( - new DoFn>>() { - @ProcessElement - public void processElement(ProcessContext c) { - int index = c.element().intValue(); - - String familyName = COLUMN_FAMILY_NAME; - if (index % 1700 == 0 ) { - familyName = "malformed"; - } - Iterable mutations = ImmutableList.of( - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setValue(testData.get(index).getValue()) - .setFamilyName(familyName)) - .build()); - c.output(KV.of(testData.get(index).getKey(), mutations)); - } - })); - ErrorHandler> errorHandler = p.registerBadRecordErrorHandler(new ErrorSinkTransform()); + PCollection>> mutations = + p.apply(GenerateSequence.from(0).to(numRows)) + .apply( + ParDo.of( + new DoFn>>() { + @ProcessElement + public void processElement(ProcessContext c) { + int index = c.element().intValue(); + + String familyName = COLUMN_FAMILY_NAME; + if (index % 600 == 0) { + familyName = "malformed"; + } + Iterable mutations = + ImmutableList.of( + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(testData.get(index).getValue()) + .setFamilyName(familyName)) + .build()); + c.output(KV.of(testData.get(index).getKey(), mutations)); + } + })); + ErrorHandler> errorHandler = + p.registerBadRecordErrorHandler(new ErrorSinkTransform()); mutations.apply( - BigtableIO.write() - .withProjectId(project) - .withInstanceId(options.getInstanceId()) - .withTableId(failureTableId) - .withErrorHandler(errorHandler)); + BigtableIO.write() + .withProjectId(project) + .withInstanceId(options.getInstanceId()) + .withTableId(tableId) + .withErrorHandler(errorHandler)); errorHandler.close(); - // PAssert.thatSingleton(Objects.requireNonNull(errorHandler.getOutput())).isEqualTo(10L); + PAssert.thatSingleton(Objects.requireNonNull(errorHandler.getOutput())).isEqualTo(2L); p.run(); // Test number of column families and column family name equality - Table table = getTable(failureTableId); + Table table = getTable(tableId); assertThat(table.getColumnFamilies(), Matchers.hasSize(1)); assertThat( table.getColumnFamilies().stream().map((c) -> c.getId()).collect(Collectors.toList()), Matchers.contains(COLUMN_FAMILY_NAME)); // Test table data equality - List> tableData = getTableData(failureTableId); - assertThat(tableData, Matchers.containsInAnyOrder(testData.toArray())); + List> tableData = getTableData(tableId); + assertEquals(998, tableData.size()); } @After public void tearDown() throws Exception { deleteTable(tableId); - deleteTable(failureTableId); if (tableAdminClient != null) { tableAdminClient.close(); } From 6d1cfebfcb3acc979409447a00a6a180e05bdcaa Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Thu, 28 Dec 2023 18:25:11 -0500 Subject: [PATCH 13/21] remove check for oversized mutation --- .../org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java | 6 +----- .../apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java | 2 +- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 81d539d009fb..3c0e56375735 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1333,11 +1333,7 @@ public BiConsumer handleMutationException( KV> record, BoundedWindow window) { return (MutateRowResponse result, Throwable exception) -> { if (exception != null) { - if (exception instanceof IllegalStateException) { - // This case indicates that an individual mutation is invalid, and should not poison the - // batch. It can safely be handled without retrying. - badRecords.add(KV.of(new BigtableWriteException(record, exception), window)); - } else if (exception instanceof NotFoundException + if (exception instanceof NotFoundException && !((NotFoundException) exception).isRetryable()) { // This case, of being an NotFoundException and not retryable, // indicates an issue with the data. However, we can't know if this record diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index 9e6a98fd7630..05d316ae42cf 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -156,7 +156,7 @@ public void processElement(ProcessContext c) { } @Test - public void testE2EBigtableWriteWithFailures() throws Exception { + public void testE2EBigtableWriteWithInvalidColumnFamilyFailures() throws Exception { final int numRows = 1000; final List> testData = generateTableData(numRows); From d606f89e0cc9429d9d22f60df81bf4fccd91c1cd Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Tue, 2 Jan 2024 10:15:14 -0500 Subject: [PATCH 14/21] spotless --- .../transforms/errorhandling/BadRecordRouter.java | 12 ++++++++---- .../apache/beam/sdk/extensions/avro/io/AvroIO.java | 2 -- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java index ec866ea2a616..1e45ad3b6cf8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java @@ -43,7 +43,7 @@ void route( throws Exception; void route( - DoFn.FinishBundleContext c, + DoFn.FinishBundleContext c, RecordT record, @Nullable Coder coder, @Nullable Exception exception, @@ -79,7 +79,7 @@ public void route( @Override public void route( - DoFn.FinishBundleContext c, + DoFn.FinishBundleContext c, RecordT record, @Nullable Coder coder, @Nullable Exception exception, @@ -120,14 +120,18 @@ public void route( @Override public void route( - DoFn.FinishBundleContext c, + DoFn.FinishBundleContext c, RecordT record, @Nullable Coder coder, @Nullable Exception exception, String description, BoundedWindow window) throws Exception { - c.output(BAD_RECORD_TAG,BadRecord.fromExceptionInformation(record, coder, exception, description),window.maxTimestamp(),window); + c.output( + BAD_RECORD_TAG, + BadRecord.fromExceptionInformation(record, coder, exception, description), + window.maxTimestamp(), + window); } } } diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java index 141b0e888a97..dcbed31b48ea 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java @@ -1435,8 +1435,6 @@ public abstract static class TypedWrite abstract @Nullable ErrorHandler getBadRecordErrorHandler(); - abstract @Nullable ErrorHandler getBadRecordErrorHandler(); - abstract @Nullable SerializableFunction getBadRecordMatcher(); /** From ebc857702f6a8899a21eb742dc287dee589b4e22 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Tue, 2 Jan 2024 10:24:19 -0500 Subject: [PATCH 15/21] fix merge error --- .../java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java index dcbed31b48ea..2e4939560ad1 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/io/AvroIO.java @@ -1435,8 +1435,6 @@ public abstract static class TypedWrite abstract @Nullable ErrorHandler getBadRecordErrorHandler(); - abstract @Nullable SerializableFunction getBadRecordMatcher(); - /** * The codec used to encode the blocks in the Avro file. String value drawn from those in * https://avro.apache.org/docs/1.7.7/api/java/org/apache/avro/file/CodecFactory.html From a420d52ff7eb0ef29654e9cddae0fa32b82c3c1a Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Wed, 3 Jan 2024 10:42:16 -0500 Subject: [PATCH 16/21] address comments --- .../errorhandling/BadRecordRouter.java | 19 +++++-------------- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 10 ++++++---- .../sdk/io/gcp/bigtable/BigtableService.java | 9 ++++++--- .../io/gcp/bigtable/BigtableServiceImpl.java | 2 +- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 6 +++--- 5 files changed, 21 insertions(+), 25 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java index 1e45ad3b6cf8..5af562cb608a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecordRouter.java @@ -61,20 +61,7 @@ public void route( @Nullable Exception exception, String description) throws Exception { - if (exception != null) { - throw exception; - } else { - Preconditions.checkArgumentNotNull(record); - String encodedRecord = - BadRecord.Record.builder() - .addHumanReadableJson(record) - .build() - .getHumanReadableJsonRecord(); - if (encodedRecord == null) { - encodedRecord = "Unable to serialize bad record"; - } - throw new RuntimeException("Encountered Bad Record: " + encodedRecord); - } + route(record, exception); } @Override @@ -86,6 +73,10 @@ public void route( String description, BoundedWindow window) throws Exception { + route(record, exception); + } + + private void route(RecordT record, @Nullable Exception exception) throws Exception { if (exception != null) { throw exception; } else { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 3c0e56375735..2c0e7a151e1a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1282,9 +1282,9 @@ private static class BigtableWriterFn private final Coder>> inputCoder; private final BadRecordRouter badRecordRouter; - private Set> badRecords = new HashSet<>(); + private transient Set> badRecords = new HashSet<>(); - private Set> unbatchedWrites = new HashSet<>(); + private transient Set> unbatchedWrites = new HashSet<>(); // Assign serviceEntry in startBundle and clear it in tearDown. @Nullable private BigtableServiceEntry serviceEntry; @@ -1336,7 +1336,9 @@ public BiConsumer handleMutationException( if (exception instanceof NotFoundException && !((NotFoundException) exception).isRetryable()) { // This case, of being an NotFoundException and not retryable, - // indicates an issue with the data. However, we can't know if this record + // indicates an issue with the data. In general, the likely cause is the user trying to + // write to a column family that doesn't exist. The frequency of this is dependent on + // how column families are determined upstream of the io. We can't know if this record // failed, or if it was another record in the batch. Thus, we retry each record // individually. retryIndividualRecord(record, window); @@ -1352,7 +1354,7 @@ public void retryIndividualRecord( try { unbatchedWrites.add( bigtableWriter - .writeRecordWithoutBatching(record) + .writeSingleRecord(record) .whenComplete( (singleMutationResult, singleException) -> { if (singleException != null) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index f39a039e76a2..acc713239c3d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -45,9 +45,12 @@ interface Writer { CompletionStage writeRecord(KV> record) throws IOException; - /** Like above, but will not batch the record. Useful for single record retries */ - CompletionStage writeRecordWithoutBatching( - KV> record) throws IOException; + /** + * Like above, but will not batch the record. Useful for single record retries. writeRecord + * should be preferred for performance reasons. + */ + CompletionStage writeSingleRecord(KV> record) + throws IOException; /** * Closes the writer. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 7513ead4cdd0..9caab8a6b4cc 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -564,7 +564,7 @@ public CompletionStage writeRecord(KV writeRecordWithoutBatching( + public CompletionStage writeSingleRecord( KV> record) { com.google.cloud.bigtable.data.v2.models.Mutation mutation = com.google.cloud.bigtable.data.v2.models.Mutation.fromProtoUnsafe(record.getValue()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 9a12729706a9..a592b06c7b37 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -1930,7 +1930,7 @@ public CompletionStage writeRecord(KV writeRecordWithoutBatching( + public CompletionStage writeSingleRecord( KV> record) throws IOException { return writeRecord(record); } @@ -1957,12 +1957,12 @@ public CompletionStage writeRecord(KV writeRecordWithoutBatching( + public CompletionStage writeSingleRecord( KV> record) throws IOException { if (failureOptions.getFailAtWriteRecord()) { throw new IOException("Fake IOException in writeRecord()"); } - return super.writeRecordWithoutBatching(record); + return super.writeSingleRecord(record); } private final FailureOptions failureOptions; From c630bc11e0882f0365c1846d42b9206b94979503 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Thu, 4 Jan 2024 11:34:58 -0500 Subject: [PATCH 17/21] add BigtableWriteIT test case. it unexpectedly passes --- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index 05d316ae42cf..0ac62ca3f8f1 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -22,6 +22,7 @@ import com.google.api.gax.rpc.ServerStream; import com.google.bigtable.v2.Mutation; +import com.google.bigtable.v2.Mutation.SetCell; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; @@ -31,6 +32,7 @@ import com.google.cloud.bigtable.data.v2.models.Query; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors.FieldDescriptor; import java.io.Serializable; import java.util.ArrayList; import java.util.Date; @@ -45,7 +47,10 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.errorhandling.BadRecord; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; @@ -213,6 +218,49 @@ public void processElement(ProcessContext c) { assertEquals(998, tableData.size()); } + @Test + public void testE2EBigtableWriteWithOversizedRowFailures() throws Exception { + ByteString rowId = ByteString.copyFromUtf8("rowId"); + + final int numRows = 400; + + createEmptyTable(tableId); + + ByteString value = ByteString.copyFrom(new byte[11_000_000]); + + Pipeline p = Pipeline.create(options); + p.apply(GenerateSequence.from(0).to(numRows)) + .apply("ConvertToMutation", + ParDo.of( + new DoFn>() { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(KV.of(rowId,Mutation.newBuilder() + .setSetCell( + SetCell.newBuilder() + .setValue(value) + .setColumnQualifier(ByteString.copyFromUtf8( + String.format("key%09d", c.element()))) + .setFamilyName(COLUMN_FAMILY_NAME)) + .build())); + } + })) + .apply(GroupByKey.create()) + .apply( + BigtableIO.write() + .withProjectId(project) + .withInstanceId(options.getInstanceId()) + .withTableId(tableId)); + p.run(); + + // Test number of column families and column family name equality + Table table = getTable(tableId); + assertThat(table.getColumnFamilies(), Matchers.hasSize(1)); + assertThat( + table.getColumnFamilies().stream().map((c) -> c.getId()).collect(Collectors.toList()), + Matchers.contains(COLUMN_FAMILY_NAME)); + } + @After public void tearDown() throws Exception { deleteTable(tableId); From 5a97efc5781727f2bd4a181d3883eaf13baee887 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Mon, 8 Jan 2024 10:53:46 -0500 Subject: [PATCH 18/21] support routing more malformed mutation types, clean ups based on PR comments --- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 62 ++--- .../io/gcp/bigtable/BigtableServiceImpl.java | 26 +- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 249 ++++++++++++------ 3 files changed, 220 insertions(+), 117 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 2c0e7a151e1a..fbeb2bce8a36 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -25,6 +25,8 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import com.google.api.gax.batching.BatchingException; +import com.google.api.gax.rpc.ApiException; +import com.google.api.gax.rpc.InvalidArgumentException; import com.google.api.gax.rpc.NotFoundException; import com.google.auto.value.AutoValue; import com.google.bigtable.v2.MutateRowResponse; @@ -45,7 +47,6 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Set; -import java.util.concurrent.CompletionStage; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.function.BiConsumer; import org.apache.beam.sdk.PipelineRunner; @@ -1282,9 +1283,7 @@ private static class BigtableWriterFn private final Coder>> inputCoder; private final BadRecordRouter badRecordRouter; - private transient Set> badRecords = new HashSet<>(); - - private transient Set> unbatchedWrites = new HashSet<>(); + private transient Set> badRecords = null; // Assign serviceEntry in startBundle and clear it in tearDown. @Nullable private BigtableServiceEntry serviceEntry; @@ -1317,7 +1316,6 @@ public void startBundle(StartBundleContext c) throws IOException { } badRecords = new HashSet<>(); - unbatchedWrites = new HashSet<>(); } @ProcessElement @@ -1329,12 +1327,11 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except seenWindows.compute(window, (key, count) -> (count != null ? count : 0) + 1); } - public BiConsumer handleMutationException( + private BiConsumer handleMutationException( KV> record, BoundedWindow window) { return (MutateRowResponse result, Throwable exception) -> { if (exception != null) { - if (exception instanceof NotFoundException - && !((NotFoundException) exception).isRetryable()) { + if (isDataException(exception)) { // This case, of being an NotFoundException and not retryable, // indicates an issue with the data. In general, the likely cause is the user trying to // write to a column family that doesn't exist. The frequency of this is dependent on @@ -1349,30 +1346,35 @@ public BiConsumer handleMutationException( }; } - public void retryIndividualRecord( + private void retryIndividualRecord( KV> record, BoundedWindow window) { try { - unbatchedWrites.add( - bigtableWriter - .writeSingleRecord(record) - .whenComplete( - (singleMutationResult, singleException) -> { - if (singleException != null) { - if (singleException instanceof NotFoundException - && !((NotFoundException) singleException).isRetryable()) { - // if we get another NotFoundException, we know this is the bad record. - badRecords.add( - KV.of(new BigtableWriteException(record, singleException), window)); - } else { - failures.add(new BigtableWriteException(record, singleException)); - } - } - })); + bigtableWriter + .writeSingleRecord(record) + .whenComplete( + (singleMutationResult, singleException) -> { + if (singleException != null) { + if (isDataException(singleException)) { + // if we get another NotFoundException, we know this is the bad record. + badRecords.add( + KV.of(new BigtableWriteException(record, singleException), window)); + } else { + failures.add(new BigtableWriteException(record, singleException)); + } + } + }); } catch (IOException e) { throw new RuntimeException(e); } } + private static boolean isDataException(Throwable e) { + if (e instanceof ApiException && !((ApiException) e).isRetryable()) { + return e instanceof NotFoundException || e instanceof InvalidArgumentException; + } + return false; + } + @FinishBundle public void finishBundle(FinishBundleContext c) throws Exception { try { @@ -1392,16 +1394,6 @@ public void finishBundle(FinishBundleContext c) throws Exception { bigtableWriter = null; } - for (CompletionStage unbatchedWrite : unbatchedWrites) { - try { - unbatchedWrite.toCompletableFuture().get(); - } catch (Exception e) { - // ignore exceptions here, they are handled by the .whenComplete as part of the - // completion stage this .get exists purely to make sure all unbatched writes are - // complete. - } - } - for (KV badRecord : badRecords) { try { badRecordRouter.route( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 9caab8a6b4cc..0096c6e9e4fa 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -54,11 +54,13 @@ import java.util.ArrayDeque; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; import java.util.Queue; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; @@ -491,6 +493,8 @@ static class BigtableWriterImpl implements Writer { private Distribution bulkSize = Metrics.distribution("BigTable-" + tableId, "batchSize"); private Distribution latency = Metrics.distribution("BigTable-" + tableId, "batchLatencyMs"); + private Set> unbatchedWrites = new HashSet<>(); + BigtableWriterImpl( BigtableDataClient client, String projectId, @@ -507,6 +511,7 @@ static class BigtableWriterImpl implements Writer { @Override public void close() throws IOException { + IOException exception = null; if (bulkMutation != null) { try { stopwatch.start(); @@ -530,16 +535,29 @@ public void close() throws IOException { // instead of tracking them separately in BigtableIOWriteFn. } catch (TimeoutException e) { // We fail because future.get() timed out - throw new IOException("BulkMutation took too long to close", e); + exception = new IOException("BulkMutation took too long to close", e); } catch (ExecutionException e) { - throw new IOException("Failed to close batch", e.getCause()); + exception = new IOException("Failed to close batch", e.getCause()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); // We fail since close() operation was interrupted. - throw new IOException(e); + exception = new IOException(e); } bulkMutation = null; } + // ensure any singleton writes are finished + for (CompletionStage unbatchedWrite : unbatchedWrites) { + try { + unbatchedWrite.toCompletableFuture().get(); + } catch (Exception e) { + // ignore exceptions here, they are handled by the .whenComplete as part of the + // completion stage this .get exists purely to make sure all unbatched writes are + // complete. + } + } + if (exception != null) { + throw exception; + } } @Override @@ -575,11 +593,11 @@ public CompletionStage writeSingleRecord( CompletableFuture result = new CompletableFuture<>(); - outstandingMutations += 1; Futures.addCallback( new VendoredListenableFutureAdapter<>(client.mutateRowAsync(rowMutation)), new WriteMutationCallback(result, serviceCallMetric), directExecutor()); + unbatchedWrites.add(result); return result; } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index 0ac62ca3f8f1..ebff9ad7f74f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -22,7 +22,6 @@ import com.google.api.gax.rpc.ServerStream; import com.google.bigtable.v2.Mutation; -import com.google.bigtable.v2.Mutation.SetCell; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; @@ -32,7 +31,6 @@ import com.google.cloud.bigtable.data.v2.models.Query; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.protobuf.ByteString; -import com.google.protobuf.Descriptors.FieldDescriptor; import java.io.Serializable; import java.util.ArrayList; import java.util.Date; @@ -47,10 +45,7 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.errorhandling.BadRecord; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler; @@ -165,33 +160,173 @@ public void testE2EBigtableWriteWithInvalidColumnFamilyFailures() throws Excepti final int numRows = 1000; final List> testData = generateTableData(numRows); + failureTest( + numRows, + new DoFn>>() { + @ProcessElement + public void processElement(ProcessContext c) { + int index = c.element().intValue(); + + String familyName = COLUMN_FAMILY_NAME; + if (index % 600 == 0) { + familyName = "malformed"; + } + Iterable mutations = + ImmutableList.of( + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(testData.get(index).getValue()) + .setFamilyName(familyName)) + .build()); + c.output(KV.of(testData.get(index).getKey(), mutations)); + } + }); + } + + @Test + public void testE2EBigtableWriteWithEmptyMutationFailures() throws Exception { + final int numRows = 1000; + final List> testData = generateTableData(numRows); + failureTest( + numRows, + new DoFn>>() { + @ProcessElement + public void processElement(ProcessContext c) { + int index = c.element().intValue(); + Iterable mutations; + if (index % 600 == 0) { + mutations = + ImmutableList.of( + Mutation.newBuilder().setSetCell(Mutation.SetCell.newBuilder()).build()); + } else { + mutations = + ImmutableList.of( + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(testData.get(index).getValue()) + .setFamilyName(COLUMN_FAMILY_NAME)) + .build()); + } + c.output(KV.of(testData.get(index).getKey(), mutations)); + } + }); + } + + @Test + public void testE2EBigtableWriteWithEmptyRowFailures() throws Exception { + final int numRows = 1000; + final List> testData = generateTableData(numRows); + + failureTest( + numRows, + new DoFn>>() { + @ProcessElement + public void processElement(ProcessContext c) { + int index = c.element().intValue(); + Iterable mutations = + ImmutableList.of( + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(testData.get(index).getValue()) + .setFamilyName(COLUMN_FAMILY_NAME)) + .build()); + + ByteString rowKey = testData.get(index).getKey(); + if (index % 600 == 0) { + rowKey = ByteString.empty(); + } + c.output(KV.of(rowKey, mutations)); + } + }); + } + + @Test + public void testE2EBigtableWriteWithInvalidTimestampFailures() throws Exception { + final int numRows = 1000; + final List> testData = generateTableData(numRows); + + failureTest( + numRows, + new DoFn>>() { + @ProcessElement + public void processElement(ProcessContext c) { + int index = c.element().intValue(); + Iterable mutations; + if (index % 600 == 0) { + mutations = + ImmutableList.of( + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(testData.get(index).getValue()) + .setFamilyName(COLUMN_FAMILY_NAME) + .setTimestampMicros(-2)) + .build()); + } else { + mutations = + ImmutableList.of( + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(testData.get(index).getValue()) + .setFamilyName(COLUMN_FAMILY_NAME)) + .build()); + } + + c.output(KV.of(testData.get(index).getKey(), mutations)); + } + }); + } + + @Test + public void testE2EBigtableWriteWithOversizedQualifierFailures() throws Exception { + final int numRows = 1000; + final List> testData = generateTableData(numRows); + + failureTest( + numRows, + new DoFn>>() { + @ProcessElement + public void processElement(ProcessContext c) { + int index = c.element().intValue(); + Iterable mutations; + if (index % 600 == 0) { + mutations = + ImmutableList.of( + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(testData.get(index).getValue()) + .setFamilyName(COLUMN_FAMILY_NAME) + .setColumnQualifier(ByteString.copyFrom(new byte[20_000]))) + .build()); + } else { + mutations = + ImmutableList.of( + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(testData.get(index).getValue()) + .setFamilyName(COLUMN_FAMILY_NAME)) + .build()); + } + + c.output(KV.of(testData.get(index).getKey(), mutations)); + } + }); + } + + public void failureTest(int numRows, DoFn>> dataGenerator) + throws Exception { + createEmptyTable(tableId); Pipeline p = Pipeline.create(options); PCollection>> mutations = - p.apply(GenerateSequence.from(0).to(numRows)) - .apply( - ParDo.of( - new DoFn>>() { - @ProcessElement - public void processElement(ProcessContext c) { - int index = c.element().intValue(); - - String familyName = COLUMN_FAMILY_NAME; - if (index % 600 == 0) { - familyName = "malformed"; - } - Iterable mutations = - ImmutableList.of( - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setValue(testData.get(index).getValue()) - .setFamilyName(familyName)) - .build()); - c.output(KV.of(testData.get(index).getKey(), mutations)); - } - })); + p.apply(GenerateSequence.from(0).to(numRows)).apply(ParDo.of(dataGenerator)); ErrorHandler> errorHandler = p.registerBadRecordErrorHandler(new ErrorSinkTransform()); mutations.apply( @@ -218,52 +353,10 @@ public void processElement(ProcessContext c) { assertEquals(998, tableData.size()); } - @Test - public void testE2EBigtableWriteWithOversizedRowFailures() throws Exception { - ByteString rowId = ByteString.copyFromUtf8("rowId"); - - final int numRows = 400; - - createEmptyTable(tableId); - - ByteString value = ByteString.copyFrom(new byte[11_000_000]); - - Pipeline p = Pipeline.create(options); - p.apply(GenerateSequence.from(0).to(numRows)) - .apply("ConvertToMutation", - ParDo.of( - new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(KV.of(rowId,Mutation.newBuilder() - .setSetCell( - SetCell.newBuilder() - .setValue(value) - .setColumnQualifier(ByteString.copyFromUtf8( - String.format("key%09d", c.element()))) - .setFamilyName(COLUMN_FAMILY_NAME)) - .build())); - } - })) - .apply(GroupByKey.create()) - .apply( - BigtableIO.write() - .withProjectId(project) - .withInstanceId(options.getInstanceId()) - .withTableId(tableId)); - p.run(); - - // Test number of column families and column family name equality - Table table = getTable(tableId); - assertThat(table.getColumnFamilies(), Matchers.hasSize(1)); - assertThat( - table.getColumnFamilies().stream().map((c) -> c.getId()).collect(Collectors.toList()), - Matchers.contains(COLUMN_FAMILY_NAME)); - } - @After public void tearDown() throws Exception { - deleteTable(tableId); + System.out.println("Table is:" + tableId); + // deleteTable(tableId); if (tableAdminClient != null) { tableAdminClient.close(); } @@ -313,10 +406,10 @@ private List> getTableData(String tableId) { return tableData; } - /** Helper function to delete a table. */ - private void deleteTable(String tableId) { - if (tableAdminClient != null) { - tableAdminClient.deleteTable(tableId); - } - } + // /** Helper function to delete a table. */ + // private void deleteTable(String tableId) { + // if (tableAdminClient != null) { + // tableAdminClient.deleteTable(tableId); + // } + // } } From 8963686bcc09ddee1730f681e51a51e5d75330f9 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Tue, 9 Jan 2024 11:43:49 -0500 Subject: [PATCH 19/21] make retrys non-async to remove race condition causing data loss --- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 25 ++++------ .../sdk/io/gcp/bigtable/BigtableService.java | 4 +- .../io/gcp/bigtable/BigtableServiceImpl.java | 47 +++++++------------ .../sdk/io/gcp/bigtable/BigtableIOTest.java | 13 +++-- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 15 +++--- 5 files changed, 42 insertions(+), 62 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index fbeb2bce8a36..0885f937219f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1349,22 +1349,15 @@ private BiConsumer handleMutationException( private void retryIndividualRecord( KV> record, BoundedWindow window) { try { - bigtableWriter - .writeSingleRecord(record) - .whenComplete( - (singleMutationResult, singleException) -> { - if (singleException != null) { - if (isDataException(singleException)) { - // if we get another NotFoundException, we know this is the bad record. - badRecords.add( - KV.of(new BigtableWriteException(record, singleException), window)); - } else { - failures.add(new BigtableWriteException(record, singleException)); - } - } - }); - } catch (IOException e) { - throw new RuntimeException(e); + bigtableWriter.writeSingleRecord(record); + } catch (ApiException e){ + if (isDataException(e)) { + // if we get another NotFoundException, we know this is the bad record. + badRecords.add( + KV.of(new BigtableWriteException(record, e), window)); + } else { + failures.add(new BigtableWriteException(record, e)); + } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index acc713239c3d..3a3de5622cd2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import com.google.api.gax.rpc.ApiException; import com.google.bigtable.v2.MutateRowResponse; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; @@ -49,8 +50,7 @@ CompletionStage writeRecord(KV * Like above, but will not batch the record. Useful for single record retries. writeRecord * should be preferred for performance reasons. */ - CompletionStage writeSingleRecord(KV> record) - throws IOException; + void writeSingleRecord(KV> record) throws ApiException; /** * Closes the writer. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 0096c6e9e4fa..883db5c6dd51 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -24,6 +24,7 @@ import com.google.api.gax.batching.BatchingException; import com.google.api.gax.grpc.GrpcCallContext; import com.google.api.gax.retrying.RetrySettings; +import com.google.api.gax.rpc.ApiException; import com.google.api.gax.rpc.ResponseObserver; import com.google.api.gax.rpc.StreamController; import com.google.bigtable.v2.Cell; @@ -493,8 +494,6 @@ static class BigtableWriterImpl implements Writer { private Distribution bulkSize = Metrics.distribution("BigTable-" + tableId, "batchSize"); private Distribution latency = Metrics.distribution("BigTable-" + tableId, "batchLatencyMs"); - private Set> unbatchedWrites = new HashSet<>(); - BigtableWriterImpl( BigtableDataClient client, String projectId, @@ -511,7 +510,6 @@ static class BigtableWriterImpl implements Writer { @Override public void close() throws IOException { - IOException exception = null; if (bulkMutation != null) { try { stopwatch.start(); @@ -535,29 +533,16 @@ public void close() throws IOException { // instead of tracking them separately in BigtableIOWriteFn. } catch (TimeoutException e) { // We fail because future.get() timed out - exception = new IOException("BulkMutation took too long to close", e); + throw new IOException("BulkMutation took too long to close", e); } catch (ExecutionException e) { - exception = new IOException("Failed to close batch", e.getCause()); + throw new IOException("Failed to close batch", e.getCause()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); // We fail since close() operation was interrupted. - exception = new IOException(e); + throw new IOException(e); } bulkMutation = null; } - // ensure any singleton writes are finished - for (CompletionStage unbatchedWrite : unbatchedWrites) { - try { - unbatchedWrite.toCompletableFuture().get(); - } catch (Exception e) { - // ignore exceptions here, they are handled by the .whenComplete as part of the - // completion stage this .get exists purely to make sure all unbatched writes are - // complete. - } - } - if (exception != null) { - throw exception; - } } @Override @@ -582,8 +567,8 @@ public CompletionStage writeRecord(KV writeSingleRecord( - KV> record) { + public void writeSingleRecord( + KV> record) throws ApiException { com.google.cloud.bigtable.data.v2.models.Mutation mutation = com.google.cloud.bigtable.data.v2.models.Mutation.fromProtoUnsafe(record.getValue()); @@ -591,14 +576,18 @@ public CompletionStage writeSingleRecord( ServiceCallMetric serviceCallMetric = createServiceCallMetric(); - CompletableFuture result = new CompletableFuture<>(); - - Futures.addCallback( - new VendoredListenableFutureAdapter<>(client.mutateRowAsync(rowMutation)), - new WriteMutationCallback(result, serviceCallMetric), - directExecutor()); - unbatchedWrites.add(result); - return result; + try { + client.mutateRow(rowMutation); + serviceCallMetric.call("ok"); + } catch (ApiException e){ + if (e.getCause() instanceof StatusRuntimeException) { + serviceCallMetric.call( + ((StatusRuntimeException) e.getCause()).getStatus().getCode().value()); + } else { + serviceCallMetric.call("unknown"); + } + throw e; + } } private ServiceCallMetric createServiceCallMetric() { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index a592b06c7b37..aff982a1fe75 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -40,6 +40,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import com.google.api.gax.rpc.ApiException; import com.google.auto.value.AutoValue; import com.google.bigtable.v2.Cell; import com.google.bigtable.v2.Column; @@ -1930,9 +1931,8 @@ public CompletionStage writeRecord(KV writeSingleRecord( - KV> record) throws IOException { - return writeRecord(record); + public void writeSingleRecord( + KV> record) { } @Override @@ -1957,12 +1957,11 @@ public CompletionStage writeRecord(KV writeSingleRecord( - KV> record) throws IOException { + public void writeSingleRecord( + KV> record) throws ApiException { if (failureOptions.getFailAtWriteRecord()) { - throw new IOException("Fake IOException in writeRecord()"); + throw new RuntimeException("Fake RuntimeException in writeRecord()"); } - return super.writeSingleRecord(record); } private final FailureOptions failureOptions; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index ebff9ad7f74f..723bf7e7b190 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -355,8 +355,7 @@ public void failureTest(int numRows, DoFn> getTableData(String tableId) { return tableData; } - // /** Helper function to delete a table. */ - // private void deleteTable(String tableId) { - // if (tableAdminClient != null) { - // tableAdminClient.deleteTable(tableId); - // } - // } + /** Helper function to delete a table. */ + private void deleteTable(String tableId) { + if (tableAdminClient != null) { + tableAdminClient.deleteTable(tableId); + } + } } From 3086274274bfd0ea375f1e59c031f412fc8f7513 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Tue, 9 Jan 2024 12:00:33 -0500 Subject: [PATCH 20/21] spotless --- .../org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java | 5 ++--- .../beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java | 7 ++----- .../apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java | 7 ++----- 3 files changed, 6 insertions(+), 13 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 0885f937219f..6ad5443bb79a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1350,11 +1350,10 @@ private void retryIndividualRecord( KV> record, BoundedWindow window) { try { bigtableWriter.writeSingleRecord(record); - } catch (ApiException e){ + } catch (ApiException e) { if (isDataException(e)) { // if we get another NotFoundException, we know this is the bad record. - badRecords.add( - KV.of(new BigtableWriteException(record, e), window)); + badRecords.add(KV.of(new BigtableWriteException(record, e), window)); } else { failures.add(new BigtableWriteException(record, e)); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 883db5c6dd51..7537a16a188b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -55,13 +55,11 @@ import java.util.ArrayDeque; import java.util.ArrayList; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; import java.util.Queue; -import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; @@ -567,8 +565,7 @@ public CompletionStage writeRecord(KV> record) throws ApiException { + public void writeSingleRecord(KV> record) throws ApiException { com.google.cloud.bigtable.data.v2.models.Mutation mutation = com.google.cloud.bigtable.data.v2.models.Mutation.fromProtoUnsafe(record.getValue()); @@ -579,7 +576,7 @@ public void writeSingleRecord( try { client.mutateRow(rowMutation); serviceCallMetric.call("ok"); - } catch (ApiException e){ + } catch (ApiException e) { if (e.getCause() instanceof StatusRuntimeException) { serviceCallMetric.call( ((StatusRuntimeException) e.getCause()).getStatus().getCode().value()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index aff982a1fe75..bffca8652089 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -1931,9 +1931,7 @@ public CompletionStage writeRecord(KV> record) { - } + public void writeSingleRecord(KV> record) {} @Override public void close() {} @@ -1957,8 +1955,7 @@ public CompletionStage writeRecord(KV> record) throws ApiException { + public void writeSingleRecord(KV> record) throws ApiException { if (failureOptions.getFailAtWriteRecord()) { throw new RuntimeException("Fake RuntimeException in writeRecord()"); } From 505319dd93f156c50fd4986d1a2bba5d75d74ed1 Mon Sep 17 00:00:00 2001 From: johnjcasey Date: Tue, 9 Jan 2024 16:05:52 -0500 Subject: [PATCH 21/21] update comment on identifying data exception --- .../apache/beam/sdk/io/gcp/bigtable/BigtableIO.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 6ad5443bb79a..da989280bd4a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1332,12 +1332,6 @@ private BiConsumer handleMutationException( return (MutateRowResponse result, Throwable exception) -> { if (exception != null) { if (isDataException(exception)) { - // This case, of being an NotFoundException and not retryable, - // indicates an issue with the data. In general, the likely cause is the user trying to - // write to a column family that doesn't exist. The frequency of this is dependent on - // how column families are determined upstream of the io. We can't know if this record - // failed, or if it was another record in the batch. Thus, we retry each record - // individually. retryIndividualRecord(record, window); } else { failures.add(new BigtableWriteException(record, exception)); @@ -1360,6 +1354,13 @@ private void retryIndividualRecord( } } + // This method checks if an exception is the result of an error in the data. + // We first check if the exception is retryable, because if it is, we want to retry it via the + // runner. If the method is retryable, we check if it is a NotFoundException, or if it's an + // InvalidArgumentException. A NotFoundException likely means that the mutation is trying to + // write to a column family that doesn't exist. An InvalidArgumentException means that the + // mutation itself is invalid, with either an empty row key, invalid timestamp (ts <= -2), + // an empty mutation, or a column qualifier that is too large. private static boolean isDataException(Throwable e) { if (e instanceof ApiException && !((ApiException) e).isRetryable()) { return e instanceof NotFoundException || e instanceof InvalidArgumentException;