You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by hvanhovell <gi...@git.apache.org> on 2016/03/28 23:54:43 UTC

[GitHub] spark pull request: [SPARK-14213][SQL] Migrate HiveQl parsing to A...

GitHub user hvanhovell opened a pull request:

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

    [SPARK-14213][SQL] Migrate HiveQl parsing to ANTLR4 parser [WIP]

    ### What changes were proposed in this pull request?
    
    This PR migrates all HiveQl parsing to the new ANTLR4 parser. This PR is build on top of https://github.com/apache/spark/pull/12011, and we should wait with merging until that one is in (hence the WIP tag).
    
    As soon as this PR is merged we can start removing much of the old parser infrastructure.
    
    ### How was this patch tested?
    
    Exisiting Hive unit tests.
    
    cc @rxin @andrewor14 @yhuai

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

    $ git pull https://github.com/hvanhovell/spark SPARK-14213

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

    https://github.com/apache/spark/pull/12015.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 #12015
    
----
commit f7c8eb5bca073d521d0f306e740cbc872cbc657e
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-03-28T19:57:20Z

    Add DDL commands to SparkSqlParser.

commit 575acdb251328b28f05b2748d73a2a46b930f967
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-03-28T20:14:04Z

    Fix doc.

commit b836eae02e10f9a0600c9c2a7e51ffa7365cb848
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-03-28T20:58:38Z

    SerdeClass is option in tableRowFormat.

commit 1fc35921782f29b318a866be04df68f0e2e9cb81
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-03-28T21:45:29Z

    Migrate HiveQl parsing to ANTLR4 parser.

