You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2021/06/29 21:29:05 UTC

[GitHub] [beam] pabloem commented on a change in pull request #15013: [BEAM-12497] BigQueryIO returns successful rows for streaming inserts

pabloem commented on a change in pull request #15013:
URL: https://github.com/apache/beam/pull/15013#discussion_r660922229



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java
##########
@@ -191,23 +192,25 @@ private BatchedStreamingWrite(
   }
 
   @Override
-  public PCollection<ErrorT> expand(PCollection<KV<String, TableRowInfo<ElementT>>> input) {
+  public PCollectionTuple expand(PCollection<KV<String, TableRowInfo<ElementT>>> input) {
     return batchViaStateful
         ? input.apply(new ViaStateful())
         : input.apply(new ViaBundleFinalization());
   }
 
   private class ViaBundleFinalization
-      extends PTransform<PCollection<KV<String, TableRowInfo<ElementT>>>, PCollection<ErrorT>> {
+      extends PTransform<PCollection<KV<String, TableRowInfo<ElementT>>>, PCollectionTuple> {
     @Override
-    public PCollection<ErrorT> expand(PCollection<KV<String, TableRowInfo<ElementT>>> input) {
+    public PCollectionTuple expand(PCollection<KV<String, TableRowInfo<ElementT>>> input) {
       PCollectionTuple result =
           input.apply(
               ParDo.of(new BatchAndInsertElements())
-                  .withOutputTags(mainOutputTag, TupleTagList.of(failedOutputTag)));
+                  .withOutputTags(
+                      mainOutputTag, TupleTagList.of(failedOutputTag).and(SUCCESSFUL_ROWS_TAG)));
       PCollection<ErrorT> failedInserts = result.get(failedOutputTag);
       failedInserts.setCoder(failedOutputCoder);

Review comment:
       Done

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java
##########
@@ -191,23 +192,25 @@ private BatchedStreamingWrite(
   }
 
   @Override
-  public PCollection<ErrorT> expand(PCollection<KV<String, TableRowInfo<ElementT>>> input) {
+  public PCollectionTuple expand(PCollection<KV<String, TableRowInfo<ElementT>>> input) {
     return batchViaStateful
         ? input.apply(new ViaStateful())
         : input.apply(new ViaBundleFinalization());
   }
 
   private class ViaBundleFinalization
-      extends PTransform<PCollection<KV<String, TableRowInfo<ElementT>>>, PCollection<ErrorT>> {
+      extends PTransform<PCollection<KV<String, TableRowInfo<ElementT>>>, PCollectionTuple> {
     @Override
-    public PCollection<ErrorT> expand(PCollection<KV<String, TableRowInfo<ElementT>>> input) {
+    public PCollectionTuple expand(PCollection<KV<String, TableRowInfo<ElementT>>> input) {
       PCollectionTuple result =
           input.apply(
               ParDo.of(new BatchAndInsertElements())
-                  .withOutputTags(mainOutputTag, TupleTagList.of(failedOutputTag)));
+                  .withOutputTags(
+                      mainOutputTag, TupleTagList.of(failedOutputTag).and(SUCCESSFUL_ROWS_TAG)));

Review comment:
       Yeah, it's curious.

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java
##########
@@ -306,10 +311,13 @@ public void finishBundle(FinishBundleContext context) throws Exception {
                       ShardedKey.Coder.of(StringUtf8Coder.of()), IterableCoder.of(valueCoder)))
               .apply(
                   ParDo.of(new InsertBatchedElements())
-                      .withOutputTags(mainOutputTag, TupleTagList.of(failedOutputTag)));
+                      .withOutputTags(
+                          mainOutputTag,
+                          TupleTagList.of(failedOutputTag).and(SUCCESSFUL_ROWS_TAG)));
       PCollection<ErrorT> failedInserts = result.get(failedOutputTag);
       failedInserts.setCoder(failedOutputCoder);

Review comment:
       Done.

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java
##########
@@ -305,20 +316,22 @@ public WriteResult expand(PCollection<KV<TableDestination, ElementT>> input) {
 
       // Auto-sharding is achieved via GroupIntoBatches.WithShardedKey transform which groups and at
       // the same time batches the TableRows to be inserted to BigQuery.
-      return unshardedTagged.apply(
-          "StreamingWrite",
-          new BatchedStreamingWrite<>(
-                  bigQueryServices,
-                  retryPolicy,
-                  failedInsertsTag,
-                  coder,
-                  errorContainer,
-                  skipInvalidRows,
-                  ignoreUnknownValues,
-                  ignoreInsertIds,
-                  toTableRow,
-                  toFailsafeTableRow)
-              .viaStateful());
+      PCollectionTuple result =

Review comment:
       Removed. Thanks!

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java
##########
@@ -334,44 +347,47 @@ public WriteResult expand(PCollection<KV<TableDestination, ElementT>> input) {
                       ShardedKeyCoder.of(StringUtf8Coder.of()),
                       TableRowInfoCoder.of(elementCoder)));
 
-      return shardedTagged
-          .apply(Reshuffle.of())
-          // Put in the global window to ensure that DynamicDestinations side inputs are accessed
-          // correctly.
-          .apply(
-              "GlobalWindow",
-              Window.<KV<ShardedKey<String>, TableRowInfo<ElementT>>>into(new GlobalWindows())
-                  .triggering(DefaultTrigger.of())
-                  .discardingFiredPanes())
-          .apply(
-              "StripShardId",
-              MapElements.via(
-                  new SimpleFunction<
-                      KV<ShardedKey<String>, TableRowInfo<ElementT>>,
-                      KV<String, TableRowInfo<ElementT>>>() {
-                    @Override
-                    public KV<String, TableRowInfo<ElementT>> apply(
-                        KV<ShardedKey<String>, TableRowInfo<ElementT>> input) {
-                      return KV.of(input.getKey().getKey(), input.getValue());
-                    }
-                  }))
-          .setCoder(KvCoder.of(StringUtf8Coder.of(), TableRowInfoCoder.of(elementCoder)))
-          // Also batch the TableRows in a best effort manner via bundle finalization before
-          // inserting to BigQuery.
-          .apply(
-              "StreamingWrite",
-              new BatchedStreamingWrite<>(
-                      bigQueryServices,
-                      retryPolicy,
-                      failedInsertsTag,
-                      coder,
-                      errorContainer,
-                      skipInvalidRows,
-                      ignoreUnknownValues,
-                      ignoreInsertIds,
-                      toTableRow,
-                      toFailsafeTableRow)
-                  .viaDoFnFinalization());
+      PCollectionTuple result =

Review comment:
       undone. Thanks!

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
##########
@@ -844,6 +903,7 @@ public void deleteDataset(String projectId, String datasetId)
                 + "as many elements as rowList");
       }
 
+      final Set<Integer> failedIndices = new HashSet<>();

Review comment:
       if all elements failed in a single request, we would have 500 failed indices by default (this setting can be tweaked by the user, but it would usually be 500 or less) - so I think it's an acceptable size. WDYT?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org