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-21531][table][hive] Introduce pluggable Parser
This closes apache#15050
- Loading branch information
1 parent
14267e6
commit e964e40
Showing
10 changed files
with
312 additions
and
46 deletions.
There are no files selected for viewing
72 changes: 72 additions & 0 deletions
72
...nnector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.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,72 @@ | ||
/* | ||
* 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.planner.delegation.hive; | ||
|
||
import org.apache.flink.table.api.TableSchema; | ||
import org.apache.flink.table.catalog.CatalogManager; | ||
import org.apache.flink.table.operations.Operation; | ||
import org.apache.flink.table.planner.calcite.CalciteParser; | ||
import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; | ||
import org.apache.flink.table.planner.calcite.SqlExprToRexConverter; | ||
import org.apache.flink.table.planner.delegation.ParserImpl; | ||
import org.apache.flink.table.planner.delegation.PlannerContext; | ||
import org.apache.flink.table.planner.plan.FlinkCalciteCatalogReader; | ||
|
||
import org.apache.calcite.tools.FrameworkConfig; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.util.List; | ||
import java.util.function.Function; | ||
import java.util.function.Supplier; | ||
|
||
/** A Parser that uses Hive's planner to parse a statement. */ | ||
public class HiveParser extends ParserImpl { | ||
|
||
private static final Logger LOG = LoggerFactory.getLogger(HiveParser.class); | ||
|
||
private final PlannerContext plannerContext; | ||
private final FlinkCalciteCatalogReader catalogReader; | ||
private final FrameworkConfig frameworkConfig; | ||
|
||
HiveParser( | ||
CatalogManager catalogManager, | ||
Supplier<FlinkPlannerImpl> validatorSupplier, | ||
Supplier<CalciteParser> calciteParserSupplier, | ||
Function<TableSchema, SqlExprToRexConverter> sqlExprToRexConverterCreator, | ||
PlannerContext plannerContext) { | ||
super( | ||
catalogManager, | ||
validatorSupplier, | ||
calciteParserSupplier, | ||
sqlExprToRexConverterCreator); | ||
this.plannerContext = plannerContext; | ||
this.catalogReader = | ||
plannerContext.createCatalogReader( | ||
false, | ||
catalogManager.getCurrentCatalog(), | ||
catalogManager.getCurrentDatabase()); | ||
this.frameworkConfig = plannerContext.createFrameworkConfig(); | ||
} | ||
|
||
@Override | ||
public List<Operation> parse(String statement) { | ||
return super.parse(statement); | ||
} | ||
} |
72 changes: 72 additions & 0 deletions
72
...-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParserFactory.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,72 @@ | ||
/* | ||
* 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.planner.delegation.hive; | ||
|
||
import org.apache.flink.table.api.SqlDialect; | ||
import org.apache.flink.table.api.config.TableConfigOptions; | ||
import org.apache.flink.table.catalog.CatalogManager; | ||
import org.apache.flink.table.delegation.Parser; | ||
import org.apache.flink.table.descriptors.DescriptorProperties; | ||
import org.apache.flink.table.planner.calcite.SqlExprToRexConverterFactory; | ||
import org.apache.flink.table.planner.delegation.ParserFactory; | ||
import org.apache.flink.table.planner.delegation.PlannerContext; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.Map; | ||
|
||
/** A Parser factory that creates {@link HiveParser}. */ | ||
public class HiveParserFactory implements ParserFactory { | ||
|
||
@Override | ||
public Parser create(CatalogManager catalogManager, PlannerContext plannerContext) { | ||
SqlExprToRexConverterFactory sqlExprToRexConverterFactory = | ||
plannerContext::createSqlExprToRexConverter; | ||
return new HiveParser( | ||
catalogManager, | ||
() -> | ||
plannerContext.createFlinkPlanner( | ||
catalogManager.getCurrentCatalog(), | ||
catalogManager.getCurrentDatabase()), | ||
plannerContext::createCalciteParser, | ||
tableSchema -> | ||
sqlExprToRexConverterFactory.create( | ||
plannerContext.getTypeFactory().buildRelNodeRowType(tableSchema)), | ||
plannerContext); | ||
} | ||
|
||
@Override | ||
public Map<String, String> optionalContext() { | ||
DescriptorProperties properties = new DescriptorProperties(); | ||
return properties.asMap(); | ||
} | ||
|
||
@Override | ||
public Map<String, String> requiredContext() { | ||
DescriptorProperties properties = new DescriptorProperties(); | ||
properties.putString( | ||
TableConfigOptions.TABLE_SQL_DIALECT.key(), SqlDialect.HIVE.name().toLowerCase()); | ||
return properties.asMap(); | ||
} | ||
|
||
@Override | ||
public List<String> supportedProperties() { | ||
return Collections.singletonList(TableConfigOptions.TABLE_SQL_DIALECT.key()); | ||
} | ||
} |
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
Oops, something went wrong.