dawidwys commented on a change in pull request #13556:
URL: https://github.com/apache/flink/pull/13556#discussion_r501796258



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/BaseSimpleTransformationTranslator.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.graph;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.streaming.api.transformations.PhysicalTransformation;
+
+import java.util.Collection;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A base class for all {@link StreamGraphTranslator StreamGraphTranslators} 
who translate
+ * {@link Transformation Transformations} that have a single operator in their 
runtime implementation.
+ * These include most of the currently supported operations.
+ *
+ * @param <OUT> The type of the output elements of the transformation being 
translated.
+ * @param <T> The type of transformation being translated.
+ */
+@Internal
+public abstract class BaseSimpleTransformationTranslator<OUT, T extends 
Transformation<OUT>>
+               implements StreamGraphTranslator<OUT, T> {
+
+       @Override
+       public Collection<Integer> translateForBatch(final T transformation, 
final Context context) {
+               checkNotNull(transformation);
+               checkNotNull(context);
+
+               final Collection<Integer> transformedIds =
+                               translateForBatchInternal(transformation, 
context);
+               configure(transformation, context);
+               return transformedIds;
+       }
+
+       @Override
+       public Collection<Integer> translateForStreaming(final T 
transformation, final Context context) {
+               checkNotNull(transformation);
+               checkNotNull(context);
+
+               final Collection<Integer> transformedIds =
+                               translateForStreamingInternal(transformation, 
context);
+               configure(transformation, context);
+               return transformedIds;
+       }
+
+       /**
+        * Translates a given {@link Transformation} to its runtime 
implementation for
+        * {@link org.apache.flink.streaming.api.RuntimeExecutionMode#BATCH 
BATCH-style} execution.
+        *
+        * @param transformation The transformation to be translated.
+        * @param context The translation context.
+        * @return The ids of the nodes corresponding to this transformation in 
the transformation graph.
+        */
+       protected abstract Collection<Integer> translateForBatchInternal(final 
T transformation, final Context context);
+
+       /**
+        * Translates a given {@link Transformation} to its runtime 
implementation for
+        * {@link org.apache.flink.streaming.api.RuntimeExecutionMode#STREAMING 
STREAMING-style} execution.
+        *
+        * @param transformation The transformation to be translated.
+        * @param context The translation context.
+        * @return The ids of the nodes corresponding to this transformation in 
the transformation graph.
+        */
+       protected abstract Collection<Integer> 
translateForStreamingInternal(final T transformation, final Context context);
+
+       private void configure(final T transformation, final Context context) {
+               final StreamGraph streamGraph = context.getStreamGraph();
+               final int transformationId = transformation.getId();
+
+               if (transformation.getBufferTimeout() >= 0) {
+                       streamGraph.setBufferTimeout(transformationId, 
transformation.getBufferTimeout());
+               } else {
+                       streamGraph.setBufferTimeout(transformationId, 
context.getDefaultBufferTimeout());
+               }
+
+               if (transformation.getUid() != null) {
+                       streamGraph.setTransformationUID(transformationId, 
transformation.getUid());
+               }
+               if (transformation.getUserProvidedNodeHash() != null) {
+                       streamGraph.setTransformationUserHash(transformationId, 
transformation.getUserProvidedNodeHash());
+               }
+
+               if 
(!streamGraph.getExecutionConfig().hasAutoGeneratedUIDsEnabled()) {
+                       if (transformation instanceof PhysicalTransformation
+                                       && 
transformation.getUserProvidedNodeHash() == null
+                                       && transformation.getUid() == null) {
+                               throw new IllegalStateException("Auto generated 
UIDs have been disabled " +
+                                               "but no UID or hash has been 
assigned to operator " + transformation.getName());
+                       }
+               }
+
+               if (transformation.getMinResources() != null && 
transformation.getPreferredResources() != null) {
+                       streamGraph.setResources(transformationId, 
transformation.getMinResources(), transformation.getPreferredResources());
+               }
+
+               streamGraph.setManagedMemoryUseCaseWeights(

Review comment:
       Just a quick comment here. I think we might need to set different 
weights/use cases depending on the runtime mode. E.g. in the BATCH runtime we 
will declare `ManagedMemoryUseCase.BATCH_OP`, whereas for streaming 
`ManagedMemoryUseCase.ROCKS_DB`.
   
   I am still not entirely sure, how this should all play together, but thats a 
concern that I have.

##########
File path: flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java
##########
@@ -513,6 +514,12 @@ public long getBufferTimeout() {
         */
        public abstract Collection<Transformation<?>> 
getTransitivePredecessors();
 
+       /**
+        * Returns the {@link Transformation transformations} that are the
+        * immediate predecessors of the current transformation in the 
transformation graph.
+        */
+       public abstract List<Transformation<?>> getInputs();

Review comment:
       very nitpicking: How about we unify the return type of 
`getTransitivePredecessors` & `getInputs` ?

##########
File path: 
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java
##########
@@ -445,9 +443,10 @@ public void testParallelismOnLimitPushDown() {
                RelNode relNode = 
planner.optimize(TableTestUtil.toRelNode(table));
                ExecNode execNode = 
planner.translateToExecNodePlan(toScala(Collections.singletonList(relNode))).get(0);
                @SuppressWarnings("unchecked")
-               Transformation transformation = 
execNode.translateToPlan(planner);
-               Assert.assertEquals(1, ((PartitionTransformation) 
((OneInputTransformation) transformation).getInput())
-                       .getInput().getParallelism());
+               Transformation transformation = (Transformation) 
((Transformation) 
execNode.translateToPlan(planner).getInputs().get(0)).getInputs().get(0);

Review comment:
       This looks extremely weird.
   
   How about:
   ```
                @SuppressWarnings("unchecked")
                ExecNode<PlannerBase, ?> execNode = (ExecNode<PlannerBase, ?>) 
planner.translateToExecNodePlan(toScala(
                        Collections.singletonList(relNode))).get(0);
                Transformation<?> transformation = 
(execNode.translateToPlan(planner).getInputs().get(0)).getInputs().get(0);
   ```

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
##########
@@ -127,6 +129,16 @@
 
        private RuntimeExecutionMode runtimeExecutionMode = 
RuntimeExecutionMode.STREAMING;
 
+       private boolean shouldExecuteInBatchMode;
+
+       private static final Map<Class<? extends Transformation>, 
StreamGraphTranslator<?, ? extends Transformation>> translatorMap;

Review comment:
       Could we add suppressions here?
   
   ```
        @SuppressWarnings("rawtypes")
        private static final Map<Class<? extends Transformation>, 
StreamGraphTranslator<?, ? extends Transformation>> translatorMap;
   
        static {
                @SuppressWarnings("rawtypes")
                Map<Class<? extends Transformation>, StreamGraphTranslator<?, ? 
extends Transformation>> tmp = new HashMap<>();
                tmp.put(OneInputTransformation.class, new 
OneInputTranslator<>());
                translatorMap = Collections.unmodifiableMap(tmp);
        }
   ```

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
##########
@@ -276,10 +289,28 @@ private boolean isUnboundedSource(final Transformation<?> 
transformation) {
                // call at least once to trigger exceptions about 
MissingTypeInfo
                transform.getOutputType();
 
+               final StreamGraphTranslator<?, Transformation<?>> translator =

Review comment:
       add suppression

##########
File path: 
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java
##########
@@ -445,9 +443,10 @@ public void testParallelismOnLimitPushDown() {
                RelNode relNode = 
planner.optimize(TableTestUtil.toRelNode(table));
                ExecNode execNode = 
planner.translateToExecNodePlan(toScala(Collections.singletonList(relNode))).get(0);
                @SuppressWarnings("unchecked")
-               Transformation transformation = 
execNode.translateToPlan(planner);
-               Assert.assertEquals(1, ((PartitionTransformation) 
((OneInputTransformation) transformation).getInput())
-                       .getInput().getParallelism());
+               Transformation transformation = (Transformation) 
((Transformation) 
execNode.translateToPlan(planner).getInputs().get(0)).getInputs().get(0);

Review comment:
       (I know it was weird before as well)




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

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


Reply via email to