twalthr commented on a change in pull request #18755:
URL: https://github.com/apache/flink/pull/18755#discussion_r806658241



##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamStatementSet.java
##########
@@ -38,18 +40,26 @@
 @PublicEvolving
 public interface StreamStatementSet extends StatementSet {
 
+    /** {@inheritDoc} */

Review comment:
       This should actually not be necessary. I remember that I also did some 
research. Did you verify this with IntelliJ?

##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TablePipeline.java
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.api;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * This interface describes a pipeline created from a single {@code INSERT 
INTO} statement, created
+ * either using Table APIs or SQL.
+ */
+@PublicEvolving
+public interface TablePipeline extends Explainable<TablePipeline>, Executable, 
Compilable {}

Review comment:
       to fill this interface with something useful. how about we provide an 
`ObjectIdentifier` for the sink table.

##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamStatementSet.java
##########
@@ -38,18 +40,26 @@
 @PublicEvolving
 public interface StreamStatementSet extends StatementSet {
 
+    /** {@inheritDoc} */
     @Override
     StreamStatementSet addInsertSql(String statement);
 
+    /** {@inheritDoc} */
     @Override
+    StreamStatementSet add(TablePipeline tablePipeline);

Review comment:
       nit: add it to the end of all `insert` methods or to the beginning

##########
File path: 
flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamStatementSetImpl.scala
##########
@@ -21,40 +21,39 @@ package org.apache.flink.table.api.bridge.scala.internal
 import org.apache.flink.annotation.Internal
 import org.apache.flink.table.api.bridge.scala.StreamStatementSet
 import org.apache.flink.table.api.internal.StatementSetImpl
-import org.apache.flink.table.api.{Table, TableDescriptor}
+import org.apache.flink.table.api.{ExplainDetail, Table, TableDescriptor, 
TablePipeline}
 
 /** Implementation for [[StreamStatementSet]]. */
 @Internal
 class StreamStatementSetImpl(tableEnvironment: StreamTableEnvironmentImpl)
     extends StatementSetImpl[StreamTableEnvironmentImpl](tableEnvironment)
     with StreamStatementSet {
 
-  override def addInsertSql(statement: String): StreamStatementSet = {
+  override def addInsertSql(statement: String): StreamStatementSet =
     super.addInsertSql(statement).asInstanceOf[StreamStatementSet]
-  }
 
-  override def addInsert(targetPath: String, table: Table): StreamStatementSet 
= {
+  override def add(tablePipeline: TablePipeline): StreamStatementSet =
+    super.add(tablePipeline).asInstanceOf[StreamStatementSet]
+
+  override def addInsert(targetPath: String, table: Table): StreamStatementSet 
=
     super.addInsert(targetPath, table).asInstanceOf[StreamStatementSet]
-  }
 
-  override def addInsert(
-      targetPath: String,
-      table: Table,
-      overwrite: Boolean)
-    : StreamStatementSet = {
+  override def addInsert(targetPath: String, table: Table, overwrite: 
Boolean): StreamStatementSet =
     super.addInsert(targetPath, table, 
overwrite).asInstanceOf[StreamStatementSet]
-  }
 
-  override def addInsert(targetDescriptor: TableDescriptor, table: Table): 
StreamStatementSet = {
+  override def addInsert(targetDescriptor: TableDescriptor, table: Table): 
StreamStatementSet =
     super.addInsert(targetDescriptor, table).asInstanceOf[StreamStatementSet]
-  }
 
   override def addInsert(
-      targetDescriptor: TableDescriptor,
-      table: Table,
-      overwrite: Boolean)
-    : StreamStatementSet = {
+                          targetDescriptor: TableDescriptor,
+                          table: Table,
+                          overwrite: Boolean)
+  : StreamStatementSet =
     super.addInsert(targetDescriptor, table, 
overwrite).asInstanceOf[StreamStatementSet]
+
+  override def printExplain(extraDetails: ExplainDetail*): StreamStatementSet 
= {
+    System.out.println(super.explain(extraDetails: _*))

Review comment:
       Scala offers `println()` without `System.out`

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java
##########
@@ -95,7 +96,19 @@ public void testExecutePlan() throws Exception {
     }
 
     @Test
-    public void testExecutePlanSql() throws Exception {
+    public void testExecutePlanTable() throws Exception {
+        List<String> data = Arrays.asList("1,1,hi", "2,1,hello", "3,2,hello 
world");

Review comment:
       use constants above

##########
File path: 
flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamStatementSetImpl.scala
##########
@@ -21,40 +21,39 @@ package org.apache.flink.table.api.bridge.scala.internal
 import org.apache.flink.annotation.Internal
 import org.apache.flink.table.api.bridge.scala.StreamStatementSet
 import org.apache.flink.table.api.internal.StatementSetImpl
-import org.apache.flink.table.api.{Table, TableDescriptor}
+import org.apache.flink.table.api.{ExplainDetail, Table, TableDescriptor, 
TablePipeline}
 
 /** Implementation for [[StreamStatementSet]]. */
 @Internal
 class StreamStatementSetImpl(tableEnvironment: StreamTableEnvironmentImpl)
     extends StatementSetImpl[StreamTableEnvironmentImpl](tableEnvironment)
     with StreamStatementSet {
 
-  override def addInsertSql(statement: String): StreamStatementSet = {
+  override def addInsertSql(statement: String): StreamStatementSet =
     super.addInsertSql(statement).asInstanceOf[StreamStatementSet]
-  }
 
-  override def addInsert(targetPath: String, table: Table): StreamStatementSet 
= {
+  override def add(tablePipeline: TablePipeline): StreamStatementSet =
+    super.add(tablePipeline).asInstanceOf[StreamStatementSet]
+
+  override def addInsert(targetPath: String, table: Table): StreamStatementSet 
=
     super.addInsert(targetPath, table).asInstanceOf[StreamStatementSet]
-  }
 
-  override def addInsert(
-      targetPath: String,
-      table: Table,
-      overwrite: Boolean)
-    : StreamStatementSet = {
+  override def addInsert(targetPath: String, table: Table, overwrite: 
Boolean): StreamStatementSet =
     super.addInsert(targetPath, table, 
overwrite).asInstanceOf[StreamStatementSet]
-  }
 
-  override def addInsert(targetDescriptor: TableDescriptor, table: Table): 
StreamStatementSet = {
+  override def addInsert(targetDescriptor: TableDescriptor, table: Table): 
StreamStatementSet =
     super.addInsert(targetDescriptor, table).asInstanceOf[StreamStatementSet]
-  }
 
   override def addInsert(
-      targetDescriptor: TableDescriptor,
-      table: Table,
-      overwrite: Boolean)
-    : StreamStatementSet = {
+                          targetDescriptor: TableDescriptor,

Review comment:
       fix indention

##########
File path: 
flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamStatementSetImpl.scala
##########
@@ -21,40 +21,39 @@ package org.apache.flink.table.api.bridge.scala.internal
 import org.apache.flink.annotation.Internal
 import org.apache.flink.table.api.bridge.scala.StreamStatementSet
 import org.apache.flink.table.api.internal.StatementSetImpl
-import org.apache.flink.table.api.{Table, TableDescriptor}
+import org.apache.flink.table.api.{ExplainDetail, Table, TableDescriptor, 
TablePipeline}
 
 /** Implementation for [[StreamStatementSet]]. */
 @Internal
 class StreamStatementSetImpl(tableEnvironment: StreamTableEnvironmentImpl)
     extends StatementSetImpl[StreamTableEnvironmentImpl](tableEnvironment)
     with StreamStatementSet {
 
-  override def addInsertSql(statement: String): StreamStatementSet = {
+  override def addInsertSql(statement: String): StreamStatementSet =
     super.addInsertSql(statement).asInstanceOf[StreamStatementSet]
-  }

Review comment:
       nit: IMHO we should leave curly braces for methods, we don't need to use 
very fancy Scala syntax, if you would add another statement, you would add the 
statement to the class init, pretty dangerous.




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