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-14490][table-planner] Extract CalciteParser from FlinkPlannerImpl
- Loading branch information
Showing
14 changed files
with
219 additions
and
108 deletions.
There are no files selected for viewing
71 changes: 71 additions & 0 deletions
71
...ble-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/CalciteParser.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,71 @@ | ||
/* | ||
* 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.calcite; | ||
|
||
import org.apache.flink.table.api.SqlParserException; | ||
|
||
import org.apache.calcite.sql.SqlIdentifier; | ||
import org.apache.calcite.sql.SqlNode; | ||
import org.apache.calcite.sql.parser.SqlParseException; | ||
import org.apache.calcite.sql.parser.SqlParser; | ||
|
||
/** | ||
* Thin wrapper around {@link SqlParser} that does exception conversion and {@link SqlNode} casting. | ||
*/ | ||
public class CalciteParser { | ||
private final SqlParser.Config config; | ||
|
||
public CalciteParser(SqlParser.Config config) { | ||
this.config = config; | ||
} | ||
|
||
/** | ||
* Parses a SQL statement into a {@link SqlNode}. The {@link SqlNode} is not yet validated. | ||
* | ||
* @param sql a sql string to parse | ||
* @return a parsed sql node | ||
* @throws SqlParserException if an exception is thrown when parsing the statement | ||
*/ | ||
public SqlNode parse(String sql) { | ||
try { | ||
SqlParser parser = SqlParser.create(sql, config); | ||
return parser.parseStmt(); | ||
} catch (SqlParseException e) { | ||
throw new SqlParserException("SQL parse failed. " + e.getMessage()); | ||
} | ||
} | ||
|
||
/** | ||
* Parses a SQL string as an identifier into a {@link SqlIdentifier}. | ||
* | ||
* @param identifier a sql string to parse as an identifier | ||
* @return a parsed sql node | ||
* @throws SqlParserException if an exception is thrown when parsing the identifier | ||
*/ | ||
public SqlIdentifier parseIdentifier(String identifier) { | ||
try { | ||
SqlParser parser = SqlParser.create(identifier, config); | ||
SqlNode sqlNode = parser.parseExpression(); | ||
return (SqlIdentifier) sqlNode; | ||
} catch (Exception e) { | ||
throw new SqlParserException(String.format( | ||
"Invalid SQL identifier %s. All SQL keywords must be escaped.", identifier)); | ||
} | ||
} | ||
} |
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
71 changes: 71 additions & 0 deletions
71
...table/flink-table-planner/src/main/java/org/apache/flink/table/calcite/CalciteParser.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,71 @@ | ||
/* | ||
* 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.calcite; | ||
|
||
import org.apache.flink.table.api.SqlParserException; | ||
|
||
import org.apache.calcite.sql.SqlIdentifier; | ||
import org.apache.calcite.sql.SqlNode; | ||
import org.apache.calcite.sql.parser.SqlParseException; | ||
import org.apache.calcite.sql.parser.SqlParser; | ||
|
||
/** | ||
* Thin wrapper around {@link SqlParser} that does exception conversion and {@link SqlNode} casting. | ||
*/ | ||
public class CalciteParser { | ||
private final SqlParser.Config config; | ||
|
||
public CalciteParser(SqlParser.Config config) { | ||
this.config = config; | ||
} | ||
|
||
/** | ||
* Parses a SQL statement into a {@link SqlNode}. The {@link SqlNode} is not yet validated. | ||
* | ||
* @param sql a sql string to parse | ||
* @return a parsed sql node | ||
* @throws SqlParserException if an exception is thrown when parsing the statement | ||
*/ | ||
public SqlNode parse(String sql) { | ||
try { | ||
SqlParser parser = SqlParser.create(sql, config); | ||
return parser.parseStmt(); | ||
} catch (SqlParseException e) { | ||
throw new SqlParserException("SQL parse failed. " + e.getMessage()); | ||
} | ||
} | ||
|
||
/** | ||
* Parses a SQL string as an identifier into a {@link SqlIdentifier}. | ||
* | ||
* @param identifier a sql string to parse as an identifier | ||
* @return a parsed sql node | ||
* @throws SqlParserException if an exception is thrown when parsing the identifier | ||
*/ | ||
public SqlIdentifier parseIdentifier(String identifier) { | ||
try { | ||
SqlParser parser = SqlParser.create(identifier, config); | ||
SqlNode sqlNode = parser.parseExpression(); | ||
return (SqlIdentifier) sqlNode; | ||
} catch (Exception e) { | ||
throw new SqlParserException(String.format( | ||
"Invalid SQL identifier %s. All SQL keywords must be escaped.", identifier)); | ||
} | ||
} | ||
} |
Oops, something went wrong.