You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by nongli <gi...@git.apache.org> on 2015/12/21 23:25:09 UTC

[GitHub] spark pull request: [SPARK-12363] [SQL] Inline Hive parser into sp...

GitHub user nongli opened a pull request:

    https://github.com/apache/spark/pull/10420

    [SPARK-12363] [SQL] Inline Hive parser into spark sql

    This is a WIP. This inlines the hive sql grammar parser into spark sql in the hive
    subproject. This should eventually be moved into the SQL core project once all the
    hive dependencies are removed.
    
    This patch does some of that by cleaning up the hive code to remove much of semantic
    analysis. The SBT plugin uses 3.5.2 (the latest in 3.5.x) and this updates the grammar
    to support that (from 3.4). Changes are very minor.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/nongli/spark hive-parser

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/10420.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #10420
    
----

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12363] [SQL] Inline Hive parser into sp...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/10420#issuecomment-166519668
  
    I think you meant https://issues.apache.org/jira/browse/SPARK-12362, not 12363?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12363] [SQL] Inline Hive parser into sp...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10420#discussion_r48458578
  
    --- Diff: sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java ---
    @@ -0,0 +1,260 @@
    +/**
    + * 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.spark.sql.parser;
    +
    +import java.util.ArrayList;
    +import org.antlr.runtime.ANTLRStringStream;
    +import org.antlr.runtime.CharStream;
    +import org.antlr.runtime.NoViableAltException;
    +import org.antlr.runtime.RecognitionException;
    +import org.antlr.runtime.Token;
    +import org.antlr.runtime.TokenRewriteStream;
    +import org.antlr.runtime.TokenStream;
    +import org.antlr.runtime.tree.CommonTree;
    +import org.antlr.runtime.tree.CommonTreeAdaptor;
    +import org.antlr.runtime.tree.TreeAdaptor;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.hadoop.hive.ql.Context;
    +
    +/**
    + * ParseDriver.
    + *
    + */
    +public class ParseDriver {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger("hive.ql.parse.ParseDriver");
    +
    +  /**
    +   * ANTLRNoCaseStringStream.
    +   *
    +   */
    +  //This class provides and implementation for a case insensitive token checker
    +  //for the lexical analysis part of antlr. By converting the token stream into
    +  //upper case at the time when lexical rules are checked, this class ensures that the
    +  //lexical rules need to just match the token with upper case letters as opposed to
    +  //combination of upper case and lower case characteres. This is purely used for matching lexical
    +  //rules. The actual token text is stored in the same way as the user input without
    +  //actually converting it into an upper case. The token values are generated by the consume()
    +  //function of the super class ANTLRStringStream. The LA() function is the lookahead funtion
    +  //and is purely used for matching lexical rules. This also means that the grammar will only
    +  //accept capitalized tokens in case it is run from other tools like antlrworks which
    +  //do not have the ANTLRNoCaseStringStream implementation.
    +  public class ANTLRNoCaseStringStream extends ANTLRStringStream {
    +
    +    public ANTLRNoCaseStringStream(String input) {
    +      super(input);
    +    }
    +
    +    @Override
    +    public int LA(int i) {
    +
    +      int returnChar = super.LA(i);
    +      if (returnChar == CharStream.EOF) {
    +        return returnChar;
    +      } else if (returnChar == 0) {
    +        return returnChar;
    +      }
    +
    +      return Character.toUpperCase((char) returnChar);
    +    }
    +  }
    +
    +  /**
    +   * HiveLexerX.
    +   *
    +   */
    +  public class HiveLexerX extends SparkSqlLexer {
    +
    +    private final ArrayList<ParseError> errors;
    +
    +    public HiveLexerX() {
    +      super();
    +      errors = new ArrayList<ParseError>();
    +    }
    +
    +    public HiveLexerX(CharStream input) {
    +      super(input);
    +      errors = new ArrayList<ParseError>();
    +    }
    +
    +    @Override
    +    public void displayRecognitionError(String[] tokenNames,
    +        RecognitionException e) {
    +
    +      errors.add(new ParseError(this, e, tokenNames));
    +    }
    +
    +    @Override
    +    public String getErrorMessage(RecognitionException e, String[] tokenNames) {
    +      String msg = null;
    +
    +      if (e instanceof NoViableAltException) {
    +        @SuppressWarnings("unused")
    +        NoViableAltException nvae = (NoViableAltException) e;
    +        // for development, can add
    +        // "decision=<<"+nvae.grammarDecisionDescription+">>"
    +        // and "(decision="+nvae.decisionNumber+") and
    +        // "state "+nvae.stateNumber
    +        msg = "character " + getCharErrorDisplay(e.c) + " not supported here";
    +      } else {
    +        msg = super.getErrorMessage(e, tokenNames);
    +      }
    +
    +      return msg;
    +    }
    +
    +    public ArrayList<ParseError> getErrors() {
    +      return errors;
    +    }
    +
    +  }
    +
    +  /**
    +   * Tree adaptor for making antlr return ASTNodes instead of CommonTree nodes
    +   * so that the graph walking algorithms and the rules framework defined in
    +   * ql.lib can be used with the AST Nodes.
    +   */
    +  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.
    +     *
    +     * @param payload
    +     *          The token.
    +     * @return Object (which is actually an ASTNode) for the token.
    +     */
    +    @Override
    +    public Object create(Token payload) {
    +      return new ASTNode(payload);
    +    }
    +
    +    @Override
    +    public Object dupNode(Object t) {
    +
    +      return create(((CommonTree)t).token);
    +    };
    +
    +    @Override
    +    public Object errorNode(TokenStream input, Token start, Token stop, RecognitionException e) {
    +      return new ASTErrorNode(input, start, stop, e);
    +    };
    +  };
    +
    +  public ASTNode parse(String command) throws ParseException {
    +    return parse(command, null);
    +  }
    +  
    +  public ASTNode parse(String command, Context ctx) 
    +      throws ParseException {
    +    return parse(command, ctx, true);
    +  }
    +
    +  /**
    +   * Parses a command, optionally assigning the parser's token stream to the
    +   * given context.
    +   *
    +   * @param command
    +   *          command to parse
    +   *
    +   * @param ctx
    +   *          context with which to associate this parser's token stream, or
    +   *          null if either no context is available or the context already has
    +   *          an existing stream
    +   *
    +   * @return parsed AST
    +   */
    +  public ASTNode parse(String command, Context ctx, boolean setTokenRewriteStream) 
    +      throws ParseException {
    +    LOG.info("Parsing command: " + command);
    +
    +    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
    +    TokenRewriteStream tokens = new TokenRewriteStream(lexer);
    +    if (ctx != null) {
    +      if ( setTokenRewriteStream) {
    +        ctx.setTokenRewriteStream(tokens);
    +      }
    +      lexer.setHiveConf(ctx.getConf());
    +    }
    +    SparkSqlParser parser = new SparkSqlParser(tokens);
    +    if (ctx != null) {
    +      parser.setHiveConf(ctx.getConf());
    --- End diff --
    
    The parser only checks the ```HiveConf``` to check if we can use SQL11 as identifiers. We could move this to our own configuration.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12363] [SQL] Inline Hive parser into sp...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10420#discussion_r48458564
  
    --- Diff: sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java ---
    @@ -0,0 +1,260 @@
    +/**
    + * 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.spark.sql.parser;
    +
    +import java.util.ArrayList;
    +import org.antlr.runtime.ANTLRStringStream;
    +import org.antlr.runtime.CharStream;
    +import org.antlr.runtime.NoViableAltException;
    +import org.antlr.runtime.RecognitionException;
    +import org.antlr.runtime.Token;
    +import org.antlr.runtime.TokenRewriteStream;
    +import org.antlr.runtime.TokenStream;
    +import org.antlr.runtime.tree.CommonTree;
    +import org.antlr.runtime.tree.CommonTreeAdaptor;
    +import org.antlr.runtime.tree.TreeAdaptor;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.hadoop.hive.ql.Context;
    +
    +/**
    + * ParseDriver.
    + *
    + */
    +public class ParseDriver {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger("hive.ql.parse.ParseDriver");
    +
    +  /**
    +   * ANTLRNoCaseStringStream.
    +   *
    +   */
    +  //This class provides and implementation for a case insensitive token checker
    +  //for the lexical analysis part of antlr. By converting the token stream into
    +  //upper case at the time when lexical rules are checked, this class ensures that the
    +  //lexical rules need to just match the token with upper case letters as opposed to
    +  //combination of upper case and lower case characteres. This is purely used for matching lexical
    +  //rules. The actual token text is stored in the same way as the user input without
    +  //actually converting it into an upper case. The token values are generated by the consume()
    +  //function of the super class ANTLRStringStream. The LA() function is the lookahead funtion
    +  //and is purely used for matching lexical rules. This also means that the grammar will only
    +  //accept capitalized tokens in case it is run from other tools like antlrworks which
    +  //do not have the ANTLRNoCaseStringStream implementation.
    +  public class ANTLRNoCaseStringStream extends ANTLRStringStream {
    +
    +    public ANTLRNoCaseStringStream(String input) {
    +      super(input);
    +    }
    +
    +    @Override
    +    public int LA(int i) {
    +
    +      int returnChar = super.LA(i);
    +      if (returnChar == CharStream.EOF) {
    +        return returnChar;
    +      } else if (returnChar == 0) {
    +        return returnChar;
    +      }
    +
    +      return Character.toUpperCase((char) returnChar);
    +    }
    +  }
    +
    +  /**
    +   * HiveLexerX.
    +   *
    +   */
    +  public class HiveLexerX extends SparkSqlLexer {
    +
    +    private final ArrayList<ParseError> errors;
    +
    +    public HiveLexerX() {
    +      super();
    +      errors = new ArrayList<ParseError>();
    +    }
    +
    +    public HiveLexerX(CharStream input) {
    +      super(input);
    +      errors = new ArrayList<ParseError>();
    +    }
    +
    +    @Override
    +    public void displayRecognitionError(String[] tokenNames,
    +        RecognitionException e) {
    +
    +      errors.add(new ParseError(this, e, tokenNames));
    +    }
    +
    +    @Override
    +    public String getErrorMessage(RecognitionException e, String[] tokenNames) {
    +      String msg = null;
    +
    +      if (e instanceof NoViableAltException) {
    +        @SuppressWarnings("unused")
    +        NoViableAltException nvae = (NoViableAltException) e;
    +        // for development, can add
    +        // "decision=<<"+nvae.grammarDecisionDescription+">>"
    +        // and "(decision="+nvae.decisionNumber+") and
    +        // "state "+nvae.stateNumber
    +        msg = "character " + getCharErrorDisplay(e.c) + " not supported here";
    +      } else {
    +        msg = super.getErrorMessage(e, tokenNames);
    +      }
    +
    +      return msg;
    +    }
    +
    +    public ArrayList<ParseError> getErrors() {
    +      return errors;
    +    }
    +
    +  }
    +
    +  /**
    +   * Tree adaptor for making antlr return ASTNodes instead of CommonTree nodes
    +   * so that the graph walking algorithms and the rules framework defined in
    +   * ql.lib can be used with the AST Nodes.
    +   */
    +  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.
    +     *
    +     * @param payload
    +     *          The token.
    +     * @return Object (which is actually an ASTNode) for the token.
    +     */
    +    @Override
    +    public Object create(Token payload) {
    +      return new ASTNode(payload);
    +    }
    +
    +    @Override
    +    public Object dupNode(Object t) {
    +
    +      return create(((CommonTree)t).token);
    +    };
    +
    +    @Override
    +    public Object errorNode(TokenStream input, Token start, Token stop, RecognitionException e) {
    +      return new ASTErrorNode(input, start, stop, e);
    +    };
    +  };
    +
    +  public ASTNode parse(String command) throws ParseException {
    +    return parse(command, null);
    +  }
    +  
    +  public ASTNode parse(String command, Context ctx) 
    +      throws ParseException {
    +    return parse(command, ctx, true);
    +  }
    +
    +  /**
    +   * Parses a command, optionally assigning the parser's token stream to the
    +   * given context.
    +   *
    +   * @param command
    +   *          command to parse
    +   *
    +   * @param ctx
    +   *          context with which to associate this parser's token stream, or
    +   *          null if either no context is available or the context already has
    +   *          an existing stream
    +   *
    +   * @return parsed AST
    +   */
    +  public ASTNode parse(String command, Context ctx, boolean setTokenRewriteStream) 
    +      throws ParseException {
    +    LOG.info("Parsing command: " + command);
    +
    +    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
    +    TokenRewriteStream tokens = new TokenRewriteStream(lexer);
    +    if (ctx != null) {
    +      if ( setTokenRewriteStream) {
    +        ctx.setTokenRewriteStream(tokens);
    +      }
    +      lexer.setHiveConf(ctx.getConf());
    --- End diff --
    
    The lexer only checks the ```HiveConf``` to check if it supports quoted ids. We could move this to our own configuration.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12363] [SQL] Inline Hive parser into sp...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10420#issuecomment-166441617
  
    **[Test build #48132 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/48132/consoleFull)** for PR 10420 at commit [`7e1a145`](https://github.com/apache/spark/commit/7e1a14582fc32fda2016072138b4a431c7ba9333).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12363] [SQL] Inline Hive parser into sp...

Posted by nongli <gi...@git.apache.org>.
Github user nongli closed the pull request at:

    https://github.com/apache/spark/pull/10420


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12363] [SQL] Inline Hive parser into sp...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/10420#discussion_r48458597
  
    --- Diff: sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java ---
    @@ -0,0 +1,260 @@
    +/**
    + * 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.spark.sql.parser;
    +
    +import java.util.ArrayList;
    +import org.antlr.runtime.ANTLRStringStream;
    +import org.antlr.runtime.CharStream;
    +import org.antlr.runtime.NoViableAltException;
    +import org.antlr.runtime.RecognitionException;
    +import org.antlr.runtime.Token;
    +import org.antlr.runtime.TokenRewriteStream;
    +import org.antlr.runtime.TokenStream;
    +import org.antlr.runtime.tree.CommonTree;
    +import org.antlr.runtime.tree.CommonTreeAdaptor;
    +import org.antlr.runtime.tree.TreeAdaptor;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.hadoop.hive.ql.Context;
    +
    +/**
    + * ParseDriver.
    + *
    + */
    +public class ParseDriver {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger("hive.ql.parse.ParseDriver");
    +
    +  /**
    +   * ANTLRNoCaseStringStream.
    +   *
    +   */
    +  //This class provides and implementation for a case insensitive token checker
    +  //for the lexical analysis part of antlr. By converting the token stream into
    +  //upper case at the time when lexical rules are checked, this class ensures that the
    +  //lexical rules need to just match the token with upper case letters as opposed to
    +  //combination of upper case and lower case characteres. This is purely used for matching lexical
    +  //rules. The actual token text is stored in the same way as the user input without
    +  //actually converting it into an upper case. The token values are generated by the consume()
    +  //function of the super class ANTLRStringStream. The LA() function is the lookahead funtion
    +  //and is purely used for matching lexical rules. This also means that the grammar will only
    +  //accept capitalized tokens in case it is run from other tools like antlrworks which
    +  //do not have the ANTLRNoCaseStringStream implementation.
    +  public class ANTLRNoCaseStringStream extends ANTLRStringStream {
    +
    +    public ANTLRNoCaseStringStream(String input) {
    +      super(input);
    +    }
    +
    +    @Override
    +    public int LA(int i) {
    +
    +      int returnChar = super.LA(i);
    +      if (returnChar == CharStream.EOF) {
    +        return returnChar;
    +      } else if (returnChar == 0) {
    +        return returnChar;
    +      }
    +
    +      return Character.toUpperCase((char) returnChar);
    +    }
    +  }
    +
    +  /**
    +   * HiveLexerX.
    +   *
    +   */
    +  public class HiveLexerX extends SparkSqlLexer {
    +
    +    private final ArrayList<ParseError> errors;
    +
    +    public HiveLexerX() {
    +      super();
    +      errors = new ArrayList<ParseError>();
    +    }
    +
    +    public HiveLexerX(CharStream input) {
    +      super(input);
    +      errors = new ArrayList<ParseError>();
    +    }
    +
    +    @Override
    +    public void displayRecognitionError(String[] tokenNames,
    +        RecognitionException e) {
    +
    +      errors.add(new ParseError(this, e, tokenNames));
    +    }
    +
    +    @Override
    +    public String getErrorMessage(RecognitionException e, String[] tokenNames) {
    +      String msg = null;
    +
    +      if (e instanceof NoViableAltException) {
    +        @SuppressWarnings("unused")
    +        NoViableAltException nvae = (NoViableAltException) e;
    +        // for development, can add
    +        // "decision=<<"+nvae.grammarDecisionDescription+">>"
    +        // and "(decision="+nvae.decisionNumber+") and
    +        // "state "+nvae.stateNumber
    +        msg = "character " + getCharErrorDisplay(e.c) + " not supported here";
    +      } else {
    +        msg = super.getErrorMessage(e, tokenNames);
    +      }
    +
    +      return msg;
    +    }
    +
    +    public ArrayList<ParseError> getErrors() {
    +      return errors;
    +    }
    +
    +  }
    +
    +  /**
    +   * Tree adaptor for making antlr return ASTNodes instead of CommonTree nodes
    +   * so that the graph walking algorithms and the rules framework defined in
    +   * ql.lib can be used with the AST Nodes.
    +   */
    +  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.
    +     *
    +     * @param payload
    +     *          The token.
    +     * @return Object (which is actually an ASTNode) for the token.
    +     */
    +    @Override
    +    public Object create(Token payload) {
    +      return new ASTNode(payload);
    +    }
    +
    +    @Override
    +    public Object dupNode(Object t) {
    +
    +      return create(((CommonTree)t).token);
    +    };
    +
    +    @Override
    +    public Object errorNode(TokenStream input, Token start, Token stop, RecognitionException e) {
    +      return new ASTErrorNode(input, start, stop, e);
    +    };
    +  };
    +
    +  public ASTNode parse(String command) throws ParseException {
    +    return parse(command, null);
    +  }
    +  
    +  public ASTNode parse(String command, Context ctx) 
    +      throws ParseException {
    +    return parse(command, ctx, true);
    +  }
    +
    +  /**
    +   * Parses a command, optionally assigning the parser's token stream to the
    +   * given context.
    +   *
    +   * @param command
    +   *          command to parse
    +   *
    +   * @param ctx
    +   *          context with which to associate this parser's token stream, or
    +   *          null if either no context is available or the context already has
    +   *          an existing stream
    +   *
    +   * @return parsed AST
    +   */
    +  public ASTNode parse(String command, Context ctx, boolean setTokenRewriteStream) 
    +      throws ParseException {
    +    LOG.info("Parsing command: " + command);
    +
    +    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
    +    TokenRewriteStream tokens = new TokenRewriteStream(lexer);
    +    if (ctx != null) {
    +      if ( setTokenRewriteStream) {
    +        ctx.setTokenRewriteStream(tokens);
    --- End diff --
    
    The ```Context.tokenRewriteStream``` is used in HiveQL. If we factor out this and the ```HiveQL``` (see comments below) we can drop the Context entirely.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12363] [SQL] Inline Hive parser into sp...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/10420#issuecomment-166519832
  
    Not sure if you have already done it. If not, we should make sure we make it clear in source code that this is coming from Hive.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12363] [SQL] Inline Hive parser into sp...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10420#issuecomment-166443494
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/48132/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12363] [SQL] Inline Hive parser into sp...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/10420#issuecomment-166443483
  
    **[Test build #48132 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/48132/consoleFull)** for PR 10420 at commit [`7e1a145`](https://github.com/apache/spark/commit/7e1a14582fc32fda2016072138b4a431c7ba9333).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-12363] [SQL] Inline Hive parser into sp...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/10420#issuecomment-166443491
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org