You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by rxin <gi...@git.apache.org> on 2016/02/11 11:10:02 UTC

[GitHub] spark pull request: [SPARK-13282][SQL] LogicalPlan toSql should ju...

GitHub user rxin opened a pull request:

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

    [SPARK-13282][SQL] LogicalPlan toSql should just return a String

    Rather than Option[String].
    
    Previously we were using Option[String] and None to indicate the case when Spark fails to generate SQL. It is easier to just use exceptions to propagate error cases, rather than having for comprehension everywhere. I also introduced a "build" function that simplifies string concatenation (i.e. no need to reason about whether we have an extra space or not).


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

    $ git pull https://github.com/rxin/spark SPARK-13282

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

    https://github.com/apache/spark/pull/11171.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 #11171
    
----
commit 9fd34fc1fa27c09cfa5426a53be85cbc5e0460c3
Author: Reynold Xin <rx...@databricks.com>
Date:   2016-02-11T10:08:15Z

    [SPARK-13282][SQL] LogicalPlan toSql should just return a String rather than Option[String]

----


---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-182792045
  
    **[Test build #51093 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51093/consoleFull)** for PR 11171 at commit [`9fd34fc`](https://github.com/apache/spark/commit/9fd34fc1fa27c09cfa5426a53be85cbc5e0460c3).


---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#discussion_r52714511
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala ---
    @@ -37,157 +39,137 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation
     class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Logging {
       def this(df: DataFrame) = this(df.queryExecution.analyzed, df.sqlContext)
     
    -  def toSQL: Option[String] = {
    +  def toSQL: String = {
         val canonicalizedPlan = Canonicalizer.execute(logicalPlan)
    -    val maybeSQL = try {
    -      toSQL(canonicalizedPlan)
    -    } catch { case cause: UnsupportedOperationException =>
    -      logInfo(s"Failed to build SQL query string because: ${cause.getMessage}")
    -      None
    -    }
    -
    -    if (maybeSQL.isDefined) {
    +    try {
    +      val generatedSQL = toSQL(canonicalizedPlan)
           logDebug(
             s"""Built SQL query string successfully from given logical plan:
    -           |
    -           |# Original logical plan:
    -           |${logicalPlan.treeString}
    -           |# Canonicalized logical plan:
    -           |${canonicalizedPlan.treeString}
    -           |# Built SQL query string:
    -           |${maybeSQL.get}
    +            |
    +            |# Original logical plan:
    +            |${logicalPlan.treeString}
    +            |# Canonicalized logical plan:
    +            |${canonicalizedPlan.treeString}
    +            |# Generated SQL:
    +            |$generatedSQL
    --- End diff --
    
    Nit: Please revert the indentation change.


---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#discussion_r52714530
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala ---
    @@ -37,157 +39,137 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation
     class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Logging {
       def this(df: DataFrame) = this(df.queryExecution.analyzed, df.sqlContext)
     
    -  def toSQL: Option[String] = {
    +  def toSQL: String = {
         val canonicalizedPlan = Canonicalizer.execute(logicalPlan)
    -    val maybeSQL = try {
    -      toSQL(canonicalizedPlan)
    -    } catch { case cause: UnsupportedOperationException =>
    -      logInfo(s"Failed to build SQL query string because: ${cause.getMessage}")
    -      None
    -    }
    -
    -    if (maybeSQL.isDefined) {
    +    try {
    +      val generatedSQL = toSQL(canonicalizedPlan)
           logDebug(
             s"""Built SQL query string successfully from given logical plan:
    -           |
    -           |# Original logical plan:
    -           |${logicalPlan.treeString}
    -           |# Canonicalized logical plan:
    -           |${canonicalizedPlan.treeString}
    -           |# Built SQL query string:
    -           |${maybeSQL.get}
    +            |
    +            |# Original logical plan:
    +            |${logicalPlan.treeString}
    +            |# Canonicalized logical plan:
    +            |${canonicalizedPlan.treeString}
    +            |# Generated SQL:
    +            |$generatedSQL
              """.stripMargin)
    -    } else {
    +      generatedSQL
    +    } catch { case NonFatal(e) =>
           logDebug(
             s"""Failed to build SQL query string from given logical plan:
    -           |
    +            |
                |# Original logical plan:
    -           |${logicalPlan.treeString}
    -           |# Canonicalized logical plan:
    -           |${canonicalizedPlan.treeString}
    +            |${logicalPlan.treeString}
    +            |# Canonicalized logical plan:
    +            |${canonicalizedPlan.treeString}
    --- End diff --
    
    Nit: Indentation is off.


---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-183257659
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51178/
    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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#discussion_r52714855
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala ---
    @@ -50,10 +50,8 @@ abstract class SQLBuilderTest extends QueryTest with TestHiveSingleton {
              """.stripMargin)
         }
     
    -    val actualSQL = maybeSQL.get
    -
         try {
    -      assert(actualSQL === expectedSQL)
    +      assert(generatedSql === expectedSQL)
    --- End diff --
    
    but i'm going to change this one to be consistent with expectedSQL


---
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-13282][SQL] LogicalPlan toSql should ju...

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

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


---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#discussion_r52714504
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala ---
    @@ -37,157 +39,137 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation
     class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Logging {
       def this(df: DataFrame) = this(df.queryExecution.analyzed, df.sqlContext)
     
    -  def toSQL: Option[String] = {
    +  def toSQL: String = {
         val canonicalizedPlan = Canonicalizer.execute(logicalPlan)
    -    val maybeSQL = try {
    -      toSQL(canonicalizedPlan)
    -    } catch { case cause: UnsupportedOperationException =>
    -      logInfo(s"Failed to build SQL query string because: ${cause.getMessage}")
    -      None
    -    }
    -
    -    if (maybeSQL.isDefined) {
    +    try {
    +      val generatedSQL = toSQL(canonicalizedPlan)
           logDebug(
             s"""Built SQL query string successfully from given logical plan:
    -           |
    -           |# Original logical plan:
    -           |${logicalPlan.treeString}
    -           |# Canonicalized logical plan:
    -           |${canonicalizedPlan.treeString}
    -           |# Built SQL query string:
    -           |${maybeSQL.get}
    +            |
    +            |# Original logical plan:
    +            |${logicalPlan.treeString}
    +            |# Canonicalized logical plan:
    +            |${canonicalizedPlan.treeString}
    +            |# Generated SQL:
    +            |$generatedSQL
              """.stripMargin)
    -    } else {
    +      generatedSQL
    +    } catch { case NonFatal(e) =>
           logDebug(
             s"""Failed to build SQL query string from given logical plan:
    -           |
    +            |
                |# Original logical plan:
    -           |${logicalPlan.treeString}
    -           |# Canonicalized logical plan:
    -           |${canonicalizedPlan.treeString}
    +            |${logicalPlan.treeString}
    +            |# Canonicalized logical plan:
    +            |${canonicalizedPlan.treeString}
              """.stripMargin)
    +      throw e
         }
    -
    -    maybeSQL
       }
     
    -  private def projectToSQL(
    -      projectList: Seq[NamedExpression],
    -      child: LogicalPlan,
    -      isDistinct: Boolean): Option[String] = {
    -    for {
    -      childSQL <- toSQL(child)
    -      listSQL = projectList.map(_.sql).mkString(", ")
    -      maybeFrom = child match {
    -        case OneRowRelation => " "
    -        case _ => " FROM "
    -      }
    -      distinct = if (isDistinct) " DISTINCT " else " "
    -    } yield s"SELECT$distinct$listSQL$maybeFrom$childSQL"
    -  }
    +  private def toSQL(node: LogicalPlan): String = node match {
    +    case Distinct(p: Project) =>
    +      projectToSQL(p, isDistinct = true)
     
    -  private def aggregateToSQL(
    -      groupingExprs: Seq[Expression],
    -      aggExprs: Seq[Expression],
    -      child: LogicalPlan): Option[String] = {
    -    val aggSQL = aggExprs.map(_.sql).mkString(", ")
    -    val groupingSQL = groupingExprs.map(_.sql).mkString(", ")
    -    val maybeGroupBy = if (groupingSQL.isEmpty) "" else " GROUP BY "
    -    val maybeFrom = child match {
    -      case OneRowRelation => " "
    -      case _ => " FROM "
    -    }
    +    case p: Project =>
    +      projectToSQL(p, isDistinct = false)
     
    -    toSQL(child).map { childSQL =>
    -      s"SELECT $aggSQL$maybeFrom$childSQL$maybeGroupBy$groupingSQL"
    -    }
    -  }
    +    case p: Aggregate =>
    +      aggregateToSQL(p)
    +
    +    case p: Limit =>
    +      s"${toSQL(p.child)} LIMIT ${p.limitExpr.sql}"
     
    -  private def toSQL(node: LogicalPlan): Option[String] = node match {
    -    case Distinct(Project(list, child)) =>
    -      projectToSQL(list, child, isDistinct = true)
    -
    -    case Project(list, child) =>
    -      projectToSQL(list, child, isDistinct = false)
    -
    -    case Aggregate(groupingExprs, aggExprs, child) =>
    -      aggregateToSQL(groupingExprs, aggExprs, child)
    -
    -    case Limit(limit, child) =>
    -      for {
    -        childSQL <- toSQL(child)
    -        limitSQL = limit.sql
    -      } yield s"$childSQL LIMIT $limitSQL"
    -
    -    case Filter(condition, child) =>
    -      for {
    -        childSQL <- toSQL(child)
    -        whereOrHaving = child match {
    -          case _: Aggregate => "HAVING"
    -          case _ => "WHERE"
    -        }
    -        conditionSQL = condition.sql
    -      } yield s"$childSQL $whereOrHaving $conditionSQL"
    -
    -    case Union(children) if children.length > 1 =>
    -      val childrenSql = children.map(toSQL(_))
    -      if (childrenSql.exists(_.isEmpty)) {
    -        None
    -      } else {
    -        Some(childrenSql.map(_.get).mkString(" UNION ALL "))
    +    case p: Filter =>
    +      val whereOrHaving = p.child match {
    +        case _: Aggregate => "HAVING"
    +        case _ => "WHERE"
    +      }
    +      build(toSQL(p.child), whereOrHaving, p.condition.sql)
    +
    +    case p: Union if p.children.length > 1 =>
    +      val childrenSql = p.children.map(toSQL(_))
    +      childrenSql.mkString(" UNION ALL ")
    +
    +    case p: Subquery =>
    +      p.child match {
    +        // Persisted data source relation
    +        case LogicalRelation(_, _, Some(TableIdentifier(table, Some(database)))) =>
    +          s"`$database`.`$table`"
    +        // Parentheses is not used for persisted data source relations
    +        // e.g., select x.c1 from (t1) as x inner join (t1) as y on x.c1 = y.c1
    +        case Subquery(_, _: LogicalRelation | _: MetastoreRelation) =>
    +          build(toSQL(p.child), "AS", p.alias)
    +        case _ =>
    +          build("(" + toSQL(p.child) + ")", "AS", p.alias)
           }
     
    -    // Persisted data source relation
    -    case Subquery(alias, LogicalRelation(_, _, Some(TableIdentifier(table, Some(database))))) =>
    -      Some(s"`$database`.`$table`")
    -
    -    case Subquery(alias, child) =>
    -      toSQL(child).map( childSQL =>
    -        child match {
    -          // Parentheses is not used for persisted data source relations
    -          // e.g., select x.c1 from (t1) as x inner join (t1) as y on x.c1 = y.c1
    -          case Subquery(_, _: LogicalRelation | _: MetastoreRelation) =>
    -            s"$childSQL AS $alias"
    -          case _ =>
    -            s"($childSQL) AS $alias"
    -        })
    -
    -    case Join(left, right, joinType, condition) =>
    -      for {
    -        leftSQL <- toSQL(left)
    -        rightSQL <- toSQL(right)
    -        joinTypeSQL = joinType.sql
    -        conditionSQL = condition.map(" ON " + _.sql).getOrElse("")
    -      } yield s"$leftSQL $joinTypeSQL JOIN $rightSQL$conditionSQL"
    -
    -    case MetastoreRelation(database, table, alias) =>
    -      val aliasSQL = alias.map(a => s" AS `$a`").getOrElse("")
    -      Some(s"`$database`.`$table`$aliasSQL")
    +    case p: Join =>
    +      build(
    +        toSQL(p.left),
    +        p.joinType.sql,
    +        "JOIN",
    +        toSQL(p.right),
    +        p.condition.map(" ON " + _.sql).getOrElse(""))
    +
    +    case p: MetastoreRelation =>
    +      build(
    +        s"`${p.databaseName}`.`${p.tableName}`",
    +        p.alias.map(a => s" AS `$a`").getOrElse("")
    +      )
     
         case Sort(orders, _, RepartitionByExpression(partitionExprs, child, _))
             if orders.map(_.child) == partitionExprs =>
    -      for {
    -        childSQL <- toSQL(child)
    -        partitionExprsSQL = partitionExprs.map(_.sql).mkString(", ")
    -      } yield s"$childSQL CLUSTER BY $partitionExprsSQL"
    -
    -    case Sort(orders, global, child) =>
    -      for {
    -        childSQL <- toSQL(child)
    -        ordersSQL = orders.map { case SortOrder(e, dir) => s"${e.sql} ${dir.sql}" }.mkString(", ")
    -        orderOrSort = if (global) "ORDER" else "SORT"
    -      } yield s"$childSQL $orderOrSort BY $ordersSQL"
    -
    -    case RepartitionByExpression(partitionExprs, child, _) =>
    -      for {
    -        childSQL <- toSQL(child)
    -        partitionExprsSQL = partitionExprs.map(_.sql).mkString(", ")
    -      } yield s"$childSQL DISTRIBUTE BY $partitionExprsSQL"
    +      build(toSQL(child), "CLUSTER BY", partitionExprs.map(_.sql).mkString(", "))
    +
    +    case p: Sort =>
    +      build(
    +        toSQL(p.child),
    +        if (p.global) "ORDER BY" else "SORT BY",
    +        p.order.map { case SortOrder(e, dir) => s"${e.sql} ${dir.sql}" }.mkString(", ")
    +      )
    +
    +    case p: RepartitionByExpression =>
    +      build(
    +        toSQL(p.child),
    +        "DISTRIBUTE BY",
    +        p.partitionExpressions.map(_.sql).mkString(", ")
    +      )
     
         case OneRowRelation =>
    -      Some("")
    +      ""
     
    -    case _ => None
    +    case _ =>
    +      throw new UnsupportedOperationException("")
    --- End diff --
    
    Probably not using empty error message here?


---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#discussion_r52714801
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala ---
    @@ -50,10 +50,8 @@ abstract class SQLBuilderTest extends QueryTest with TestHiveSingleton {
              """.stripMargin)
         }
     
    -    val actualSQL = maybeSQL.get
    -
         try {
    -      assert(actualSQL === expectedSQL)
    +      assert(generatedSql === expectedSQL)
    --- End diff --
    
    `generatedSQL`? Do we have any naming rules related to acronyms (`Sql` or `SQL`)?


---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-183226591
  
    Updated.
    
    cc @gatorsmile unfortunately you will have to rebase your pull requests, although it should be easy to do.



---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-183434335
  
    Thanks - I've merged this.



---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-183257656
  
    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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-182821712
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51093/
    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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-183257408
  
    **[Test build #51178 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51178/consoleFull)** for PR 11171 at commit [`cfccf21`](https://github.com/apache/spark/commit/cfccf213a1bb89f68553f883c205e6b626f5be67).
     * 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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#discussion_r52714696
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala ---
    @@ -50,10 +50,8 @@ abstract class SQLBuilderTest extends QueryTest with TestHiveSingleton {
              """.stripMargin)
         }
     
    -    val actualSQL = maybeSQL.get
    -
         try {
    -      assert(actualSQL === expectedSQL)
    +      assert(generatedSql === expectedSQL)
    --- End diff --
    
    `generatedSQL`? Do we have any naming rules related to acronyms (`Sql` or `SQL`)?


---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-183423770
  
    It is easy to rebase. After the merge, I will submit PRs for the other related JIRAs. 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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-182821707
  
    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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-182790954
  
    cc @liancheng 


---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-182820588
  
    **[Test build #51093 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51093/consoleFull)** for PR 11171 at commit [`9fd34fc`](https://github.com/apache/spark/commit/9fd34fc1fa27c09cfa5426a53be85cbc5e0460c3).
     * 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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#issuecomment-183227971
  
    **[Test build #51178 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51178/consoleFull)** for PR 11171 at commit [`cfccf21`](https://github.com/apache/spark/commit/cfccf213a1bb89f68553f883c205e6b626f5be67).


---
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-13282][SQL] LogicalPlan toSql should ju...

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

    https://github.com/apache/spark/pull/11171#discussion_r52714791
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala ---
    @@ -50,10 +50,8 @@ abstract class SQLBuilderTest extends QueryTest with TestHiveSingleton {
              """.stripMargin)
         }
     
    -    val actualSQL = maybeSQL.get
    -
         try {
    -      assert(actualSQL === expectedSQL)
    +      assert(generatedSql === expectedSQL)
    --- End diff --
    
    i think both are acceptable according to java conventions. if the acronym is longer than certain characters, than the preference is to camel case.
    



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