forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-17106][table-planner] Support create and drop view in both pla…
…nners This closes apache#11727
- Loading branch information
Showing
17 changed files
with
1,512 additions
and
12 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
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
80 changes: 80 additions & 0 deletions
80
...ble-api-java/src/main/java/org/apache/flink/table/operations/ddl/CreateViewOperation.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:https://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.table.catalog.CatalogView; | ||
import org.apache.flink.table.catalog.ObjectIdentifier; | ||
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; | ||
|
||
/** | ||
* Operation to describe a CREATE VIEW statement. | ||
*/ | ||
public class CreateViewOperation implements CreateOperation { | ||
private final ObjectIdentifier viewIdentifier; | ||
private CatalogView catalogView; | ||
private boolean ignoreIfExists; | ||
private boolean isTemporary; | ||
|
||
public CreateViewOperation( | ||
ObjectIdentifier viewIdentifier, | ||
CatalogView catalogView, | ||
boolean ignoreIfExists, | ||
boolean isTemporary) { | ||
this.viewIdentifier = viewIdentifier; | ||
this.catalogView = catalogView; | ||
this.ignoreIfExists = ignoreIfExists; | ||
this.isTemporary = isTemporary; | ||
} | ||
|
||
public CatalogView getCatalogView() { | ||
return catalogView; | ||
} | ||
|
||
public ObjectIdentifier getViewIdentifier() { | ||
return viewIdentifier; | ||
} | ||
|
||
public boolean isIgnoreIfExists() { | ||
return ignoreIfExists; | ||
} | ||
|
||
public boolean isTemporary() { | ||
return isTemporary; | ||
} | ||
|
||
@Override | ||
public String asSummaryString() { | ||
Map<String, Object> params = new LinkedHashMap<>(); | ||
params.put("originalQuery", catalogView.getOriginalQuery()); | ||
params.put("expandedQuery", catalogView.getExpandedQuery()); | ||
params.put("identifier", viewIdentifier); | ||
params.put("ignoreIfExists", ignoreIfExists); | ||
params.put("isTemporary", isTemporary); | ||
return OperationUtils.formatWithChildren( | ||
"CREATE VIEW", | ||
params, | ||
Collections.emptyList(), | ||
Operation::asSummaryString); | ||
} | ||
} |
69 changes: 69 additions & 0 deletions
69
...table-api-java/src/main/java/org/apache/flink/table/operations/ddl/DropViewOperation.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,69 @@ | ||
/* | ||
* 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:https://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.table.catalog.ObjectIdentifier; | ||
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; | ||
|
||
/** | ||
* Operation to describe a DROP VIEW statement. | ||
*/ | ||
public class DropViewOperation implements DropOperation { | ||
|
||
private final ObjectIdentifier viewIdentifier; | ||
private final boolean ifExists; | ||
private final boolean isTemporary; | ||
|
||
public DropViewOperation(ObjectIdentifier viewIdentifier, boolean ifExists, boolean isTemporary) { | ||
this.viewIdentifier = viewIdentifier; | ||
this.ifExists = ifExists; | ||
this.isTemporary = isTemporary; | ||
} | ||
|
||
public ObjectIdentifier getViewIdentifier() { | ||
return this.viewIdentifier; | ||
} | ||
|
||
public boolean isIfExists() { | ||
return this.ifExists; | ||
} | ||
|
||
public boolean isTemporary() { | ||
return this.isTemporary; | ||
} | ||
|
||
@Override | ||
public String asSummaryString() { | ||
Map<String, Object> params = new LinkedHashMap<>(); | ||
params.put("identifier", viewIdentifier); | ||
params.put("ifExists", ifExists); | ||
params.put("isTemporary", isTemporary); | ||
|
||
return OperationUtils.formatWithChildren( | ||
"DROP VIEW", | ||
params, | ||
Collections.emptyList(), | ||
Operation::asSummaryString); | ||
} | ||
} |
Oops, something went wrong.