Abacn commented on code in PR #34560:
URL: https://github.com/apache/beam/pull/34560#discussion_r2058897947


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java:
##########
@@ -28,7 +28,7 @@
 import org.apache.spark.api.java.JavaSparkContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Tuple2;
+import scala.Tuple3;

Review Comment:
   any chance this will grow to Tuple4, ..., in the future? Shall we create a 
POJO class instead (similar to SideInputBroadcast) and make the value of the 
Map this POJO class



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/SparkTransformOverrides.java:
##########
@@ -50,6 +53,18 @@ public static List<PTransformOverride> 
getDefaultOverrides(boolean streaming) {
               PTransformOverride.of(
                   
PTransformMatchers.urnEqualTo(PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN),
                   new SplittableParDoNaiveBounded.OverrideFactory()));
+    } else {
+      // For streaming pipelines, this override is applied only when the 
PTransform has the same URN

Review Comment:
   I understand this comment goes to the first item of "PTransformOverride.of("



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java:
##########
@@ -66,21 +69,21 @@ public SparkSideInputReader(
     // --- match the appropriate sideInput window.
     // a tag will point to all matching sideInputs, that is all windows.
     // now that we've obtained the appropriate sideInputWindow, all that's 
left is to filter by it.
+    final SideInputBroadcast<?> sideInputBroadcast = 
windowedBroadcastHelper.getValue();
     Iterable<WindowedValue<?>> availableSideInputs =
-        (Iterable<WindowedValue<?>>) 
windowedBroadcastHelper.getValue().getValue();
-    Iterable<?> sideInputForWindow =
+        (Iterable<WindowedValue<?>>) sideInputBroadcast.getValue();

Review Comment:
   Thanks. This improved the readability



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputReaderUtils.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.spark.util;
+
+import java.util.Map;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.WindowingStrategy;
+
+/**
+ * Utility class for creating and managing side input readers in the Spark 
runner.
+ *
+ * <p>This class provides factory methods to create appropriate {@link 
SideInputReader}
+ * implementations based on the execution mode (streaming or batch) to 
optimize side input access
+ * patterns.
+ */
+public class SideInputReaderUtils {

Review Comment:
   From Beam naming convention it sounds this is a "SideInputReaderFactory". 
And the static method can be named after "create"



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