forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-32517][table] Support to execute [CREATE OR] REPLACE TABLE AS …
…statement (apache#22949)
- Loading branch information
1 parent
1354d2f
commit 08ef36e
Showing
11 changed files
with
681 additions
and
7 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
86 changes: 86 additions & 0 deletions
86
...ble-api-java/src/main/java/org/apache/flink/table/operations/ReplaceTableAsOperation.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,86 @@ | ||
/* | ||
* 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.table.catalog.CatalogManager; | ||
import org.apache.flink.table.operations.ddl.CreateTableOperation; | ||
|
||
import java.util.Collections; | ||
import java.util.LinkedHashMap; | ||
import java.util.Map; | ||
|
||
/** Operation to describe a [CREATE OR] REPLACE TABLE AS statement. */ | ||
@Internal | ||
public class ReplaceTableAsOperation implements ModifyOperation { | ||
|
||
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; | ||
} | ||
|
||
public boolean isCreateOrReplace() { | ||
return isCreateOrReplace; | ||
} | ||
|
||
public SinkModifyOperation toSinkModifyOperation(CatalogManager catalogManager) { | ||
return new SinkModifyOperation( | ||
catalogManager.getTableOrError(createTableOperation.getTableIdentifier()), | ||
sinkModifyQuery, | ||
Collections.emptyMap(), | ||
null, // targetColumns | ||
false, | ||
Collections.emptyMap()); | ||
} | ||
|
||
@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); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
89 changes: 89 additions & 0 deletions
89
...java/org/apache/flink/table/planner/operations/converters/SqlReplaceTableAsConverter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,89 @@ | ||
/* | ||
* 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.flink.table.planner.utils.OperationConverterUtils; | ||
|
||
import org.apache.calcite.sql.SqlNode; | ||
|
||
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 | ||
String tableComment = | ||
OperationConverterUtils.getTableComment(sqlReplaceTableAs.getComment()); | ||
|
||
// get table properties | ||
Map<String, String> properties = new HashMap<>(); | ||
sqlReplaceTableAs | ||
.getPropertyList() | ||
.getList() | ||
.forEach( | ||
p -> | ||
properties.put( | ||
((SqlTableOption) p).getKeyString(), | ||
((SqlTableOption) p).getValueString())); | ||
|
||
// get table | ||
CatalogTable catalogTable = | ||
CatalogTable.of( | ||
Schema.newBuilder().fromResolvedSchema(query.getResolvedSchema()).build(), | ||
tableComment, | ||
Collections.emptyList(), | ||
properties); | ||
|
||
CreateTableOperation createTableOperation = | ||
new CreateTableOperation( | ||
identifier, | ||
catalogTable, | ||
sqlReplaceTableAs.isIfNotExists(), | ||
sqlReplaceTableAs.isTemporary()); | ||
|
||
return new ReplaceTableAsOperation( | ||
createTableOperation, query, sqlReplaceTableAs.isCreateOrReplace()); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.