----


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#issuecomment-202629016
  
    **[Test build #54367 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54367/consoleFull)** for PR 12015 at commit [`1fc3592`](https://github.com/apache/spark/commit/1fc35921782f29b318a866be04df68f0e2e9cb81).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class HiveSqlAstBuilder extends SparkSqlAstBuilder `


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

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


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#discussion_r57946081
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.hive.execution
    +
    +import scala.collection.JavaConverters._
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.apache.hadoop.hive.conf.HiveConf
    +import org.apache.hadoop.hive.conf.HiveConf.ConfVars
    +import org.apache.hadoop.hive.ql.exec.FunctionRegistry
    +import org.apache.hadoop.hive.ql.parse.EximUtil
    +import org.apache.hadoop.hive.ql.session.SessionState
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +
    +import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogStorageFormat, CatalogTable, CatalogTableType}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ng._
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkSqlAstBuilder
    +import org.apache.spark.sql.hive.{CreateTableAsSelect => CTAS, CreateViewAsSelect => CreateView}
    +import org.apache.spark.sql.hive.{HiveGenericUDTF, HiveSerDe}
    +import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper
    +
    +/**
    + * Concrete parser for HiveQl statements.
    + */
    +object HiveSqlParser extends AbstractSqlParser {
    +  val astBuilder = new HiveSqlAstBuilder
    +
    +  override protected def nativeCommand(sqlText: String): LogicalPlan = {
    +    HiveNativeCommand(sqlText)
    +  }
    +}
    +
    +/**
    + * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier.
    + */
    +class HiveSqlAstBuilder extends SparkSqlAstBuilder {
    +  import ParserUtils._
    +
    +  /**
    +   * Get the current Hive Configuration.
    +   */
    +  private[this] def hiveConf: HiveConf = {
    +    var ss = SessionState.get()
    +    // SessionState is lazy initialization, it can be null here
    +    if (ss == null) {
    +      val original = Thread.currentThread().getContextClassLoader
    +      val conf = new HiveConf(classOf[SessionState])
    +      conf.setClassLoader(original)
    +      ss = new SessionState(conf)
    +      SessionState.start(ss)
    +    }
    +    ss.getConf
    +  }
    +
    +  /**
    +   * Pass a command to Hive using a [[HiveNativeCommand]].
    +   */
    +  override def visitExecuteNativeCommand(
    +      ctx: ExecuteNativeCommandContext): LogicalPlan = withOrigin(ctx) {
    +    HiveNativeCommand(command(ctx))
    +  }
    +
    +  /**
    +   * Fail an unsupported Hive native command.
    +   */
    +  override def visitFailNativeCommand(
    +      ctx: FailNativeCommandContext): LogicalPlan = withOrigin(ctx) {
    +    val keywords = if (ctx.kws != null) {
    +      Seq(ctx.kws.kw1, ctx.kws.kw2, ctx.kws.kw3).filter(_ != null).map(_.getText).mkString(" ")
    +    } else {
    +      // SET ROLE is the exception to the rule, because we handle this before other SET commands.
    +      "SET ROLE"
    +    }
    +    throw new ParseException(s"Unsupported operation: $keywords", ctx)
    +  }
    +
    +  /**
    +   * Create an [[AddJar]] or [[AddFile]] command depending on the requested resource.
    +   */
    +  override def visitAddResource(ctx: AddResourceContext): LogicalPlan = withOrigin(ctx) {
    +    ctx.identifier.getText.toLowerCase match {
    +      case "file" => AddFile(remainder(ctx.identifier).trim)
    +      case "jar" => AddJar(remainder(ctx.identifier).trim)
    +      case other => throw new ParseException(s"Unsupported resource type '$other'.", ctx)
    +    }
    +  }
    +
    +  /**
    +   * Create a [[DropTable]] command.
    +   */
    +  override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) {
    +    if (ctx.PURGE != null) {
    +      logWarning("PURGE option is ignored.")
    +    }
    +    if (ctx.REPLICATION != null) {
    +      logWarning("REPLICATION clause is ignored.")
    +    }
    +    DropTable(visitTableIdentifier(ctx.tableIdentifier).toString, ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Create an [[AnalyzeTable]] command. This currently only implements the NOSCAN option (other
    +   * options are passed on to Hive) e.g.:
    +   * {{{
    +   *   ANALYZE TABLE table COMPUTE STATISTICS NOSCAN;
    +   * }}}
    +   */
    +  override def visitAnalyze(ctx: AnalyzeContext): LogicalPlan = withOrigin(ctx) {
    +    if (ctx.partitionSpec == null &&
    +      ctx.identifier != null &&
    +      ctx.identifier.getText.toLowerCase == "noscan") {
    +      AnalyzeTable(visitTableIdentifier(ctx.tableIdentifier).toString)
    +    } else {
    +      HiveNativeCommand(command(ctx))
    +    }
    +  }
    +
    +  /**
    +   * Create a [[CreateTableAsSelect]] command.
    +   */
    +  override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = {
    +    if (ctx.query == null) {
    +      HiveNativeCommand(command(ctx))
    +    } else {
    +      // Get the table header.
    +      val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader)
    +      val tableType = if (external) {
    +        CatalogTableType.EXTERNAL_TABLE
    +      } else {
    +        CatalogTableType.MANAGED_TABLE
    +      }
    +
    +      // Unsupported clauses.
    +      if (temp) {
    +        logWarning("TEMPORARY clause is ignored.")
    +      }
    +      if (ctx.bucketSpec != null) {
    +        // TODO add this - we need cluster columns in the CatalogTable for this to work.
    +        logWarning("CLUSTERED BY ... [ORDERED BY ...] INTO ... BUCKETS clause is ignored.")
    +      }
    +      if (ctx.skewSpec != null) {
    +        logWarning("SKEWED BY ... ON ... [STORED AS DIRECTORIES] clause is ignored.")
    +      }
    +
    +      // Create the schema.
    +      val schema = Option(ctx.colTypeList).toSeq.flatMap(_.colType.asScala).map { col =>
    +        CatalogColumn(
    +          col.identifier.getText,
    +          col.dataType.getText.toLowerCase, // TODO validate this?
    +          nullable = true,
    +          Option(col.STRING).map(string))
    +      }
    +
    +      // Get the column by which the table is partitioned.
    +      val partitionCols = Option(ctx.identifierList).toSeq.flatMap(visitIdentifierList).map {
    +        CatalogColumn(_, null, nullable = true, None)
    +      }
    +
    +      // Create the storage.
    +      def format(fmt: ParserRuleContext): CatalogStorageFormat = {
    +        Option(fmt).map(typedVisit[CatalogStorageFormat]).getOrElse(EmptyStorageFormat)
    +      }
    +      // Default storage.
    +      val defaultStorageType = hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT)
    +      val hiveSerDe = HiveSerDe.sourceToSerDe(defaultStorageType, hiveConf).getOrElse {
    +        HiveSerDe(
    +          inputFormat = Option("org.apache.hadoop.mapred.TextInputFormat"),
    +          outputFormat = Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat"))
    +      }
    +      // Defined storage.
    +      val fileStorage = format(ctx.createFileFormat)
    +      val rowStorage = format(ctx.rowFormat)
    +      val storage = CatalogStorageFormat(
    +        Option(ctx.locationSpec).map(visitLocationSpec),
    +        fileStorage.inputFormat.orElse(hiveSerDe.inputFormat),
    +        fileStorage.outputFormat.orElse(hiveSerDe.outputFormat),
    +        rowStorage.serde.orElse(hiveSerDe.serde).orElse(fileStorage.serde),
    +        rowStorage.serdeProperties ++ fileStorage.serdeProperties
    +      )
    +
    +      val tableDesc = CatalogTable(
    +        identifier = table,
    +        tableType = tableType,
    +        schema = schema,
    +        partitionColumns = partitionCols,
    +        storage = storage,
    +        properties = Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty),
    +        // TODO support the sql text - have a proper location for this!
    +        viewText = Option(ctx.STRING).map(string))
    +      CTAS(tableDesc, plan(ctx.query), ifNotExists)
    +    }
    +  }
    +
    +  /**
    +   * Create or replace a view. This creates a [[CreateViewAsSelect]] command.
    +   */
    +  override def visitCreateView(ctx: CreateViewContext): LogicalPlan = withOrigin(ctx) {
    +    // Pass a partitioned view on to hive.
    +    if (ctx.identifierList != null) {
    +      HiveNativeCommand(command(ctx))
    +    } else {
    +      if (ctx.STRING != null) {
    +        logWarning("COMMENT clause is ignored.")
    +      }
    +      val identifiers = Option(ctx.identifierCommentList).toSeq.flatMap(_.identifierComment.asScala)
    +      val schema = identifiers.map { ic =>
    +        CatalogColumn(ic.identifier.getText, null, nullable = true, Option(ic.STRING).map(string))
    +      }
    +      createView(
    +        ctx,
    +        ctx.tableIdentifier,
    +        schema,
    +        ctx.query,
    +        Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty),
    +        ctx.EXISTS != null,
    +        ctx.REPLACE != null
    +      )
    +    }
    +  }
    +
    +  /**
    +   * Alter the query of a view. This creates a [[CreateViewAsSelect]] command.
    +   */
    +  override def visitAlterViewQuery(ctx: AlterViewQueryContext): LogicalPlan = withOrigin(ctx) {
    +    createView(
    +      ctx,
    +      ctx.tableIdentifier,
    +      Seq.empty,
    +      ctx.query,
    +      Map.empty,
    +      allowExist = false,
    +      replace = true)
    +  }
    +
    +  /**
    +   * Create a [[CreateViewAsSelect]] command.
    +   */
    +  private def createView(
    +      ctx: ParserRuleContext,
    +      name: TableIdentifierContext,
    +      schema: Seq[CatalogColumn],
    +      query: QueryContext,
    +      properties: Map[String, String],
    +      allowExist: Boolean,
    +      replace: Boolean): LogicalPlan = {
    +    val sql = Option(source(query))
    +    val tableDesc = CatalogTable(
    +      identifier = visitTableIdentifier(name),
    +      tableType = CatalogTableType.VIRTUAL_VIEW,
    +      schema = schema,
    +      storage = EmptyStorageFormat,
    +      properties = properties,
    +      viewOriginalText = sql,
    +      viewText = sql)
    +    CreateView(tableDesc, plan(query), allowExist, replace, command(ctx))
    --- End diff --
    
    @gatorsmile I renamed the import to `CreateView`. The actual class is `org.apache.spark.sql.hive.CreateViewAsSelect`. See the following line in `HiveSqlParser`:
    
        import org.apache.spark.sql.hive.{CreateTableAsSelect => CTAS, CreateViewAsSelect => CreateView}


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#issuecomment-202677405
  
    Merged build finished. Test PASSed.


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#discussion_r57945455
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.hive.execution
    +
    +import scala.collection.JavaConverters._
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.apache.hadoop.hive.conf.HiveConf
    +import org.apache.hadoop.hive.conf.HiveConf.ConfVars
    +import org.apache.hadoop.hive.ql.exec.FunctionRegistry
    +import org.apache.hadoop.hive.ql.parse.EximUtil
    +import org.apache.hadoop.hive.ql.session.SessionState
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +
    +import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogStorageFormat, CatalogTable, CatalogTableType}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ng._
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkSqlAstBuilder
    +import org.apache.spark.sql.hive.{CreateTableAsSelect => CTAS, CreateViewAsSelect => CreateView}
    +import org.apache.spark.sql.hive.{HiveGenericUDTF, HiveSerDe}
    +import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper
    +
    +/**
    + * Concrete parser for HiveQl statements.
    + */
    +object HiveSqlParser extends AbstractSqlParser {
    +  val astBuilder = new HiveSqlAstBuilder
    +
    +  override protected def nativeCommand(sqlText: String): LogicalPlan = {
    +    HiveNativeCommand(sqlText)
    +  }
    +}
    +
    +/**
    + * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier.
    + */
    +class HiveSqlAstBuilder extends SparkSqlAstBuilder {
    +  import ParserUtils._
    +
    +  /**
    +   * Get the current Hive Configuration.
    +   */
    +  private[this] def hiveConf: HiveConf = {
    +    var ss = SessionState.get()
    +    // SessionState is lazy initialization, it can be null here
    +    if (ss == null) {
    +      val original = Thread.currentThread().getContextClassLoader
    +      val conf = new HiveConf(classOf[SessionState])
    +      conf.setClassLoader(original)
    +      ss = new SessionState(conf)
    +      SessionState.start(ss)
    +    }
    +    ss.getConf
    +  }
    +
    +  /**
    +   * Pass a command to Hive using a [[HiveNativeCommand]].
    +   */
    +  override def visitExecuteNativeCommand(
    +      ctx: ExecuteNativeCommandContext): LogicalPlan = withOrigin(ctx) {
    +    HiveNativeCommand(command(ctx))
    +  }
    +
    +  /**
    +   * Fail an unsupported Hive native command.
    +   */
    +  override def visitFailNativeCommand(
    +      ctx: FailNativeCommandContext): LogicalPlan = withOrigin(ctx) {
    +    val keywords = if (ctx.kws != null) {
    +      Seq(ctx.kws.kw1, ctx.kws.kw2, ctx.kws.kw3).filter(_ != null).map(_.getText).mkString(" ")
    +    } else {
    +      // SET ROLE is the exception to the rule, because we handle this before other SET commands.
    +      "SET ROLE"
    +    }
    +    throw new ParseException(s"Unsupported operation: $keywords", ctx)
    +  }
    +
    +  /**
    +   * Create an [[AddJar]] or [[AddFile]] command depending on the requested resource.
    +   */
    +  override def visitAddResource(ctx: AddResourceContext): LogicalPlan = withOrigin(ctx) {
    +    ctx.identifier.getText.toLowerCase match {
    +      case "file" => AddFile(remainder(ctx.identifier).trim)
    +      case "jar" => AddJar(remainder(ctx.identifier).trim)
    +      case other => throw new ParseException(s"Unsupported resource type '$other'.", ctx)
    +    }
    +  }
    +
    +  /**
    +   * Create a [[DropTable]] command.
    +   */
    +  override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) {
    +    if (ctx.PURGE != null) {
    +      logWarning("PURGE option is ignored.")
    +    }
    +    if (ctx.REPLICATION != null) {
    +      logWarning("REPLICATION clause is ignored.")
    +    }
    +    DropTable(visitTableIdentifier(ctx.tableIdentifier).toString, ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Create an [[AnalyzeTable]] command. This currently only implements the NOSCAN option (other
    +   * options are passed on to Hive) e.g.:
    +   * {{{
    +   *   ANALYZE TABLE table COMPUTE STATISTICS NOSCAN;
    +   * }}}
    +   */
    +  override def visitAnalyze(ctx: AnalyzeContext): LogicalPlan = withOrigin(ctx) {
    +    if (ctx.partitionSpec == null &&
    +      ctx.identifier != null &&
    +      ctx.identifier.getText.toLowerCase == "noscan") {
    +      AnalyzeTable(visitTableIdentifier(ctx.tableIdentifier).toString)
    +    } else {
    +      HiveNativeCommand(command(ctx))
    +    }
    +  }
    +
    +  /**
    +   * Create a [[CreateTableAsSelect]] command.
    +   */
    +  override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = {
    +    if (ctx.query == null) {
    +      HiveNativeCommand(command(ctx))
    +    } else {
    +      // Get the table header.
    +      val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader)
    +      val tableType = if (external) {
    +        CatalogTableType.EXTERNAL_TABLE
    +      } else {
    +        CatalogTableType.MANAGED_TABLE
    +      }
    +
    +      // Unsupported clauses.
    +      if (temp) {
    +        logWarning("TEMPORARY clause is ignored.")
    +      }
    +      if (ctx.bucketSpec != null) {
    +        // TODO add this - we need cluster columns in the CatalogTable for this to work.
    +        logWarning("CLUSTERED BY ... [ORDERED BY ...] INTO ... BUCKETS clause is ignored.")
    +      }
    +      if (ctx.skewSpec != null) {
    +        logWarning("SKEWED BY ... ON ... [STORED AS DIRECTORIES] clause is ignored.")
    +      }
    +
    +      // Create the schema.
    +      val schema = Option(ctx.colTypeList).toSeq.flatMap(_.colType.asScala).map { col =>
    +        CatalogColumn(
    +          col.identifier.getText,
    +          col.dataType.getText.toLowerCase, // TODO validate this?
    +          nullable = true,
    +          Option(col.STRING).map(string))
    +      }
    +
    +      // Get the column by which the table is partitioned.
    +      val partitionCols = Option(ctx.identifierList).toSeq.flatMap(visitIdentifierList).map {
    +        CatalogColumn(_, null, nullable = true, None)
    +      }
    +
    +      // Create the storage.
    +      def format(fmt: ParserRuleContext): CatalogStorageFormat = {
    +        Option(fmt).map(typedVisit[CatalogStorageFormat]).getOrElse(EmptyStorageFormat)
    +      }
    +      // Default storage.
    +      val defaultStorageType = hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT)
    +      val hiveSerDe = HiveSerDe.sourceToSerDe(defaultStorageType, hiveConf).getOrElse {
    +        HiveSerDe(
    +          inputFormat = Option("org.apache.hadoop.mapred.TextInputFormat"),
    +          outputFormat = Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat"))
    +      }
    +      // Defined storage.
    +      val fileStorage = format(ctx.createFileFormat)
    +      val rowStorage = format(ctx.rowFormat)
    +      val storage = CatalogStorageFormat(
    +        Option(ctx.locationSpec).map(visitLocationSpec),
    +        fileStorage.inputFormat.orElse(hiveSerDe.inputFormat),
    +        fileStorage.outputFormat.orElse(hiveSerDe.outputFormat),
    +        rowStorage.serde.orElse(hiveSerDe.serde).orElse(fileStorage.serde),
    +        rowStorage.serdeProperties ++ fileStorage.serdeProperties
    +      )
    +
    +      val tableDesc = CatalogTable(
    +        identifier = table,
    +        tableType = tableType,
    +        schema = schema,
    +        partitionColumns = partitionCols,
    +        storage = storage,
    +        properties = Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty),
    +        // TODO support the sql text - have a proper location for this!
    +        viewText = Option(ctx.STRING).map(string))
    +      CTAS(tableDesc, plan(ctx.query), ifNotExists)
    +    }
    +  }
    +
    +  /**
    +   * Create or replace a view. This creates a [[CreateViewAsSelect]] command.
    +   */
    +  override def visitCreateView(ctx: CreateViewContext): LogicalPlan = withOrigin(ctx) {
    +    // Pass a partitioned view on to hive.
    +    if (ctx.identifierList != null) {
    +      HiveNativeCommand(command(ctx))
    +    } else {
    +      if (ctx.STRING != null) {
    +        logWarning("COMMENT clause is ignored.")
    +      }
    +      val identifiers = Option(ctx.identifierCommentList).toSeq.flatMap(_.identifierComment.asScala)
    +      val schema = identifiers.map { ic =>
    +        CatalogColumn(ic.identifier.getText, null, nullable = true, Option(ic.STRING).map(string))
    +      }
    +      createView(
    +        ctx,
    +        ctx.tableIdentifier,
    +        schema,
    +        ctx.query,
    +        Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty),
    +        ctx.EXISTS != null,
    +        ctx.REPLACE != null
    +      )
    +    }
    +  }
    +
    +  /**
    +   * Alter the query of a view. This creates a [[CreateViewAsSelect]] command.
    +   */
    +  override def visitAlterViewQuery(ctx: AlterViewQueryContext): LogicalPlan = withOrigin(ctx) {
    +    createView(
    +      ctx,
    +      ctx.tableIdentifier,
    +      Seq.empty,
    +      ctx.query,
    +      Map.empty,
    +      allowExist = false,
    +      replace = true)
    +  }
    +
    +  /**
    +   * Create a [[CreateViewAsSelect]] command.
    +   */
    +  private def createView(
    +      ctx: ParserRuleContext,
    +      name: TableIdentifierContext,
    +      schema: Seq[CatalogColumn],
    +      query: QueryContext,
    +      properties: Map[String, String],
    +      allowExist: Boolean,
    +      replace: Boolean): LogicalPlan = {
    +    val sql = Option(source(query))
    +    val tableDesc = CatalogTable(
    +      identifier = visitTableIdentifier(name),
    +      tableType = CatalogTableType.VIRTUAL_VIEW,
    +      schema = schema,
    +      storage = EmptyStorageFormat,
    +      properties = properties,
    +      viewOriginalText = sql,
    +      viewText = sql)
    +    CreateView(tableDesc, plan(query), allowExist, replace, command(ctx))
    --- End diff --
    
    @hvanhovell I search the whole code base, but still unable to find the implementation of this class `CreateView`? I am just wondering how Parser can recognize it? Thanks! 


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#issuecomment-202629547
  
    **[Test build #54381 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54381/consoleFull)** for PR 12015 at commit [`cb398aa`](https://github.com/apache/spark/commit/cb398aaea6813ac3a2e1412767b77d746e3d8f3e).
     * 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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#discussion_r57954149
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.hive.execution
    +
    +import scala.collection.JavaConverters._
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.apache.hadoop.hive.conf.HiveConf
    +import org.apache.hadoop.hive.conf.HiveConf.ConfVars
    +import org.apache.hadoop.hive.ql.exec.FunctionRegistry
    +import org.apache.hadoop.hive.ql.parse.EximUtil
    +import org.apache.hadoop.hive.ql.session.SessionState
    +import org.apache.hadoop.hive.serde.serdeConstants
    +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
    +
    +import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogStorageFormat, CatalogTable, CatalogTableType}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ng._
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SparkSqlAstBuilder
    +import org.apache.spark.sql.hive.{CreateTableAsSelect => CTAS, CreateViewAsSelect => CreateView}
    +import org.apache.spark.sql.hive.{HiveGenericUDTF, HiveSerDe}
    +import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper
    +
    +/**
    + * Concrete parser for HiveQl statements.
    + */
    +object HiveSqlParser extends AbstractSqlParser {
    +  val astBuilder = new HiveSqlAstBuilder
    +
    +  override protected def nativeCommand(sqlText: String): LogicalPlan = {
    +    HiveNativeCommand(sqlText)
    +  }
    +}
    +
    +/**
    + * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier.
    + */
    +class HiveSqlAstBuilder extends SparkSqlAstBuilder {
    +  import ParserUtils._
    +
    +  /**
    +   * Get the current Hive Configuration.
    +   */
    +  private[this] def hiveConf: HiveConf = {
    +    var ss = SessionState.get()
    +    // SessionState is lazy initialization, it can be null here
    +    if (ss == null) {
    +      val original = Thread.currentThread().getContextClassLoader
    +      val conf = new HiveConf(classOf[SessionState])
    +      conf.setClassLoader(original)
    +      ss = new SessionState(conf)
    +      SessionState.start(ss)
    +    }
    +    ss.getConf
    +  }
    +
    +  /**
    +   * Pass a command to Hive using a [[HiveNativeCommand]].
    +   */
    +  override def visitExecuteNativeCommand(
    +      ctx: ExecuteNativeCommandContext): LogicalPlan = withOrigin(ctx) {
    +    HiveNativeCommand(command(ctx))
    +  }
    +
    +  /**
    +   * Fail an unsupported Hive native command.
    +   */
    +  override def visitFailNativeCommand(
    +      ctx: FailNativeCommandContext): LogicalPlan = withOrigin(ctx) {
    +    val keywords = if (ctx.kws != null) {
    +      Seq(ctx.kws.kw1, ctx.kws.kw2, ctx.kws.kw3).filter(_ != null).map(_.getText).mkString(" ")
    +    } else {
    +      // SET ROLE is the exception to the rule, because we handle this before other SET commands.
    +      "SET ROLE"
    +    }
    +    throw new ParseException(s"Unsupported operation: $keywords", ctx)
    +  }
    +
    +  /**
    +   * Create an [[AddJar]] or [[AddFile]] command depending on the requested resource.
    +   */
    +  override def visitAddResource(ctx: AddResourceContext): LogicalPlan = withOrigin(ctx) {
    +    ctx.identifier.getText.toLowerCase match {
    +      case "file" => AddFile(remainder(ctx.identifier).trim)
    +      case "jar" => AddJar(remainder(ctx.identifier).trim)
    +      case other => throw new ParseException(s"Unsupported resource type '$other'.", ctx)
    +    }
    +  }
    +
    +  /**
    +   * Create a [[DropTable]] command.
    +   */
    +  override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) {
    +    if (ctx.PURGE != null) {
    +      logWarning("PURGE option is ignored.")
    +    }
    +    if (ctx.REPLICATION != null) {
    +      logWarning("REPLICATION clause is ignored.")
    +    }
    +    DropTable(visitTableIdentifier(ctx.tableIdentifier).toString, ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Create an [[AnalyzeTable]] command. This currently only implements the NOSCAN option (other
    +   * options are passed on to Hive) e.g.:
    +   * {{{
    +   *   ANALYZE TABLE table COMPUTE STATISTICS NOSCAN;
    +   * }}}
    +   */
    +  override def visitAnalyze(ctx: AnalyzeContext): LogicalPlan = withOrigin(ctx) {
    +    if (ctx.partitionSpec == null &&
    +      ctx.identifier != null &&
    +      ctx.identifier.getText.toLowerCase == "noscan") {
    +      AnalyzeTable(visitTableIdentifier(ctx.tableIdentifier).toString)
    +    } else {
    +      HiveNativeCommand(command(ctx))
    +    }
    +  }
    +
    +  /**
    +   * Create a [[CreateTableAsSelect]] command.
    +   */
    +  override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = {
    +    if (ctx.query == null) {
    +      HiveNativeCommand(command(ctx))
    +    } else {
    +      // Get the table header.
    +      val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader)
    +      val tableType = if (external) {
    +        CatalogTableType.EXTERNAL_TABLE
    +      } else {
    +        CatalogTableType.MANAGED_TABLE
    +      }
    +
    +      // Unsupported clauses.
    +      if (temp) {
    +        logWarning("TEMPORARY clause is ignored.")
    +      }
    +      if (ctx.bucketSpec != null) {
    +        // TODO add this - we need cluster columns in the CatalogTable for this to work.
    +        logWarning("CLUSTERED BY ... [ORDERED BY ...] INTO ... BUCKETS clause is ignored.")
    +      }
    +      if (ctx.skewSpec != null) {
    +        logWarning("SKEWED BY ... ON ... [STORED AS DIRECTORIES] clause is ignored.")
    +      }
    +
    +      // Create the schema.
    +      val schema = Option(ctx.colTypeList).toSeq.flatMap(_.colType.asScala).map { col =>
    +        CatalogColumn(
    +          col.identifier.getText,
    +          col.dataType.getText.toLowerCase, // TODO validate this?
    +          nullable = true,
    +          Option(col.STRING).map(string))
    +      }
    +
    +      // Get the column by which the table is partitioned.
    +      val partitionCols = Option(ctx.identifierList).toSeq.flatMap(visitIdentifierList).map {
    +        CatalogColumn(_, null, nullable = true, None)
    +      }
    +
    +      // Create the storage.
    +      def format(fmt: ParserRuleContext): CatalogStorageFormat = {
    +        Option(fmt).map(typedVisit[CatalogStorageFormat]).getOrElse(EmptyStorageFormat)
    +      }
    +      // Default storage.
    +      val defaultStorageType = hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT)
    +      val hiveSerDe = HiveSerDe.sourceToSerDe(defaultStorageType, hiveConf).getOrElse {
    +        HiveSerDe(
    +          inputFormat = Option("org.apache.hadoop.mapred.TextInputFormat"),
    +          outputFormat = Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat"))
    +      }
    +      // Defined storage.
    +      val fileStorage = format(ctx.createFileFormat)
    +      val rowStorage = format(ctx.rowFormat)
    +      val storage = CatalogStorageFormat(
    +        Option(ctx.locationSpec).map(visitLocationSpec),
    +        fileStorage.inputFormat.orElse(hiveSerDe.inputFormat),
    +        fileStorage.outputFormat.orElse(hiveSerDe.outputFormat),
    +        rowStorage.serde.orElse(hiveSerDe.serde).orElse(fileStorage.serde),
    +        rowStorage.serdeProperties ++ fileStorage.serdeProperties
    +      )
    +
    +      val tableDesc = CatalogTable(
    +        identifier = table,
    +        tableType = tableType,
    +        schema = schema,
    +        partitionColumns = partitionCols,
    +        storage = storage,
    +        properties = Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty),
    +        // TODO support the sql text - have a proper location for this!
    +        viewText = Option(ctx.STRING).map(string))
    +      CTAS(tableDesc, plan(ctx.query), ifNotExists)
    +    }
    +  }
    +
    +  /**
    +   * Create or replace a view. This creates a [[CreateViewAsSelect]] command.
    +   */
    +  override def visitCreateView(ctx: CreateViewContext): LogicalPlan = withOrigin(ctx) {
    +    // Pass a partitioned view on to hive.
    +    if (ctx.identifierList != null) {
    +      HiveNativeCommand(command(ctx))
    +    } else {
    +      if (ctx.STRING != null) {
    +        logWarning("COMMENT clause is ignored.")
    +      }
    +      val identifiers = Option(ctx.identifierCommentList).toSeq.flatMap(_.identifierComment.asScala)
    +      val schema = identifiers.map { ic =>
    +        CatalogColumn(ic.identifier.getText, null, nullable = true, Option(ic.STRING).map(string))
    +      }
    +      createView(
    +        ctx,
    +        ctx.tableIdentifier,
    +        schema,
    +        ctx.query,
    +        Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty),
    +        ctx.EXISTS != null,
    +        ctx.REPLACE != null
    +      )
    +    }
    +  }
    +
    +  /**
    +   * Alter the query of a view. This creates a [[CreateViewAsSelect]] command.
    +   */
    +  override def visitAlterViewQuery(ctx: AlterViewQueryContext): LogicalPlan = withOrigin(ctx) {
    +    createView(
    +      ctx,
    +      ctx.tableIdentifier,
    +      Seq.empty,
    +      ctx.query,
    +      Map.empty,
    +      allowExist = false,
    +      replace = true)
    +  }
    +
    +  /**
    +   * Create a [[CreateViewAsSelect]] command.
    +   */
    +  private def createView(
    +      ctx: ParserRuleContext,
    +      name: TableIdentifierContext,
    +      schema: Seq[CatalogColumn],
    +      query: QueryContext,
    +      properties: Map[String, String],
    +      allowExist: Boolean,
    +      replace: Boolean): LogicalPlan = {
    +    val sql = Option(source(query))
    +    val tableDesc = CatalogTable(
    +      identifier = visitTableIdentifier(name),
    +      tableType = CatalogTableType.VIRTUAL_VIEW,
    +      schema = schema,
    +      storage = EmptyStorageFormat,
    +      properties = properties,
    +      viewOriginalText = sql,
    +      viewText = sql)
    +    CreateView(tableDesc, plan(query), allowExist, replace, command(ctx))
    --- End diff --
    
    NVM, `CreateViewAsSelect => CreateView`. I found it. Thanks!


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

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


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#issuecomment-202629558
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54381/
    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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#issuecomment-202596509
  
    **[Test build #54367 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54367/consoleFull)** for PR 12015 at commit [`1fc3592`](https://github.com/apache/spark/commit/1fc35921782f29b318a866be04df68f0e2e9cb81).


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#issuecomment-202676854
  
    **[Test build #54392 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54392/consoleFull)** for PR 12015 at commit [`1232590`](https://github.com/apache/spark/commit/1232590ac492fa50e3dcbd5b5a6dbaad8b980c87).
     * This patch passes all tests.
     * 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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#issuecomment-202686717
  
    Thanks - I've merged this in master.



---
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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#issuecomment-202629636
  
    Merged build finished. Test PASSed.


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#issuecomment-202629556
  
    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


[GitHub] spark pull request: [SPARK-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#issuecomment-202626323
  
    **[Test build #54381 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54381/consoleFull)** for PR 12015 at commit [`cb398aa`](https://github.com/apache/spark/commit/cb398aaea6813ac3a2e1412767b77d746e3d8f3e).


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

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


---
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-14213][SQL] Migrate HiveQl parsing to A...

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

    https://github.com/apache/spark/pull/12015#issuecomment-202647638
  
    **[Test build #54392 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54392/consoleFull)** for PR 12015 at commit [`1232590`](https://github.com/apache/spark/commit/1232590ac492fa50e3dcbd5b5a6dbaad8b980c87).


---
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