Skip to content

Commit

Permalink
[FLINK-21888][hive] Maintain our own ASTNode class (apache#15301)
Browse files Browse the repository at this point in the history
  • Loading branch information
lirui-apache authored Mar 27, 2021
1 parent cb6b5c1 commit 23afdbc
Showing 22 changed files with 1,326 additions and 267 deletions.
Original file line number Diff line number Diff line change
@@ -37,6 +37,7 @@
import org.apache.flink.table.planner.delegation.hive.parse.HiveASTParseException;
import org.apache.flink.table.planner.delegation.hive.parse.HiveASTParseUtils;
import org.apache.flink.table.planner.delegation.hive.parse.HiveASTParser;
import org.apache.flink.table.planner.delegation.hive.parse.HiveParserASTNode;
import org.apache.flink.table.planner.delegation.hive.parse.HiveParserDDLSemanticAnalyzer;
import org.apache.flink.table.planner.parse.CalciteParser;
import org.apache.flink.table.planner.plan.FlinkCalciteCatalogReader;
@@ -49,7 +50,6 @@
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.ql.lockmgr.LockException;
import org.apache.hadoop.hive.ql.metadata.Hive;
import org.apache.hadoop.hive.ql.parse.ASTNode;
import org.apache.hadoop.hive.ql.parse.SemanticException;
import org.apache.hadoop.hive.ql.session.SessionState;
import org.slf4j.Logger;
@@ -78,7 +78,7 @@ public class HiveParser extends ParserImpl {
HiveReflectionUtils.tryGetMethod(
SessionState.class, "setupQueryCurrentTimestamp", new Class[0]);

// need to maintain the ASTNode types for DDLs
// need to maintain the HiveParserASTNode types for DDLs
private static final Set<Integer> DDL_NODES;

static {
@@ -201,7 +201,7 @@ private List<Operation> processCmd(
try {
final HiveParserContext context = new HiveParserContext(hiveConf);
// parse statement to get AST
final ASTNode node = HiveASTParseUtils.parse(cmd, context);
final HiveParserASTNode node = HiveASTParseUtils.parse(cmd, context);
if (DDL_NODES.contains(node.getType())) {
HiveParserQueryState queryState = new HiveParserQueryState(hiveConf);
HiveParserDDLSemanticAnalyzer ddlAnalyzer =
Original file line number Diff line number Diff line change
@@ -20,8 +20,7 @@

import org.apache.flink.table.planner.delegation.hive.parse.HiveASTParseDriver;
import org.apache.flink.table.planner.delegation.hive.parse.HiveASTParser;

import org.apache.hadoop.hive.ql.parse.ASTNode;
import org.apache.flink.table.planner.delegation.hive.parse.HiveParserASTNode;

/** Counterpart of hive's org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTBuilder. */
public class HiveParserASTBuilder {
@@ -32,11 +31,11 @@ public static HiveParserASTBuilder construct(int tokenType, String text) {
return b;
}

public static ASTNode createAST(int tokenType, String text) {
return (ASTNode) HiveASTParseDriver.ADAPTOR.create(tokenType, text);
public static HiveParserASTNode createAST(int tokenType, String text) {
return (HiveParserASTNode) HiveASTParseDriver.ADAPTOR.create(tokenType, text);
}

public static ASTNode qualifiedName(String tableName, String colName) {
public static HiveParserASTNode qualifiedName(String tableName, String colName) {
HiveParserASTBuilder b =
HiveParserASTBuilder.construct(HiveASTParser.DOT, ".")
.add(
@@ -47,9 +46,9 @@ public static ASTNode qualifiedName(String tableName, String colName) {
return b.node();
}

ASTNode curr;
HiveParserASTNode curr;

public ASTNode node() {
public HiveParserASTNode node() {
return curr;
}

Original file line number Diff line number Diff line change
@@ -19,18 +19,18 @@
package org.apache.flink.table.planner.delegation.hive;

import org.apache.flink.table.planner.delegation.hive.parse.HiveASTParser;
import org.apache.flink.table.planner.delegation.hive.parse.HiveParserASTNode;
import org.apache.flink.table.planner.delegation.hive.parse.HiveParserBaseSemanticAnalyzer;

import org.apache.hadoop.hive.metastore.api.PrincipalType;
import org.apache.hadoop.hive.ql.parse.ASTNode;
import org.apache.hadoop.hive.ql.plan.PrincipalDesc;

/** Counterpart of hive's org.apache.hadoop.hive.ql.parse.authorization.AuthorizationParseUtils. */
public class HiveParserAuthorizationParseUtils {

private HiveParserAuthorizationParseUtils() {}

public static PrincipalDesc getPrincipalDesc(ASTNode principal) {
public static PrincipalDesc getPrincipalDesc(HiveParserASTNode principal) {
PrincipalType type = getPrincipalType(principal);
if (type != null) {
String text = principal.getChild(0).getText();
@@ -40,7 +40,7 @@ public static PrincipalDesc getPrincipalDesc(ASTNode principal) {
return null;
}

private static PrincipalType getPrincipalType(ASTNode principal) {
private static PrincipalType getPrincipalType(HiveParserASTNode principal) {
switch (principal.getType()) {
case HiveASTParser.TOK_USER:
return PrincipalType.USER;
Original file line number Diff line number Diff line change
@@ -19,11 +19,11 @@
package org.apache.flink.table.planner.delegation.hive;

import org.apache.flink.table.planner.delegation.hive.parse.HiveASTParser;
import org.apache.flink.table.planner.delegation.hive.parse.HiveParserASTNode;

import org.antlr.runtime.TokenRewriteStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.ql.parse.ASTNode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

@@ -71,8 +71,8 @@ public String toString() {
}
}

/** The suffix is always relative to a given ASTNode. */
public DestClausePrefix getDestNamePrefix(ASTNode curNode) {
/** The suffix is always relative to a given HiveParserASTNode. */
public DestClausePrefix getDestNamePrefix(HiveParserASTNode curNode) {
assert curNode != null : "must supply curNode";
assert curNode.getType() == HiveASTParser.TOK_INSERT_INTO
|| curNode.getType() == HiveASTParser.TOK_DESTINATION;
Original file line number Diff line number Diff line change
@@ -18,15 +18,14 @@

package org.apache.flink.table.planner.delegation.hive;

import org.apache.hadoop.hive.ql.ErrorMsg;
import org.apache.hadoop.hive.ql.parse.ASTNode;
import org.apache.flink.table.planner.delegation.hive.parse.HiveParserASTNode;

/** Util class for the hive parser. */
public class HiveParserUtils {

private HiveParserUtils() {}

public static String generateErrorMessage(ASTNode ast, String message) {
public static String generateErrorMessage(HiveParserASTNode ast, String message) {
StringBuilder sb = new StringBuilder();
if (ast == null) {
sb.append(message).append(". Cannot tell the position of null AST.");
@@ -38,8 +37,15 @@ public static String generateErrorMessage(ASTNode ast, String message) {
sb.append(" ");
sb.append(message);
sb.append(". Error encountered near token '");
sb.append(ErrorMsg.getText(ast));
sb.append(getText(ast));
sb.append("'");
return sb.toString();
}

private static String getText(HiveParserASTNode tree) {
if (tree.getChildCount() == 0) {
return tree.getText();
}
return getText((HiveParserASTNode) tree.getChild(tree.getChildCount() - 1));
}
}
Original file line number Diff line number Diff line change
@@ -18,7 +18,7 @@

package org.apache.flink.table.planner.delegation.hive.desc;

import org.apache.hadoop.hive.ql.parse.ASTNode;
import org.apache.flink.table.planner.delegation.hive.parse.HiveParserASTNode;

import java.io.Serializable;

@@ -28,9 +28,9 @@ public class CreateTableASDesc implements Serializable {
private static final long serialVersionUID = 1L;

private final HiveParserCreateTableDesc createTableDesc;
private final ASTNode query;
private final HiveParserASTNode query;

public CreateTableASDesc(HiveParserCreateTableDesc createTableDesc, ASTNode query) {
public CreateTableASDesc(HiveParserCreateTableDesc createTableDesc, HiveParserASTNode query) {
this.createTableDesc = createTableDesc;
this.query = query;
}
@@ -39,7 +39,7 @@ public HiveParserCreateTableDesc getCreateTableDesc() {
return createTableDesc;
}

public ASTNode getQuery() {
public HiveParserASTNode getQuery() {
return query;
}
}
Original file line number Diff line number Diff line change
@@ -18,8 +18,9 @@

package org.apache.flink.table.planner.delegation.hive.desc;

import org.apache.flink.table.planner.delegation.hive.parse.HiveParserASTNode;

import org.apache.hadoop.hive.metastore.api.FieldSchema;
import org.apache.hadoop.hive.ql.parse.ASTNode;

import java.io.Serializable;
import java.util.List;
@@ -35,7 +36,7 @@ public class HiveParserCreateViewDesc implements Serializable {
private final Map<String, String> tblProps;
private final boolean ifNotExists;
private final boolean isAlterViewAs;
private final ASTNode query;
private final HiveParserASTNode query;

private List<FieldSchema> schema;
private String originalText;
@@ -48,7 +49,7 @@ public HiveParserCreateViewDesc(
Map<String, String> tblProps,
boolean ifNotExists,
boolean isAlterViewAs,
ASTNode query) {
HiveParserASTNode query) {
this.compoundName = compoundName;
this.schema = schema;
this.comment = comment;
@@ -102,7 +103,7 @@ public void setExpandedText(String expandedText) {
this.expandedText = expandedText;
}

public ASTNode getQuery() {
public HiveParserASTNode getQuery() {
return query;
}

Original file line number Diff line number Diff line change
@@ -20,7 +20,7 @@ parser grammar FromClauseASTParser;
options
{
output=AST;
ASTLabelType=ASTNode;
ASTLabelType=HiveParserASTNode;
backtrack=false;
k=3;
}
Original file line number Diff line number Diff line change
@@ -21,7 +21,7 @@ options
{
tokenVocab=HiveASTLexer;
output=AST;
ASTLabelType=ASTNode;
ASTLabelType=HiveParserASTNode;
backtrack=false;
k=3;
}
@@ -39,7 +39,7 @@ package org.apache.flink.table.planner.delegation.hive.parse;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.ql.parse.ASTNode;
import org.apache.flink.table.planner.delegation.hive.parse.HiveParserASTNode;
}


Original file line number Diff line number Diff line change
@@ -22,7 +22,7 @@ package org.apache.flink.table.planner.delegation.hive.parse;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.ql.parse.ASTNode;
import org.apache.flink.table.planner.delegation.hive.parse.HiveParserASTNode;
}

@lexer::members {
Original file line number Diff line number Diff line change
@@ -31,7 +31,6 @@
import org.antlr.runtime.tree.CommonTreeAdaptor;
import org.antlr.runtime.tree.TreeAdaptor;
import org.apache.hadoop.hive.ql.parse.ASTErrorNode;
import org.apache.hadoop.hive.ql.parse.ASTNode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

@@ -122,15 +121,15 @@ public ArrayList<HiveASTParseError> getErrors() {
public static final TreeAdaptor ADAPTOR =
new CommonTreeAdaptor() {
/**
* Creates an ASTNode for the given token. The ASTNode is a wrapper around antlr's
* CommonTree class that implements the Node interface.
* Creates an HiveParserASTNode for the given token. The HiveParserASTNode is a
* wrapper around antlr's CommonTree class that implements the Node interface.
*
* @param payload The token.
* @return Object (which is actually an ASTNode) for the token.
* @return Object (which is actually an HiveParserASTNode) for the token.
*/
@Override
public Object create(Token payload) {
return new ASTNode(payload);
return new HiveParserASTNode(payload);
}

@Override
@@ -153,7 +152,8 @@ public Object errorNode(
* context is available or the context already has an existing stream
* @return parsed AST
*/
public ASTNode parse(String command, HiveParserContext ctx, String viewFullyQualifiedName)
public HiveParserASTNode parse(
String command, HiveParserContext ctx, String viewFullyQualifiedName)
throws HiveASTParseException {
if (LOG.isDebugEnabled()) {
LOG.debug("Parsing command: " + command);
@@ -191,15 +191,15 @@ public ASTNode parse(String command, HiveParserContext ctx, String viewFullyQual
throw new HiveASTParseException(parser.errors);
}

ASTNode tree = r.getTree();
HiveParserASTNode tree = r.getTree();
tree.setUnknownTokenBoundaries();
return tree;
}

/*
* Parse a string as a query hint.
*/
public ASTNode parseHint(String command) throws HiveASTParseException {
public HiveParserASTNode parseHint(String command) throws HiveASTParseException {
LOG.info("Parsing hint: " + command);

HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
Loading

0 comments on commit 23afdbc

Please sign in to comment.