gaoyunhaii commented on code in PR #20147:
URL: https://github.com/apache/flink/pull/20147#discussion_r929639303


##########
flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java:
##########
@@ -184,4 +188,24 @@ CompletableFuture<String> triggerSavepoint(
      */
     CompletableFuture<CoordinationResponse> sendCoordinationRequest(
             JobID jobId, OperatorID operatorId, CoordinationRequest request);
+
+    /**
+     * Return a set of ids of the completely cached intermediate dataset.
+     *
+     * @return A set of ids of the completely cached intermediate dataset.
+     */
+    default CompletableFuture<Set<AbstractID>> listCachedIntermediateDataset() 
{

Review Comment:
   Might be renamed to `listClusterDatasets`.



##########
flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java:
##########
@@ -184,4 +188,24 @@ CompletableFuture<String> triggerSavepoint(
      */
     CompletableFuture<CoordinationResponse> sendCoordinationRequest(
             JobID jobId, OperatorID operatorId, CoordinationRequest request);
+
+    /**
+     * Return a set of ids of the completely cached intermediate dataset.
+     *
+     * @return A set of ids of the completely cached intermediate dataset.
+     */
+    default CompletableFuture<Set<AbstractID>> listCachedIntermediateDataset() 
{
+        return CompletableFuture.completedFuture(Collections.emptySet());
+    }
+
+    /**
+     * Invalidate the cached intermediate dataset with the given id.
+     *
+     * @param intermediateDataSetId id of the cached intermediate dataset to 
be invalidated.
+     * @return Future which will be completed when the cached dataset is 
invalidated.
+     */
+    default CompletableFuture<Void> invalidateCachedIntermediateDataset(

Review Comment:
   Might be renamed to `invalidateClusterDataset(AbstractId datasetId)`



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java:
##########
@@ -2203,13 +2197,24 @@ public StreamGraph getStreamGraph() {
      */
     @Internal
     public StreamGraph getStreamGraph(boolean clearTransformations) {
+        updateCacheTransformation();

Review Comment:
   Might be renamed to `synchronizeClusterDatasetStatus`. 
   
   We should skip this check at least if `cachedTransformation.size() == 0`. If 
possible I think we should also skip the check if all the transformation has 
been marked as `cached`. 



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java:
##########
@@ -2591,4 +2596,70 @@ private <OUT, T extends TypeInformation<OUT>> T 
getTypeInfo(
     public List<Transformation<?>> getTransformations() {
         return transformations;
     }
+
+    @Internal
+    public <T> void addCache(AbstractID intermediateDataSetID, 
CacheTransformation<T> t) {

Review Comment:
   might be renamed to `registerCachedTransformation`?



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java:
##########
@@ -611,14 +612,25 @@ public String getSlotSharingGroup(Integer id) {
     }
 
     public void addEdge(Integer upStreamVertexID, Integer downStreamVertexID, 
int typeNumber) {
+        addEdge(upStreamVertexID, downStreamVertexID, typeNumber, null, false);
+    }
+
+    public void addEdge(
+            Integer upStreamVertexID,
+            Integer downStreamVertexID,
+            int typeNumber,
+            IntermediateDataSetID intermediateDataSetId,
+            boolean shouldCache) {

Review Comment:
   `shouldCache` might be removed for all the methods regarding addEdge in 
`StreamGraph` and `StreamEdge`?



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CachedDataStream.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.flink.streaming.api.datastream;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.transformations.CacheTransformation;
+
+/**
+ * {@link CachedDataStream} represents a {@link DataStream} whose intermediate 
result will be cached
+ * at the first time when it is computed. And the cached intermediate result 
can be used in later
+ * job that using the same {@link CachedDataStream} to avoid re-computing the 
intermediate result.
+ *
+ * @param <T> The type of the elements in this stream.
+ */
+@PublicEvolving
+public class CachedDataStream<T> extends DataStream<T> implements 
AutoCloseable {
+    /**
+     * Create a new {@link CachedDataStream} in the given execution 
environment that wrap the given
+     * physical transformation to indicates that the transformation should be 
cached.
+     *
+     * @param environment The StreamExecutionEnvironment
+     * @param transformation The physical transformation whose intermediate 
result should be cached.
+     */
+    public CachedDataStream(
+            StreamExecutionEnvironment environment, Transformation<T> 
transformation) {
+        super(
+                environment,
+                new CacheTransformation<>(
+                        transformation,
+                        String.format("Cache: %s", transformation.getName()),
+                        transformation.getOutputType(),
+                        transformation.getParallelism()));
+
+        final CacheTransformation<T> t = (CacheTransformation<T>) 
this.getTransformation();
+        environment.addCache(t.getIntermediateDataSetID(), t);
+    }
+
+    /**
+     * Invalidate the cache intermediate result of this DataStream to release 
the physical
+     * resources. Users are not required to invoke this method to release 
physical resources unless
+     * they want to. The CachedDataStream should not be used after it is 
closed.
+     */
+    @Override
+    public void close() throws Exception {

Review Comment:
   I still tend to use `invalidate` since I think it should also has the 
meaning of not be able to be used, but I think I might not very strongly argue 
for that. 



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java:
##########
@@ -399,7 +399,7 @@ public SingleOutputStreamOperator<T> 
slotSharingGroup(SlotSharingGroup slotShari
      * @see 
org.apache.flink.streaming.api.functions.ProcessFunction.Context#output(OutputTag,
      *     Object)
      */
-    public <X> DataStream<X> getSideOutput(OutputTag<X> sideOutputTag) {
+    public <X> SingleOutputStreamOperator<X> getSideOutput(OutputTag<X> 
sideOutputTag) {

Review Comment:
   I'm a bit concern on this change in that with this change: sideOutput does 
not has a corresponding operator, this means if users write code like
   
   ```
   a.getSideOutput(...).setMaxParallelism(xxx);
   ```
   The set value would be just ignored, which might cause confusion.
   
   Is it possible we do not make this change, instead, we simply move cache() 
method to the `DataStream` class? Currently it seems we have already have 
checks on the transformation types. 
   



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/CacheTransformation.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.flink.streaming.api.transformations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.util.AbstractID;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * When in batch mode, the {@link CacheTransformation} represents the 
intermediate result of the
+ * upper stream should be cached when it is computed at the first time. And it 
consumes the cached
+ * intermediate result in later jobs. In stream mode, it has no affect.
+ *
+ * @param <T> The type of the elements in the cache intermediate result.
+ */
+@Internal
+public class CacheTransformation<T> extends Transformation<T> {
+    private final Transformation<T> transformationToCache;
+    private final AbstractID intermediateDataSetID;

Review Comment:
   Might be renamed to `datasetId` ?



##########
flink-core/src/main/java/org/apache/flink/core/execution/CacheSupportedPipelineExecutor.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.flink.core.execution;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.AbstractID;
+
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+/** The pipeline executor that support caching intermediate dataset. */
+@Internal
+public interface CacheSupportedPipelineExecutor extends PipelineExecutor {
+
+    /**
+     * Return a set of ids of the completely cached intermediate dataset.
+     *
+     * @param configuration the {@link Configuration} with the required 
parameters
+     * @param userCodeClassloader the {@link ClassLoader} to deserialize 
usercode
+     * @return A set of ids of the completely cached intermediate dataset.
+     */
+    CompletableFuture<Set<AbstractID>> listCachedIntermediateDataset(

Review Comment:
   Should we unify the terminology to `ClusterDataset` ?
   We might change the method to `listClusterDatasets()`.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java:
##########
@@ -2591,4 +2596,70 @@ private <OUT, T extends TypeInformation<OUT>> T 
getTypeInfo(
     public List<Transformation<?>> getTransformations() {
         return transformations;
     }
+
+    @Internal
+    public <T> void addCache(AbstractID intermediateDataSetID, 
CacheTransformation<T> t) {
+        cachedTransformations.put(intermediateDataSetID, t);
+    }
+
+    @Internal
+    public void invalidateCache(AbstractID intermediateDataSetId) throws 
Exception {

Review Comment:
   Might be renamed to `invalidateClusterDataset(AbstractId datasetId)`.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/CacheTransformation.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.flink.streaming.api.transformations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * When in batch mode, the {@link CacheTransformation} represents the 
intermediate result of the
+ * upper stream should be cached when it is computed at the first time. And it 
consumes the cached
+ * intermediate result in later jobs. In stream mode, it has no affect.
+ *
+ * @param <T>
+ */
+@Internal
+public class CacheTransformation<T> extends Transformation<T> {
+    private final Transformation<T> transformationToCache;
+    private final IntermediateDataSetID intermediateDataSetID;
+    private boolean isCached;
+    /**
+     * Creates a new {@code Transformation} with the given name, output type 
and parallelism.
+     *
+     * @param name The name of the {@code Transformation}, this will be shown 
in Visualizations and
+     *     the Log
+     * @param outputType The output type of this {@code Transformation}
+     * @param parallelism The parallelism of this {@code Transformation}
+     */
+    public CacheTransformation(
+            Transformation<T> transformationToCache,
+            String name,
+            TypeInformation<T> outputType,
+            int parallelism) {
+        super(name, outputType, parallelism);
+        this.transformationToCache = transformationToCache;
+
+        this.intermediateDataSetID = new IntermediateDataSetID();
+        this.isCached = false;
+    }
+
+    @Override
+    public List<Transformation<?>> getTransitivePredecessors() {
+        List<Transformation<?>> result = Lists.newArrayList();
+        if (isCached) {
+            return result;
+        }
+        result.add(this);

Review Comment:
   I think `this` should also be added when `isCached = true` ?



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java:
##########
@@ -2591,4 +2596,70 @@ private <OUT, T extends TypeInformation<OUT>> T 
getTypeInfo(
     public List<Transformation<?>> getTransformations() {
         return transformations;
     }
+
+    @Internal
+    public <T> void addCache(AbstractID intermediateDataSetID, 
CacheTransformation<T> t) {
+        cachedTransformations.put(intermediateDataSetID, t);
+    }
+
+    @Internal
+    public void invalidateCache(AbstractID intermediateDataSetId) throws 
Exception {
+        if (!cachedTransformations.containsKey(intermediateDataSetId)) {
+            throw new RuntimeException(
+                    String.format("IntermediateDataset %s is not found", 
intermediateDataSetId));
+        }
+        final PipelineExecutor executor = getPipelineExecutor();
+
+        if (!(executor instanceof CacheSupportedPipelineExecutor)) {
+            return;
+        }
+
+        ((CacheSupportedPipelineExecutor) executor)
+                .invalidateCachedIntermediateDataset(
+                        intermediateDataSetId, configuration, userClassloader)
+                .get();
+    }
+
+    @Internal
+    public Set<AbstractID> listCachedIntermediateDatasetIds() {

Review Comment:
   Could we make this method private?
   
   Might be renamed to `listClusterDatasets`. 



##########
flink-core/src/main/java/org/apache/flink/core/execution/CacheSupportedPipelineExecutor.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.flink.core.execution;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.AbstractID;
+
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+/** The pipeline executor that support caching intermediate dataset. */
+@Internal
+public interface CacheSupportedPipelineExecutor extends PipelineExecutor {
+
+    /**
+     * Return a set of ids of the completely cached intermediate dataset.
+     *
+     * @param configuration the {@link Configuration} with the required 
parameters
+     * @param userCodeClassloader the {@link ClassLoader} to deserialize 
usercode
+     * @return A set of ids of the completely cached intermediate dataset.
+     */
+    CompletableFuture<Set<AbstractID>> listCachedIntermediateDataset(
+            final Configuration configuration, final ClassLoader 
userCodeClassloader)
+            throws Exception;
+
+    /**
+     * Invalidate the cached intermediate dataset with the given id.
+     *
+     * @param intermediateDataSetId id of the cached intermediate dataset to 
be invalidated.
+     * @param configuration the {@link Configuration} with the required 
parameters
+     * @param userCodeClassloader the {@link ClassLoader} to deserialize 
usercode
+     * @return Future which will be completed when the cached dataset is 
invalidated.
+     */
+    CompletableFuture<Void> invalidateCachedIntermediateDataset(

Review Comment:
   Might be renamed to `invalidateClusterDataset(AbstractID datasetId)`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to