[GitHub] [flink] godfreyhe commented on a change in pull request #14380: [FLINK-20515][table-planner-blink] Separate the implementation of BatchExecMultipleInput and StreamExecMultipleInput

2020-12-14 Thread GitBox


godfreyhe commented on a change in pull request #14380:
URL: https://github.com/apache/flink/pull/14380#discussion_r543012556



##
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.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.flink.table.planner.plan.nodes.exec.batch;
+
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import 
org.apache.flink.streaming.api.transformations.MultipleInputTransformation;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.delegation.BatchPlanner;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import 
org.apache.flink.table.runtime.operators.multipleinput.BatchMultipleInputStreamOperatorFactory;
+import 
org.apache.flink.table.runtime.operators.multipleinput.TableOperatorWrapperGenerator;
+import org.apache.flink.table.runtime.operators.multipleinput.input.InputSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Batch exec node for multiple input which contains a sub-graph of {@link 
ExecNode}s.
+ * The root node of the sub-graph is {@link #outputNode}, and the leaf nodes 
of the sub-graph are
+ * the output nodes of the {@link #getInputNodes()}.
+ *
+ * The following example shows a graph of {@code ExecNode}s with multiple 
input node:
+ * {@code
+ *  Sink
+ *   |
+ * +-++
+ * | ||
+ * |   Join   |
+ * | / \  | BatchExecMultipleInput
+ * |   Agg1Agg2   |
+ * ||   | |
+ * ++---+-+
+ *  |   |
+ * Exchange1 Exchange2
+ *  |   |
+ *Scan1   Scan2
+ * }
+ *
+ * The multiple input node contains three nodes: `Join`, `Agg1` and `Agg2`.
+ * `Join` is the root node ({@link #outputNode}) of the sub-graph,
+ * `Agg1` and `Agg2` are the leaf nodes of the sub-graph,
+ * `Exchange1` and `Exchange2` are the input nodes.
+ */
+public class BatchExecMultipleInput extends BatchExecNode {
+
+   private final ExecNode outputNode;
+
+   public BatchExecMultipleInput(
+   List> inputNodes,
+   List inputEdges,
+   ExecNode outputNode,

Review comment:
   yes, multiple input could support DAG, but currently it only support 
tree.





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




[GitHub] [flink] godfreyhe commented on a change in pull request #14380: [FLINK-20515][table-planner-blink] Separate the implementation of BatchExecMultipleInput and StreamExecMultipleInput

2020-12-14 Thread GitBox


godfreyhe commented on a change in pull request #14380:
URL: https://github.com/apache/flink/pull/14380#discussion_r542990713



##
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java
##
@@ -0,0 +1,72 @@
+/*
+ * 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.table.planner.plan.nodes.exec.stream;
+
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.delegation.StreamPlanner;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Stream exec node for multiple input which contains a sub-graph of {@link 
ExecNode}s.
+ * The root node of the sub-graph is {@link #outputNode}, and the leaf nodes 
of the sub-graph are
+ * the output nodes of the {@link #getInputNodes()}.
+ *
+ * The following example shows a graph of {@code ExecNode}s with multiple 
input node:
+ * {@code
+ *  Sink
+ *   |
+ * +-++
+ * | ||
+ * |   Join   |
+ * | / \  | BatchExecMultipleInput
+ * |   Agg1Agg2   |
+ * ||   | |
+ * ++---+-+
+ *  |   |
+ * Exchange1 Exchange2
+ *  |   |
+ *Scan1   Scan2
+ * }
+ *
+ * The multiple input node contains three nodes: `Join`, `Agg1` and `Agg2`.
+ * `Join` is the root node ({@link #outputNode}) of the sub-graph,
+ * `Agg1` and `Agg2` are the leaf nodes of the sub-graph,
+ * `Exchange1` and `Exchange2` are the input nodes.
+ */
+public class StreamExecMultipleInput extends StreamExecNode {
+
+   private final ExecNode outputNode;
+
+   public StreamExecMultipleInput(
+   List> inputNodes,
+   ExecNode outputNode,

Review comment:
   `outputNode` is used for translating the sub-graph into transformations





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




[GitHub] [flink] godfreyhe commented on a change in pull request #14380: [FLINK-20515][table-planner-blink] Separate the implementation of BatchExecMultipleInput and StreamExecMultipleInput

2020-12-14 Thread GitBox


godfreyhe commented on a change in pull request #14380:
URL: https://github.com/apache/flink/pull/14380#discussion_r542993092



##
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.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.flink.table.planner.plan.nodes.exec.batch;
+
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import 
org.apache.flink.streaming.api.transformations.MultipleInputTransformation;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.delegation.BatchPlanner;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import 
org.apache.flink.table.runtime.operators.multipleinput.BatchMultipleInputStreamOperatorFactory;
+import 
org.apache.flink.table.runtime.operators.multipleinput.TableOperatorWrapperGenerator;
+import org.apache.flink.table.runtime.operators.multipleinput.input.InputSpec;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Batch exec node for multiple input which contains a sub-graph of {@link 
ExecNode}s.
+ * The root node of the sub-graph is {@link #outputNode}, and the leaf nodes 
of the sub-graph are
+ * the output nodes of the {@link #getInputNodes()}.
+ *
+ * The following example shows a graph of {@code ExecNode}s with multiple 
input node:
+ * {@code
+ *  Sink
+ *   |
+ * +-++
+ * | ||
+ * |   Join   |
+ * | / \  | BatchExecMultipleInput
+ * |   Agg1Agg2   |
+ * ||   | |
+ * ++---+-+
+ *  |   |
+ * Exchange1 Exchange2
+ *  |   |
+ *Scan1   Scan2
+ * }
+ *
+ * The multiple input node contains three nodes: `Join`, `Agg1` and `Agg2`.
+ * `Join` is the root node ({@link #outputNode}) of the sub-graph,
+ * `Agg1` and `Agg2` are the leaf nodes of the sub-graph,
+ * `Exchange1` and `Exchange2` are the input nodes.
+ */
+public class BatchExecMultipleInput extends BatchExecNode {
+
+   private final ExecNode outputNode;
+
+   public BatchExecMultipleInput(
+   List> inputNodes,
+   List inputEdges,
+   ExecNode outputNode,

Review comment:
   In operator chaining, `tailNode` is appropriate, because the operators 
are list. While in multiple, the operators are graph (more strictly, it is a 
tree). so what about` rootNode` ?





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




[GitHub] [flink] godfreyhe commented on a change in pull request #14380: [FLINK-20515][table-planner-blink] Separate the implementation of BatchExecMultipleInput and StreamExecMultipleInput

2020-12-14 Thread GitBox


godfreyhe commented on a change in pull request #14380:
URL: https://github.com/apache/flink/pull/14380#discussion_r542990713



##
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java
##
@@ -0,0 +1,72 @@
+/*
+ * 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.table.planner.plan.nodes.exec.stream;
+
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.delegation.StreamPlanner;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Stream exec node for multiple input which contains a sub-graph of {@link 
ExecNode}s.
+ * The root node of the sub-graph is {@link #outputNode}, and the leaf nodes 
of the sub-graph are
+ * the output nodes of the {@link #getInputNodes()}.
+ *
+ * The following example shows a graph of {@code ExecNode}s with multiple 
input node:
+ * {@code
+ *  Sink
+ *   |
+ * +-++
+ * | ||
+ * |   Join   |
+ * | / \  | BatchExecMultipleInput
+ * |   Agg1Agg2   |
+ * ||   | |
+ * ++---+-+
+ *  |   |
+ * Exchange1 Exchange2
+ *  |   |
+ *Scan1   Scan2
+ * }
+ *
+ * The multiple input node contains three nodes: `Join`, `Agg1` and `Agg2`.
+ * `Join` is the root node ({@link #outputNode}) of the sub-graph,
+ * `Agg1` and `Agg2` are the leaf nodes of the sub-graph,
+ * `Exchange1` and `Exchange2` are the input nodes.
+ */
+public class StreamExecMultipleInput extends StreamExecNode {
+
+   private final ExecNode outputNode;
+
+   public StreamExecMultipleInput(
+   List> inputNodes,
+   ExecNode outputNode,

Review comment:
   `outputNode` is used for translating the sub-graph into transformation





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