fsk119 commented on a change in pull request #15317:
URL: https://github.com/apache/flink/pull/15317#discussion_r647390727



##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java
##########
@@ -942,16 +941,19 @@ private Operation convertShowViews(SqlShowViews 
sqlShowViews) {
     private Operation convertRichExplain(SqlRichExplain sqlExplain) {
         Operation operation;
         SqlNode sqlNode = sqlExplain.getStatement();
+        Set<String> explainDetails = sqlExplain.getExplainDetails();
+        /*
+            Link to FLINK-22155,EXPLAIN statement should validate insert and 
query.if sql is a insert statement,
+            it will parse to RichSqlInsert, But if sql is a select statement 
and union exists,
+            then it will be converted to sqlBasicCall with union as operator 
instead of SqlSelect SqlNode.
+            So if it is not an insert statement, we will use convertSqlQuery 
to parse the statement
+        */
         if (sqlNode instanceof RichSqlInsert) {
             operation = convertSqlInsert((RichSqlInsert) sqlNode);
-        } else if (sqlNode instanceof SqlSelect) {
-            operation = convertSqlQuery(sqlExplain.getStatement());
         } else {

Review comment:
       Would be better to 
   
   ```
        else if (sqlNode instanceof SqlSelect || sqlNode instanceof 
SqlBasicCall)
   ```
   
   Please also add tests in `SqlToOperationConverterTest` to verify it works.

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java
##########
@@ -942,16 +941,19 @@ private Operation convertShowViews(SqlShowViews 
sqlShowViews) {
     private Operation convertRichExplain(SqlRichExplain sqlExplain) {
         Operation operation;
         SqlNode sqlNode = sqlExplain.getStatement();
+        Set<String> explainDetails = sqlExplain.getExplainDetails();

Review comment:
       Would be better to convert the String to the `ExplainDetail` in the 
`convertRichExplain`. Because we always use the Flink concepts in the operation.

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java
##########
@@ -942,16 +941,19 @@ private Operation convertShowViews(SqlShowViews 
sqlShowViews) {
     private Operation convertRichExplain(SqlRichExplain sqlExplain) {
         Operation operation;
         SqlNode sqlNode = sqlExplain.getStatement();
+        Set<String> explainDetails = sqlExplain.getExplainDetails();
+        /*
+            Link to FLINK-22155,EXPLAIN statement should validate insert and 
query.if sql is a insert statement,
+            it will parse to RichSqlInsert, But if sql is a select statement 
and union exists,
+            then it will be converted to sqlBasicCall with union as operator 
instead of SqlSelect SqlNode.
+            So if it is not an insert statement, we will use convertSqlQuery 
to parse the statement
+        */

Review comment:
       In Flink, we always use `\\` to comment in the method. I think you just 
need to tell us the the query contains `UNION` is a `SqlBasicCall`.  

##########
File path: 
flink-table/flink-sql-parser/src/main/resources/org.apache.flink.sql.parser.utils/ParserResource.properties
##########
@@ -19,3 +19,4 @@
 MultipleWatermarksUnsupported=Multiple WATERMARK statements is not supported 
yet.
 OverwriteIsOnlyUsedWithInsert=OVERWRITE expression is only used with INSERT 
statement.
 createSystemFunctionOnlySupportTemporary=CREATE SYSTEM FUNCTION is not 
supported, system functions can only be registered as temporary function, you 
can use CREATE TEMPORARY SYSTEM FUNCTION instead.
+explainDetailIsDuplicate=EXPLAINDETAIL is duplicate.

Review comment:
       It's better to use `EXPLAIN DETAIL are duplicate` ? 

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java
##########
@@ -649,14 +648,16 @@ private Operation convertShowViews(SqlShowViews 
sqlShowViews) {
     private Operation convertRichExplain(SqlRichExplain sqlExplain) {
         Operation operation;
         SqlNode sqlNode = sqlExplain.getStatement();
+        /*
+            Link to FLINK-22155,EXPLAIN statement should validate insert and 
query.if sql is a insert statement,
+            it will parse to RichSqlInsert, But if sql is a select statement 
and union exists,
+            then it will be converted to sqlBasicCall with union as operator 
instead of SqlSelect SqlNode.
+            So if it is not an insert statement, we will use convertSqlQuery 
to parse the statement
+        */
         if (sqlNode instanceof RichSqlInsert) {
             operation = convertSqlInsert((RichSqlInsert) sqlNode);
-        } else if (sqlNode instanceof SqlSelect) {
-            operation = convertSqlQuery(sqlExplain.getStatement());
         } else {

Review comment:
       I think we should throw UnsupportedException if the `SqlNode` contains 
explain details.
   
   By the way, we have already removed this module now...

##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/ExplainDetail.java
##########
@@ -36,5 +36,19 @@
     CHANGELOG_MODE,
 
     /** The execution plan in json format of the program. */
-    JSON_EXECUTION_PLAN
+    JSON_EXECUTION_PLAN;
+
+    public static ExplainDetail getExplainDetailByName(String detailName) {
+        switch (detailName.toUpperCase()) {
+            case "ESTIMATED_COST":
+                return ExplainDetail.ESTIMATED_COST;
+            case "CHANGELOG_MODE":
+                return ExplainDetail.CHANGELOG_MODE;
+            case "JSON_EXECUTION_PLAN":
+                return ExplainDetail.JSON_EXECUTION_PLAN;
+            default:
+                throw new TableException(
+                        String.format("Unsupported EXPLAIN DETAIL: %s", 
detailName));

Review comment:
       Please add `.` if the sentence finishes.

##########
File path: 
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala
##########
@@ -1375,6 +1377,126 @@ class TableEnvironmentTest {
     assertEquals(replaceStageId(expected), replaceStageId(actual))
   }
 
+  @Test
+  def testExecuteSqlWithExplainDetailsSelect(): Unit = {
+    val createTableStmt =
+      """
+        |CREATE TABLE MyTable (
+        |  a bigint,
+        |  b int,
+        |  c varchar
+        |) with (
+        |  'connector' = 'COLLECTION',
+        |  'is-bounded' = 'false'
+        |)
+      """.stripMargin
+    val tableResult1 = tableEnv.executeSql(createTableStmt)
+    assertEquals(ResultKind.SUCCESS, tableResult1.getResultKind)
+
+    val tableResult2 = tableEnv
+      .executeSql("explain changelog_mode, estimated_cost, json_execution_plan 
" +
+        "select * from MyTable where a > 10")
+    assertEquals(ResultKind.SUCCESS_WITH_CONTENT, tableResult2.getResultKind)
+    val it = tableResult2.collect()
+    assertTrue(it.hasNext)
+    val row = it.next()
+    assertEquals(1, row.getArity)
+    val actual = replaceStreamNodeId(row.getField(0).toString.trim)
+    val expected = replaceStreamNodeId(TableTestUtil
+      
.readFromResource("/explain/testExecuteSqlWithExplainDetailsSelect.out").trim)
+    assertEquals(replaceStageId(expected), replaceStageId(actual))
+    assertFalse(it.hasNext)

Review comment:
       Would be better to introduce a method like 
   
   ```
   private def runExplainTest(sql: String, resultPath: String): Unit = {
       val explainResult = tableEnv.executeSql(sql)
       assertEquals(ResultKind.SUCCESS_WITH_CONTENT, 
explainResult.getResultKind)
       val actual = 
replaceStreamNodeId(explainResult.collect().next().getField(0).toString.trim)
       val expected = replaceStreamNodeId(TableTestUtil
         .readFromResource(resultPath).trim)
       assertEquals(replaceStageId(expected), replaceStageId(actual))
     }
   ```
   
   to reduce the duplcate codes.




-- 
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:
[email protected]


Reply via email to