luoyuxia commented on code in PR #22949:
URL: https://github.com/apache/flink/pull/22949#discussion_r1253044112


##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlRTASNodeToOperationConverterTest.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.operations;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.SqlDialect;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.operations.Operation;
+import org.apache.flink.table.operations.ReplaceTableAsOperation;
+import org.apache.flink.table.operations.ddl.CreateTableOperation;
+import org.apache.flink.table.planner.calcite.FlinkPlannerImpl;
+import org.apache.flink.table.planner.parse.CalciteParser;
+import org.apache.flink.table.types.AbstractDataType;
+
+import org.apache.calcite.sql.SqlNode;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test base for testing convert [CREATE OR] REPLACE TABLE AS statement to 
operation. */
+public class SqlRTASNodeToOperationConverterTest extends 
SqlNodeToOperationConversionTestBase {
+
+    @Test
+    public void testReplaceTableAS() {
+        String tableName = "replace_table";
+        String sql =

Review Comment:
   Please add comment with chinese.



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java:
##########
@@ -784,11 +786,33 @@ public CompiledPlan compilePlan(List<ModifyOperation> 
operations) {
     public TableResultInternal executeInternal(List<ModifyOperation> 
operations) {
         List<ModifyOperation> mapOperations = new ArrayList<>();
         for (ModifyOperation modify : operations) {
-            // execute CREATE TABLE first for CTAS statements
             if (modify instanceof CreateTableASOperation) {
+                // execute CREATE TABLE first for CTAS statements
                 CreateTableASOperation ctasOperation = 
(CreateTableASOperation) modify;
                 executeInternal(ctasOperation.getCreateTableOperation());
                 
mapOperations.add(ctasOperation.toSinkModifyOperation(catalogManager));
+            } else if (modify instanceof ReplaceTableAsOperation) {
+                ReplaceTableAsOperation rtasOperation = 
(ReplaceTableAsOperation) modify;

Review Comment:
   Please extract the code lines to a method



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/OperationConverterUtils.java:
##########
@@ -85,6 +87,12 @@ public static List<TableChange> buildModifyColumnChange(
                 .orElse(null);
     }
 
+    public static @Nullable String 
getTableComment(Optional<SqlCharStringLiteral> tableComment) {
+        return tableComment

Review Comment:
   Please follow the implementation of `getComment(SqlTableColumn column)`



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlCreateTableConverter.java:
##########
@@ -171,11 +171,7 @@ private CatalogTable createCatalogTable(SqlCreateTable 
sqlCreateTable) {
                         mergingStrategies);
         verifyPartitioningColumnsExist(mergedSchema, partitionKeys);
 
-        String tableComment =
-                sqlCreateTable
-                        .getComment()
-                        .map(comment -> 
comment.getValueAs(NlsString.class).getValue())
-                        .orElse(null);
+        String tableComment = 
OperationConverterUtils.getTableComment(sqlCreateTable.getComment());

Review Comment:
   I'm wondering it works when the comment is Chinese? Can you please verify 
with Chinese comment?
   I mean, will it cause the problem like 
   https://issues.apache.org/jira/browse/FLINK-32249?
   



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/RTASITCase.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.runtime.stream.sql;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.table.planner.runtime.utils.StreamingTestBase;
+import org.apache.flink.table.planner.runtime.utils.TestData;
+import org.apache.flink.table.types.AbstractDataType;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** IT Case for [CREATE OR] REPLACE TABLE AS SELECT statement. */
+public class RTASITCase extends StreamingTestBase {
+
+    @BeforeEach
+    @Override
+    public void before() throws Exception {
+        super.before();
+        String dataId1 = 
TestValuesTableFactory.registerData(TestData.smallData3());
+        tEnv().executeSql(
+                        String.format(
+                                "CREATE TABLE source(a int, b bigint, c 
string)"
+                                        + " WITH ('connector' = 'values', 
'bounded' = 'true', 'data-id' = '%s')",
+                                dataId1));
+        tEnv().executeSql(
+                        "CREATE TABLE target(a int, b bigint, c string)"
+                                + " WITH ('connector' = 'values')");
+    }
+
+    @Test
+    void testReplaceTableAS() throws Exception {
+        tEnv().executeSql(
+                        "REPLACE TABLE target WITH ('connector' = 'values',"
+                                + " 'bounded' = 'true')"
+                                + " AS SELECT * FROM source")
+                .await();
+
+        // verify written rows
+        assertThat(TestValuesTableFactory.getResults("target").toString())
+                .isEqualTo("[+I[1, 1, Hi], +I[2, 2, Hello], +I[3, 2, Hello 
world]]");
+
+        // verify the table after replacing
+        CatalogTable expectCatalogTable =
+                getExpectCatalogTable(
+                        new String[] {"a", "b", "c"},
+                        new AbstractDataType[] {
+                            DataTypes.INT(), DataTypes.BIGINT(), 
DataTypes.STRING()
+                        });
+        verifyCatalogTable(expectCatalogTable, getCatalogTable("target"));
+    }
+
+    @Test
+    void testReplaceTableASWithTableNotExist() {
+        assertThatThrownBy(() -> tEnv().executeSql("REPLACE TABLE t AS SELECT 
* FROM source"))
+                .isInstanceOf(TableException.class)
+                .hasMessage(
+                        "The table `default_catalog`.`default_database`.`t` to 
be replaced doesn't exist."
+                                + " You can try to use CREATE TABLE AS 
statement or CREATE OR REPLACE TABLE AS statement.");
+    }
+
+    @Test
+    void testCreateOrReplaceTableAS() throws Exception {
+        tEnv().executeSql(
+                        "CREATE OR REPLACE TABLE target WITH ('connector' = 
'values',"
+                                + " 'bounded' = 'true')"
+                                + " AS SELECT a, c FROM source")
+                .await();
+
+        // verify written rows
+        assertThat(TestValuesTableFactory.getResults("target").toString())
+                .isEqualTo("[+I[1, Hi], +I[2, Hello], +I[3, Hello world]]");
+
+        // verify the table after replacing
+        CatalogTable expectCatalogTable =
+                getExpectCatalogTable(
+                        new String[] {"a", "c"},
+                        new AbstractDataType[] {DataTypes.INT(), 
DataTypes.STRING()});
+        verifyCatalogTable(expectCatalogTable, getCatalogTable("target"));
+    }
+
+    @Test
+    void testCreateOrReplaceTableASWithTableNotExist() throws Exception {
+        tEnv().executeSql(
+                        "CREATE OR REPLACE TABLE not_exist_target WITH 
('connector' = 'values',"
+                                + " 'bounded' = 'true')"
+                                + " AS SELECT a, c FROM source")
+                .await();
+
+        // verify written rows
+        
assertThat(TestValuesTableFactory.getResults("not_exist_target").toString())
+                .isEqualTo("[+I[1, Hi], +I[2, Hello], +I[3, Hello world]]");
+
+        // verify the table after replacing
+        CatalogTable expectCatalogTable =
+                getExpectCatalogTable(
+                        new String[] {"a", "c"},
+                        new AbstractDataType[] {DataTypes.INT(), 
DataTypes.STRING()});
+        verifyCatalogTable(expectCatalogTable, 
getCatalogTable("not_exist_target"));
+    }
+
+    private CatalogTable getExpectCatalogTable(String[] cols, 
AbstractDataType[] fieldDataTypes) {

Review Comment:
   ```suggestion
       private CatalogTable getExpectCatalogTable(String[] cols, 
AbstractDataType<?>[] fieldDataTypes) {
   ```



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/batch/sql/RTASITCase.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.runtime.batch.sql;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.table.planner.runtime.utils.BatchTestBase;
+import org.apache.flink.table.planner.runtime.utils.TestData;
+import org.apache.flink.table.types.AbstractDataType;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** IT Case for [CREATE OR] REPLACE TABLE AS SELECT statement. */
+class RTASITCase extends BatchTestBase {
+
+    @BeforeEach
+    @Override
+    public void before() {
+        String dataId1 = 
TestValuesTableFactory.registerData(TestData.smallData3());
+        tEnv().executeSql(
+                        String.format(
+                                "CREATE TABLE source(a int, b bigint, c 
string)"
+                                        + " WITH ('connector' = 'values', 
'bounded' = 'true', 'data-id' = '%s')",
+                                dataId1));
+        tEnv().executeSql(
+                        "CREATE TABLE target(a int, b bigint, c string)"
+                                + " WITH ('connector' = 'values')");
+    }
+
+    @Test
+    void testReplaceTableAS() throws Exception {
+        tEnv().executeSql(
+                        "REPLACE TABLE target WITH ('connector' = 'values',"
+                                + " 'bounded' = 'true')"
+                                + " AS SELECT * FROM source")
+                .await();
+
+        // verify written rows
+        assertThat(TestValuesTableFactory.getResults("target").toString())
+                .isEqualTo("[+I[1, 1, Hi], +I[2, 2, Hello], +I[3, 2, Hello 
world]]");
+
+        // verify the table after replacing
+        CatalogTable expectCatalogTable =
+                getExpectCatalogTable(
+                        new String[] {"a", "b", "c"},
+                        new AbstractDataType[] {
+                            DataTypes.INT(), DataTypes.BIGINT(), 
DataTypes.STRING()
+                        });
+        verifyCatalogTable(expectCatalogTable, getCatalogTable("target"));
+    }
+
+    @Test
+    void testReplaceTableASWithTableNotExist() {
+        assertThatThrownBy(() -> tEnv().executeSql("REPLACE TABLE t AS SELECT 
* FROM source"))
+                .isInstanceOf(TableException.class)
+                .hasMessage(
+                        "The table `default_catalog`.`default_database`.`t` to 
be replaced doesn't exist."
+                                + " You can try to use CREATE TABLE AS 
statement or CREATE OR REPLACE TABLE AS statement.");
+    }
+
+    @Test
+    void testCreateOrReplaceTableAS() throws Exception {
+        tEnv().executeSql(
+                        "CREATE OR REPLACE TABLE target WITH ('connector' = 
'values',"
+                                + " 'bounded' = 'true')"
+                                + " AS SELECT a, c FROM source")
+                .await();
+
+        // verify written rows
+        assertThat(TestValuesTableFactory.getResults("target").toString())
+                .isEqualTo("[+I[1, Hi], +I[2, Hello], +I[3, Hello world]]");
+
+        // verify the table after replacing
+        CatalogTable expectCatalogTable =
+                getExpectCatalogTable(
+                        new String[] {"a", "c"},
+                        new AbstractDataType[] {DataTypes.INT(), 
DataTypes.STRING()});
+        verifyCatalogTable(expectCatalogTable, getCatalogTable("target"));
+    }
+
+    @Test
+    void testCreateOrReplaceTableASWithTableNotExist() throws Exception {
+        tEnv().executeSql(
+                        "CREATE OR REPLACE TABLE not_exist_target WITH 
('connector' = 'values',"
+                                + " 'bounded' = 'true')"
+                                + " AS SELECT a, c FROM source")
+                .await();
+
+        // verify written rows
+        
assertThat(TestValuesTableFactory.getResults("not_exist_target").toString())
+                .isEqualTo("[+I[1, Hi], +I[2, Hello], +I[3, Hello world]]");
+
+        // verify the table after replacing
+        CatalogTable expectCatalogTable =
+                getExpectCatalogTable(
+                        new String[] {"a", "c"},
+                        new AbstractDataType[] {DataTypes.INT(), 
DataTypes.STRING()});
+        verifyCatalogTable(expectCatalogTable, 
getCatalogTable("not_exist_target"));
+    }
+
+    private CatalogTable getExpectCatalogTable(String[] cols, 
AbstractDataType[] fieldDataTypes) {

Review Comment:
   ```suggestion
       private CatalogTable getExpectCatalogTable(String[] cols, 
AbstractDataType<?>[] fieldDataTypes) {
   ```



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