prodriguezdefino commented on code in PR #32529:
URL: https://github.com/apache/beam/pull/32529#discussion_r1777920699


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java:
##########
@@ -344,14 +358,39 @@ public void process(ProcessContext c) {}
 
     private static class RowDynamicDestinations extends 
DynamicDestinations<Row, String> {
       Schema schema;
+      String fixedDestination = null;
+      List<String> primaryKey = null;
 
       RowDynamicDestinations(Schema schema) {
         this.schema = schema;
       }
 
+      RowDynamicDestinations withFixedDestination(String destination) {
+        this.fixedDestination = destination;
+        return this;
+      }
+
+      RowDynamicDestinations withPrimaryKey(List<String> primaryKey) {
+        this.primaryKey = primaryKey;
+        return this;
+      }
+
       @Override
       public String getDestination(ValueInSingleWindow<Row> element) {
-        return element.getValue().getString("destination");
+        return fixedDestination != null
+            ? fixedDestination
+            : element.getValue().getString("destination");
+      }
+
+      @Override
+      public TableConstraints getTableConstraints(String destination) {
+        return Optional.ofNullable(this.primaryKey)
+            .filter(pk -> !pk.isEmpty())
+            .map(
+                pk ->
+                    new TableConstraints()
+                        .setPrimaryKey(new 
TableConstraints.PrimaryKey().setColumns(pk)))
+            .orElse(null);

Review Comment:
   @ahmedabu98 I kept the modifications on the Row Dynamic Destinations given 
the access constraints on the delegating DynDest and that the shared check only 
applies to the case when a dynamic destination is not added into the BQIO. 
   
   Let me know if you want me to make the change on the `expand` logic for BQIO.



-- 
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-unsubscr...@beam.apache.org

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

Reply via email to