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.
- Loading branch information
1 parent
44f9144
commit a3f586e
Showing
6 changed files
with
299 additions
and
0 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
86 changes: 86 additions & 0 deletions
86
...table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterCatalog.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.sql.parser.ddl; | ||
|
||
import org.apache.flink.sql.parser.SqlUnparseUtils; | ||
|
||
import org.apache.calcite.sql.*; | ||
import org.apache.calcite.sql.parser.SqlParserPos; | ||
import org.apache.calcite.util.ImmutableNullableList; | ||
|
||
import java.util.List; | ||
|
||
import static java.util.Objects.requireNonNull; | ||
|
||
/** ALTER CATALOG DDL sql call. */ | ||
public class SqlAlterCatalog extends SqlCreate { | ||
|
||
public static final SqlSpecialOperator OPERATOR = | ||
new SqlSpecialOperator("ALTER CATALOG", SqlKind.OTHER_DDL); | ||
|
||
private final SqlIdentifier catalogName; | ||
|
||
private final SqlNodeList propertyList; | ||
|
||
public SqlAlterCatalog( | ||
SqlParserPos position, SqlIdentifier catalogName, SqlNodeList propertyList) { | ||
super(OPERATOR, position, false, false); | ||
this.catalogName = requireNonNull(catalogName, "catalogName cannot be null"); | ||
this.propertyList = requireNonNull(propertyList, "propertyList cannot be null"); | ||
} | ||
|
||
@Override | ||
public SqlOperator getOperator() { | ||
return OPERATOR; | ||
} | ||
|
||
@Override | ||
public List<SqlNode> getOperandList() { | ||
return ImmutableNullableList.of(catalogName, propertyList); | ||
} | ||
|
||
public SqlIdentifier getCatalogName() { | ||
return catalogName; | ||
} | ||
|
||
public SqlNodeList getPropertyList() { | ||
return propertyList; | ||
} | ||
|
||
@Override | ||
public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { | ||
writer.keyword("ALTER CATALOG"); | ||
catalogName.unparse(writer, leftPrec, rightPrec); | ||
|
||
if (this.propertyList.size() > 0) { | ||
writer.keyword("SET"); | ||
SqlWriter.Frame withFrame = writer.startList("(", ")"); | ||
for (SqlNode property : propertyList) { | ||
SqlUnparseUtils.printIndent(writer); | ||
property.unparse(writer, leftPrec, rightPrec); | ||
} | ||
writer.newlineAndIndent(); | ||
writer.endList(withFrame); | ||
} | ||
} | ||
|
||
public String catalogName() { | ||
return catalogName.getSimple(); | ||
} | ||
} |
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
80 changes: 80 additions & 0 deletions
80
...e-api-java/src/main/java/org/apache/flink/table/operations/ddl/AlterCatalogOperation.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,80 @@ | ||
/* | ||
* 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.ddl; | ||
|
||
import org.apache.flink.annotation.Internal; | ||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.table.api.ValidationException; | ||
import org.apache.flink.table.api.internal.TableResultImpl; | ||
import org.apache.flink.table.api.internal.TableResultInternal; | ||
import org.apache.flink.table.catalog.CatalogDescriptor; | ||
import org.apache.flink.table.catalog.exceptions.CatalogException; | ||
import org.apache.flink.table.operations.Operation; | ||
import org.apache.flink.table.operations.OperationUtils; | ||
|
||
import java.util.Collections; | ||
import java.util.LinkedHashMap; | ||
import java.util.Map; | ||
|
||
import static org.apache.flink.util.Preconditions.checkNotNull; | ||
|
||
/** Operation to describe a ALTER CATALOG statement. */ | ||
@Internal | ||
public class AlterCatalogOperation implements AlterOperation { | ||
private final String catalogName; | ||
private final Map<String, String> properties; | ||
|
||
public AlterCatalogOperation(String catalogName, Map<String, String> properties) { | ||
this.catalogName = checkNotNull(catalogName); | ||
this.properties = Collections.unmodifiableMap(checkNotNull(properties)); | ||
} | ||
|
||
public String getCatalogName() { | ||
return catalogName; | ||
} | ||
|
||
public Map<String, String> getProperties() { | ||
return properties; | ||
} | ||
|
||
@Override | ||
public String asSummaryString() { | ||
Map<String, Object> params = new LinkedHashMap<>(); | ||
params.put("catalogName", catalogName); | ||
params.put("properties", properties); | ||
|
||
return OperationUtils.formatWithChildren( | ||
"ALTER CATALOG", params, Collections.emptyList(), Operation::asSummaryString); | ||
} | ||
|
||
@Override | ||
public TableResultInternal execute(Context ctx) { | ||
try { | ||
ctx.getCatalogManager() | ||
.alterCatalog( | ||
catalogName, | ||
CatalogDescriptor.of(catalogName, Configuration.fromMap(properties))); | ||
|
||
return TableResultImpl.TABLE_RESULT_OK; | ||
} catch (CatalogException e) { | ||
throw new ValidationException( | ||
String.format("Could not execute %s", asSummaryString()), e); | ||
} | ||
} | ||
} |
65 changes: 65 additions & 0 deletions
65
...n/java/org/apache/flink/table/planner/operations/converters/SqlAlterCatalogConverter.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,65 @@ | ||
/* | ||
* 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.SqlAlterCatalog; | ||
import org.apache.flink.sql.parser.ddl.SqlCreateCatalog; | ||
import org.apache.flink.sql.parser.ddl.SqlTableOption; | ||
import org.apache.flink.table.api.ValidationException; | ||
import org.apache.flink.table.catalog.Catalog; | ||
import org.apache.flink.table.catalog.CatalogDescriptor; | ||
import org.apache.flink.table.operations.Operation; | ||
import org.apache.flink.table.operations.ddl.AlterCatalogOperation; | ||
|
||
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
|
||
/** A converter for {@link SqlAlterCatalog}. */ | ||
public class SqlAlterCatalogConverter implements SqlNodeConverter<SqlAlterCatalog> { | ||
|
||
@Override | ||
public Operation convertSqlNode(SqlAlterCatalog sqlAlterCatalog, ConvertContext context) { | ||
String catalogName = sqlAlterCatalog.catalogName(); | ||
Optional<Catalog> catalog = context.getCatalogManager().getCatalog(catalogName); | ||
final Map<String, String> properties; | ||
if (catalog.isPresent()) { | ||
Optional<CatalogDescriptor> catalogDescriptor = | ||
context.getCatalogManager().getCatalogDescriptor(catalogName); | ||
properties = | ||
new HashMap<>( | ||
catalogDescriptor.isPresent() | ||
? catalogDescriptor.get().getConfiguration().toMap() | ||
: Collections.emptyMap()); | ||
} else { | ||
throw new ValidationException(String.format("Catalog %s not exists", catalogName)); | ||
} | ||
// set with properties | ||
sqlAlterCatalog | ||
.getPropertyList() | ||
.getList() | ||
.forEach( | ||
p -> | ||
properties.put( | ||
((SqlTableOption) p).getKeyString(), | ||
((SqlTableOption) p).getValueString())); | ||
return new AlterCatalogOperation(catalogName, properties); | ||
} | ||
} |
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