[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546804
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
--- End diff --

yes


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546686
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
+exprId: ExprId)
+  extends SubqueryExpression with CodegenFallback {
--- End diff --

yes, I think it's fine, one boxing is not the end of world.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546631
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
+  s"${e.query.treeString}")
+  }
+  // Analyzer will make sure that it only return on column
+  if (rows.length > 0) {
--- End diff --

Yea again, it'd be better to make it more explicit, e.g.

```
if (rows.length == 0) {
  e.updateResult(null, e.dataType)
} else {
  assert(rows.length == 1)
  e.updateResult(rows(0).get(0, e.dataType))
}


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546610
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -88,7 +88,19 @@ abstract class Optimizer extends 
RuleExecutor[LogicalPlan] {
 Batch("Decimal Optimizations", FixedPoint(100),
   DecimalAggregates) ::
 Batch("LocalRelation", FixedPoint(100),
-  ConvertToLocalRelation) :: Nil
+  ConvertToLocalRelation) ::
+Batch("Subquery", Once,
+  Subquery) :: Nil
+  }
+
+  /**
+   * Optimize all the subqueries inside expression.
+   */
+  object Subquery extends Rule[LogicalPlan] {
+def apply(plan: LogicalPlan): LogicalPlan = plan 
transformAllExpressions {
+  case subquery: SubqueryExpression =>
+subquery.withNewPlan(execute(subquery.query))
--- End diff --

Maybe replace it with "Optimizer.this.execute" so it is more clear. 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546562
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
+  s"${e.query.treeString}")
+  }
+  // Analyzer will make sure that it only return on column
+  if (rows.length > 0) {
--- End diff --

Repeated: `rows.length` could be 0, then the value will be null, will add a 
comment for that.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-10749][MESOS] Support multiple roles wi...

2016-02-19 Thread tnachen
Github user tnachen commented on the pull request:

https://github.com/apache/spark/pull/8872#issuecomment-186534096
  
@andrewor14 Sorry for the mess up, I kept thinking the code was ready just 
needed to rebase and address comments. The rebase and comments did cause some 
style problems in the end, will be more careful next time. I took a pass again 
and I don't see anything any more. PTAL when you can.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546478
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
--- End diff --

not 100% sure. maybe it's better to just say more than one, so we don't 
need to run the whole plan (e..g i'm thinking maybe we should inject a limit to 
subquery)


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-10749][MESOS] Support multiple roles wi...

2016-02-19 Thread tnachen
Github user tnachen commented on a diff in the pull request:

https://github.com/apache/spark/pull/8872#discussion_r53546468
  
--- Diff: 
core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
 ---
@@ -442,9 +443,12 @@ private[spark] class MesosClusterScheduler(
 options
   }
 
-  private class ResourceOffer(val offer: Offer, var cpu: Double, var mem: 
Double) {
+  private class ResourceOffer(
+  val offerId: OfferID,
--- End diff --

Sounds good, will remember 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546444
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
--- End diff --

The current error message has more information than postgres', should we 
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546416
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
--- End diff --

I have never thought we should match the exactly error message with 
PostgreSQL, that's great.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546398
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
+
+  /**
+   * The underline plan for the query, could be logical plan or physical 
plan.
+   *
+   * This is used to generate tree string.
+   */
+  def plan: QueryPlan[_]
+
+  /**
+   * Updates the query with new logical plan.
+   */
+  def withNewPlan(plan: LogicalPlan): SubqueryExpression
--- End diff --

We will have ExistsSubquery, InSubquery shortly (or next release). 


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546368
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala 
---
@@ -226,4 +227,29 @@ abstract class QueryPlan[PlanType <: 
TreeNode[PlanType]] extends TreeNode[PlanTy
   protected def statePrefix = if (missingInput.nonEmpty && 
children.nonEmpty) "!" else ""
 
   override def simpleString: String = statePrefix + super.simpleString
+
+  override def generateTreeString(
+  depth: Int, lastChildren: Seq[Boolean], builder: StringBuilder): 
StringBuilder = {
+if (depth > 0) {
+  lastChildren.init.foreach { isLast =>
+val prefixFragment = if (isLast) "   " else ":  "
+builder.append(prefixFragment)
+  }
+
+  val branch = if (lastChildren.last) "+- " else ":- "
+  builder.append(branch)
+}
+
+builder.append(simpleString)
+builder.append("\n")
+
+val allSubqueries = expressions.flatMap(_.collect {case e: 
SubqueryExpression => e})
--- End diff --

If I could figure out a way, would not copy this function here and ask you.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546355
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
+
+  /**
+   * The underline plan for the query, could be logical plan or physical 
plan.
+   *
+   * This is used to generate tree string.
+   */
+  def plan: QueryPlan[_]
+
+  /**
+   * Updates the query with new logical plan.
+   */
+  def withNewPlan(plan: LogicalPlan): SubqueryExpression
--- End diff --

I meant ScalarSubquery. That's already the one isn't it?



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546358
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -88,7 +88,19 @@ abstract class Optimizer extends 
RuleExecutor[LogicalPlan] {
 Batch("Decimal Optimizations", FixedPoint(100),
   DecimalAggregates) ::
 Batch("LocalRelation", FixedPoint(100),
-  ConvertToLocalRelation) :: Nil
+  ConvertToLocalRelation) ::
+Batch("Subquery", Once,
+  Subquery) :: Nil
+  }
+
+  /**
+   * Optimize all the subqueries inside expression.
+   */
+  object Subquery extends Rule[LogicalPlan] {
+def apply(plan: LogicalPlan): LogicalPlan = plan 
transformAllExpressions {
+  case subquery: SubqueryExpression =>
+subquery.withNewPlan(execute(subquery.query))
--- End diff --

The parent class, Optimizer.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546295
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
+
+  /**
+   * The underline plan for the query, could be logical plan or physical 
plan.
+   *
+   * This is used to generate tree string.
+   */
+  def plan: QueryPlan[_]
+
+  /**
+   * Updates the query with new logical plan.
+   */
+  def withNewPlan(plan: LogicalPlan): SubqueryExpression
--- End diff --

Then should we have LogicalSubqueryExpression ?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546298
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
--- End diff --

Analyzer and Optimizer only applies to logical plan right?



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546277
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
--- End diff --

This is the base class for both logical plan and physical plan, kind of 
weird. This is to make the generateTreeString works in QueryPlan


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546257
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
--- End diff --

This is an helper function used in Analyzer and Optimizer, or we need to do 
type conversion.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12154] Upgrade to Jersey 2

2016-02-19 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/11223#issuecomment-186529563
  
@JoshRosen the dependency change is pretty scary (adding a lot of 
libraries). Is this change necessary?



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13408][Core]Ignore errors when it's alr...

2016-02-19 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13408][Core]Ignore errors when it's alr...

2016-02-19 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/11280#issuecomment-186529434
  
LGTM, merging this into master, 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53546149
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
--- End diff --

e.executedPlan  rather than e.plan, if you want to keep executedPlan


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-186528815
  
I would also just remove support for 192/256, so we don't have to explain 
the JCE stuff.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/10527#discussion_r53546094
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
 ---
@@ -442,3 +444,90 @@ case class PrintToStderr(child: Expression) extends 
UnaryExpression {
""".stripMargin)
   }
 }
+
+/**
+ * A function that encrypts input using AES. Key lengths of 128, 192 or 
256 bits can be used. 192
+ * and 256 bits keys can be used if Java Cryptography Extension (JCE) 
Unlimited Strength Jurisdic-
+ * tion Policy Files are installed. If either argument is NULL, the result 
will also be null. If
+ * input is invalid, key length is not one of the permitted values or 
using 192/256 bits key before
+ * installing JCE, an exception will be thrown.
+ */
+@ExpressionDescription(
+  usage = "_FUNC_(input, key) - Encrypts input using AES.",
+  extended = "> SELECT Base64(_FUNC_('ABC', '1234567890123456'));\n 
'y6Ss+zCYObpCbgfWfyNWTw=='")
+case class AesEncrypt(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def dataType: DataType = BinaryType
+  override def inputTypes: Seq[DataType] = Seq(BinaryType, BinaryType)
+
+  protected override def nullSafeEval(input1: Any, input2: Any): Any = {
+val cipher = Cipher.getInstance("AES")
+val secretKey: SecretKeySpec = new 
SecretKeySpec(input2.asInstanceOf[Array[Byte]], 0,
+  input2.asInstanceOf[Array[Byte]].length, "AES")
+cipher.init(Cipher.ENCRYPT_MODE, secretKey)
+cipher.doFinal(input1.asInstanceOf[Array[Byte]], 0, 
input1.asInstanceOf[Array[Byte]].length)
+  }
+
+  override def genCode(ctx: CodegenContext, ev: ExprCode): String = {
+nullSafeCodeGen(ctx, ev, (str, key) => {
+  val Cipher = "javax.crypto.Cipher"
+  val SecretKeySpec = "javax.crypto.spec.SecretKeySpec"
+  s"""
+  try {
+$Cipher cipher = $Cipher.getInstance("AES");
+$SecretKeySpec secret = new $SecretKeySpec($key, 0, 
$key.length, "AES");
+cipher.init($Cipher.ENCRYPT_MODE, secret);
+${ev.value} = cipher.doFinal($str, 0, $str.length);
+  } catch (java.security.GeneralSecurityException e) {
+org.apache.spark.unsafe.Platform.throwException(e);
+  }
+  """
+})
+  }
+}
+
+/**
+ * A function that decrypts input using AES. Key lengths of 128, 192 or 
256 bits can be used. 192
+ * and 256 bits keys can be used if Java Cryptography Extension (JCE) 
Unlimited Strength Jurisdic-
+ * tion Policy Files are installed. If either argument is NULL, the result 
will also be null. If
+ * input is invalid, key length is not one of the permitted values or 
using 192/256 bits key before
+ * installing JCE, an exception will be thrown.
+ */
+@ExpressionDescription(
+  usage = "_FUNC_(input, key) - Decrypts input using AES.",
+  extended = "> SELECT 
_FUNC_(UnBase64('y6Ss+zCYObpCbgfWfyNWTw=='),'1234567890123456');\n 'ABC'")
+case class AesDecrypt(left: Expression, right: Expression)
--- End diff --

also if the key is literal, i'd just do some input data type checking in 
analysis (override checkInputTypes) to make sure the key is in acceptable range.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13233][SQL][WIP] Python Dataset

2016-02-19 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7#issuecomment-186527435
  
**[Test build #51588 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51588/consoleFull)**
 for PR 7 at commit 
[`97dcac2`](https://github.com/apache/spark/commit/97dcac2d4ba4994fc6c9a5167be0c69e724c56bb).


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/10527#discussion_r53546060
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
 ---
@@ -442,3 +444,90 @@ case class PrintToStderr(child: Expression) extends 
UnaryExpression {
""".stripMargin)
   }
 }
+
+/**
+ * A function that encrypts input using AES. Key lengths of 128, 192 or 
256 bits can be used. 192
+ * and 256 bits keys can be used if Java Cryptography Extension (JCE) 
Unlimited Strength Jurisdic-
+ * tion Policy Files are installed. If either argument is NULL, the result 
will also be null. If
+ * input is invalid, key length is not one of the permitted values or 
using 192/256 bits key before
+ * installing JCE, an exception will be thrown.
+ */
+@ExpressionDescription(
+  usage = "_FUNC_(input, key) - Encrypts input using AES.",
+  extended = "> SELECT Base64(_FUNC_('ABC', '1234567890123456'));\n 
'y6Ss+zCYObpCbgfWfyNWTw=='")
+case class AesEncrypt(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def dataType: DataType = BinaryType
+  override def inputTypes: Seq[DataType] = Seq(BinaryType, BinaryType)
+
+  protected override def nullSafeEval(input1: Any, input2: Any): Any = {
+val cipher = Cipher.getInstance("AES")
+val secretKey: SecretKeySpec = new 
SecretKeySpec(input2.asInstanceOf[Array[Byte]], 0,
+  input2.asInstanceOf[Array[Byte]].length, "AES")
+cipher.init(Cipher.ENCRYPT_MODE, secretKey)
+cipher.doFinal(input1.asInstanceOf[Array[Byte]], 0, 
input1.asInstanceOf[Array[Byte]].length)
+  }
+
+  override def genCode(ctx: CodegenContext, ev: ExprCode): String = {
+nullSafeCodeGen(ctx, ev, (str, key) => {
+  val Cipher = "javax.crypto.Cipher"
+  val SecretKeySpec = "javax.crypto.spec.SecretKeySpec"
+  s"""
+  try {
+$Cipher cipher = $Cipher.getInstance("AES");
+$SecretKeySpec secret = new $SecretKeySpec($key, 0, 
$key.length, "AES");
+cipher.init($Cipher.ENCRYPT_MODE, secret);
+${ev.value} = cipher.doFinal($str, 0, $str.length);
+  } catch (java.security.GeneralSecurityException e) {
+org.apache.spark.unsafe.Platform.throwException(e);
+  }
+  """
+})
+  }
+}
+
+/**
+ * A function that decrypts input using AES. Key lengths of 128, 192 or 
256 bits can be used. 192
+ * and 256 bits keys can be used if Java Cryptography Extension (JCE) 
Unlimited Strength Jurisdic-
+ * tion Policy Files are installed. If either argument is NULL, the result 
will also be null. If
+ * input is invalid, key length is not one of the permitted values or 
using 192/256 bits key before
+ * installing JCE, an exception will be thrown.
+ */
+@ExpressionDescription(
+  usage = "_FUNC_(input, key) - Decrypts input using AES.",
+  extended = "> SELECT 
_FUNC_(UnBase64('y6Ss+zCYObpCbgfWfyNWTw=='),'1234567890123456');\n 'ABC'")
+case class AesDecrypt(left: Expression, right: Expression)
--- End diff --

i find it confusing to have left/right here (e.g. input, key)

Let's give them a proper name, and then just `override def left: Expression 
= input`.




---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/10527#discussion_r53546023
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1931,6 +1931,42 @@ object functions extends LegacyFunctions {
 new Murmur3Hash(cols.map(_.expr))
   }
 
+  /**
+   * Encrypts input using AES and Returns the result as a binary column.
+   * Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits 
keys can be used if Java
+   * Cryptography Extension (JCE) Unlimited Strength Jurisdiction Policy 
Files are installed. If
+   * either argument is NULL, the result will also be null. If input is 
invalid, key length is not
+   * one of the permitted values or using 192/256 bits key before 
installing JCE, an exception will
+   * be thrown.
+   *
+   * @param input binary column to encrypt input
+   * @param key binary column of 128, 192 or 256 bits key
+   *
+   * @group misc_funcs
+   * @since 2.0.0
+   */
+  def aes_encrypt(input: Column, key: Column): Column = withExpr {
--- End diff --

What are the other examples?



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread vectorijk
Github user vectorijk commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-186523636
  
Ok, Sure. I will 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-186523782
  
I did not look into this closely, since @cloud-fan already reviewed this 
many rounds. sorry for the rush.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-186523445
  
@vectorijk I've reverted the patch. Can you reopen the pull request and fix 
the return types?



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/10527#discussion_r53546006
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1931,6 +1931,42 @@ object functions extends LegacyFunctions {
 new Murmur3Hash(cols.map(_.expr))
   }
 
+  /**
+   * Encrypts input using AES and Returns the result as a binary column.
+   * Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits 
keys can be used if Java
+   * Cryptography Extension (JCE) Unlimited Strength Jurisdiction Policy 
Files are installed. If
+   * either argument is NULL, the result will also be null. If input is 
invalid, key length is not
+   * one of the permitted values or using 192/256 bits key before 
installing JCE, an exception will
+   * be thrown.
+   *
+   * @param input binary column to encrypt input
+   * @param key binary column of 128, 192 or 256 bits key
+   *
+   * @group misc_funcs
+   * @since 2.0.0
+   */
+  def aes_encrypt(input: Column, key: Column): Column = withExpr {
--- End diff --

Some of the functions take literals, some not., In 2.0, should we clean up 
all the mess?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-186523374
  
Sorry I'm not convinced this is correct (decrypt returning string type), 
and there are other issues with it. I'm just going to revert the patch, since 
it is unlikely other things will conflict with 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-186522894
  
@davies does this implementation even make sense? decrypt always return a 
string, while encrypt always take in a binary? The two are not symmetric.




---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread vectorijk
Github user vectorijk commented on a diff in the pull request:

https://github.com/apache/spark/pull/10527#discussion_r53545904
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1931,6 +1931,42 @@ object functions extends LegacyFunctions {
 new Murmur3Hash(cols.map(_.expr))
   }
 
+  /**
+   * Encrypts input using AES and Returns the result as a binary column.
+   * Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits 
keys can be used if Java
+   * Cryptography Extension (JCE) Unlimited Strength Jurisdiction Policy 
Files are installed. If
+   * either argument is NULL, the result will also be null. If input is 
invalid, key length is not
+   * one of the permitted values or using 192/256 bits key before 
installing JCE, an exception will
+   * be thrown.
+   *
+   * @param input binary column to encrypt input
+   * @param key binary column of 128, 192 or 256 bits key
+   *
+   * @group misc_funcs
+   * @since 2.0.0
+   */
+  def aes_encrypt(input: Column, key: Column): Column = withExpr {
--- End diff --

I also think so. The examples is just one of those cases. Key also could be 
`abcdef1234567890`.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/10527#discussion_r53545906
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1931,6 +1931,42 @@ object functions extends LegacyFunctions {
 new Murmur3Hash(cols.map(_.expr))
   }
 
+  /**
+   * Encrypts input using AES and Returns the result as a binary column.
+   * Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits 
keys can be used if Java
+   * Cryptography Extension (JCE) Unlimited Strength Jurisdiction Policy 
Files are installed. If
+   * either argument is NULL, the result will also be null. If input is 
invalid, key length is not
+   * one of the permitted values or using 192/256 bits key before 
installing JCE, an exception will
+   * be thrown.
+   *
+   * @param input binary column to encrypt input
+   * @param key binary column of 128, 192 or 256 bits key
+   *
+   * @group misc_funcs
+   * @since 2.0.0
+   */
+  def aes_encrypt(input: Column, key: Column): Column = withExpr {
--- End diff --

then we should probably just take a string.

the thing is we want to facilitate the most common cases, e.g. if 99% of 
the time people are passing in keys directly rather than relying on some other 
columns' values, we should just let them pass the literals.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13329] [SQL] considering output for sta...

2016-02-19 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/11210#issuecomment-186520570
  
@andrewor14 I saw that, will investigate it.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/10527#discussion_r53545875
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1931,6 +1931,42 @@ object functions extends LegacyFunctions {
 new Murmur3Hash(cols.map(_.expr))
   }
 
+  /**
+   * Encrypts input using AES and Returns the result as a binary column.
+   * Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits 
keys can be used if Java
+   * Cryptography Extension (JCE) Unlimited Strength Jurisdiction Policy 
Files are installed. If
+   * either argument is NULL, the result will also be null. If input is 
invalid, key length is not
+   * one of the permitted values or using 192/256 bits key before 
installing JCE, an exception will
+   * be thrown.
+   *
+   * @param input binary column to encrypt input
+   * @param key binary column of 128, 192 or 256 bits key
+   *
+   * @group misc_funcs
+   * @since 2.0.0
+   */
+  def aes_encrypt(input: Column, key: Column): Column = withExpr {
--- End diff --

Key is not integer, it's binary, see examples.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545862
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
--- End diff --

Broadcast will be issued before 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/10527#discussion_r53545860
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1931,6 +1931,42 @@ object functions extends LegacyFunctions {
 new Murmur3Hash(cols.map(_.expr))
   }
 
+  /**
+   * Encrypts input using AES and Returns the result as a binary column.
+   * Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits 
keys can be used if Java
+   * Cryptography Extension (JCE) Unlimited Strength Jurisdiction Policy 
Files are installed. If
+   * either argument is NULL, the result will also be null. If input is 
invalid, key length is not
+   * one of the permitted values or using 192/256 bits key before 
installing JCE, an exception will
+   * be thrown.
+   *
+   * @param input binary column to encrypt input
+   * @param key binary column of 128, 192 or 256 bits key
+   *
+   * @group misc_funcs
+   * @since 2.0.0
+   */
+  def aes_encrypt(input: Column, key: Column): Column = withExpr {
--- End diff --

key should just be an int


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12594] [SQL] Outer Join Elimination by ...

2016-02-19 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/10527#discussion_r53545856
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1931,6 +1931,40 @@ object functions extends LegacyFunctions {
 new Murmur3Hash(cols.map(_.expr))
   }
 
+  /**
+   * Encrypts input using AES and Returns the result as a binary column.
+   * Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits 
keys can be used if
+   * Java Cryptography Extension (JCE) Unlimited Strength Jurisdiction 
Policy Files are installed.
+   * If either argument is NULL or the key length is not one of the 
permitted values,
+   * the result will also be null.
+   *
+   * @param input binary column to encrypt input
+   * @param key binary column of 128, 192 or 256 bits key
+   *
+   * @group misc_funcs
+   * @since 2.0.0
+   */
+  def aes_encrypt(input: Column, key: Column): Column = withExpr {
+AesEncrypt(input.expr, key.expr)
+  }
+
+  /**
+   * Decrypts input using AES and Returns the result as a string column.
+   * Key lengths of 128, 192 or 256 bits can be used. 192 and 256 bits 
keys can be used if
+   * Java Cryptography Extension (JCE) Unlimited Strength Jurisdiction 
Policy Files are installed.
+   * If either argument is NULL or the key length is not one of the 
permitted values,
+   * the result will also be null.
+   *
+   * @param input binary column to decrypt input
+   * @param key binary column of 128, 192 or 256 bits key
+   *
+   * @group misc_funcs
+   * @since 2.0.0
+   */
+  def aes_decrypt(input: Column, key: Column): Column = withExpr {
--- End diff --

key should just be an integer 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545847
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
--- End diff --

what if there is a broadcast join after 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12594] [SQL] Outer Join Elimination by ...

2016-02-19 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/10567#issuecomment-186520244
  
Merging this into master, 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545843
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala 
---
@@ -226,4 +227,29 @@ abstract class QueryPlan[PlanType <: 
TreeNode[PlanType]] extends TreeNode[PlanTy
   protected def statePrefix = if (missingInput.nonEmpty && 
children.nonEmpty) "!" else ""
 
   override def simpleString: String = statePrefix + super.simpleString
+
+  override def generateTreeString(
+  depth: Int, lastChildren: Seq[Boolean], builder: StringBuilder): 
StringBuilder = {
+if (depth > 0) {
+  lastChildren.init.foreach { isLast =>
+val prefixFragment = if (isLast) "   " else ":  "
+builder.append(prefixFragment)
+  }
+
+  val branch = if (lastChildren.last) "+- " else ":- "
+  builder.append(branch)
+}
+
+builder.append(simpleString)
+builder.append("\n")
+
+val allSubqueries = expressions.flatMap(_.collect {case e: 
SubqueryExpression => e})
--- End diff --

can you try? I don't have time to think/write this one.

It just feels bad to do it here. If it is impossible, then of course we 
have to do it 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-02-19 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-186520228
  
LGTM, merging this into master, 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545835
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,68 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.types.DataType
+
+/**
+  * A interface for subquery that is used in expressions.
+  */
+trait SubqueryExpression extends LeafExpression {
+  def query: LogicalPlan
+  def withNewPlan(plan: LogicalPlan): SubqueryExpression
+}
+
+/**
+  * A subquery that will return only one row and one column.
+  */
+case class ScalarSubquery(query: LogicalPlan) extends SubqueryExpression 
with CodegenFallback {
+
+  override lazy val resolved: Boolean = query.resolved
+
+  override def dataType: DataType = query.schema.fields.head.dataType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+if (query.schema.length != 1) {
+  TypeCheckResult.TypeCheckFailure("Scalar subquery can only have 1 
column, but got " +
--- End diff --

"Scalar subquery must return only one column, but got " ...


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12543] [SPARK-4226] [SQL] Subquery in e...

2016-02-19 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/10706#issuecomment-186520013
  
I'd like to keep this open so I can easily find this branch.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12543] [SPARK-4226] [SQL] Subquery in e...

2016-02-19 Thread davies
Github user davies closed the pull request at:

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


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545824
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
 ---
@@ -114,6 +114,12 @@ class AnalysisErrorSuite extends AnalysisTest {
   val dateLit = Literal.create(null, DateType)
 
   errorTest(
+"invalid scalar subquery",
+ testRelation.select(
+   (ScalarSubquery(testRelation.select('a, dateLit.as('b))) + 
Literal(1)).as('a)),
+ "Scalar subquery can only have 1 column, but got 2" :: Nil)
--- End diff --

add a test case for 0 column if possible


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545809
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala 
---
@@ -226,4 +227,29 @@ abstract class QueryPlan[PlanType <: 
TreeNode[PlanType]] extends TreeNode[PlanTy
   protected def statePrefix = if (missingInput.nonEmpty && 
children.nonEmpty) "!" else ""
 
   override def simpleString: String = statePrefix + super.simpleString
+
+  override def generateTreeString(
+  depth: Int, lastChildren: Seq[Boolean], builder: StringBuilder): 
StringBuilder = {
+if (depth > 0) {
+  lastChildren.init.foreach { isLast =>
+val prefixFragment = if (isLast) "   " else ":  "
+builder.append(prefixFragment)
+  }
+
+  val branch = if (lastChildren.last) "+- " else ":- "
+  builder.append(branch)
+}
+
+builder.append(simpleString)
+builder.append("\n")
+
+val allSubqueries = expressions.flatMap(_.collect {case e: 
SubqueryExpression => e})
--- End diff --

How to do that?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545811
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
--- End diff --

we can use postgres' error message: "more than one row returned by a 
subquery used as an expression"




---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545778
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
--- End diff --

This is called after `doPrepare()`, and after prepare() of it's children, 
so it will NOT block broadcasting (will happen in the same time).


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545771
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
+  s"${e.query.treeString}")
+  }
+  // Analyzer will make sure that it only return on column
+  if (rows.length > 0) {
--- End diff --

and same thing applies - the test coverage for this is pretty bad. add a 
test case where the subquery returns 0 or more than 1 rows.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545766
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
+  s"${e.query.treeString}")
+  }
+  // Analyzer will make sure that it only return on column
+  if (rows.length > 0) {
--- End diff --

ok makes sense.

please change the check to rows.length == 1

it's pretty confusing to first check it's greater than 1, and then check it 
is greater than 0, when you are just expecting 1.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545756
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
+
+  /**
+   * The underline plan for the query, could be logical plan or physical 
plan.
+   *
+   * This is used to generate tree string.
+   */
+  def plan: QueryPlan[_]
+
+  /**
+   * Updates the query with new logical plan.
+   */
+  def withNewPlan(plan: LogicalPlan): SubqueryExpression
--- End diff --

i think you can just remove this and move it into the logical subquery 
expression, since it's only used for logical plan anyway?



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545749
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
+exprId: ExprId)
+  extends SubqueryExpression with CodegenFallback {
--- End diff --

but then it boxes everything?



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545740
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
+  s"${e.query.treeString}")
+  }
+  // Analyzer will make sure that it only return on column
+  if (rows.length > 0) {
--- End diff --

`rows.length` means number of rows, not number of columns.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545741
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
+exprId: ExprId)
--- End diff --

please document it.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545735
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -88,7 +88,19 @@ abstract class Optimizer extends 
RuleExecutor[LogicalPlan] {
 Batch("Decimal Optimizations", FixedPoint(100),
   DecimalAggregates) ::
 Batch("LocalRelation", FixedPoint(100),
-  ConvertToLocalRelation) :: Nil
+  ConvertToLocalRelation) ::
+Batch("Subquery", Once,
+  Subquery) :: Nil
+  }
+
+  /**
+   * Optimize all the subqueries inside expression.
+   */
+  object Subquery extends Rule[LogicalPlan] {
+def apply(plan: LogicalPlan): LogicalPlan = plan 
transformAllExpressions {
+  case subquery: SubqueryExpression =>
+subquery.withNewPlan(execute(subquery.query))
--- End diff --

where is execute coming from?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545727
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
+exprId: ExprId)
+  extends SubqueryExpression with CodegenFallback {
--- End diff --

This is LeafExpression, whole stage codegen support that (passing null to 
eval()).


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545716
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
+exprId: ExprId)
--- End diff --

yes


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545713
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
--- End diff --

`plan` is already used to return a `Subquery`


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545693
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
+
+  /**
+   * The underline plan for the query, could be logical plan or physical 
plan.
+   *
+   * This is used to generate tree string.
+   */
+  def plan: QueryPlan[_]
+
+  /**
+   * Updates the query with new logical plan.
+   */
+  def withNewPlan(plan: LogicalPlan): SubqueryExpression
--- End diff --

This should be `copy()`, but I did not figure out how to make copy() work 
for different kind of SubqueryExpression.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12594] [SQL] Outer Join Elimination by ...

2016-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/10567#issuecomment-186519535
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51587/
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12594] [SQL] Outer Join Elimination by ...

2016-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/10567#issuecomment-186519534
  
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12543] [SPARK-4226] [SQL] Subquery in e...

2016-02-19 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/10706#issuecomment-186519506
  
Can we close this first, and create a new one when we get to correlated 
subqueries?



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12594] [SQL] Outer Join Elimination by ...

2016-02-19 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10567#issuecomment-186519491
  
**[Test build #51587 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51587/consoleFull)**
 for PR 10567 at commit 
[`82357e0`](https://github.com/apache/spark/commit/82357e0c0e396f5631d53d3d8151e9f406c5337d).
 * 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13376] [SQL] improve column pruning

2016-02-19 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/11256#issuecomment-186519466
  
cc @cloud-fan 



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-8000][SQL] Support for auto-detecting d...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11270#discussion_r53545637
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala
 ---
@@ -130,7 +141,49 @@ object ResolvedDataSource extends Logging {
   bucketSpec: Option[BucketSpec],
   provider: String,
   options: Map[String, String]): ResolvedDataSource = {
-val clazz: Class[_] = lookupDataSource(provider)
+// Here, it tries to find out data source by file extensions if the 
`format()` is not called.
+// The auto-detection is based on given paths and it recognizes glob 
pattern as well but
+// it does not recursively check the sub-paths even if the given paths 
are directories.
+// This source detection goes the following steps
+//
+//   1. Check `provider` and use this if this is not `null`.
+//   2. If `provider` is not given, then it tries to detect the source 
types by extension.
+//  at this point, if detects only if all the given paths have the 
same extension.
+//   3. if it fails to detect, use the datasource given to 
`spark.sql.sources.default`.
+//
+val paths = {
--- End diff --

note that i'd move this detection code into a separate class, so we can 
unit test it.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: SPARK-13403: Pass hadoopConfiguration to HiveC...

2016-02-19 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/11273#issuecomment-186519373
  
cc @marmbrus 


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545595
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
+
+  /**
+   * The underline plan for the query, could be logical plan or physical 
plan.
+   *
+   * This is used to generate tree string.
+   */
+  def plan: QueryPlan[_]
+
+  /**
+   * Updates the query with new logical plan.
+   */
+  def withNewPlan(plan: LogicalPlan): SubqueryExpression
--- End diff --

can't this be just in the logical plan itself?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545583
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
--- End diff --

why is this needed?




---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545592
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
+
+  /**
+   * The underline plan for the query, could be logical plan or physical 
plan.
--- End diff --

"Either a logical plan or a physical plan. The generated tree string 
(explain output) uses this field to explain the subquery."


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545565
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
+exprId: ExprId)
--- End diff --

document why we need exprId (for explain string output?)


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r5354
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
--- End diff --

i'd just call this physicalPlan, or plan.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545560
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
--- End diff --

actually plan is probably the best. 


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545541
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
--- End diff --

or a subquery is now blocking broadcasting ...



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545455
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
+
+  /**
+   * The logical plan of the query.
+   */
+  def query: LogicalPlan
+
+  /**
+   * The underline plan for the query, could be logical plan or physical 
plan.
+   *
+   * This is used to generate tree string.
+   */
+  def plan: QueryPlan[_]
+
+  /**
+   * Updates the query with new logical plan.
+   */
+  def withNewPlan(plan: LogicalPlan): SubqueryExpression
+}
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is not evaluable, it should be converted into SparkScalaSubquery.
--- End diff --

This will be converted into [[execution.ScalarSubquery]] during planning.

(make sure it links properly)


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545426
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
--- End diff --

ah ok you can't have a general execute.

I guess this is why some query engines have init and then prepare.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545400
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
--- End diff --

we should move the blocking phase into execute, otherwise if multiple nodes 
have subqueries, it becomes blocking.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545335
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
+exprId: ExprId)
+  extends SubqueryExpression with CodegenFallback {
+
+  override def query: LogicalPlan = throw new UnsupportedOperationException
+  override def withNewPlan(plan: LogicalPlan): SubqueryExpression = {
+throw new UnsupportedOperationException
+  }
+  override def plan: SparkPlan = Subquery(simpleString, executedPlan)
+
+  override def dataType: DataType = 
executedPlan.schema.fields.head.dataType
+  override def nullable: Boolean = true
+  override def toString: String = s"subquery#${exprId.id}"
+
+  // the first column in first row from `query`.
+  private var result: Any = null
+
+  def updateResult(v: Any): Unit = {
+result = v
+  }
+
+  override def eval(input: InternalRow): Any = result
+}
+
+/**
+ * Convert the subquery from logical plan into executed plan.
+ */
+private[sql] case class ConvertSubquery(sqlContext: SQLContext) extends 
Rule[SparkPlan] {
--- End diff --

ConvertSubquery -> PlanSubquery


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-11624][SPARK-11972][SQL]fix commands th...

2016-02-19 Thread adrian-wang
Github user adrian-wang commented on a diff in the pull request:

https://github.com/apache/spark/pull/9589#discussion_r53545337
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -104,29 +104,39 @@ private[hive] class HiveClientImpl(
 }
 
 val ret = try {
-  val initialConf = new HiveConf(classOf[SessionState])
-  // HiveConf is a Hadoop Configuration, which has a field of 
classLoader and
-  // the initial value will be the current thread's context class 
loader
-  // (i.e. initClassLoader at here).
-  // We call initialConf.setClassLoader(initClassLoader) at here to 
make
-  // this action explicit.
-  initialConf.setClassLoader(initClassLoader)
-  config.foreach { case (k, v) =>
-if (k.toLowerCase.contains("password")) {
-  logDebug(s"Hive Config: $k=xxx")
-} else {
-  logDebug(s"Hive Config: $k=$v")
+  // originState will be created if not exists, will never be null
+  val originalState = SessionState.get()
+  if (originalState.isInstanceOf[CliSessionState]) {
--- End diff --

the `SessionState.get()` method would create a instance of `SessionState` 
if not exists.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545306
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * An interface for subquery that is used in expressions.
+ */
+abstract class SubqueryExpression extends LeafExpression{
--- End diff --

add a space after LeafExpression


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545236
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -88,7 +88,19 @@ abstract class Optimizer extends 
RuleExecutor[LogicalPlan] {
 Batch("Decimal Optimizations", FixedPoint(100),
   DecimalAggregates) ::
 Batch("LocalRelation", FixedPoint(100),
-  ConvertToLocalRelation) :: Nil
+  ConvertToLocalRelation) ::
+Batch("Subquery", Once,
+  Subquery) :: Nil
+  }
+
+  /**
+   * Optimize all the subqueries inside expression.
+   */
+  object Subquery extends Rule[LogicalPlan] {
--- End diff --

Subquery -> OptimizeSubqueries


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545226
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala 
---
@@ -226,4 +227,29 @@ abstract class QueryPlan[PlanType <: 
TreeNode[PlanType]] extends TreeNode[PlanTy
   protected def statePrefix = if (missingInput.nonEmpty && 
children.nonEmpty) "!" else ""
 
   override def simpleString: String = statePrefix + super.simpleString
+
+  override def generateTreeString(
+  depth: Int, lastChildren: Seq[Boolean], builder: StringBuilder): 
StringBuilder = {
+if (depth > 0) {
+  lastChildren.init.foreach { isLast =>
+val prefixFragment = if (isLast) "   " else ":  "
+builder.append(prefixFragment)
+  }
+
+  val branch = if (lastChildren.last) "+- " else ":- "
+  builder.append(branch)
+}
+
+builder.append(simpleString)
+builder.append("\n")
+
+val allSubqueries = expressions.flatMap(_.collect {case e: 
SubqueryExpression => e})
--- End diff --

Can't we do this in subquery itself? It is pretty odd to have this general 
base class depend on some specific expression


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545188
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
+  s"${e.query.treeString}")
+  }
+  // Analyzer will make sure that it only return on column
+  if (rows.length > 0) {
--- End diff --

Also if it is possible to have 0 column, we also need to add a test 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545182
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
+  s"${e.query.treeString}")
+  }
+  // Analyzer will make sure that it only return on column
+  if (rows.length > 0) {
--- End diff --

How do we write a query with 0 column? The comment above said the analyzer 
would make sure there's only one column.

If it is possible to have 0 column, then I'd make it explicitly here to set 
the value to null.




---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545111
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -2105,6 +2105,22 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
 assert(error.getMessage contains "grouping_id() can only be used with 
GroupingSets/Cube/Rollup")
   }
 
+  test("uncorrelated scalar subquery") {
--- End diff --

Also we should test the behavior when there is no rows returned.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545093
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
+exprId: ExprId)
+  extends SubqueryExpression with CodegenFallback {
+
+  override def query: LogicalPlan = throw new UnsupportedOperationException
+  override def withNewPlan(plan: LogicalPlan): SubqueryExpression = {
+throw new UnsupportedOperationException
+  }
+  override def plan: SparkPlan = Subquery(simpleString, executedPlan)
+
+  override def dataType: DataType = 
executedPlan.schema.fields.head.dataType
+  override def nullable: Boolean = true
+  override def toString: String = s"subquery#${exprId.id}"
+
+  // the first column in first row from `query`.
+  private var result: Any = null
+
+  def updateResult(v: Any): Unit = {
+result = v
+  }
+
+  override def eval(input: InternalRow): Any = result
+}
+
+/**
+ * Convert the subquery from logical plan into executed plan.
+ */
+private[sql] case class ConvertSubquery(sqlContext: SQLContext) extends 
Rule[SparkPlan] {
+  def apply(plan: SparkPlan): SparkPlan = {
+plan.transformAllExpressions {
+  // Only scalar subquery will be executed separately, all others will 
be written as join.
--- End diff --

"will be" -> "should have been".

They should've been planned as joins by the time we get 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545076
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
+@transient executedPlan: SparkPlan,
+exprId: ExprId)
+  extends SubqueryExpression with CodegenFallback {
--- End diff --

How does this interact with whole stage codegen?



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545061
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala ---
@@ -0,0 +1,71 @@
+/*
+ * 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.execution
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{ExprId, ScalarSubquery, 
SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.DataType
+
+/**
+ * A subquery that will return only one row and one column.
+ *
+ * This is the physical copy of ScalarSubquery to be used inside SparkPlan.
+ */
+case class SparkScalarSubquery(
--- End diff --

Name this ScalarSubquery to be more consistent with rest of the physical vs 
logical plan naming.



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545044
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -2105,6 +2105,22 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
 assert(error.getMessage contains "grouping_id() can only be used with 
GroupingSets/Cube/Rollup")
   }
 
+  test("uncorrelated scalar subquery") {
--- End diff --

About test coverage:

let's create a subquery suite and move the test cases there.

Also it would be great to have at least once test case that actually runs 
on a dataset that is not generated by just select x, because I worry in the 
future we add some special optimizations and then all the test cases here 
become no-op.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53545045
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala 
---
@@ -201,4 +202,49 @@ class CatalystQlSuite extends PlanTest {
 parser.parsePlan("select sum(product + 1) over (partition by (product 
+ (1)) order by 2) " +
   "from windowData")
   }
+
+  test("subquery") {
+comparePlans(
+  parser.parsePlan("select (select max(b) from s) ss from t"),
+  Project(
--- End diff --

@hvanhovell I'm going to remove these plan checking, it's very easy to 
break. The details of plan does not mean much, we will have other tests to 
verify the correctness.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53544993
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
+  s"${e.query.treeString}")
+  }
+  // Analyzer will make sure that it only return on column
+  if (rows.length > 0) {
--- End diff --

the length could be zero, then the value is null.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53544977
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -231,6 +269,11 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   }
 }
 
+object SparkPlan {
+  private[execution] val subqueryExecutionContext = 
ExecutionContext.fromExecutorService(
--- End diff --

This could be refactored later, use the same thread pool for all of them.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53544978
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
+  s"${e.query.treeString}")
+  }
+  // Analyzer will make sure that it only return on column
+  if (rows.length > 0) {
--- End diff --

can rows.length ever be 0 here? if it can only be 1, why we are testing > 0 
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53544969
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(SparkScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
SparkScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = Future {
+  e.plan.executeCollect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
+  if (rows.length > 1) {
+sys.error(s"Scalar subquery should return at most one row, but 
got ${rows.length}: " +
+  s"${e.query.treeString}")
+  }
+  // Analyzer will make sure that it only return on column
--- End diff --

"Analyzer should make sure this only returns one column"

and add an assert after 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/11190#discussion_r53544966
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -122,7 +125,33 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
   final def prepare(): Unit = {
 if (prepareCalled.compareAndSet(false, true)) {
   doPrepare()
+
+  // collect all the subqueries and submit jobs to execute them in 
background
+  val queryResults = ArrayBuffer[(ScalarSubquery, 
Future[Array[InternalRow]])]()
+  val allSubqueries = expressions.flatMap(_.collect {case e: 
ScalarSubquery => e})
+  allSubqueries.foreach { e =>
+val futureResult = scala.concurrent.future {
+  val df = DataFrame(sqlContext, e.query)
+  df.queryExecution.toRdd.collect()
+}(SparkPlan.subqueryExecutionContext)
+queryResults += e -> futureResult
+  }
+
   children.foreach(_.prepare())
+
+  // fill in the result of subqueries
+  queryResults.foreach {
+case (e, futureResult) =>
+  val rows = Await.result(futureResult, Duration.Inf)
--- End diff --

After offline discussion with @rxin and @marmbrus , we decided to not have 
a timeout here, we can see the spark job in UI and could cancel it anytime.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13306] [SQL] uncorrelated scalar subque...

2016-02-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11190#issuecomment-186514679
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51586/
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



  1   2   3   4   5   6   >