kennknowles commented on code in PR #31490:
URL: https://github.com/apache/beam/pull/31490#discussion_r1635258681


##########
runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RedistributeByKeyOverrideFactory.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.runners.dataflow;
+
+import java.util.Collections;
+import org.apache.beam.sdk.runners.AppliedPTransform;
+import org.apache.beam.sdk.runners.PTransformOverrideFactory;
+import 
org.apache.beam.sdk.runners.PTransformOverrideFactory.PTransformReplacement;
+import org.apache.beam.sdk.transforms.DataflowGroupByKey;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.Element;
+import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
+import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Redistribute.RedistributeByKey;
+import org.apache.beam.sdk.transforms.Reify;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger;
+import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.IdentityWindowFn;
+import org.apache.beam.sdk.util.construction.PTransformReplacements;
+import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.ValueInSingleWindow;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.joda.time.Duration;
+
+class RedistributeByKeyOverrideFactory<K, V>
+    extends SingleInputOutputOverrideFactory<
+        PCollection<KV<K, V>>, PCollection<KV<K, V>>, RedistributeByKey<K, V>> 
{
+
+  @Override
+  public PTransformReplacement<PCollection<KV<K, V>>, PCollection<KV<K, V>>>
+      getReplacementTransform(
+          AppliedPTransform<PCollection<KV<K, V>>, PCollection<KV<K, V>>, 
RedistributeByKey<K, V>>
+              transform) {
+    return PTransformOverrideFactory.PTransformReplacement.of(
+        PTransformReplacements.getSingletonMainInput(transform),
+        new DataflowRedistributeByKey<>(transform.getTransform()));
+  }
+
+  /** Specialized implementation of {@link RedistributeByKey} for Dataflow 
pipelines. */
+  private static class DataflowRedistributeByKey<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, V>>> {
+
+    private final RedistributeByKey<K, V> originalTransform;
+
+    private DataflowRedistributeByKey(RedistributeByKey<K, V> 
originalTransform) {
+      this.originalTransform = originalTransform;
+    }
+
+    @Override
+    public PCollection<KV<K, V>> expand(PCollection<KV<K, V>> input) {
+      WindowingStrategy<?, ?> originalStrategy = input.getWindowingStrategy();
+      Window<KV<K, V>> rewindow =
+          Window.<KV<K, V>>into(
+                  new 
IdentityWindowFn<>(originalStrategy.getWindowFn().windowCoder()))
+              .triggering(new ReshuffleTrigger<>())
+              .discardingFiredPanes()
+              .withTimestampCombiner(TimestampCombiner.EARLIEST)
+              
.withAllowedLateness(Duration.millis(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()));
+
+      PCollection<KV<K, ValueInSingleWindow<V>>> reified =
+          input
+              .apply("SetIdentityWindow", rewindow)
+              .apply("ReifyOriginalMetadata", Reify.windowsInValue());
+
+      @SuppressWarnings("nullness") // Cannot figure out how to make this 
typecheck
+      PCollection<KV<K, Iterable<ValueInSingleWindow<V>>>> grouped =
+          reified.apply(
+              originalTransform != null && 
originalTransform.getAllowDuplicates()

Review Comment:
   Maybe try assigning this to a local variable with an explicit type and see 
what you learn



-- 
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