je-ik commented on code in PR #30545:
URL: https://github.com/apache/beam/pull/30545#discussion_r1521012592


##########
sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * 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;
+
+import static org.apache.beam.sdk.TestUtils.KvMatcher.isKv;
+import static 
org.apache.beam.sdk.util.construction.PTransformTranslation.REDISTRIBUTE_ARBITRARILY_URN;
+import static 
org.apache.beam.sdk.util.construction.PTransformTranslation.REDISTRIBUTE_BY_KEY_URN;
+import static org.apache.beam.sdk.values.TypeDescriptors.integers;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+import static 
org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+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;
+import org.apache.beam.sdk.testing.ValidatesRunner;
+import org.apache.beam.sdk.transforms.Redistribute.AssignShardFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.construction.PTransformTranslation;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.ValueInSingleWindow;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link Redistribute}. */
+@RunWith(JUnit4.class)
+public class RedistributeTest implements Serializable {

Review Comment:
   There seems to be no test for `Redistribute.byKey`. What is the invariant 
there? I suppose that (as far as only the test is concerned), we could do 
`Redistribute.byKey` -> `ParDo` and verify elements with identical keys are 
processed not processed in parallel?



##########
runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java:
##########
@@ -290,6 +296,26 @@ public FlinkPortablePipelineTranslator.Executor translate(
     return context;
   }
 
+  private static <K, V> void translateRedistributeByKey(
+      PTransformNode transform, RunnerApi.Pipeline pipeline, 
BatchTranslationContext context) {
+    DataSet<WindowedValue<KV<K, V>>> inputDataSet =
+        context.getDataSetOrThrow(
+            
Iterables.getOnlyElement(transform.getTransform().getInputsMap().values()));
+    context.addDataSet(
+        
Iterables.getOnlyElement(transform.getTransform().getOutputsMap().values()),
+        inputDataSet.rebalance());

Review Comment:
   I wonder if it is OK to translate `REDISTRIBUTE_BY_KEY` and 
`REDISTRIBUTE_ARBITRARILY` the same way. Do the semantics actually differ? If 
yes, maybe semantically correct would be 
[DataSet#groupBy](https://nightlies.apache.org/flink/flink-docs-release-1.16/api/java/org/apache/flink/api/java/DataSet.html#groupBy-org.apache.flink.api.java.functions.KeySelector-)
 -> 
[DataSet#reduceGroup(identity)](https://nightlies.apache.org/flink/flink-docs-release-1.16/api/java/org/apache/flink/api/java/operators/UnsortedGrouping.html#reduceGroup-org.apache.flink.api.common.functions.GroupReduceFunction-)?



##########
runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java:
##########
@@ -921,6 +926,38 @@ public void translateNode(
     }
   }
 
+  private static class RedistributeByKeyTranslatorStreaming<K, InputT>
+      extends FlinkStreamingPipelineTranslator.StreamTransformTranslator<
+          PTransform<PCollection<KV<K, InputT>>, PCollection<KV<K, InputT>>>> {
+
+    @Override
+    public void translateNode(
+        PTransform<PCollection<KV<K, InputT>>, PCollection<KV<K, InputT>>> 
transform,
+        FlinkStreamingTranslationContext context) {
+
+      DataStream<WindowedValue<KV<K, InputT>>> inputDataSet =
+          context.getInputDataStream(context.getInput(transform));
+
+      context.setOutputDataStream(context.getOutput(transform), 
inputDataSet.rebalance());

Review Comment:
   `keyBy`?



##########
runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java:
##########
@@ -283,6 +287,24 @@ private <K, V> void translateReshuffle(
         Iterables.getOnlyElement(transform.getOutputsMap().values()), 
inputDataStream.rebalance());
   }
 
+  private <K, V> void translateRedistributeByKey(
+      String id, RunnerApi.Pipeline pipeline, StreamingTranslationContext 
context) {
+    RunnerApi.PTransform transform = 
pipeline.getComponents().getTransformsOrThrow(id);
+    DataStream<WindowedValue<KV<K, V>>> inputDataStream =
+        
context.getDataStreamOrThrow(Iterables.getOnlyElement(transform.getInputsMap().values()));
+    context.addDataStream(
+        Iterables.getOnlyElement(transform.getOutputsMap().values()), 
inputDataStream.rebalance());

Review Comment:
   This might want to use `DataStream#keyBy`



##########
runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java:
##########
@@ -422,6 +428,73 @@ public void translateNode(
     }
   }
 
+  private static class RedistributeByKeyTranslatorBatch<K, InputT>
+      implements FlinkBatchPipelineTranslator.BatchTransformTranslator<
+          Redistribute.RedistributeByKey<K, InputT>> {
+
+    @Override
+    public void translateNode(
+        Redistribute.RedistributeByKey<K, InputT> transform, 
FlinkBatchTranslationContext context) {
+      final DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
+          context.getInputDataSet(context.getInput(transform));
+      // Construct an instance of CoderTypeInformation which contains the 
pipeline options.
+      // This will be used to initialized FileSystems.
+      final CoderTypeInformation<WindowedValue<KV<K, InputT>>> outputType =
+          ((CoderTypeInformation<WindowedValue<KV<K, InputT>>>) 
inputDataSet.getType())
+              .withPipelineOptions(context.getPipelineOptions());
+      // We insert a NOOP here to initialize the FileSystems via the above 
CoderTypeInformation.
+      // The output type coder may be relying on file system access. The 
shuffled data may have to
+      // be deserialized on a different machine using this coder where 
FileSystems has not been
+      // initialized.
+      final DataSet<WindowedValue<KV<K, InputT>>> retypedDataSet =
+          new MapOperator<>(
+              inputDataSet,
+              outputType,
+              FlinkIdentityFunction.of(),
+              getCurrentTransformName(context));
+      final Configuration partitionOptions = new Configuration();
+      partitionOptions.setString(
+          Optimizer.HINT_SHIP_STRATEGY, 
Optimizer.HINT_SHIP_STRATEGY_REPARTITION);

Review Comment:
   @dmvk Does this have the required effect? I'd think this applies to SQL 
optimizer, mostly. No?



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