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


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlReplaceTableAsConverter.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.converters;
+
+import org.apache.flink.sql.parser.ddl.SqlReplaceTableAs;
+import org.apache.flink.sql.parser.ddl.SqlTableOption;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.UnresolvedIdentifier;
+import org.apache.flink.table.operations.Operation;
+import org.apache.flink.table.operations.QueryOperation;
+import org.apache.flink.table.operations.ReplaceTableAsOperation;
+import org.apache.flink.table.operations.ddl.CreateTableOperation;
+import org.apache.flink.table.planner.operations.PlannerQueryOperation;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.NlsString;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/** A converter for {@link SqlReplaceTableAs}. */
+public class SqlReplaceTableAsConverter implements 
SqlNodeConverter<SqlReplaceTableAs> {
+
+    @Override
+    public Operation convertSqlNode(SqlReplaceTableAs sqlReplaceTableAs, 
ConvertContext context) {
+        CatalogManager catalogManager = context.getCatalogManager();
+        UnresolvedIdentifier unresolvedIdentifier =
+                UnresolvedIdentifier.of(sqlReplaceTableAs.fullTableName());
+        ObjectIdentifier identifier = 
catalogManager.qualifyIdentifier(unresolvedIdentifier);
+
+        SqlNode asQuerySqlNode = sqlReplaceTableAs.getAsQuery();
+        context.getSqlValidator().validate(asQuerySqlNode);
+        QueryOperation query =
+                new 
PlannerQueryOperation(context.toRelRoot(asQuerySqlNode).project());
+
+        // get table comment

Review Comment:
   warning from my IDE
   ```
   Duplicated code fragment (16 lines long) 
   ```



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlRTASNodeToOperationConverterTest.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.sql.parser.error.SqlValidateException;
+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;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test base for testing convert [CREATE OR] REPLACE TABLE AS statement to 
operation. */
+public class SqlRTASNodeToOperationConverterTest extends 
SqlNodeToOperationConversionTestBase {
+
+    @Test
+    public void testReplaceTableAS() {
+        String sql = "REPLACE TABLE t_t WITH ('k1' = 'v1', 'k2' = 'v2') as 
SELECT * FROM t1";
+        ObjectIdentifier expectedIdentifier = ObjectIdentifier.of("builtin", 
"default", "t_t");
+        Operation operation = parseAndConvert(sql);
+        Map<String, String> expectedOptions = new HashMap<>();
+        expectedOptions.put("k1", "v1");
+        expectedOptions.put("k2", "v2");
+        Schema expectedSchema =
+                Schema.newBuilder()
+                        .fromFields(
+                                new String[] {"a", "b", "c", "d"},
+                                new AbstractDataType[] {
+                                    DataTypes.BIGINT(),
+                                    DataTypes.STRING(),
+                                    DataTypes.INT(),
+                                    DataTypes.STRING()
+                                })
+                        .build();
+        CatalogTable expectedCatalogTable =
+                CatalogTable.of(expectedSchema, null, Collections.emptyList(), 
expectedOptions);
+        verifyReplaceTableAsOperation(operation, expectedIdentifier, 
expectedCatalogTable);
+    }
+
+    @Test
+    public void testReplaceTableAsThrowException() {

Review Comment:
   I don't think we need to test the case that throws an exception since we 
have covered it in the previous pr.



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlRTASNodeToOperationConverterTest.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.sql.parser.error.SqlValidateException;
+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;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test base for testing convert [CREATE OR] REPLACE TABLE AS statement to 
operation. */
+public class SqlRTASNodeToOperationConverterTest extends 
SqlNodeToOperationConversionTestBase {
+
+    @Test
+    public void testReplaceTableAS() {
+        String sql = "REPLACE TABLE t_t WITH ('k1' = 'v1', 'k2' = 'v2') as 
SELECT * FROM t1";
+        ObjectIdentifier expectedIdentifier = ObjectIdentifier.of("builtin", 
"default", "t_t");
+        Operation operation = parseAndConvert(sql);
+        Map<String, String> expectedOptions = new HashMap<>();
+        expectedOptions.put("k1", "v1");
+        expectedOptions.put("k2", "v2");
+        Schema expectedSchema =
+                Schema.newBuilder()
+                        .fromFields(
+                                new String[] {"a", "b", "c", "d"},
+                                new AbstractDataType[] {
+                                    DataTypes.BIGINT(),
+                                    DataTypes.STRING(),
+                                    DataTypes.INT(),
+                                    DataTypes.STRING()
+                                })
+                        .build();
+        CatalogTable expectedCatalogTable =
+                CatalogTable.of(expectedSchema, null, Collections.emptyList(), 
expectedOptions);
+        verifyReplaceTableAsOperation(operation, expectedIdentifier, 
expectedCatalogTable);
+    }
+
+    @Test
+    public void testReplaceTableAsThrowException() {
+        // should throw exception for specifying column
+        final String sql1 =
+                "REPLACE TABLE t_t (b_1 string, c_1 bigint METADATA, c_1 as 1 
+ 1)"
+                        + " WITH ('k1' = 'v1', 'k2' = 'v2') as SELECT b, c 
FROM t1";
+        assertThatThrownBy(() -> parseAndConvert(sql1))
+                .isInstanceOf(SqlValidateException.class)
+                .hasMessage(
+                        "REPLACE TABLE AS SELECT syntax does not support to 
specify explicit columns yet.");
+
+        // should throw exception for specifying watermark
+        final String sql2 =
+                "REPLACE TABLE t_t (WATERMARK FOR eventTime AS eventTime - 
INTERVAL '5' SECOND)"
+                        + " WITH ('k1' = 'v1', 'k2' = 'v2') as SELECT b, c 
FROM t1";
+        assertThatThrownBy(() -> parseAndConvert(sql2))
+                .isInstanceOf(SqlValidateException.class)
+                .hasMessage(
+                        "REPLACE TABLE AS SELECT syntax does not support to 
specify explicit watermark yet.");
+
+        // should throw exception for tmp table
+        final String sql3 = "REPLACE TEMPORARY TABLE t WITH ('test' = 'zm') AS 
SELECT b, c FROM t1";
+        assertThatThrownBy(() -> parseAndConvert(sql3))
+                .isInstanceOf(SqlValidateException.class)
+                .hasMessage("REPLACE TABLE AS SELECT syntax does not support 
temporary table yet.");
+
+        // should throw exception for partition key
+        final String sql4 =
+                "REPLACE TABLE t PARTITIONED BY(b) WITH ('test' = 'zm') AS 
SELECT b FROM t1";
+        assertThatThrownBy(() -> parseAndConvert(sql4))
+                .isInstanceOf(SqlValidateException.class)
+                .hasMessage(
+                        "REPLACE TABLE AS SELECT syntax does not support to 
create partitioned table yet.");
+    }
+
+    @Test
+    public void testCreateOrReplaceTableAS() {
+        String sql =
+                "CREATE OR REPLACE TABLE t_t WITH ('k1' = 'v1', 'k2' = 'v2') 
as SELECT * FROM t1";
+        ObjectIdentifier expectedIdentifier = ObjectIdentifier.of("builtin", 
"default", "t_t");
+        Operation operation = parseAndConvert(sql);
+        Map<String, String> expectedOptions = new HashMap<>();

Review Comment:
   Please extract the following codes to a method so that method 
`testReplaceTableAS` can use it 



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ReplaceTableAsOperation.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.operations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.internal.TableResultInternal;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.operations.ddl.CreateTableOperation;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.table.api.config.TableConfigOptions.TABLE_DML_SYNC;
+
+/** Operation to describe a [CREATE OR] REPLACE TABLE AS statement. */
+@Internal
+public class ReplaceTableAsOperation implements ModifyOperation, 
ExecutableOperation {
+
+    private final CreateTableOperation createTableOperation;
+    private final QueryOperation sinkModifyQuery;
+    private final boolean isCreateOrReplace;
+
+    public ReplaceTableAsOperation(
+            CreateTableOperation createTableOperation,
+            QueryOperation sinkModifyQuery,
+            boolean isCreateOrReplace) {
+        this.createTableOperation = createTableOperation;
+        this.sinkModifyQuery = sinkModifyQuery;
+        this.isCreateOrReplace = isCreateOrReplace;
+    }
+
+    @Override
+    public QueryOperation getChild() {
+        return sinkModifyQuery;
+    }
+
+    @Override
+    public <T> T accept(ModifyOperationVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    public CreateTableOperation getCreateTableOperation() {
+        return createTableOperation;
+    }
+
+    @Override
+    public String asSummaryString() {
+        Map<String, Object> params = new LinkedHashMap<>();
+        params.put("catalogTable", 
getCreateTableOperation().getCatalogTable());
+        params.put("identifier", 
getCreateTableOperation().getTableIdentifier());
+
+        return OperationUtils.formatWithChildren(
+                isCreateOrReplace ? "CREATE OR REPLACE TABLE AS" : "REPLACE 
TABLE AS",
+                params,
+                Collections.singletonList(sinkModifyQuery),
+                Operation::asSummaryString);
+    }
+
+    @Override
+    public TableResultInternal execute(Context ctx) {
+        CatalogManager catalogManager = ctx.getCatalogManager();
+        ObjectIdentifier tableIdentifier = 
createTableOperation.getTableIdentifier();
+        Optional<Catalog> optionalCatalog =
+                
ctx.getCatalogManager().getCatalog(tableIdentifier.getCatalogName());
+        ObjectPath objectPath = tableIdentifier.toObjectPath();
+
+        Catalog catalog = optionalCatalog.get();
+        // rtas drop table first, then create
+        try {
+            // if is create or replace statement will ignore table not exist
+            catalog.dropTable(objectPath, isCreateOrReplace);
+        } catch (TableNotExistException e) {
+            throw new TableException(
+                    String.format(
+                            "The table %s to be replaced doesn't exist. You 
may want to use CREATE TABLE AS statement or CREATE OR REPLACE TABLE AS 
statement.",
+                            tableIdentifier));
+        }
+
+        // first create table
+        CreateTableOperation executableCreateTableOperation =
+                new CreateTableOperation(
+                        createTableOperation.getTableIdentifier(),
+                        createTableOperation.getCatalogTable(),
+                        false,
+                        false);
+        executableCreateTableOperation.execute(ctx);
+
+        // then insert into sink
+        SinkModifyOperation sinkModifyOperation =
+                new SinkModifyOperation(
+                        
catalogManager.getTableOrError(createTableOperation.getTableIdentifier()),
+                        sinkModifyQuery,
+                        Collections.emptyMap(),
+                        null, // targetColumns
+                        false,
+                        Collections.emptyMap());
+
+        List<ModifyOperation> modifyOperations = 
Collections.singletonList(sinkModifyOperation);
+        List<Transformation<?>> transformations = 
ctx.translate(modifyOperations);
+        List<String> sinkIdentifierNames = 
extractSinkIdentifierNames(modifyOperations);
+        TableResultInternal result = ctx.executeInternal(transformations, 
sinkIdentifierNames);
+        if (ctx.getTableConfig().get(TABLE_DML_SYNC)) {
+            try {
+                result.await();
+            } catch (InterruptedException | ExecutionException e) {
+                result.getJobClient().ifPresent(JobClient::cancel);
+                throw new TableException("Fail to wait execution finish.", e);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * extract sink identifier names from {@link ModifyOperation}s and 
deduplicate them with {@link
+     * #deduplicateSinkIdentifierNames(List)}.
+     */
+    private List<String> extractSinkIdentifierNames(List<ModifyOperation> 
operations) {
+        List<String> tableNames = new ArrayList<>(operations.size());
+        for (ModifyOperation operation : operations) {
+            if (operation instanceof SinkModifyOperation) {
+                String fullName =
+                        ((SinkModifyOperation) operation)
+                                .getContextResolvedTable()
+                                .getIdentifier()
+                                .asSummaryString();
+                tableNames.add(fullName);
+            } else {
+                throw new UnsupportedOperationException("Unsupported 
operation: " + operation);
+            }
+        }
+        return deduplicateSinkIdentifierNames(tableNames);
+    }
+
+    /**
+     * Deduplicate sink identifier names. If there are multiple tables with 
the same name, an index
+     * suffix will be added at the end of the name to ensure each name is 
unique.
+     */
+    private List<String> deduplicateSinkIdentifierNames(List<String> 
tableNames) {

Review Comment:
   Btw, I'd like not to let `ReplaceTableAsOperation` implement 
`ExecutableOperation` but just do like `CreateTableAsOperation` with old stack 
in the first step.
   We can migrate them to the new stacks anyway at one shot since they're 
almost similar.



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/RTASITCase.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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 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 =

Review Comment:
   Please extract it to a common method



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ReplaceTableAsOperation.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.operations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.internal.TableResultInternal;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.operations.ddl.CreateTableOperation;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.table.api.config.TableConfigOptions.TABLE_DML_SYNC;
+
+/** Operation to describe a [CREATE OR] REPLACE TABLE AS statement. */
+@Internal
+public class ReplaceTableAsOperation implements ModifyOperation, 
ExecutableOperation {
+
+    private final CreateTableOperation createTableOperation;
+    private final QueryOperation sinkModifyQuery;
+    private final boolean isCreateOrReplace;
+
+    public ReplaceTableAsOperation(
+            CreateTableOperation createTableOperation,
+            QueryOperation sinkModifyQuery,
+            boolean isCreateOrReplace) {
+        this.createTableOperation = createTableOperation;
+        this.sinkModifyQuery = sinkModifyQuery;
+        this.isCreateOrReplace = isCreateOrReplace;
+    }
+
+    @Override
+    public QueryOperation getChild() {
+        return sinkModifyQuery;
+    }
+
+    @Override
+    public <T> T accept(ModifyOperationVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    public CreateTableOperation getCreateTableOperation() {
+        return createTableOperation;
+    }
+
+    @Override
+    public String asSummaryString() {
+        Map<String, Object> params = new LinkedHashMap<>();
+        params.put("catalogTable", 
getCreateTableOperation().getCatalogTable());
+        params.put("identifier", 
getCreateTableOperation().getTableIdentifier());
+
+        return OperationUtils.formatWithChildren(
+                isCreateOrReplace ? "CREATE OR REPLACE TABLE AS" : "REPLACE 
TABLE AS",
+                params,
+                Collections.singletonList(sinkModifyQuery),
+                Operation::asSummaryString);
+    }
+
+    @Override
+    public TableResultInternal execute(Context ctx) {
+        CatalogManager catalogManager = ctx.getCatalogManager();
+        ObjectIdentifier tableIdentifier = 
createTableOperation.getTableIdentifier();
+        Optional<Catalog> optionalCatalog =
+                
ctx.getCatalogManager().getCatalog(tableIdentifier.getCatalogName());
+        ObjectPath objectPath = tableIdentifier.toObjectPath();
+
+        Catalog catalog = optionalCatalog.get();
+        // rtas drop table first, then create
+        try {
+            // if is create or replace statement will ignore table not exist
+            catalog.dropTable(objectPath, isCreateOrReplace);
+        } catch (TableNotExistException e) {
+            throw new TableException(
+                    String.format(
+                            "The table %s to be replaced doesn't exist. You 
may want to use CREATE TABLE AS statement or CREATE OR REPLACE TABLE AS 
statement.",

Review Comment:
   ```suggestion
                               "The table %s to be replaced doesn't exist. You 
can try to use CREATE TABLE AS statement or CREATE OR REPLACE TABLE AS 
statement.",
   ```



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlRTASNodeToOperationConverterTest.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.sql.parser.error.SqlValidateException;
+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;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test base for testing convert [CREATE OR] REPLACE TABLE AS statement to 
operation. */
+public class SqlRTASNodeToOperationConverterTest extends 
SqlNodeToOperationConversionTestBase {
+
+    @Test
+    public void testReplaceTableAS() {
+        String sql = "REPLACE TABLE t_t WITH ('k1' = 'v1', 'k2' = 'v2') as 
SELECT * FROM t1";
+        ObjectIdentifier expectedIdentifier = ObjectIdentifier.of("builtin", 
"default", "t_t");
+        Operation operation = parseAndConvert(sql);
+        Map<String, String> expectedOptions = new HashMap<>();
+        expectedOptions.put("k1", "v1");
+        expectedOptions.put("k2", "v2");
+        Schema expectedSchema =
+                Schema.newBuilder()
+                        .fromFields(
+                                new String[] {"a", "b", "c", "d"},
+                                new AbstractDataType[] {
+                                    DataTypes.BIGINT(),
+                                    DataTypes.STRING(),
+                                    DataTypes.INT(),
+                                    DataTypes.STRING()
+                                })
+                        .build();
+        CatalogTable expectedCatalogTable =
+                CatalogTable.of(expectedSchema, null, Collections.emptyList(), 
expectedOptions);
+        verifyReplaceTableAsOperation(operation, expectedIdentifier, 
expectedCatalogTable);
+    }
+
+    @Test
+    public void testReplaceTableAsThrowException() {
+        // should throw exception for specifying column
+        final String sql1 =
+                "REPLACE TABLE t_t (b_1 string, c_1 bigint METADATA, c_1 as 1 
+ 1)"
+                        + " WITH ('k1' = 'v1', 'k2' = 'v2') as SELECT b, c 
FROM t1";
+        assertThatThrownBy(() -> parseAndConvert(sql1))
+                .isInstanceOf(SqlValidateException.class)
+                .hasMessage(
+                        "REPLACE TABLE AS SELECT syntax does not support to 
specify explicit columns yet.");
+
+        // should throw exception for specifying watermark
+        final String sql2 =
+                "REPLACE TABLE t_t (WATERMARK FOR eventTime AS eventTime - 
INTERVAL '5' SECOND)"
+                        + " WITH ('k1' = 'v1', 'k2' = 'v2') as SELECT b, c 
FROM t1";
+        assertThatThrownBy(() -> parseAndConvert(sql2))
+                .isInstanceOf(SqlValidateException.class)
+                .hasMessage(
+                        "REPLACE TABLE AS SELECT syntax does not support to 
specify explicit watermark yet.");
+
+        // should throw exception for tmp table
+        final String sql3 = "REPLACE TEMPORARY TABLE t WITH ('test' = 'zm') AS 
SELECT b, c FROM t1";
+        assertThatThrownBy(() -> parseAndConvert(sql3))
+                .isInstanceOf(SqlValidateException.class)
+                .hasMessage("REPLACE TABLE AS SELECT syntax does not support 
temporary table yet.");
+
+        // should throw exception for partition key
+        final String sql4 =
+                "REPLACE TABLE t PARTITIONED BY(b) WITH ('test' = 'zm') AS 
SELECT b FROM t1";
+        assertThatThrownBy(() -> parseAndConvert(sql4))
+                .isInstanceOf(SqlValidateException.class)
+                .hasMessage(
+                        "REPLACE TABLE AS SELECT syntax does not support to 
create partitioned table yet.");
+    }
+
+    @Test
+    public void testCreateOrReplaceTableAS() {
+        String sql =
+                "CREATE OR REPLACE TABLE t_t WITH ('k1' = 'v1', 'k2' = 'v2') 
as SELECT * FROM t1";
+        ObjectIdentifier expectedIdentifier = ObjectIdentifier.of("builtin", 
"default", "t_t");
+        Operation operation = parseAndConvert(sql);
+        Map<String, String> expectedOptions = new HashMap<>();
+        expectedOptions.put("k1", "v1");
+        expectedOptions.put("k2", "v2");
+        Schema expectedSchema =
+                Schema.newBuilder()
+                        .fromFields(
+                                new String[] {"a", "b", "c", "d"},
+                                new AbstractDataType[] {
+                                    DataTypes.BIGINT(),
+                                    DataTypes.STRING(),
+                                    DataTypes.INT(),
+                                    DataTypes.STRING()
+                                })
+                        .build();
+        CatalogTable expectedCatalogTable =
+                CatalogTable.of(expectedSchema, null, Collections.emptyList(), 
expectedOptions);
+        verifyReplaceTableAsOperation(operation, expectedIdentifier, 
expectedCatalogTable);
+    }
+
+    @Test
+    public void testCreateOrReplaceTableAsThrowException() {

Review Comment:
   dito



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ReplaceTableAsOperation.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.operations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.internal.TableResultInternal;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.operations.ddl.CreateTableOperation;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.table.api.config.TableConfigOptions.TABLE_DML_SYNC;
+
+/** Operation to describe a [CREATE OR] REPLACE TABLE AS statement. */
+@Internal
+public class ReplaceTableAsOperation implements ModifyOperation, 
ExecutableOperation {
+
+    private final CreateTableOperation createTableOperation;
+    private final QueryOperation sinkModifyQuery;
+    private final boolean isCreateOrReplace;
+
+    public ReplaceTableAsOperation(
+            CreateTableOperation createTableOperation,
+            QueryOperation sinkModifyQuery,
+            boolean isCreateOrReplace) {
+        this.createTableOperation = createTableOperation;
+        this.sinkModifyQuery = sinkModifyQuery;
+        this.isCreateOrReplace = isCreateOrReplace;
+    }
+
+    @Override
+    public QueryOperation getChild() {
+        return sinkModifyQuery;
+    }
+
+    @Override
+    public <T> T accept(ModifyOperationVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    public CreateTableOperation getCreateTableOperation() {
+        return createTableOperation;
+    }
+
+    @Override
+    public String asSummaryString() {
+        Map<String, Object> params = new LinkedHashMap<>();
+        params.put("catalogTable", 
getCreateTableOperation().getCatalogTable());
+        params.put("identifier", 
getCreateTableOperation().getTableIdentifier());
+
+        return OperationUtils.formatWithChildren(
+                isCreateOrReplace ? "CREATE OR REPLACE TABLE AS" : "REPLACE 
TABLE AS",
+                params,
+                Collections.singletonList(sinkModifyQuery),
+                Operation::asSummaryString);
+    }
+
+    @Override
+    public TableResultInternal execute(Context ctx) {
+        CatalogManager catalogManager = ctx.getCatalogManager();
+        ObjectIdentifier tableIdentifier = 
createTableOperation.getTableIdentifier();
+        Optional<Catalog> optionalCatalog =
+                
ctx.getCatalogManager().getCatalog(tableIdentifier.getCatalogName());
+        ObjectPath objectPath = tableIdentifier.toObjectPath();
+
+        Catalog catalog = optionalCatalog.get();
+        // rtas drop table first, then create
+        try {
+            // if is create or replace statement will ignore table not exist
+            catalog.dropTable(objectPath, isCreateOrReplace);
+        } catch (TableNotExistException e) {
+            throw new TableException(
+                    String.format(
+                            "The table %s to be replaced doesn't exist. You 
may want to use CREATE TABLE AS statement or CREATE OR REPLACE TABLE AS 
statement.",
+                            tableIdentifier));
+        }
+
+        // first create table
+        CreateTableOperation executableCreateTableOperation =
+                new CreateTableOperation(
+                        createTableOperation.getTableIdentifier(),
+                        createTableOperation.getCatalogTable(),
+                        false,
+                        false);
+        executableCreateTableOperation.execute(ctx);
+
+        // then insert into sink
+        SinkModifyOperation sinkModifyOperation =
+                new SinkModifyOperation(
+                        
catalogManager.getTableOrError(createTableOperation.getTableIdentifier()),
+                        sinkModifyQuery,
+                        Collections.emptyMap(),
+                        null, // targetColumns
+                        false,
+                        Collections.emptyMap());
+
+        List<ModifyOperation> modifyOperations = 
Collections.singletonList(sinkModifyOperation);
+        List<Transformation<?>> transformations = 
ctx.translate(modifyOperations);
+        List<String> sinkIdentifierNames = 
extractSinkIdentifierNames(modifyOperations);
+        TableResultInternal result = ctx.executeInternal(transformations, 
sinkIdentifierNames);
+        if (ctx.getTableConfig().get(TABLE_DML_SYNC)) {
+            try {
+                result.await();
+            } catch (InterruptedException | ExecutionException e) {
+                result.getJobClient().ifPresent(JobClient::cancel);
+                throw new TableException("Fail to wait execution finish.", e);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * extract sink identifier names from {@link ModifyOperation}s and 
deduplicate them with {@link
+     * #deduplicateSinkIdentifierNames(List)}.
+     */
+    private List<String> extractSinkIdentifierNames(List<ModifyOperation> 
operations) {

Review Comment:
   warning from my IDE:
   `Duplicated code fragment (18 lines long) `



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ReplaceTableAsOperation.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.operations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.internal.TableResultInternal;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.operations.ddl.CreateTableOperation;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.table.api.config.TableConfigOptions.TABLE_DML_SYNC;
+
+/** Operation to describe a [CREATE OR] REPLACE TABLE AS statement. */
+@Internal
+public class ReplaceTableAsOperation implements ModifyOperation, 
ExecutableOperation {
+
+    private final CreateTableOperation createTableOperation;
+    private final QueryOperation sinkModifyQuery;
+    private final boolean isCreateOrReplace;
+
+    public ReplaceTableAsOperation(
+            CreateTableOperation createTableOperation,
+            QueryOperation sinkModifyQuery,
+            boolean isCreateOrReplace) {
+        this.createTableOperation = createTableOperation;
+        this.sinkModifyQuery = sinkModifyQuery;
+        this.isCreateOrReplace = isCreateOrReplace;
+    }
+
+    @Override
+    public QueryOperation getChild() {
+        return sinkModifyQuery;
+    }
+
+    @Override
+    public <T> T accept(ModifyOperationVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    public CreateTableOperation getCreateTableOperation() {
+        return createTableOperation;
+    }
+
+    @Override
+    public String asSummaryString() {
+        Map<String, Object> params = new LinkedHashMap<>();
+        params.put("catalogTable", 
getCreateTableOperation().getCatalogTable());
+        params.put("identifier", 
getCreateTableOperation().getTableIdentifier());
+
+        return OperationUtils.formatWithChildren(
+                isCreateOrReplace ? "CREATE OR REPLACE TABLE AS" : "REPLACE 
TABLE AS",
+                params,
+                Collections.singletonList(sinkModifyQuery),
+                Operation::asSummaryString);
+    }
+
+    @Override
+    public TableResultInternal execute(Context ctx) {
+        CatalogManager catalogManager = ctx.getCatalogManager();
+        ObjectIdentifier tableIdentifier = 
createTableOperation.getTableIdentifier();
+        Optional<Catalog> optionalCatalog =
+                
ctx.getCatalogManager().getCatalog(tableIdentifier.getCatalogName());
+        ObjectPath objectPath = tableIdentifier.toObjectPath();
+
+        Catalog catalog = optionalCatalog.get();
+        // rtas drop table first, then create
+        try {
+            // if is create or replace statement will ignore table not exist
+            catalog.dropTable(objectPath, isCreateOrReplace);
+        } catch (TableNotExistException e) {
+            throw new TableException(
+                    String.format(
+                            "The table %s to be replaced doesn't exist. You 
may want to use CREATE TABLE AS statement or CREATE OR REPLACE TABLE AS 
statement.",
+                            tableIdentifier));
+        }
+
+        // first create table
+        CreateTableOperation executableCreateTableOperation =
+                new CreateTableOperation(
+                        createTableOperation.getTableIdentifier(),
+                        createTableOperation.getCatalogTable(),
+                        false,
+                        false);
+        executableCreateTableOperation.execute(ctx);
+
+        // then insert into sink
+        SinkModifyOperation sinkModifyOperation =
+                new SinkModifyOperation(
+                        
catalogManager.getTableOrError(createTableOperation.getTableIdentifier()),
+                        sinkModifyQuery,
+                        Collections.emptyMap(),
+                        null, // targetColumns
+                        false,
+                        Collections.emptyMap());
+
+        List<ModifyOperation> modifyOperations = 
Collections.singletonList(sinkModifyOperation);
+        List<Transformation<?>> transformations = 
ctx.translate(modifyOperations);
+        List<String> sinkIdentifierNames = 
extractSinkIdentifierNames(modifyOperations);
+        TableResultInternal result = ctx.executeInternal(transformations, 
sinkIdentifierNames);
+        if (ctx.getTableConfig().get(TABLE_DML_SYNC)) {
+            try {
+                result.await();
+            } catch (InterruptedException | ExecutionException e) {
+                result.getJobClient().ifPresent(JobClient::cancel);
+                throw new TableException("Fail to wait execution finish.", e);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * extract sink identifier names from {@link ModifyOperation}s and 
deduplicate them with {@link
+     * #deduplicateSinkIdentifierNames(List)}.
+     */
+    private List<String> extractSinkIdentifierNames(List<ModifyOperation> 
operations) {
+        List<String> tableNames = new ArrayList<>(operations.size());
+        for (ModifyOperation operation : operations) {
+            if (operation instanceof SinkModifyOperation) {
+                String fullName =
+                        ((SinkModifyOperation) operation)
+                                .getContextResolvedTable()
+                                .getIdentifier()
+                                .asSummaryString();
+                tableNames.add(fullName);
+            } else {
+                throw new UnsupportedOperationException("Unsupported 
operation: " + operation);
+            }
+        }
+        return deduplicateSinkIdentifierNames(tableNames);
+    }
+
+    /**
+     * Deduplicate sink identifier names. If there are multiple tables with 
the same name, an index
+     * suffix will be added at the end of the name to ensure each name is 
unique.
+     */
+    private List<String> deduplicateSinkIdentifierNames(List<String> 
tableNames) {

Review Comment:
   dito



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