You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/11/17 09:48:19 UTC

[GitHub] [beam] echauchot commented on a diff in pull request #24009: [Spark dataset runner] Cache datasets if used multiple times

echauchot commented on code in PR #24009:
URL: https://github.com/apache/beam/pull/24009#discussion_r1021700719


##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.util.Collection;
+import java.util.concurrent.Callable;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables;
+import org.apache.spark.api.java.function.ForeachFunction;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.execution.ExplainMode;
+import org.apache.spark.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The {@link EvaluationContext} is the result of a pipeline {@link PipelineTranslator#translate
+ * translation} and can be used to evaluate / run the pipeline.
+ *
+ * <p>However, in some cases pipeline translation involves the early evaluation of some parts of the
+ * pipeline. For example, this is necessary to materialize side-inputs. The {@link
+ * EvaluationContext} won't re-evaluate such datasets.
+ */
+@Internal
+public final class EvaluationContext {
+  private static final Logger LOG = LoggerFactory.getLogger(EvaluationContext.class);
+
+  interface NamedDataset<T> {
+    String name();
+
+    @Nullable
+    Dataset<WindowedValue<T>> dataset();
+  }
+
+  private final Collection<? extends NamedDataset<?>> leaveDatasets;
+  private final SparkSession session;
+
+  EvaluationContext(Collection<? extends NamedDataset<?>> leaveDatasets, SparkSession session) {
+    this.leaveDatasets = leaveDatasets;
+    this.session = session;
+  }
+
+  /** Trigger evaluation of all leave datasets. */
+  public void evaluate() {
+    for (NamedDataset<?> ds : leaveDatasets) {
+      final Dataset<?> dataset = ds.dataset();
+      if (dataset == null) {
+        continue;
+      }
+      if (LOG.isDebugEnabled()) {
+        ExplainMode explainMode = ExplainMode.fromString("simple");
+        String execPlan = dataset.queryExecution().explainString(explainMode);
+        LOG.debug("Evaluating dataset {}:\n{}", ds.name(), execPlan);
+      }
+      // force evaluation using a dummy foreach action
+      evaluate(ds.name(), () -> dataset.foreach(NOOP));
+    }
+  }
+
+  /**
+   * The purpose of this utility is to mark the evaluation of Spark actions, both during Pipeline
+   * translation, when evaluation is required, and when finally evaluating the pipeline.
+   */
+  public static void evaluate(String name, Runnable action) {
+    long startMs = System.currentTimeMillis();
+    try {
+      action.run();
+      LOG.info("Evaluated dataset {} in {}", name, durationSince(startMs));
+    } catch (RuntimeException e) {
+      LOG.error("Failed to evaluate dataset {}: {}", name, Throwables.getRootCause(e).getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * The purpose of this utility is to mark the evaluation of Spark actions, both during Pipeline
+   * translation, when evaluation is required, and when finally evaluating the pipeline.
+   */
+  public static <T> T evaluate(String name, Callable<T> action) {

Review Comment:
   unused



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java:
##########
@@ -81,27 +80,13 @@ public class PipelineTranslatorBatch extends PipelineTranslator {
 
     TRANSFORM_TRANSLATORS.put(
         SplittableParDo.PrimitiveBoundedRead.class, new ReadSourceTranslatorBatch<>());
-

Review Comment:
   It is true that the CreatePCollectionView translation was doing nothing except setting the view inside the translation context. You don't need this anymore ? This is the preparation for [#24035](https://github.com/apache/beam/issues/24035) you mentioned ?



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java:
##########
@@ -118,69 +126,53 @@ public <T> PCollection<T> getOutput(TupleTag<T> tag) {
       return pc;
     }
 
-    public Map<TupleTag<?>, PCollection<?>> getOutputs() {
-      return transform.getOutputs();
-    }
-
     public AppliedPTransform<InT, OutT, PTransform<InT, OutT>> getCurrentTransform() {
       return transform;
     }
 
+    @Override
     public <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection) {
-      return cxt.getDataset(pCollection);
+      return state.getDataset(pCollection);
     }
 
-    public <T> void putDataset(PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset) {
-      cxt.putDataset(pCollection, dataset);
+    @Override
+    public <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache) {

Review Comment:
   nit: I'm not a big fan of double negations: cache = false seems better than noCache = true



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java:
##########
@@ -17,170 +17,336 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation;
 
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
+import org.apache.beam.runners.spark.SparkCommonPipelineOptions;
+import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderProvider;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.AppliedPTransform;
-import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
-import org.checkerframework.checker.nullness.qual.Nullable;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.storage.StorageLevel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * {@link Pipeline.PipelineVisitor} that translates the Beam operators to their Spark counterparts.
- * It also does the pipeline preparation: mode detection, transforms replacement, classpath
- * preparation.
+ * The pipeline translator translates a Beam {@link Pipeline} into a Spark correspondence, that can
+ * then be evaluated.
+ *
+ * <p>The translation involves traversing the hierarchy of a pipeline multiple times:
+ *
+ * <ol>
+ *   <li>Detect if {@link StreamingOptions#setStreaming streaming} mode is required.
+ *   <li>Identify datasets that are repeatedly used as input and should be cached.
+ *   <li>And finally, translate each primitive or composite {@link PTransform} that is {@link
+ *       #getTransformTranslator known} and {@link TransformTranslator#canTranslate supported} into
+ *       its Spark correspondence. If a composite is not supported, it will be expanded further into
+ *       its parts and translated then.
+ * </ol>
  */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
+@Internal
+public abstract class PipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class);
-  protected TranslationContext translationContext;
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline preparation methods
-  // --------------------------------------------------------------------------------------------
   public static void replaceTransforms(Pipeline pipeline, StreamingOptions options) {
     pipeline.replaceAll(SparkTransformOverrides.getDefaultOverrides(options.isStreaming()));
   }
 
   /**
-   * Visit the pipeline to determine the translation mode (batch/streaming) and update options
-   * accordingly.
+   * Analyse the pipeline to determine if we have to switch to streaming mode for the pipeline
+   * translation and update {@link StreamingOptions} accordingly.
    */
-  public static void detectTranslationMode(Pipeline pipeline, StreamingOptions options) {
-    TranslationModeDetector detector = new TranslationModeDetector();
+  public static void detectStreamingMode(Pipeline pipeline, StreamingOptions options) {
+    StreamingModeDetector detector = new StreamingModeDetector(options.isStreaming());
     pipeline.traverseTopologically(detector);
-    if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) {
-      options.setStreaming(true);
+    options.setStreaming(detector.streaming);
+  }
+
+  /** Returns a {@link TransformTranslator} for the given {@link PTransform} if known. */
+  protected abstract @Nullable <
+          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
+      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(TransformT transform);
+
+  /**
+   * Translates a Beam pipeline into its Spark correspondence using the Spark SQL / Dataset API.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted to be able to continue with the translation.
+   *
+   * @return The result of the translation is an {@link EvaluationContext} that can trigger the
+   *     evaluation of the Spark pipeline.
+   */
+  public EvaluationContext translate(

Review Comment:
   I like this API :+1: 



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java:
##########
@@ -17,170 +17,336 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation;
 
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
+import org.apache.beam.runners.spark.SparkCommonPipelineOptions;
+import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderProvider;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.AppliedPTransform;
-import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
-import org.checkerframework.checker.nullness.qual.Nullable;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.storage.StorageLevel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * {@link Pipeline.PipelineVisitor} that translates the Beam operators to their Spark counterparts.
- * It also does the pipeline preparation: mode detection, transforms replacement, classpath
- * preparation.
+ * The pipeline translator translates a Beam {@link Pipeline} into a Spark correspondence, that can
+ * then be evaluated.
+ *
+ * <p>The translation involves traversing the hierarchy of a pipeline multiple times:
+ *
+ * <ol>
+ *   <li>Detect if {@link StreamingOptions#setStreaming streaming} mode is required.
+ *   <li>Identify datasets that are repeatedly used as input and should be cached.
+ *   <li>And finally, translate each primitive or composite {@link PTransform} that is {@link
+ *       #getTransformTranslator known} and {@link TransformTranslator#canTranslate supported} into
+ *       its Spark correspondence. If a composite is not supported, it will be expanded further into
+ *       its parts and translated then.
+ * </ol>
  */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
+@Internal
+public abstract class PipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class);
-  protected TranslationContext translationContext;
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline preparation methods
-  // --------------------------------------------------------------------------------------------
   public static void replaceTransforms(Pipeline pipeline, StreamingOptions options) {
     pipeline.replaceAll(SparkTransformOverrides.getDefaultOverrides(options.isStreaming()));
   }
 
   /**
-   * Visit the pipeline to determine the translation mode (batch/streaming) and update options
-   * accordingly.
+   * Analyse the pipeline to determine if we have to switch to streaming mode for the pipeline
+   * translation and update {@link StreamingOptions} accordingly.
    */
-  public static void detectTranslationMode(Pipeline pipeline, StreamingOptions options) {
-    TranslationModeDetector detector = new TranslationModeDetector();
+  public static void detectStreamingMode(Pipeline pipeline, StreamingOptions options) {
+    StreamingModeDetector detector = new StreamingModeDetector(options.isStreaming());
     pipeline.traverseTopologically(detector);
-    if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) {
-      options.setStreaming(true);
+    options.setStreaming(detector.streaming);
+  }
+
+  /** Returns a {@link TransformTranslator} for the given {@link PTransform} if known. */
+  protected abstract @Nullable <
+          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
+      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(TransformT transform);
+
+  /**
+   * Translates a Beam pipeline into its Spark correspondence using the Spark SQL / Dataset API.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted to be able to continue with the translation.
+   *
+   * @return The result of the translation is an {@link EvaluationContext} that can trigger the
+   *     evaluation of the Spark pipeline.
+   */
+  public EvaluationContext translate(
+      Pipeline pipeline, SparkSession session, SparkCommonPipelineOptions options) {
+    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
+    DependencyVisitor dependencies = new DependencyVisitor();
+    pipeline.traverseTopologically(dependencies);
+
+    TranslatingVisitor translator = new TranslatingVisitor(session, options, dependencies.results);
+    pipeline.traverseTopologically(translator);
+
+    return new EvaluationContext(translator.leaves, session);
+  }
+
+  /**
+   * The correspondence of a {@link PCollection} as result of translating a {@link PTransform}
+   * including additional metadata (such as name and dependents).
+   */
+  private static final class TranslationResult<T> implements EvaluationContext.NamedDataset<T> {
+    private final String name;
+    private @Nullable Dataset<WindowedValue<T>> dataset = null;
+    private final Set<PTransform<?, ?>> dependentTransforms = new HashSet<>();
+
+    private TranslationResult(PCollection<?> pCol) {
+      this.name = pCol.getName();
+    }
+
+    @Override
+    public String name() {
+      return name;
+    }
+
+    @Override
+    public @Nullable Dataset<WindowedValue<T>> dataset() {
+      return dataset;
     }
   }
 
-  /** The translation mode of the Beam Pipeline. */
-  private enum TranslationMode {
+  /** Shared, mutable state during the translation of a pipeline and omitted afterwards. */
+  interface TranslationState extends EncoderProvider {

Review Comment:
   elegant use of state and transparent encoder capabilities :+1: 



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderProvider.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.helpers;
+
+import static org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers.kvEncoder;
+
+import java.util.function.Function;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.values.KV;
+import org.apache.spark.sql.Encoder;
+
+@Internal
+public interface EncoderProvider {
+  interface Factory<T> extends Function<Coder<T>, Encoder<T>> {
+    Factory<?> INSTANCE = EncoderHelpers::encoderFor;
+  }
+
+  <T> Encoder<T> encoderOf(Coder<T> coder, Factory<T> factory);
+
+  default <T> Encoder<T> encoderOf(Coder<T> coder) {
+    return coder instanceof KvCoder
+        ? (Encoder<T>) kvEncoderOf((KvCoder) coder)
+        : encoderOf(coder, encoderFactory());
+  }
+
+  default <K, V> Encoder<KV<K, V>> kvEncoderOf(KvCoder<K, V> coder) {
+    return encoderOf(coder, c -> kvEncoder(keyEncoderOf(coder), valueEncoderOf(coder)));
+  }
+
+  default <K, V> Encoder<K> keyEncoderOf(KvCoder<K, V> coder) {
+    return encoderOf(coder.getKeyCoder(), encoderFactory());
+  }
+
+  default <K, V> Encoder<V> valueEncoderOf(KvCoder<K, V> coder) {
+    return encoderOf(coder.getValueCoder(), encoderFactory());
+  }
+
+  default <T> Factory<T> encoderFactory() {
+    return (Factory<T>) Factory.INSTANCE;

Review Comment:
   I would prefer that you inline the INSTANCE here (as it is used only here) and leave Factory as a simple tagging interface over Function. That would remove the need for the cast and the strange Factory containing a Factory.



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java:
##########
@@ -17,170 +17,336 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation;
 
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
+import org.apache.beam.runners.spark.SparkCommonPipelineOptions;
+import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderProvider;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.AppliedPTransform;
-import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
-import org.checkerframework.checker.nullness.qual.Nullable;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.storage.StorageLevel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * {@link Pipeline.PipelineVisitor} that translates the Beam operators to their Spark counterparts.
- * It also does the pipeline preparation: mode detection, transforms replacement, classpath
- * preparation.
+ * The pipeline translator translates a Beam {@link Pipeline} into a Spark correspondence, that can
+ * then be evaluated.
+ *
+ * <p>The translation involves traversing the hierarchy of a pipeline multiple times:
+ *
+ * <ol>
+ *   <li>Detect if {@link StreamingOptions#setStreaming streaming} mode is required.
+ *   <li>Identify datasets that are repeatedly used as input and should be cached.
+ *   <li>And finally, translate each primitive or composite {@link PTransform} that is {@link
+ *       #getTransformTranslator known} and {@link TransformTranslator#canTranslate supported} into
+ *       its Spark correspondence. If a composite is not supported, it will be expanded further into
+ *       its parts and translated then.
+ * </ol>
  */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
+@Internal
+public abstract class PipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class);
-  protected TranslationContext translationContext;
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline preparation methods
-  // --------------------------------------------------------------------------------------------
   public static void replaceTransforms(Pipeline pipeline, StreamingOptions options) {
     pipeline.replaceAll(SparkTransformOverrides.getDefaultOverrides(options.isStreaming()));
   }
 
   /**
-   * Visit the pipeline to determine the translation mode (batch/streaming) and update options
-   * accordingly.
+   * Analyse the pipeline to determine if we have to switch to streaming mode for the pipeline
+   * translation and update {@link StreamingOptions} accordingly.
    */
-  public static void detectTranslationMode(Pipeline pipeline, StreamingOptions options) {
-    TranslationModeDetector detector = new TranslationModeDetector();
+  public static void detectStreamingMode(Pipeline pipeline, StreamingOptions options) {
+    StreamingModeDetector detector = new StreamingModeDetector(options.isStreaming());
     pipeline.traverseTopologically(detector);
-    if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) {
-      options.setStreaming(true);
+    options.setStreaming(detector.streaming);
+  }
+
+  /** Returns a {@link TransformTranslator} for the given {@link PTransform} if known. */
+  protected abstract @Nullable <
+          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
+      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(TransformT transform);
+
+  /**
+   * Translates a Beam pipeline into its Spark correspondence using the Spark SQL / Dataset API.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted to be able to continue with the translation.
+   *
+   * @return The result of the translation is an {@link EvaluationContext} that can trigger the
+   *     evaluation of the Spark pipeline.
+   */
+  public EvaluationContext translate(
+      Pipeline pipeline, SparkSession session, SparkCommonPipelineOptions options) {
+    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
+    DependencyVisitor dependencies = new DependencyVisitor();
+    pipeline.traverseTopologically(dependencies);
+
+    TranslatingVisitor translator = new TranslatingVisitor(session, options, dependencies.results);
+    pipeline.traverseTopologically(translator);
+
+    return new EvaluationContext(translator.leaves, session);
+  }
+
+  /**
+   * The correspondence of a {@link PCollection} as result of translating a {@link PTransform}
+   * including additional metadata (such as name and dependents).
+   */
+  private static final class TranslationResult<T> implements EvaluationContext.NamedDataset<T> {
+    private final String name;
+    private @Nullable Dataset<WindowedValue<T>> dataset = null;
+    private final Set<PTransform<?, ?>> dependentTransforms = new HashSet<>();
+
+    private TranslationResult(PCollection<?> pCol) {
+      this.name = pCol.getName();
+    }
+
+    @Override
+    public String name() {
+      return name;
+    }
+
+    @Override
+    public @Nullable Dataset<WindowedValue<T>> dataset() {
+      return dataset;
     }
   }
 
-  /** The translation mode of the Beam Pipeline. */
-  private enum TranslationMode {
+  /** Shared, mutable state during the translation of a pipeline and omitted afterwards. */
+  interface TranslationState extends EncoderProvider {

Review Comment:
   good also to avoid storing all the datasets and just keep the current one and then forget about it



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java:
##########
@@ -17,170 +17,336 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation;
 
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
+import org.apache.beam.runners.spark.SparkCommonPipelineOptions;
+import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderProvider;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.AppliedPTransform;
-import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
-import org.checkerframework.checker.nullness.qual.Nullable;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.storage.StorageLevel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * {@link Pipeline.PipelineVisitor} that translates the Beam operators to their Spark counterparts.
- * It also does the pipeline preparation: mode detection, transforms replacement, classpath
- * preparation.
+ * The pipeline translator translates a Beam {@link Pipeline} into a Spark correspondence, that can
+ * then be evaluated.
+ *
+ * <p>The translation involves traversing the hierarchy of a pipeline multiple times:
+ *
+ * <ol>
+ *   <li>Detect if {@link StreamingOptions#setStreaming streaming} mode is required.
+ *   <li>Identify datasets that are repeatedly used as input and should be cached.
+ *   <li>And finally, translate each primitive or composite {@link PTransform} that is {@link
+ *       #getTransformTranslator known} and {@link TransformTranslator#canTranslate supported} into
+ *       its Spark correspondence. If a composite is not supported, it will be expanded further into
+ *       its parts and translated then.
+ * </ol>
  */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
+@Internal
+public abstract class PipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class);
-  protected TranslationContext translationContext;
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline preparation methods
-  // --------------------------------------------------------------------------------------------
   public static void replaceTransforms(Pipeline pipeline, StreamingOptions options) {
     pipeline.replaceAll(SparkTransformOverrides.getDefaultOverrides(options.isStreaming()));
   }
 
   /**
-   * Visit the pipeline to determine the translation mode (batch/streaming) and update options
-   * accordingly.
+   * Analyse the pipeline to determine if we have to switch to streaming mode for the pipeline
+   * translation and update {@link StreamingOptions} accordingly.
    */
-  public static void detectTranslationMode(Pipeline pipeline, StreamingOptions options) {
-    TranslationModeDetector detector = new TranslationModeDetector();
+  public static void detectStreamingMode(Pipeline pipeline, StreamingOptions options) {
+    StreamingModeDetector detector = new StreamingModeDetector(options.isStreaming());
     pipeline.traverseTopologically(detector);
-    if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) {
-      options.setStreaming(true);
+    options.setStreaming(detector.streaming);
+  }
+
+  /** Returns a {@link TransformTranslator} for the given {@link PTransform} if known. */
+  protected abstract @Nullable <
+          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
+      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(TransformT transform);
+
+  /**
+   * Translates a Beam pipeline into its Spark correspondence using the Spark SQL / Dataset API.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted to be able to continue with the translation.
+   *
+   * @return The result of the translation is an {@link EvaluationContext} that can trigger the
+   *     evaluation of the Spark pipeline.
+   */
+  public EvaluationContext translate(
+      Pipeline pipeline, SparkSession session, SparkCommonPipelineOptions options) {
+    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
+    DependencyVisitor dependencies = new DependencyVisitor();
+    pipeline.traverseTopologically(dependencies);
+
+    TranslatingVisitor translator = new TranslatingVisitor(session, options, dependencies.results);
+    pipeline.traverseTopologically(translator);
+
+    return new EvaluationContext(translator.leaves, session);
+  }
+
+  /**
+   * The correspondence of a {@link PCollection} as result of translating a {@link PTransform}
+   * including additional metadata (such as name and dependents).
+   */
+  private static final class TranslationResult<T> implements EvaluationContext.NamedDataset<T> {
+    private final String name;
+    private @Nullable Dataset<WindowedValue<T>> dataset = null;
+    private final Set<PTransform<?, ?>> dependentTransforms = new HashSet<>();
+
+    private TranslationResult(PCollection<?> pCol) {
+      this.name = pCol.getName();
+    }
+
+    @Override
+    public String name() {
+      return name;
+    }
+
+    @Override
+    public @Nullable Dataset<WindowedValue<T>> dataset() {
+      return dataset;
     }
   }
 
-  /** The translation mode of the Beam Pipeline. */
-  private enum TranslationMode {
+  /** Shared, mutable state during the translation of a pipeline and omitted afterwards. */
+  interface TranslationState extends EncoderProvider {
+    <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection);
+
+    <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache);
 
-    /** Uses the batch mode. */
-    BATCH,
+    default <T> void putDataset(PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset) {
+      putDataset(pCollection, dataset, false);
+    }
 
-    /** Uses the streaming mode. */
-    STREAMING
+    SerializablePipelineOptions getSerializableOptions();
+
+    SparkSession getSparkSession();
   }
 
-  /** Traverses the Pipeline to determine the {@link TranslationMode} for this pipeline. */
-  private static class TranslationModeDetector extends Pipeline.PipelineVisitor.Defaults {
-    private static final Logger LOG = LoggerFactory.getLogger(TranslationModeDetector.class);
+  /**
+   * {@link PTransformVisitor} that translates supported {@link PTransform PTransforms} into their
+   * Spark correspondence.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted.
+   */
+  private class TranslatingVisitor extends PTransformVisitor implements TranslationState {
+    private final Map<PCollection<?>, TranslationResult<?>> translationResults;
+    private final Map<Coder<?>, ExpressionEncoder<?>> encoders;
+    private final SparkSession sparkSession;
+    private final SerializablePipelineOptions serializableOptions;
+    private final StorageLevel storageLevel;
+
+    private final Set<TranslationResult<?>> leaves;
+
+    public TranslatingVisitor(
+        SparkSession sparkSession,
+        SparkCommonPipelineOptions options,
+        Map<PCollection<?>, TranslationResult<?>> translationResults) {
+      this.sparkSession = sparkSession;
+      this.translationResults = translationResults;
+      this.serializableOptions = new SerializablePipelineOptions(options);
+      this.storageLevel = StorageLevel.fromString(options.getStorageLevel());
+      this.encoders = new HashMap<>();
+      this.leaves = new HashSet<>();
+    }
 
-    private TranslationMode translationMode;
+    @Override
+    <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator) {
+
+      AppliedPTransform<InT, OutT, PTransform<InT, OutT>> appliedTransform =
+          (AppliedPTransform) node.toAppliedPTransform(getPipeline());
+      try {
+        LOG.info(
+            "Translating {}: {}",
+            node.isCompositeNode() ? "composite" : "primitive",
+            node.getFullName());
+        translator.translate(transform, appliedTransform, this);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
 
-    TranslationModeDetector(TranslationMode defaultMode) {
-      this.translationMode = defaultMode;
+    @Override
+    public <T> Encoder<T> encoderOf(Coder<T> coder, Factory<T> factory) {
+      return (Encoder<T>) encoders.computeIfAbsent(coder, (Factory) factory);
     }
 
-    TranslationModeDetector() {
-      this(TranslationMode.BATCH);
+    private <T> TranslationResult<T> getResult(PCollection<T> pCollection) {
+      return (TranslationResult<T>) checkStateNotNull(translationResults.get(pCollection));
     }
 
-    TranslationMode getTranslationMode() {
-      return translationMode;
+    @Override
+    public <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection) {
+      return checkStateNotNull(getResult(pCollection).dataset);
     }
 
     @Override
-    public void visitValue(PValue value, TransformHierarchy.Node producer) {
-      if (translationMode.equals(TranslationMode.BATCH)) {
-        if (value instanceof PCollection
-            && ((PCollection) value).isBounded() == PCollection.IsBounded.UNBOUNDED) {
-          LOG.info(
-              "Found unbounded PCollection {}. Switching to streaming execution.", value.getName());
-          translationMode = TranslationMode.STREAMING;
+    public <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache) {
+      TranslationResult<T> result = getResult(pCollection);
+      if (!noCache && result.dependentTransforms.size() > 1) {
+        LOG.info("Dataset {} will be cached.", result.name);
+        result.dataset = dataset.persist(storageLevel); // use NONE to disable
+      } else {
+        result.dataset = dataset;
+        if (result.dependentTransforms.isEmpty()) {
+          leaves.add(result);
         }
       }
     }
-  }
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline utility methods
-  // --------------------------------------------------------------------------------------------
+    @Override
+    public SerializablePipelineOptions getSerializableOptions() {
+      return serializableOptions;
+    }
 
-  /** Get a {@link TransformTranslator} for the given {@link TransformHierarchy.Node}. */
-  protected abstract @Nullable <
-          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
-      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(
-          @Nullable TransformT transform);
-
-  /** Apply the given TransformTranslator to the given node. */
-  private <InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
-      void applyTransformTranslator(
-          TransformHierarchy.Node node,
-          TransformT transform,
-          TransformTranslator<InT, OutT, TransformT> transformTranslator) {
-    // create the applied PTransform on the translationContext
-    AppliedPTransform<InT, OutT, PTransform<InT, OutT>> appliedTransform =
-        (AppliedPTransform) node.toAppliedPTransform(getPipeline());
-    try {
-      transformTranslator.translate(transform, appliedTransform, translationContext);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
+    @Override
+    public SparkSession getSparkSession() {
+      return sparkSession;
     }
   }
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline visitor entry point
-  // --------------------------------------------------------------------------------------------
-
   /**
-   * Translates the pipeline by passing this class as a visitor.
+   * {@link PTransformVisitor} that analyses dependencies of supported {@link PTransform
+   * PTransforms} to help identify cache candidates.
    *
-   * @param pipeline The pipeline to be translated
+   * <p>The visitor may throw if a {@link PTransform} is observed that uses unsupported features.
    */
-  public void translate(Pipeline pipeline) {
-    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
-    pipeline.traverseTopologically(this);
+  private class DependencyVisitor extends PTransformVisitor {
+    private final Map<PCollection<?>, TranslationResult<?>> results = new HashMap<>();
+
+    @Override
+    <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator) {
+      for (PCollection<?> pOut : node.getOutputs().values()) {
+        results.put(pOut, new TranslationResult<>(pOut));
+        for (Map.Entry<TupleTag<?>, PCollection<?>> entry : node.getInputs().entrySet()) {
+          TranslationResult<?> input = checkStateNotNull(results.get(entry.getValue()));
+          input.dependentTransforms.add(transform);
+        }
+      }
+    }

Review Comment:
   I don't understand this algo especially the nested loops because `pOut` is not referenced inside the inner loop.
   Why don't you populate `results` based on outputs first and then do the second loop?
   
   There might be something I don't get, if it's the case, please add some comments to the algo for ease of maintenance.



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java:
##########
@@ -17,170 +17,336 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation;
 
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
+import org.apache.beam.runners.spark.SparkCommonPipelineOptions;
+import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderProvider;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.AppliedPTransform;
-import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
-import org.checkerframework.checker.nullness.qual.Nullable;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.storage.StorageLevel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * {@link Pipeline.PipelineVisitor} that translates the Beam operators to their Spark counterparts.
- * It also does the pipeline preparation: mode detection, transforms replacement, classpath
- * preparation.
+ * The pipeline translator translates a Beam {@link Pipeline} into a Spark correspondence, that can
+ * then be evaluated.
+ *
+ * <p>The translation involves traversing the hierarchy of a pipeline multiple times:
+ *
+ * <ol>
+ *   <li>Detect if {@link StreamingOptions#setStreaming streaming} mode is required.
+ *   <li>Identify datasets that are repeatedly used as input and should be cached.
+ *   <li>And finally, translate each primitive or composite {@link PTransform} that is {@link
+ *       #getTransformTranslator known} and {@link TransformTranslator#canTranslate supported} into
+ *       its Spark correspondence. If a composite is not supported, it will be expanded further into
+ *       its parts and translated then.
+ * </ol>
  */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
+@Internal
+public abstract class PipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class);
-  protected TranslationContext translationContext;
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline preparation methods
-  // --------------------------------------------------------------------------------------------
   public static void replaceTransforms(Pipeline pipeline, StreamingOptions options) {
     pipeline.replaceAll(SparkTransformOverrides.getDefaultOverrides(options.isStreaming()));
   }
 
   /**
-   * Visit the pipeline to determine the translation mode (batch/streaming) and update options
-   * accordingly.
+   * Analyse the pipeline to determine if we have to switch to streaming mode for the pipeline
+   * translation and update {@link StreamingOptions} accordingly.
    */
-  public static void detectTranslationMode(Pipeline pipeline, StreamingOptions options) {
-    TranslationModeDetector detector = new TranslationModeDetector();
+  public static void detectStreamingMode(Pipeline pipeline, StreamingOptions options) {
+    StreamingModeDetector detector = new StreamingModeDetector(options.isStreaming());
     pipeline.traverseTopologically(detector);
-    if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) {
-      options.setStreaming(true);
+    options.setStreaming(detector.streaming);
+  }
+
+  /** Returns a {@link TransformTranslator} for the given {@link PTransform} if known. */
+  protected abstract @Nullable <
+          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
+      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(TransformT transform);
+
+  /**
+   * Translates a Beam pipeline into its Spark correspondence using the Spark SQL / Dataset API.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted to be able to continue with the translation.
+   *
+   * @return The result of the translation is an {@link EvaluationContext} that can trigger the
+   *     evaluation of the Spark pipeline.
+   */
+  public EvaluationContext translate(
+      Pipeline pipeline, SparkSession session, SparkCommonPipelineOptions options) {
+    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
+    DependencyVisitor dependencies = new DependencyVisitor();
+    pipeline.traverseTopologically(dependencies);
+
+    TranslatingVisitor translator = new TranslatingVisitor(session, options, dependencies.results);
+    pipeline.traverseTopologically(translator);
+
+    return new EvaluationContext(translator.leaves, session);
+  }
+
+  /**
+   * The correspondence of a {@link PCollection} as result of translating a {@link PTransform}
+   * including additional metadata (such as name and dependents).
+   */
+  private static final class TranslationResult<T> implements EvaluationContext.NamedDataset<T> {
+    private final String name;
+    private @Nullable Dataset<WindowedValue<T>> dataset = null;
+    private final Set<PTransform<?, ?>> dependentTransforms = new HashSet<>();
+
+    private TranslationResult(PCollection<?> pCol) {
+      this.name = pCol.getName();
+    }
+
+    @Override
+    public String name() {
+      return name;
+    }
+
+    @Override
+    public @Nullable Dataset<WindowedValue<T>> dataset() {
+      return dataset;
     }
   }
 
-  /** The translation mode of the Beam Pipeline. */
-  private enum TranslationMode {
+  /** Shared, mutable state during the translation of a pipeline and omitted afterwards. */
+  interface TranslationState extends EncoderProvider {
+    <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection);
+
+    <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache);
 
-    /** Uses the batch mode. */
-    BATCH,
+    default <T> void putDataset(PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset) {
+      putDataset(pCollection, dataset, false);
+    }
 
-    /** Uses the streaming mode. */
-    STREAMING
+    SerializablePipelineOptions getSerializableOptions();
+
+    SparkSession getSparkSession();
   }
 
-  /** Traverses the Pipeline to determine the {@link TranslationMode} for this pipeline. */
-  private static class TranslationModeDetector extends Pipeline.PipelineVisitor.Defaults {
-    private static final Logger LOG = LoggerFactory.getLogger(TranslationModeDetector.class);
+  /**
+   * {@link PTransformVisitor} that translates supported {@link PTransform PTransforms} into their
+   * Spark correspondence.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted.
+   */
+  private class TranslatingVisitor extends PTransformVisitor implements TranslationState {
+    private final Map<PCollection<?>, TranslationResult<?>> translationResults;
+    private final Map<Coder<?>, ExpressionEncoder<?>> encoders;
+    private final SparkSession sparkSession;
+    private final SerializablePipelineOptions serializableOptions;
+    private final StorageLevel storageLevel;
+
+    private final Set<TranslationResult<?>> leaves;
+
+    public TranslatingVisitor(
+        SparkSession sparkSession,
+        SparkCommonPipelineOptions options,
+        Map<PCollection<?>, TranslationResult<?>> translationResults) {
+      this.sparkSession = sparkSession;
+      this.translationResults = translationResults;
+      this.serializableOptions = new SerializablePipelineOptions(options);
+      this.storageLevel = StorageLevel.fromString(options.getStorageLevel());
+      this.encoders = new HashMap<>();
+      this.leaves = new HashSet<>();
+    }
 
-    private TranslationMode translationMode;
+    @Override
+    <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator) {
+
+      AppliedPTransform<InT, OutT, PTransform<InT, OutT>> appliedTransform =
+          (AppliedPTransform) node.toAppliedPTransform(getPipeline());
+      try {
+        LOG.info(
+            "Translating {}: {}",
+            node.isCompositeNode() ? "composite" : "primitive",
+            node.getFullName());
+        translator.translate(transform, appliedTransform, this);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
 
-    TranslationModeDetector(TranslationMode defaultMode) {
-      this.translationMode = defaultMode;
+    @Override
+    public <T> Encoder<T> encoderOf(Coder<T> coder, Factory<T> factory) {
+      return (Encoder<T>) encoders.computeIfAbsent(coder, (Factory) factory);
     }
 
-    TranslationModeDetector() {
-      this(TranslationMode.BATCH);
+    private <T> TranslationResult<T> getResult(PCollection<T> pCollection) {
+      return (TranslationResult<T>) checkStateNotNull(translationResults.get(pCollection));
     }
 
-    TranslationMode getTranslationMode() {
-      return translationMode;
+    @Override
+    public <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection) {
+      return checkStateNotNull(getResult(pCollection).dataset);
     }
 
     @Override
-    public void visitValue(PValue value, TransformHierarchy.Node producer) {
-      if (translationMode.equals(TranslationMode.BATCH)) {
-        if (value instanceof PCollection
-            && ((PCollection) value).isBounded() == PCollection.IsBounded.UNBOUNDED) {
-          LOG.info(
-              "Found unbounded PCollection {}. Switching to streaming execution.", value.getName());
-          translationMode = TranslationMode.STREAMING;
+    public <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache) {
+      TranslationResult<T> result = getResult(pCollection);
+      if (!noCache && result.dependentTransforms.size() > 1) {
+        LOG.info("Dataset {} will be cached.", result.name);
+        result.dataset = dataset.persist(storageLevel); // use NONE to disable
+      } else {
+        result.dataset = dataset;
+        if (result.dependentTransforms.isEmpty()) {
+          leaves.add(result);
         }
       }
     }
-  }
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline utility methods
-  // --------------------------------------------------------------------------------------------
+    @Override
+    public SerializablePipelineOptions getSerializableOptions() {
+      return serializableOptions;
+    }
 
-  /** Get a {@link TransformTranslator} for the given {@link TransformHierarchy.Node}. */
-  protected abstract @Nullable <
-          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
-      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(
-          @Nullable TransformT transform);
-
-  /** Apply the given TransformTranslator to the given node. */
-  private <InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
-      void applyTransformTranslator(
-          TransformHierarchy.Node node,
-          TransformT transform,
-          TransformTranslator<InT, OutT, TransformT> transformTranslator) {
-    // create the applied PTransform on the translationContext
-    AppliedPTransform<InT, OutT, PTransform<InT, OutT>> appliedTransform =
-        (AppliedPTransform) node.toAppliedPTransform(getPipeline());
-    try {
-      transformTranslator.translate(transform, appliedTransform, translationContext);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
+    @Override
+    public SparkSession getSparkSession() {
+      return sparkSession;
     }
   }
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline visitor entry point
-  // --------------------------------------------------------------------------------------------
-
   /**
-   * Translates the pipeline by passing this class as a visitor.
+   * {@link PTransformVisitor} that analyses dependencies of supported {@link PTransform
+   * PTransforms} to help identify cache candidates.
    *
-   * @param pipeline The pipeline to be translated
+   * <p>The visitor may throw if a {@link PTransform} is observed that uses unsupported features.
    */
-  public void translate(Pipeline pipeline) {
-    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
-    pipeline.traverseTopologically(this);
+  private class DependencyVisitor extends PTransformVisitor {
+    private final Map<PCollection<?>, TranslationResult<?>> results = new HashMap<>();
+
+    @Override
+    <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator) {
+      for (PCollection<?> pOut : node.getOutputs().values()) {
+        results.put(pOut, new TranslationResult<>(pOut));
+        for (Map.Entry<TupleTag<?>, PCollection<?>> entry : node.getInputs().entrySet()) {
+          TranslationResult<?> input = checkStateNotNull(results.get(entry.getValue()));
+          input.dependentTransforms.add(transform);
+        }
+      }
+    }
   }
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline Visitor Methods
-  // --------------------------------------------------------------------------------------------
+  /**
+   * An abstract {@link PipelineVisitor} that visits all translatable {@link PTransform} pipeline
+   * nodes of a pipeline with the respective {@link TransformTranslator}.
+   *
+   * <p>The visitor may throw if a {@link PTransform} is observed that uses unsupported features.
+   */
+  private abstract class PTransformVisitor extends PipelineVisitor.Defaults {
 
-  @Override
-  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
-    PTransform<PInput, POutput> transform = (PTransform<PInput, POutput>) node.getTransform();
-    TransformTranslator<PInput, POutput, PTransform<PInput, POutput>> transformTranslator =
-        getTransformTranslator(transform);
+    /** Visit the {@link PTransform} with its respective {@link TransformTranslator}. */
+    abstract <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator);
 
-    if (transformTranslator != null) {
-      LOG.info("Translating composite: {}", node.getFullName());
-      applyTransformTranslator(node, transform, transformTranslator);
-      return CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
-    } else {
-      return CompositeBehavior.ENTER_TRANSFORM;
+    @Override
+    public final CompositeBehavior enterCompositeTransform(Node node) {
+      PTransform<PInput, POutput> transform = (PTransform<PInput, POutput>) node.getTransform();
+      TransformTranslator<PInput, POutput, PTransform<PInput, POutput>> translator =
+          getTranslator(transform);
+      if (transform != null && translator != null) {
+        visit(node, transform, translator);
+        return DO_NOT_ENTER_TRANSFORM;
+      } else {
+        return ENTER_TRANSFORM;
+      }
+    }
+
+    @Override
+    public final void visitPrimitiveTransform(Node node) {
+      PTransform<PInput, POutput> transform = (PTransform<PInput, POutput>) node.getTransform();
+      if (transform == null || transform.getClass().equals(View.CreatePCollectionView.class)) {
+        return; // ignore, nothing to be translated here
+      }
+      TransformTranslator<PInput, POutput, PTransform<PInput, POutput>> translator =
+          getTranslator(transform);
+      if (translator == null) {
+        String urn = PTransformTranslation.urnForTransform(transform);
+        throw new UnsupportedOperationException("Transform " + urn + " is not supported.");
+      }
+      visit(node, transform, translator);
     }
-  }
 
-  @Override
-  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
-    LOG.info("Translating primitive: {}", node.getFullName());
-    // get the transformation corresponding to the node we are
-    // currently visiting and translate it into its Spark alternative.
-    PTransform<PInput, POutput> transform = (PTransform<PInput, POutput>) node.getTransform();
-    TransformTranslator<PInput, POutput, PTransform<PInput, POutput>> transformTranslator =
-        getTransformTranslator(transform);
-
-    if (transformTranslator == null) {
-      String transformUrn = PTransformTranslation.urnForTransform(node.getTransform());
-      throw new UnsupportedOperationException(
-          "The transform " + transformUrn + " is currently not supported.");
-    }
-    applyTransformTranslator(node, transform, transformTranslator);
+    /** {@link TransformTranslator} for {@link PTransform} if translation is known and supported. */
+    private @Nullable TransformTranslator<PInput, POutput, PTransform<PInput, POutput>>

Review Comment:
   please rename to getTransformTranslatorIfTranslatable for clarity



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java:
##########
@@ -220,8 +225,9 @@ private SideInputBroadcast createBroadcastSideInputs(
       Coder<WindowedValue<?>> windowedValueCoder =
           (Coder<WindowedValue<?>>)
               (Coder<?>) WindowedValue.getFullCoder(pc.getCoder(), windowCoder);
-      Dataset<WindowedValue<?>> broadcastSet = context.getSideInputDataset(sideInput);
-      List<WindowedValue<?>> valuesList = broadcastSet.collectAsList();
+      Dataset<WindowedValue<?>> broadcastSet = context.getDataset((PCollection) pc);
+      List<WindowedValue<?>> valuesList =

Review Comment:
   now I see the preparation for side inputs you mentioned. :+1: 
   You evaluate the associated PCollection when you need to broadcast it for side inputs



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java:
##########
@@ -17,170 +17,336 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation;
 
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
+import org.apache.beam.runners.spark.SparkCommonPipelineOptions;
+import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderProvider;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.AppliedPTransform;
-import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
-import org.checkerframework.checker.nullness.qual.Nullable;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.storage.StorageLevel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * {@link Pipeline.PipelineVisitor} that translates the Beam operators to their Spark counterparts.
- * It also does the pipeline preparation: mode detection, transforms replacement, classpath
- * preparation.
+ * The pipeline translator translates a Beam {@link Pipeline} into a Spark correspondence, that can
+ * then be evaluated.
+ *
+ * <p>The translation involves traversing the hierarchy of a pipeline multiple times:
+ *
+ * <ol>
+ *   <li>Detect if {@link StreamingOptions#setStreaming streaming} mode is required.
+ *   <li>Identify datasets that are repeatedly used as input and should be cached.
+ *   <li>And finally, translate each primitive or composite {@link PTransform} that is {@link
+ *       #getTransformTranslator known} and {@link TransformTranslator#canTranslate supported} into
+ *       its Spark correspondence. If a composite is not supported, it will be expanded further into
+ *       its parts and translated then.
+ * </ol>
  */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
+@Internal
+public abstract class PipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class);
-  protected TranslationContext translationContext;
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline preparation methods
-  // --------------------------------------------------------------------------------------------
   public static void replaceTransforms(Pipeline pipeline, StreamingOptions options) {
     pipeline.replaceAll(SparkTransformOverrides.getDefaultOverrides(options.isStreaming()));
   }
 
   /**
-   * Visit the pipeline to determine the translation mode (batch/streaming) and update options
-   * accordingly.
+   * Analyse the pipeline to determine if we have to switch to streaming mode for the pipeline
+   * translation and update {@link StreamingOptions} accordingly.
    */
-  public static void detectTranslationMode(Pipeline pipeline, StreamingOptions options) {
-    TranslationModeDetector detector = new TranslationModeDetector();
+  public static void detectStreamingMode(Pipeline pipeline, StreamingOptions options) {
+    StreamingModeDetector detector = new StreamingModeDetector(options.isStreaming());
     pipeline.traverseTopologically(detector);
-    if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) {
-      options.setStreaming(true);
+    options.setStreaming(detector.streaming);
+  }
+
+  /** Returns a {@link TransformTranslator} for the given {@link PTransform} if known. */
+  protected abstract @Nullable <
+          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
+      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(TransformT transform);
+
+  /**
+   * Translates a Beam pipeline into its Spark correspondence using the Spark SQL / Dataset API.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted to be able to continue with the translation.
+   *
+   * @return The result of the translation is an {@link EvaluationContext} that can trigger the
+   *     evaluation of the Spark pipeline.
+   */
+  public EvaluationContext translate(
+      Pipeline pipeline, SparkSession session, SparkCommonPipelineOptions options) {
+    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
+    DependencyVisitor dependencies = new DependencyVisitor();
+    pipeline.traverseTopologically(dependencies);
+
+    TranslatingVisitor translator = new TranslatingVisitor(session, options, dependencies.results);
+    pipeline.traverseTopologically(translator);
+
+    return new EvaluationContext(translator.leaves, session);

Review Comment:
   Very clear ! :+1: 



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java:
##########
@@ -51,22 +48,24 @@
 import scala.reflect.ClassTag;
 
 /**
- * Supports translation between a Beam transform, and Spark's operations on Datasets.
+ * A {@link TransformTranslator} provides the capability to translate a specific primitive or
+ * composite {@link PTransform} into its Spark correspondence.
  *
- * <p>WARNING: Do not make this class serializable! It could easily hide situations where
- * unnecessary references leak into Spark closures.
+ * <p>WARNING: {@link TransformTranslator TransformTranslators} should never be serializable! This
+ * could easily hide situations where unnecessary references leak into Spark closures.
  */
+@Internal
 public abstract class TransformTranslator<
     InT extends PInput, OutT extends POutput, TransformT extends PTransform<? extends InT, OutT>> {
 
   protected abstract void translate(TransformT transform, Context cxt) throws IOException;
 
-  public final void translate(
+  protected final void translate(

Review Comment:
   can be package local now that this method is no more the one that is overridden



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java:
##########
@@ -17,170 +17,336 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation;
 
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
+import org.apache.beam.runners.spark.SparkCommonPipelineOptions;
+import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderProvider;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.AppliedPTransform;
-import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
-import org.checkerframework.checker.nullness.qual.Nullable;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.storage.StorageLevel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * {@link Pipeline.PipelineVisitor} that translates the Beam operators to their Spark counterparts.
- * It also does the pipeline preparation: mode detection, transforms replacement, classpath
- * preparation.
+ * The pipeline translator translates a Beam {@link Pipeline} into a Spark correspondence, that can
+ * then be evaluated.
+ *
+ * <p>The translation involves traversing the hierarchy of a pipeline multiple times:
+ *
+ * <ol>
+ *   <li>Detect if {@link StreamingOptions#setStreaming streaming} mode is required.
+ *   <li>Identify datasets that are repeatedly used as input and should be cached.
+ *   <li>And finally, translate each primitive or composite {@link PTransform} that is {@link
+ *       #getTransformTranslator known} and {@link TransformTranslator#canTranslate supported} into
+ *       its Spark correspondence. If a composite is not supported, it will be expanded further into
+ *       its parts and translated then.
+ * </ol>
  */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
+@Internal
+public abstract class PipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class);
-  protected TranslationContext translationContext;
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline preparation methods
-  // --------------------------------------------------------------------------------------------
   public static void replaceTransforms(Pipeline pipeline, StreamingOptions options) {
     pipeline.replaceAll(SparkTransformOverrides.getDefaultOverrides(options.isStreaming()));
   }
 
   /**
-   * Visit the pipeline to determine the translation mode (batch/streaming) and update options
-   * accordingly.
+   * Analyse the pipeline to determine if we have to switch to streaming mode for the pipeline
+   * translation and update {@link StreamingOptions} accordingly.
    */
-  public static void detectTranslationMode(Pipeline pipeline, StreamingOptions options) {
-    TranslationModeDetector detector = new TranslationModeDetector();
+  public static void detectStreamingMode(Pipeline pipeline, StreamingOptions options) {
+    StreamingModeDetector detector = new StreamingModeDetector(options.isStreaming());
     pipeline.traverseTopologically(detector);
-    if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) {
-      options.setStreaming(true);
+    options.setStreaming(detector.streaming);
+  }
+
+  /** Returns a {@link TransformTranslator} for the given {@link PTransform} if known. */
+  protected abstract @Nullable <
+          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
+      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(TransformT transform);
+
+  /**
+   * Translates a Beam pipeline into its Spark correspondence using the Spark SQL / Dataset API.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted to be able to continue with the translation.
+   *
+   * @return The result of the translation is an {@link EvaluationContext} that can trigger the
+   *     evaluation of the Spark pipeline.
+   */
+  public EvaluationContext translate(
+      Pipeline pipeline, SparkSession session, SparkCommonPipelineOptions options) {
+    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
+    DependencyVisitor dependencies = new DependencyVisitor();
+    pipeline.traverseTopologically(dependencies);
+
+    TranslatingVisitor translator = new TranslatingVisitor(session, options, dependencies.results);
+    pipeline.traverseTopologically(translator);
+
+    return new EvaluationContext(translator.leaves, session);
+  }
+
+  /**
+   * The correspondence of a {@link PCollection} as result of translating a {@link PTransform}
+   * including additional metadata (such as name and dependents).
+   */
+  private static final class TranslationResult<T> implements EvaluationContext.NamedDataset<T> {
+    private final String name;
+    private @Nullable Dataset<WindowedValue<T>> dataset = null;
+    private final Set<PTransform<?, ?>> dependentTransforms = new HashSet<>();
+
+    private TranslationResult(PCollection<?> pCol) {
+      this.name = pCol.getName();
+    }
+
+    @Override
+    public String name() {
+      return name;
+    }
+
+    @Override
+    public @Nullable Dataset<WindowedValue<T>> dataset() {
+      return dataset;
     }
   }
 
-  /** The translation mode of the Beam Pipeline. */
-  private enum TranslationMode {
+  /** Shared, mutable state during the translation of a pipeline and omitted afterwards. */
+  interface TranslationState extends EncoderProvider {
+    <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection);
+
+    <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache);
 
-    /** Uses the batch mode. */
-    BATCH,
+    default <T> void putDataset(PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset) {
+      putDataset(pCollection, dataset, false);
+    }
 
-    /** Uses the streaming mode. */
-    STREAMING
+    SerializablePipelineOptions getSerializableOptions();
+
+    SparkSession getSparkSession();
   }
 
-  /** Traverses the Pipeline to determine the {@link TranslationMode} for this pipeline. */
-  private static class TranslationModeDetector extends Pipeline.PipelineVisitor.Defaults {
-    private static final Logger LOG = LoggerFactory.getLogger(TranslationModeDetector.class);
+  /**
+   * {@link PTransformVisitor} that translates supported {@link PTransform PTransforms} into their
+   * Spark correspondence.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted.
+   */
+  private class TranslatingVisitor extends PTransformVisitor implements TranslationState {
+    private final Map<PCollection<?>, TranslationResult<?>> translationResults;
+    private final Map<Coder<?>, ExpressionEncoder<?>> encoders;
+    private final SparkSession sparkSession;
+    private final SerializablePipelineOptions serializableOptions;
+    private final StorageLevel storageLevel;
+
+    private final Set<TranslationResult<?>> leaves;
+
+    public TranslatingVisitor(
+        SparkSession sparkSession,
+        SparkCommonPipelineOptions options,
+        Map<PCollection<?>, TranslationResult<?>> translationResults) {
+      this.sparkSession = sparkSession;
+      this.translationResults = translationResults;
+      this.serializableOptions = new SerializablePipelineOptions(options);
+      this.storageLevel = StorageLevel.fromString(options.getStorageLevel());
+      this.encoders = new HashMap<>();
+      this.leaves = new HashSet<>();
+    }
 
-    private TranslationMode translationMode;
+    @Override
+    <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator) {
+
+      AppliedPTransform<InT, OutT, PTransform<InT, OutT>> appliedTransform =

Review Comment:
   Better use of types :+1: 



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java:
##########
@@ -81,27 +80,13 @@ public class PipelineTranslatorBatch extends PipelineTranslator {
 
     TRANSFORM_TRANSLATORS.put(
         SplittableParDo.PrimitiveBoundedRead.class, new ReadSourceTranslatorBatch<>());
-
-    TRANSFORM_TRANSLATORS.put(
-        View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch<>());
-  }
-
-  public PipelineTranslatorBatch(SparkStructuredStreamingPipelineOptions options) {
-    translationContext = new TranslationContext(options);
   }
 
-  /** Returns a translator for the given node, if it is possible, otherwise null. */
+  /** Returns a {@link TransformTranslator} for the given {@link PTransform} if known. */
   @Override
   @Nullable
   protected <InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
-      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(
-          @Nullable TransformT transform) {
-    // Root of the graph is null
-    if (transform == null) {
-      return null;
-    }
-    TransformTranslator<InT, OutT, TransformT> translator =
-        TRANSFORM_TRANSLATORS.get(transform.getClass());
-    return translator != null && translator.canTranslate(transform) ? translator : null;

Review Comment:
   you no more check that the transform can be translated here but you rather do it in the visitor. Right?



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java:
##########
@@ -17,170 +17,336 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation;
 
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
+import org.apache.beam.runners.spark.SparkCommonPipelineOptions;
+import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderProvider;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.AppliedPTransform;
-import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
-import org.checkerframework.checker.nullness.qual.Nullable;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.storage.StorageLevel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * {@link Pipeline.PipelineVisitor} that translates the Beam operators to their Spark counterparts.
- * It also does the pipeline preparation: mode detection, transforms replacement, classpath
- * preparation.
+ * The pipeline translator translates a Beam {@link Pipeline} into a Spark correspondence, that can
+ * then be evaluated.
+ *
+ * <p>The translation involves traversing the hierarchy of a pipeline multiple times:
+ *
+ * <ol>
+ *   <li>Detect if {@link StreamingOptions#setStreaming streaming} mode is required.
+ *   <li>Identify datasets that are repeatedly used as input and should be cached.
+ *   <li>And finally, translate each primitive or composite {@link PTransform} that is {@link
+ *       #getTransformTranslator known} and {@link TransformTranslator#canTranslate supported} into
+ *       its Spark correspondence. If a composite is not supported, it will be expanded further into
+ *       its parts and translated then.
+ * </ol>
  */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
+@Internal
+public abstract class PipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class);
-  protected TranslationContext translationContext;
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline preparation methods
-  // --------------------------------------------------------------------------------------------
   public static void replaceTransforms(Pipeline pipeline, StreamingOptions options) {
     pipeline.replaceAll(SparkTransformOverrides.getDefaultOverrides(options.isStreaming()));
   }
 
   /**
-   * Visit the pipeline to determine the translation mode (batch/streaming) and update options
-   * accordingly.
+   * Analyse the pipeline to determine if we have to switch to streaming mode for the pipeline
+   * translation and update {@link StreamingOptions} accordingly.
    */
-  public static void detectTranslationMode(Pipeline pipeline, StreamingOptions options) {
-    TranslationModeDetector detector = new TranslationModeDetector();
+  public static void detectStreamingMode(Pipeline pipeline, StreamingOptions options) {
+    StreamingModeDetector detector = new StreamingModeDetector(options.isStreaming());
     pipeline.traverseTopologically(detector);
-    if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) {
-      options.setStreaming(true);
+    options.setStreaming(detector.streaming);
+  }
+
+  /** Returns a {@link TransformTranslator} for the given {@link PTransform} if known. */
+  protected abstract @Nullable <
+          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
+      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(TransformT transform);
+
+  /**
+   * Translates a Beam pipeline into its Spark correspondence using the Spark SQL / Dataset API.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted to be able to continue with the translation.
+   *
+   * @return The result of the translation is an {@link EvaluationContext} that can trigger the
+   *     evaluation of the Spark pipeline.
+   */
+  public EvaluationContext translate(
+      Pipeline pipeline, SparkSession session, SparkCommonPipelineOptions options) {
+    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
+    DependencyVisitor dependencies = new DependencyVisitor();
+    pipeline.traverseTopologically(dependencies);
+
+    TranslatingVisitor translator = new TranslatingVisitor(session, options, dependencies.results);
+    pipeline.traverseTopologically(translator);
+
+    return new EvaluationContext(translator.leaves, session);
+  }
+
+  /**
+   * The correspondence of a {@link PCollection} as result of translating a {@link PTransform}
+   * including additional metadata (such as name and dependents).
+   */
+  private static final class TranslationResult<T> implements EvaluationContext.NamedDataset<T> {
+    private final String name;
+    private @Nullable Dataset<WindowedValue<T>> dataset = null;
+    private final Set<PTransform<?, ?>> dependentTransforms = new HashSet<>();
+
+    private TranslationResult(PCollection<?> pCol) {
+      this.name = pCol.getName();
+    }
+
+    @Override
+    public String name() {
+      return name;
+    }
+
+    @Override
+    public @Nullable Dataset<WindowedValue<T>> dataset() {
+      return dataset;
     }
   }
 
-  /** The translation mode of the Beam Pipeline. */
-  private enum TranslationMode {
+  /** Shared, mutable state during the translation of a pipeline and omitted afterwards. */
+  interface TranslationState extends EncoderProvider {
+    <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection);
+
+    <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache);
 
-    /** Uses the batch mode. */
-    BATCH,
+    default <T> void putDataset(PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset) {
+      putDataset(pCollection, dataset, false);
+    }
 
-    /** Uses the streaming mode. */
-    STREAMING
+    SerializablePipelineOptions getSerializableOptions();
+
+    SparkSession getSparkSession();
   }
 
-  /** Traverses the Pipeline to determine the {@link TranslationMode} for this pipeline. */
-  private static class TranslationModeDetector extends Pipeline.PipelineVisitor.Defaults {
-    private static final Logger LOG = LoggerFactory.getLogger(TranslationModeDetector.class);
+  /**
+   * {@link PTransformVisitor} that translates supported {@link PTransform PTransforms} into their
+   * Spark correspondence.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted.
+   */
+  private class TranslatingVisitor extends PTransformVisitor implements TranslationState {
+    private final Map<PCollection<?>, TranslationResult<?>> translationResults;
+    private final Map<Coder<?>, ExpressionEncoder<?>> encoders;
+    private final SparkSession sparkSession;
+    private final SerializablePipelineOptions serializableOptions;
+    private final StorageLevel storageLevel;
+
+    private final Set<TranslationResult<?>> leaves;
+
+    public TranslatingVisitor(
+        SparkSession sparkSession,
+        SparkCommonPipelineOptions options,
+        Map<PCollection<?>, TranslationResult<?>> translationResults) {
+      this.sparkSession = sparkSession;
+      this.translationResults = translationResults;
+      this.serializableOptions = new SerializablePipelineOptions(options);
+      this.storageLevel = StorageLevel.fromString(options.getStorageLevel());
+      this.encoders = new HashMap<>();
+      this.leaves = new HashSet<>();
+    }
 
-    private TranslationMode translationMode;
+    @Override
+    <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator) {
+
+      AppliedPTransform<InT, OutT, PTransform<InT, OutT>> appliedTransform =
+          (AppliedPTransform) node.toAppliedPTransform(getPipeline());
+      try {
+        LOG.info(
+            "Translating {}: {}",
+            node.isCompositeNode() ? "composite" : "primitive",
+            node.getFullName());
+        translator.translate(transform, appliedTransform, this);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
 
-    TranslationModeDetector(TranslationMode defaultMode) {
-      this.translationMode = defaultMode;
+    @Override
+    public <T> Encoder<T> encoderOf(Coder<T> coder, Factory<T> factory) {
+      return (Encoder<T>) encoders.computeIfAbsent(coder, (Factory) factory);
     }
 
-    TranslationModeDetector() {
-      this(TranslationMode.BATCH);
+    private <T> TranslationResult<T> getResult(PCollection<T> pCollection) {
+      return (TranslationResult<T>) checkStateNotNull(translationResults.get(pCollection));
     }
 
-    TranslationMode getTranslationMode() {
-      return translationMode;
+    @Override
+    public <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection) {
+      return checkStateNotNull(getResult(pCollection).dataset);
     }
 
     @Override
-    public void visitValue(PValue value, TransformHierarchy.Node producer) {
-      if (translationMode.equals(TranslationMode.BATCH)) {
-        if (value instanceof PCollection
-            && ((PCollection) value).isBounded() == PCollection.IsBounded.UNBOUNDED) {
-          LOG.info(
-              "Found unbounded PCollection {}. Switching to streaming execution.", value.getName());
-          translationMode = TranslationMode.STREAMING;
+    public <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache) {
+      TranslationResult<T> result = getResult(pCollection);
+      if (!noCache && result.dependentTransforms.size() > 1) {
+        LOG.info("Dataset {} will be cached.", result.name);
+        result.dataset = dataset.persist(storageLevel); // use NONE to disable
+      } else {
+        result.dataset = dataset;
+        if (result.dependentTransforms.isEmpty()) {
+          leaves.add(result);
         }
       }
     }
-  }
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline utility methods
-  // --------------------------------------------------------------------------------------------
+    @Override
+    public SerializablePipelineOptions getSerializableOptions() {
+      return serializableOptions;
+    }
 
-  /** Get a {@link TransformTranslator} for the given {@link TransformHierarchy.Node}. */
-  protected abstract @Nullable <
-          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
-      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(
-          @Nullable TransformT transform);
-
-  /** Apply the given TransformTranslator to the given node. */
-  private <InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
-      void applyTransformTranslator(
-          TransformHierarchy.Node node,
-          TransformT transform,
-          TransformTranslator<InT, OutT, TransformT> transformTranslator) {
-    // create the applied PTransform on the translationContext
-    AppliedPTransform<InT, OutT, PTransform<InT, OutT>> appliedTransform =
-        (AppliedPTransform) node.toAppliedPTransform(getPipeline());
-    try {
-      transformTranslator.translate(transform, appliedTransform, translationContext);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
+    @Override
+    public SparkSession getSparkSession() {
+      return sparkSession;
     }
   }
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline visitor entry point
-  // --------------------------------------------------------------------------------------------
-
   /**
-   * Translates the pipeline by passing this class as a visitor.
+   * {@link PTransformVisitor} that analyses dependencies of supported {@link PTransform
+   * PTransforms} to help identify cache candidates.
    *
-   * @param pipeline The pipeline to be translated
+   * <p>The visitor may throw if a {@link PTransform} is observed that uses unsupported features.
    */
-  public void translate(Pipeline pipeline) {
-    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
-    pipeline.traverseTopologically(this);
+  private class DependencyVisitor extends PTransformVisitor {
+    private final Map<PCollection<?>, TranslationResult<?>> results = new HashMap<>();
+
+    @Override
+    <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator) {
+      for (PCollection<?> pOut : node.getOutputs().values()) {
+        results.put(pOut, new TranslationResult<>(pOut));
+        for (Map.Entry<TupleTag<?>, PCollection<?>> entry : node.getInputs().entrySet()) {
+          TranslationResult<?> input = checkStateNotNull(results.get(entry.getValue()));
+          input.dependentTransforms.add(transform);
+        }
+      }
+    }
   }
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline Visitor Methods
-  // --------------------------------------------------------------------------------------------
+  /**
+   * An abstract {@link PipelineVisitor} that visits all translatable {@link PTransform} pipeline
+   * nodes of a pipeline with the respective {@link TransformTranslator}.
+   *
+   * <p>The visitor may throw if a {@link PTransform} is observed that uses unsupported features.
+   */
+  private abstract class PTransformVisitor extends PipelineVisitor.Defaults {
 
-  @Override
-  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
-    PTransform<PInput, POutput> transform = (PTransform<PInput, POutput>) node.getTransform();
-    TransformTranslator<PInput, POutput, PTransform<PInput, POutput>> transformTranslator =
-        getTransformTranslator(transform);
+    /** Visit the {@link PTransform} with its respective {@link TransformTranslator}. */
+    abstract <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator);
 
-    if (transformTranslator != null) {
-      LOG.info("Translating composite: {}", node.getFullName());
-      applyTransformTranslator(node, transform, transformTranslator);
-      return CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
-    } else {
-      return CompositeBehavior.ENTER_TRANSFORM;
+    @Override
+    public final CompositeBehavior enterCompositeTransform(Node node) {
+      PTransform<PInput, POutput> transform = (PTransform<PInput, POutput>) node.getTransform();
+      TransformTranslator<PInput, POutput, PTransform<PInput, POutput>> translator =
+          getTranslator(transform);
+      if (transform != null && translator != null) {
+        visit(node, transform, translator);
+        return DO_NOT_ENTER_TRANSFORM;
+      } else {
+        return ENTER_TRANSFORM;
+      }
+    }
+
+    @Override
+    public final void visitPrimitiveTransform(Node node) {
+      PTransform<PInput, POutput> transform = (PTransform<PInput, POutput>) node.getTransform();
+      if (transform == null || transform.getClass().equals(View.CreatePCollectionView.class)) {
+        return; // ignore, nothing to be translated here

Review Comment:
   cf question on [PipelineTranslatorBatch.java](https://github.com/apache/beam/pull/24009/files#diff-135056b6cae8cfffc97af038ebe5d427e0ce4f58a0c947e510a748e0879cb2b3). Can you put a comment to elaborate the PCollectionView case ?



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java:
##########
@@ -17,170 +17,336 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation;
 
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
+import org.apache.beam.runners.spark.SparkCommonPipelineOptions;
+import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderProvider;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.AppliedPTransform;
-import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
-import org.checkerframework.checker.nullness.qual.Nullable;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.storage.StorageLevel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * {@link Pipeline.PipelineVisitor} that translates the Beam operators to their Spark counterparts.
- * It also does the pipeline preparation: mode detection, transforms replacement, classpath
- * preparation.
+ * The pipeline translator translates a Beam {@link Pipeline} into a Spark correspondence, that can
+ * then be evaluated.
+ *
+ * <p>The translation involves traversing the hierarchy of a pipeline multiple times:
+ *
+ * <ol>
+ *   <li>Detect if {@link StreamingOptions#setStreaming streaming} mode is required.
+ *   <li>Identify datasets that are repeatedly used as input and should be cached.
+ *   <li>And finally, translate each primitive or composite {@link PTransform} that is {@link
+ *       #getTransformTranslator known} and {@link TransformTranslator#canTranslate supported} into
+ *       its Spark correspondence. If a composite is not supported, it will be expanded further into
+ *       its parts and translated then.
+ * </ol>
  */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
+@Internal
+public abstract class PipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class);
-  protected TranslationContext translationContext;
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline preparation methods
-  // --------------------------------------------------------------------------------------------
   public static void replaceTransforms(Pipeline pipeline, StreamingOptions options) {
     pipeline.replaceAll(SparkTransformOverrides.getDefaultOverrides(options.isStreaming()));
   }
 
   /**
-   * Visit the pipeline to determine the translation mode (batch/streaming) and update options
-   * accordingly.
+   * Analyse the pipeline to determine if we have to switch to streaming mode for the pipeline
+   * translation and update {@link StreamingOptions} accordingly.
    */
-  public static void detectTranslationMode(Pipeline pipeline, StreamingOptions options) {
-    TranslationModeDetector detector = new TranslationModeDetector();
+  public static void detectStreamingMode(Pipeline pipeline, StreamingOptions options) {
+    StreamingModeDetector detector = new StreamingModeDetector(options.isStreaming());
     pipeline.traverseTopologically(detector);
-    if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) {
-      options.setStreaming(true);
+    options.setStreaming(detector.streaming);
+  }
+
+  /** Returns a {@link TransformTranslator} for the given {@link PTransform} if known. */
+  protected abstract @Nullable <
+          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
+      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(TransformT transform);
+
+  /**
+   * Translates a Beam pipeline into its Spark correspondence using the Spark SQL / Dataset API.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted to be able to continue with the translation.
+   *
+   * @return The result of the translation is an {@link EvaluationContext} that can trigger the
+   *     evaluation of the Spark pipeline.
+   */
+  public EvaluationContext translate(
+      Pipeline pipeline, SparkSession session, SparkCommonPipelineOptions options) {
+    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
+    DependencyVisitor dependencies = new DependencyVisitor();
+    pipeline.traverseTopologically(dependencies);
+
+    TranslatingVisitor translator = new TranslatingVisitor(session, options, dependencies.results);
+    pipeline.traverseTopologically(translator);
+
+    return new EvaluationContext(translator.leaves, session);
+  }
+
+  /**
+   * The correspondence of a {@link PCollection} as result of translating a {@link PTransform}
+   * including additional metadata (such as name and dependents).
+   */
+  private static final class TranslationResult<T> implements EvaluationContext.NamedDataset<T> {
+    private final String name;
+    private @Nullable Dataset<WindowedValue<T>> dataset = null;
+    private final Set<PTransform<?, ?>> dependentTransforms = new HashSet<>();
+
+    private TranslationResult(PCollection<?> pCol) {
+      this.name = pCol.getName();
+    }
+
+    @Override
+    public String name() {
+      return name;
+    }
+
+    @Override
+    public @Nullable Dataset<WindowedValue<T>> dataset() {
+      return dataset;
     }
   }
 
-  /** The translation mode of the Beam Pipeline. */
-  private enum TranslationMode {
+  /** Shared, mutable state during the translation of a pipeline and omitted afterwards. */
+  interface TranslationState extends EncoderProvider {
+    <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection);
+
+    <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache);
 
-    /** Uses the batch mode. */
-    BATCH,
+    default <T> void putDataset(PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset) {
+      putDataset(pCollection, dataset, false);
+    }
 
-    /** Uses the streaming mode. */
-    STREAMING
+    SerializablePipelineOptions getSerializableOptions();
+
+    SparkSession getSparkSession();
   }
 
-  /** Traverses the Pipeline to determine the {@link TranslationMode} for this pipeline. */
-  private static class TranslationModeDetector extends Pipeline.PipelineVisitor.Defaults {
-    private static final Logger LOG = LoggerFactory.getLogger(TranslationModeDetector.class);
+  /**
+   * {@link PTransformVisitor} that translates supported {@link PTransform PTransforms} into their
+   * Spark correspondence.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted.
+   */
+  private class TranslatingVisitor extends PTransformVisitor implements TranslationState {
+    private final Map<PCollection<?>, TranslationResult<?>> translationResults;
+    private final Map<Coder<?>, ExpressionEncoder<?>> encoders;
+    private final SparkSession sparkSession;
+    private final SerializablePipelineOptions serializableOptions;
+    private final StorageLevel storageLevel;
+
+    private final Set<TranslationResult<?>> leaves;
+
+    public TranslatingVisitor(
+        SparkSession sparkSession,
+        SparkCommonPipelineOptions options,
+        Map<PCollection<?>, TranslationResult<?>> translationResults) {
+      this.sparkSession = sparkSession;
+      this.translationResults = translationResults;
+      this.serializableOptions = new SerializablePipelineOptions(options);
+      this.storageLevel = StorageLevel.fromString(options.getStorageLevel());
+      this.encoders = new HashMap<>();
+      this.leaves = new HashSet<>();
+    }
 
-    private TranslationMode translationMode;
+    @Override
+    <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator) {
+
+      AppliedPTransform<InT, OutT, PTransform<InT, OutT>> appliedTransform =
+          (AppliedPTransform) node.toAppliedPTransform(getPipeline());
+      try {
+        LOG.info(
+            "Translating {}: {}",
+            node.isCompositeNode() ? "composite" : "primitive",
+            node.getFullName());
+        translator.translate(transform, appliedTransform, this);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
 
-    TranslationModeDetector(TranslationMode defaultMode) {
-      this.translationMode = defaultMode;
+    @Override
+    public <T> Encoder<T> encoderOf(Coder<T> coder, Factory<T> factory) {
+      return (Encoder<T>) encoders.computeIfAbsent(coder, (Factory) factory);
     }
 
-    TranslationModeDetector() {
-      this(TranslationMode.BATCH);
+    private <T> TranslationResult<T> getResult(PCollection<T> pCollection) {
+      return (TranslationResult<T>) checkStateNotNull(translationResults.get(pCollection));
     }
 
-    TranslationMode getTranslationMode() {
-      return translationMode;
+    @Override
+    public <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection) {
+      return checkStateNotNull(getResult(pCollection).dataset);
     }
 
     @Override
-    public void visitValue(PValue value, TransformHierarchy.Node producer) {
-      if (translationMode.equals(TranslationMode.BATCH)) {
-        if (value instanceof PCollection
-            && ((PCollection) value).isBounded() == PCollection.IsBounded.UNBOUNDED) {
-          LOG.info(
-              "Found unbounded PCollection {}. Switching to streaming execution.", value.getName());
-          translationMode = TranslationMode.STREAMING;
+    public <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache) {
+      TranslationResult<T> result = getResult(pCollection);
+      if (!noCache && result.dependentTransforms.size() > 1) {

Review Comment:
   nit: for symmetry with else branch: use `!result.dependentTransforms.isEmpty()`



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java:
##########
@@ -17,170 +17,336 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation;
 
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
+import static org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
+import org.apache.beam.runners.spark.SparkCommonPipelineOptions;
+import org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderProvider;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.AppliedPTransform;
-import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
-import org.checkerframework.checker.nullness.qual.Nullable;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
+import org.apache.spark.storage.StorageLevel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * {@link Pipeline.PipelineVisitor} that translates the Beam operators to their Spark counterparts.
- * It also does the pipeline preparation: mode detection, transforms replacement, classpath
- * preparation.
+ * The pipeline translator translates a Beam {@link Pipeline} into a Spark correspondence, that can
+ * then be evaluated.
+ *
+ * <p>The translation involves traversing the hierarchy of a pipeline multiple times:
+ *
+ * <ol>
+ *   <li>Detect if {@link StreamingOptions#setStreaming streaming} mode is required.
+ *   <li>Identify datasets that are repeatedly used as input and should be cached.
+ *   <li>And finally, translate each primitive or composite {@link PTransform} that is {@link
+ *       #getTransformTranslator known} and {@link TransformTranslator#canTranslate supported} into
+ *       its Spark correspondence. If a composite is not supported, it will be expanded further into
+ *       its parts and translated then.
+ * </ol>
  */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
-public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
+@Internal
+public abstract class PipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class);
-  protected TranslationContext translationContext;
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline preparation methods
-  // --------------------------------------------------------------------------------------------
   public static void replaceTransforms(Pipeline pipeline, StreamingOptions options) {
     pipeline.replaceAll(SparkTransformOverrides.getDefaultOverrides(options.isStreaming()));
   }
 
   /**
-   * Visit the pipeline to determine the translation mode (batch/streaming) and update options
-   * accordingly.
+   * Analyse the pipeline to determine if we have to switch to streaming mode for the pipeline
+   * translation and update {@link StreamingOptions} accordingly.
    */
-  public static void detectTranslationMode(Pipeline pipeline, StreamingOptions options) {
-    TranslationModeDetector detector = new TranslationModeDetector();
+  public static void detectStreamingMode(Pipeline pipeline, StreamingOptions options) {
+    StreamingModeDetector detector = new StreamingModeDetector(options.isStreaming());
     pipeline.traverseTopologically(detector);
-    if (detector.getTranslationMode().equals(TranslationMode.STREAMING)) {
-      options.setStreaming(true);
+    options.setStreaming(detector.streaming);
+  }
+
+  /** Returns a {@link TransformTranslator} for the given {@link PTransform} if known. */
+  protected abstract @Nullable <
+          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
+      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(TransformT transform);
+
+  /**
+   * Translates a Beam pipeline into its Spark correspondence using the Spark SQL / Dataset API.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted to be able to continue with the translation.
+   *
+   * @return The result of the translation is an {@link EvaluationContext} that can trigger the
+   *     evaluation of the Spark pipeline.
+   */
+  public EvaluationContext translate(
+      Pipeline pipeline, SparkSession session, SparkCommonPipelineOptions options) {
+    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
+    DependencyVisitor dependencies = new DependencyVisitor();
+    pipeline.traverseTopologically(dependencies);
+
+    TranslatingVisitor translator = new TranslatingVisitor(session, options, dependencies.results);
+    pipeline.traverseTopologically(translator);
+
+    return new EvaluationContext(translator.leaves, session);
+  }
+
+  /**
+   * The correspondence of a {@link PCollection} as result of translating a {@link PTransform}
+   * including additional metadata (such as name and dependents).
+   */
+  private static final class TranslationResult<T> implements EvaluationContext.NamedDataset<T> {
+    private final String name;
+    private @Nullable Dataset<WindowedValue<T>> dataset = null;
+    private final Set<PTransform<?, ?>> dependentTransforms = new HashSet<>();
+
+    private TranslationResult(PCollection<?> pCol) {
+      this.name = pCol.getName();
+    }
+
+    @Override
+    public String name() {
+      return name;
+    }
+
+    @Override
+    public @Nullable Dataset<WindowedValue<T>> dataset() {
+      return dataset;
     }
   }
 
-  /** The translation mode of the Beam Pipeline. */
-  private enum TranslationMode {
+  /** Shared, mutable state during the translation of a pipeline and omitted afterwards. */
+  interface TranslationState extends EncoderProvider {
+    <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection);
+
+    <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache);
 
-    /** Uses the batch mode. */
-    BATCH,
+    default <T> void putDataset(PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset) {
+      putDataset(pCollection, dataset, false);
+    }
 
-    /** Uses the streaming mode. */
-    STREAMING
+    SerializablePipelineOptions getSerializableOptions();
+
+    SparkSession getSparkSession();
   }
 
-  /** Traverses the Pipeline to determine the {@link TranslationMode} for this pipeline. */
-  private static class TranslationModeDetector extends Pipeline.PipelineVisitor.Defaults {
-    private static final Logger LOG = LoggerFactory.getLogger(TranslationModeDetector.class);
+  /**
+   * {@link PTransformVisitor} that translates supported {@link PTransform PTransforms} into their
+   * Spark correspondence.
+   *
+   * <p>Note, in some cases this involves the early evaluation of some parts of the pipeline. For
+   * example, in order to use a side-input {@link org.apache.beam.sdk.values.PCollectionView
+   * PCollectionView} in a translation the corresponding Spark {@link
+   * org.apache.beam.runners.spark.translation.Dataset Dataset} might have to be collected and
+   * broadcasted.
+   */
+  private class TranslatingVisitor extends PTransformVisitor implements TranslationState {
+    private final Map<PCollection<?>, TranslationResult<?>> translationResults;
+    private final Map<Coder<?>, ExpressionEncoder<?>> encoders;
+    private final SparkSession sparkSession;
+    private final SerializablePipelineOptions serializableOptions;
+    private final StorageLevel storageLevel;
+
+    private final Set<TranslationResult<?>> leaves;
+
+    public TranslatingVisitor(
+        SparkSession sparkSession,
+        SparkCommonPipelineOptions options,
+        Map<PCollection<?>, TranslationResult<?>> translationResults) {
+      this.sparkSession = sparkSession;
+      this.translationResults = translationResults;
+      this.serializableOptions = new SerializablePipelineOptions(options);
+      this.storageLevel = StorageLevel.fromString(options.getStorageLevel());
+      this.encoders = new HashMap<>();
+      this.leaves = new HashSet<>();
+    }
 
-    private TranslationMode translationMode;
+    @Override
+    <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator) {
+
+      AppliedPTransform<InT, OutT, PTransform<InT, OutT>> appliedTransform =
+          (AppliedPTransform) node.toAppliedPTransform(getPipeline());
+      try {
+        LOG.info(
+            "Translating {}: {}",
+            node.isCompositeNode() ? "composite" : "primitive",
+            node.getFullName());
+        translator.translate(transform, appliedTransform, this);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
 
-    TranslationModeDetector(TranslationMode defaultMode) {
-      this.translationMode = defaultMode;
+    @Override
+    public <T> Encoder<T> encoderOf(Coder<T> coder, Factory<T> factory) {
+      return (Encoder<T>) encoders.computeIfAbsent(coder, (Factory) factory);
     }
 
-    TranslationModeDetector() {
-      this(TranslationMode.BATCH);
+    private <T> TranslationResult<T> getResult(PCollection<T> pCollection) {
+      return (TranslationResult<T>) checkStateNotNull(translationResults.get(pCollection));
     }
 
-    TranslationMode getTranslationMode() {
-      return translationMode;
+    @Override
+    public <T> Dataset<WindowedValue<T>> getDataset(PCollection<T> pCollection) {
+      return checkStateNotNull(getResult(pCollection).dataset);
     }
 
     @Override
-    public void visitValue(PValue value, TransformHierarchy.Node producer) {
-      if (translationMode.equals(TranslationMode.BATCH)) {
-        if (value instanceof PCollection
-            && ((PCollection) value).isBounded() == PCollection.IsBounded.UNBOUNDED) {
-          LOG.info(
-              "Found unbounded PCollection {}. Switching to streaming execution.", value.getName());
-          translationMode = TranslationMode.STREAMING;
+    public <T> void putDataset(
+        PCollection<T> pCollection, Dataset<WindowedValue<T>> dataset, boolean noCache) {
+      TranslationResult<T> result = getResult(pCollection);
+      if (!noCache && result.dependentTransforms.size() > 1) {
+        LOG.info("Dataset {} will be cached.", result.name);
+        result.dataset = dataset.persist(storageLevel); // use NONE to disable
+      } else {
+        result.dataset = dataset;
+        if (result.dependentTransforms.isEmpty()) {
+          leaves.add(result);
         }
       }
     }
-  }
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline utility methods
-  // --------------------------------------------------------------------------------------------
+    @Override
+    public SerializablePipelineOptions getSerializableOptions() {
+      return serializableOptions;
+    }
 
-  /** Get a {@link TransformTranslator} for the given {@link TransformHierarchy.Node}. */
-  protected abstract @Nullable <
-          InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
-      TransformTranslator<InT, OutT, TransformT> getTransformTranslator(
-          @Nullable TransformT transform);
-
-  /** Apply the given TransformTranslator to the given node. */
-  private <InT extends PInput, OutT extends POutput, TransformT extends PTransform<InT, OutT>>
-      void applyTransformTranslator(
-          TransformHierarchy.Node node,
-          TransformT transform,
-          TransformTranslator<InT, OutT, TransformT> transformTranslator) {
-    // create the applied PTransform on the translationContext
-    AppliedPTransform<InT, OutT, PTransform<InT, OutT>> appliedTransform =
-        (AppliedPTransform) node.toAppliedPTransform(getPipeline());
-    try {
-      transformTranslator.translate(transform, appliedTransform, translationContext);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
+    @Override
+    public SparkSession getSparkSession() {
+      return sparkSession;
     }
   }
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline visitor entry point
-  // --------------------------------------------------------------------------------------------
-
   /**
-   * Translates the pipeline by passing this class as a visitor.
+   * {@link PTransformVisitor} that analyses dependencies of supported {@link PTransform
+   * PTransforms} to help identify cache candidates.
    *
-   * @param pipeline The pipeline to be translated
+   * <p>The visitor may throw if a {@link PTransform} is observed that uses unsupported features.
    */
-  public void translate(Pipeline pipeline) {
-    LOG.debug("starting translation of the pipeline using {}", getClass().getName());
-    pipeline.traverseTopologically(this);
+  private class DependencyVisitor extends PTransformVisitor {
+    private final Map<PCollection<?>, TranslationResult<?>> results = new HashMap<>();
+
+    @Override
+    <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator) {
+      for (PCollection<?> pOut : node.getOutputs().values()) {
+        results.put(pOut, new TranslationResult<>(pOut));
+        for (Map.Entry<TupleTag<?>, PCollection<?>> entry : node.getInputs().entrySet()) {
+          TranslationResult<?> input = checkStateNotNull(results.get(entry.getValue()));
+          input.dependentTransforms.add(transform);
+        }
+      }
+    }
   }
 
-  // --------------------------------------------------------------------------------------------
-  //  Pipeline Visitor Methods
-  // --------------------------------------------------------------------------------------------
+  /**
+   * An abstract {@link PipelineVisitor} that visits all translatable {@link PTransform} pipeline
+   * nodes of a pipeline with the respective {@link TransformTranslator}.
+   *
+   * <p>The visitor may throw if a {@link PTransform} is observed that uses unsupported features.
+   */
+  private abstract class PTransformVisitor extends PipelineVisitor.Defaults {
 
-  @Override
-  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
-    PTransform<PInput, POutput> transform = (PTransform<PInput, POutput>) node.getTransform();
-    TransformTranslator<PInput, POutput, PTransform<PInput, POutput>> transformTranslator =
-        getTransformTranslator(transform);
+    /** Visit the {@link PTransform} with its respective {@link TransformTranslator}. */
+    abstract <InT extends PInput, OutT extends POutput> void visit(
+        Node node,
+        PTransform<InT, OutT> transform,
+        TransformTranslator<InT, OutT, PTransform<InT, OutT>> translator);
 
-    if (transformTranslator != null) {
-      LOG.info("Translating composite: {}", node.getFullName());
-      applyTransformTranslator(node, transform, transformTranslator);
-      return CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
-    } else {
-      return CompositeBehavior.ENTER_TRANSFORM;
+    @Override
+    public final CompositeBehavior enterCompositeTransform(Node node) {
+      PTransform<PInput, POutput> transform = (PTransform<PInput, POutput>) node.getTransform();
+      TransformTranslator<PInput, POutput, PTransform<PInput, POutput>> translator =
+          getTranslator(transform);
+      if (transform != null && translator != null) {
+        visit(node, transform, translator);
+        return DO_NOT_ENTER_TRANSFORM;
+      } else {
+        return ENTER_TRANSFORM;
+      }
+    }
+
+    @Override
+    public final void visitPrimitiveTransform(Node node) {
+      PTransform<PInput, POutput> transform = (PTransform<PInput, POutput>) node.getTransform();
+      if (transform == null || transform.getClass().equals(View.CreatePCollectionView.class)) {
+        return; // ignore, nothing to be translated here
+      }
+      TransformTranslator<PInput, POutput, PTransform<PInput, POutput>> translator =
+          getTranslator(transform);
+      if (translator == null) {
+        String urn = PTransformTranslation.urnForTransform(transform);
+        throw new UnsupportedOperationException("Transform " + urn + " is not supported.");
+      }
+      visit(node, transform, translator);
     }
-  }
 
-  @Override
-  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
-    LOG.info("Translating primitive: {}", node.getFullName());
-    // get the transformation corresponding to the node we are
-    // currently visiting and translate it into its Spark alternative.
-    PTransform<PInput, POutput> transform = (PTransform<PInput, POutput>) node.getTransform();
-    TransformTranslator<PInput, POutput, PTransform<PInput, POutput>> transformTranslator =
-        getTransformTranslator(transform);
-
-    if (transformTranslator == null) {
-      String transformUrn = PTransformTranslation.urnForTransform(node.getTransform());
-      throw new UnsupportedOperationException(
-          "The transform " + transformUrn + " is currently not supported.");
-    }
-    applyTransformTranslator(node, transform, transformTranslator);
+    /** {@link TransformTranslator} for {@link PTransform} if translation is known and supported. */
+    private @Nullable TransformTranslator<PInput, POutput, PTransform<PInput, POutput>>
+        getTranslator(@Nullable PTransform<PInput, POutput> transform) {
+      if (transform == null) {
+        return null;
+      }
+      TransformTranslator<PInput, POutput, PTransform<PInput, POutput>> translator =
+          getTransformTranslator(transform);
+      return translator != null && translator.canTranslate(transform) ? translator : null;
+    }
   }
 
-  public TranslationContext getTranslationContext() {
-    return translationContext;
+  /**
+   * Traverse the pipeline to check for unbounded {@link PCollection PCollections} that would
+   * require streaming mode unless streaming mode is already enabled.
+   */
+  private static class StreamingModeDetector extends PipelineVisitor.Defaults {
+    private boolean streaming;
+
+    StreamingModeDetector(boolean streaming) {
+      this.streaming = streaming;
+    }
+
+    @Override
+    public CompositeBehavior enterCompositeTransform(Node node) {
+      return streaming ? DO_NOT_ENTER_TRANSFORM : ENTER_TRANSFORM; // stop if in streaming mode

Review Comment:
   good improvement to stop traversing the hierarchy if the streaming mode is already forced !



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org