reuvenlax commented on a change in pull request #15731:
URL: https://github.com/apache/beam/pull/15731#discussion_r740639484



##########
File path: 
runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
##########
@@ -716,13 +716,13 @@ private void addOutput(String name, PValue value, 
Coder<?> valueCoder) {
       translator.registerOutputName(value, name);
 
       // If the output requires runner determined sharding, also append 
necessary input properties.
-      if (value instanceof PCollection
-          && 
translator.runner.doesPCollectionRequireAutoSharding((PCollection<?>) value)) {
-        addInput(PropertyNames.ALLOWS_SHARDABLE_STATE, "true");
-        // Currently we only allow auto-sharding to be enabled through the 
GroupIntoBatches
-        // transform. So we also add the following property which 
GroupIntoBatchesDoFn has, to allow
-        // the backend to perform graph optimization.
-        addInput(PropertyNames.PRESERVES_KEYS, "true");
+      if (value instanceof PCollection) {
+        if 
(translator.runner.doesPCollectionRequireAutoSharding((PCollection<?>) value)) {
+          addInput(PropertyNames.ALLOWS_SHARDABLE_STATE, "true");
+        }
+        if (translator.runner.doesPCollectionPreserveKeys((PCollection<?>) 
value)) {
+          addInput(PropertyNames.PRESERVES_KEYS, "true");

Review comment:
       Would like to generalize and also make this a model feature. There are 
some other things worth adding as well, such as PARTITIONED_BY_KEY (right now 
the model doesn't really know if a PCollection<KV> is already partitioned by 
key. runners can infer it by looking for a GroupByKey followed by only 
key-preserving operations. however what is a source produces items already 
partitioned by key?)




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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to