aromanenko-dev commented on code in PR #24288:
URL: https://github.com/apache/beam/pull/24288#discussion_r1036906888


##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java:
##########
@@ -69,4 +133,75 @@ private static SparkSession.Builder sessionBuilder(
     }
     return SparkSession.builder().config(sparkConf);
   }
+
+  public static class SparkKryoRegistrator implements KryoRegistrator {

Review Comment:
   Should it be `public`? Please, add Javadoc for this class



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java:
##########
@@ -83,61 +86,74 @@
       ClassTag.apply(Tuple2.class);
 
   @Override
-  public void translate(ParDo.MultiOutput<InputT, OutputT> transform, Context 
cxt)
-      throws IOException {
-    String stepName = cxt.getCurrentTransform().getFullName();
-
-    SparkCommonPipelineOptions opts = 
cxt.getOptions().as(SparkCommonPipelineOptions.class);
-    StorageLevel storageLevel = 
StorageLevel.fromString(opts.getStorageLevel());
+  public boolean canTranslate(ParDo.MultiOutput<InputT, OutputT> transform) {
+    DoFn<InputT, OutputT> doFn = transform.getFn();
+    DoFnSignature signature = DoFnSignatures.signatureForDoFn(doFn);
 
-    // Check for not supported advanced features
     // TODO: add support of Splittable DoFn
-    DoFn<InputT, OutputT> doFn = transform.getFn();
     checkState(
-        !DoFnSignatures.isSplittable(doFn),
+        !signature.processElement().isSplittable(),
         "Not expected to directly translate splittable DoFn, should have been 
overridden: %s",
         doFn);
 
     // TODO: add support of states and timers
     checkState(
-        !DoFnSignatures.isStateful(doFn), "States and timers are not supported 
for the moment.");
+        !signature.usesState() && !signature.usesTimers(),
+        "States and timers are not supported for the moment.");
 
     checkState(
-        !DoFnSignatures.requiresTimeSortedInput(doFn),
+        signature.onWindowExpiration() == null, "onWindowExpiration is not 
supported: %s", doFn);
+
+    checkState(
+        !signature.processElement().requiresTimeSortedInput(),
         "@RequiresTimeSortedInput is not supported for the moment");
 
+    
SparkSideInputReader.validateMaterializations(transform.getSideInputs().values());
+    return true;

Review Comment:
   Does it always return `true` in case of passing all checks above?



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java:
##########
@@ -135,6 +135,12 @@ public <T> Dataset<WindowedValue<T>> 
getDataset(PCollection<T> pCollection) {
       return state.getDataset(pCollection);
     }
 
+    @Override
+    public <T> Broadcast<SideInputValues<T>> getSideInputBroadcast(

Review Comment:
   Should it be `public`?



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java:
##########
@@ -83,61 +86,74 @@
       ClassTag.apply(Tuple2.class);
 
   @Override
-  public void translate(ParDo.MultiOutput<InputT, OutputT> transform, Context 
cxt)
-      throws IOException {
-    String stepName = cxt.getCurrentTransform().getFullName();
-
-    SparkCommonPipelineOptions opts = 
cxt.getOptions().as(SparkCommonPipelineOptions.class);
-    StorageLevel storageLevel = 
StorageLevel.fromString(opts.getStorageLevel());
+  public boolean canTranslate(ParDo.MultiOutput<InputT, OutputT> transform) {

Review Comment:
   I'd call it something like `isTranslationSupported()`



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java:
##########
@@ -56,13 +57,13 @@
  */
 @Internal
 public abstract class TransformTranslator<
-    InT extends PInput, OutT extends POutput, TransformT extends PTransform<? 
extends InT, OutT>> {
+    InT extends PInput, OutT extends POutput, TransformT extends PTransform<? 
super InT, OutT>> {

Review Comment:
   Why it was changed?



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/CachedSideInputReader.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.structuredstreaming.translation.batch.functions;
+
+import static 
org.apache.beam.sdk.transforms.Materializations.MULTIMAP_MATERIALIZATION_URN;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import org.apache.beam.runners.core.SideInputReader;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.transforms.Materialization;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheStats;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * SideInputReader that caches results for costly {@link Materialization 
Materializations}.
+ * Concurrent access is not expected, but it won't impact correctness.
+ */
+@Internal
+public class CachedSideInputReader implements SideInputReader {

Review Comment:
   Does it support dynamic changes of SideInput ?



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java:
##########
@@ -53,6 +107,16 @@ private static SparkSession.Builder sessionBuilder(
       sparkConf.setJars(jars.toArray(new String[0]));
     }
 
+    if (!sparkConf.contains("spark.serializer")) {

Review Comment:
   Please, add log messages about configuration changes.



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