[GitHub] spark issue #13371: [SPARK-15639][SQL] Try to push down filter at RowGroups ...

2016-06-09 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/13371
  
@yhuai Your step 3 may not work. We are going to filter the row groups for 
each parquet file to read in `VectorizedParquetRecordReader`. I think we don't 
do anything regarding creating splits?


---
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 #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

2016-06-09 Thread frreiss
Github user frreiss commented on a diff in the pull request:

https://github.com/apache/spark/pull/13155#discussion_r66564793
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends 
Rule[LogicalPlan] {
   }
 
   /**
+   * Statically evaluate an expression containing zero or more 
placeholders, given a set
+   * of bindings for placeholder values.
+   */
+  private def evalExpr(expr : Expression, bindings : Map[Long, 
Option[Any]]) : Option[Any] = {
+val rewrittenExpr = expr transform {
+  case r @ AttributeReference(_, dataType, _, _) =>
+bindings(r.exprId.id) match {
+  case Some(v) => Literal.create(v, dataType)
+  case None => Literal.default(NullType)
+}
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate an expression containing one or more aggregates 
on an empty input.
+   */
+  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
+// AggregateExpressions are Unevaluable, so we need to replace all 
aggregates
+// in the expression with the value they would return for zero input 
tuples.
+val rewrittenExpr = expr transform {
+  case a @ AggregateExpression(aggFunc, _, _, resultId) =>
+aggFunc.defaultResult.getOrElse(Literal.default(NullType))
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate a scalar subquery on an empty input.
+   *
+   * WARNING: This method only covers subqueries that pass the 
checks under
+   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the 
checks in
+   * CheckAnalysis become less restrictive, this method will need to 
change.
+   */
+  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
+// Inputs to this method will start with a chain of zero or more 
SubqueryAlias
+// and Project operators, followed by an optional Filter, followed by 
an
+// Aggregate. Traverse the operators recursively.
+def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
+  lp match {
+case SubqueryAlias(_, child) => evalPlan(child)
+case Filter(condition, child) =>
+  val bindings = evalPlan(child)
+  if (bindings.size == 0) bindings
+  else {
+val exprResult = evalExpr(condition, bindings).getOrElse(false)
+  .asInstanceOf[Boolean]
+if (exprResult) bindings else Map()
+  }
+
+case Project(projectList, child) =>
+  val bindings = evalPlan(child)
+  if (bindings.size == 0) {
+bindings
+  } else {
+projectList.map(ne => (ne.exprId.id, evalExpr(ne, 
bindings))).toMap
+  }
+
+case Aggregate(_, aggExprs, _) =>
+  // Some of the expressions under the Aggregate node are the join 
columns
+  // for joining with the outer query block. Fill those 
expressions in with
+  // nulls and statically evaluate the remainder.
+  aggExprs.map(ne => ne match {
+case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
+case Alias(AttributeReference(_, _, _, _), _) => 
(ne.exprId.id, None)
+case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
+  }).toMap
+
+case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
+  }
+}
+
+val resultMap = evalPlan(plan)
+
+// By convention, the scalar subquery result is the leftmost field.
+resultMap(plan.output.head.exprId.id)
+  }
+
+  /**
+   * Split the plan for a scalar subquery into the parts above the 
Aggregate node
+   * (first part of returned value) and the parts below the Aggregate 
node, including
+   * the Aggregate (second part of returned value)
+   */
+  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], 
Aggregate] = {
+var topPart = List[LogicalPlan]()
+var bottomPart : LogicalPlan = plan
+while (! bottomPart.isInstanceOf[Aggregate]) {
+  topPart = bottomPart :: topPart
+  bottomPart = bottomPart.children.head
+}
+(topPart, bottomPart.asInstanceOf[Aggregate])
+  }
+
+  /**
+   * Rewrite the nodes above the Aggregate in a subquery so that they 
generate an
+   * auxiliary column "isFiltered"
+   * @param subqueryPlan plan before rewrite
+   * @param filteredId expression ID for the "isFiltered" column
+   */
+  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : 
ExprId) : LogicalPlan = {
+val 

[GitHub] spark pull request #13585: [SPARK-15859][SQL] Optimize the partition pruning...

2016-06-09 Thread wangyang1992
Github user wangyang1992 commented on a diff in the pull request:

https://github.com/apache/spark/pull/13585#discussion_r66564745
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala ---
@@ -65,4 +69,95 @@ class QueryPartitionSuite extends QueryTest with 
SQLTestUtils with TestHiveSingl
   sql("DROP TABLE IF EXISTS createAndInsertTest")
 }
   }
+
+  test("partition pruning in disjunction") {
+withSQLConf((SQLConf.HIVE_VERIFY_PARTITION_PATH.key, "true")) {
+  val testData = sparkContext.parallelize(
+(1 to 10).map(i => TestData(i, i.toString))).toDF()
+  testData.registerTempTable("testData")
+
+  val testData2 = sparkContext.parallelize(
+(11 to 20).map(i => TestData(i, i.toString))).toDF()
+  testData2.registerTempTable("testData2")
+
+  val testData3 = sparkContext.parallelize(
+(21 to 30).map(i => TestData(i, i.toString))).toDF()
+  testData3.registerTempTable("testData3")
+
+  val testData4 = sparkContext.parallelize(
+(31 to 40).map(i => TestData(i, i.toString))).toDF()
+  testData4.registerTempTable("testData4")
+
+  val tmpDir = Files.createTempDir()
+  // create the table for test
+  sql(s"CREATE TABLE table_with_partition(key int,value string) " +
+s"PARTITIONED by (ds string, ds2 string) location 
'${tmpDir.toURI.toString}' ")
+  sql("INSERT OVERWRITE TABLE table_with_partition  partition (ds='1', 
ds2='d1') " +
+"SELECT key,value FROM testData")
+  sql("INSERT OVERWRITE TABLE table_with_partition  partition (ds='2', 
ds2='d1') " +
+"SELECT key,value FROM testData2")
+  sql("INSERT OVERWRITE TABLE table_with_partition  partition (ds='3', 
ds2='d3') " +
+"SELECT key,value FROM testData3")
+  sql("INSERT OVERWRITE TABLE table_with_partition  partition (ds='4', 
ds2='d4') " +
+"SELECT key,value FROM testData4")
+
+  checkAnswer(sql("select key,value from table_with_partition"),
+testData.collect ++ testData2.collect ++ testData3.collect ++ 
testData4.collect)
+
+  checkAnswer(
+sql(
+  """select key,value from table_with_partition
+| where (ds='4' and key=38) or (ds='3' and 
key=22)""".stripMargin),
+  Row(38, "38") :: Row(22, "22") :: Nil)
+
+  checkAnswer(
+sql(
+  """select key,value from table_with_partition
+| where (key<40 and key>38) or (ds='3' and 
key=22)""".stripMargin),
+Row(39, "39") :: Row(22, "22") :: Nil)
+
+  sql("DROP TABLE table_with_partition")
+  sql("DROP TABLE createAndInsertTest")
--- End diff --

Not really sure why we should drop "createAndInsertTest", I can find it 
anywhere. Maybe those temp tables named  "testData*" are the ones should be 
dropped. ^_^


---
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 #13413: [SPARK-15663][SQL] SparkSession.catalog.listFunct...

2016-06-09 Thread techaddict
Github user techaddict commented on a diff in the pull request:

https://github.com/apache/spark/pull/13413#discussion_r66564698
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -1481,17 +1481,7 @@ def test_list_functions(self):
 spark.sql("CREATE DATABASE some_db")
 functions = dict((f.name, f) for f in 
spark.catalog.listFunctions())
 functionsDefault = dict((f.name, f) for f in 
spark.catalog.listFunctions("default"))
-self.assertTrue(len(functions) > 200)
-self.assertTrue("+" in functions)
-self.assertTrue("like" in functions)
-self.assertTrue("month" in functions)
-self.assertTrue("to_unix_timestamp" in functions)
-self.assertTrue("current_database" in functions)
-self.assertEquals(functions["+"], Function(
-name="+",
-description=None,
-className="org.apache.spark.sql.catalyst.expressions.Add",
-isTemporary=True))
+self.assertEquals(len(functions), 0)
--- End diff --

@yhuai I meant there are python tests already here 
(https://github.com/apache/spark/pull/13413/files#diff-7c2fe8530271c0635fb99f7b49e0c4a4L1496)
 Let me know if you anything specific in mind too.


---
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 #13436: [SPARK-15696][SQL] Improve `crosstab` to have a c...

2016-06-09 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 issue #13415: [SPARK-15676] [SQL] Disallow Column Names as Partition C...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13415
  
**[Test build #60275 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60275/consoleFull)**
 for PR 13415 at commit 
[`f4207e3`](https://github.com/apache/spark/commit/f4207e3c185a13a7f2866b0f12fcde5b28b8d948).


---
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 issue #13436: [SPARK-15696][SQL] Improve `crosstab` to have a consiste...

2016-06-09 Thread rxin
Github user rxin commented on the issue:

https://github.com/apache/spark/pull/13436
  
LGTM - merging in master/2.0.



---
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 issue #13415: [SPARK-15676] [SQL] Disallow Column Names as Partition C...

2016-06-09 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/13415
  
retest this please


---
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 issue #13371: [SPARK-15639][SQL] Try to push down filter at RowGroups ...

2016-06-09 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/13371
  
@yhuai Parquet also does this filtering at ParquetRecordReader 
(https://github.com/apache/parquet-mr/blob/4b1ff8f4b9dfa0ccb064ef286cf2953bfb2c492d/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordReader.java#L178)
 and 
ParquetReader(https://github.com/apache/parquet-mr/blob/4b1ff8f4b9dfa0ccb064ef286cf2953bfb2c492d/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetReader.java#L145).

In Spark, we also did this at SpecificParquetRecordReaderBase 
(https://github.com/apache/spark/blob/f958c1c3e292aba98d283637606890f353a9836c/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java#L103).

I've manually tested it. But it should be good to have a formal test case 
for it as you said. I will try to add it later, maybe when I come back to work 
few days later...



---
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 #13585: [SPARK-15859][SQL] Optimize the partition pruning...

2016-06-09 Thread wangyang1992
Github user wangyang1992 commented on a diff in the pull request:

https://github.com/apache/spark/pull/13585#discussion_r66563744
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
 ---
@@ -92,6 +92,36 @@ object PhysicalOperation extends PredicateHelper {
   .map(Alias(_, a.name)(a.exprId, a.qualifier, isGenerated = 
a.isGenerated)).getOrElse(a)
 }
   }
+
+  /**
+   * Drop the non-partition key expression in the disjunctions, to 
optimize the partition pruning.
--- End diff --

"Drop the non-partition key expression in the disjunctions". Should it be 
"conjunctions"? 


---
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 #13576: [SPARK-15840][SQL] Add two missing options in doc...

2016-06-09 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13576#discussion_r66563599
  
--- Diff: python/pyspark/sql/readwriter.py ---
@@ -327,6 +334,8 @@ def csv(self, path, schema=None, sep=None, 
encoding=None, quote=None, escape=Non
 character. By default (None), it is disabled.
 :param header: uses the first line as names of columns. If None is 
set, it uses the
default value, ``false``.
+:param inferSchema: infers the input schema automatically from 
data. It requires one extra
--- End diff --

isn't inferSchema automatically off if schema is set?



---
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 #13569: [SPARK-15791] Fix NPE in ScalarSubquery

2016-06-09 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 #13576: [SPARK-15840][SQL] Add two missing options in doc...

2016-06-09 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13576#discussion_r66563370
  
--- Diff: python/pyspark/sql/readwriter.py ---
@@ -207,9 +207,10 @@ def json(self, path, schema=None, 
primitivesAsString=None, prefersDecimal=None,
 *  ``FAILFAST`` : throws an exception when it meets 
corrupted records.
 
 :param columnNameOfCorruptRecord: allows renaming the new field 
having malformed string
-  created by ``PERMISSIVE`` mode. 
This overrides
+  created by ``PERMISSIVE`` mode. 
This overwrites
--- End diff --

i think technically "overrides" is more accurate than "overwrites" 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 issue #13496: [SPARK-15753][SQL] Move Analyzer stuff to Analyzer from ...

2016-06-09 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/13496
  
try to ping @yhuai 


---
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 #13413: [SPARK-15663][SQL] SparkSession.catalog.listFunct...

2016-06-09 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/13413#discussion_r66563299
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -58,15 +60,39 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
 
   test("show functions") {
 def getFunctions(pattern: String): Seq[Row] = {
-  
StringUtils.filterPattern(spark.sessionState.functionRegistry.listFunction(), 
pattern)
+  StringUtils.filterPattern(
+
spark.sessionState.catalog.listFunctions("default").map(_.funcName), pattern)
 .map(Row(_))
 }
+
+def createFunction(names: Seq[String]): Unit = {
+  names.foreach { name =>
+spark.udf.register(name, (arg1: Int, arg2: String) => arg2 + arg1)
+  }
+}
+
+def dropFunction(names: Seq[String]): Unit = {
+  names.foreach { name =>
+spark.sessionState.catalog.dropTempFunction(name, false)
+  }
+}
+
+val functions = Seq("ilog", "logi", "logii", "logiii", "crc32i", 
"cubei", "cume_disti",
+  "isize", "ispace", "to_datei", "date_addi", "current_datei")
+
+assert(sql("SHOW functions").collect().isEmpty)
+
+createFunction(functions)
+
 checkAnswer(sql("SHOW functions"), getFunctions("*"))
+assert(sql("SHOW functions").collect().size === functions.size)
--- End diff --

Seems we want to check the content of returned function list instead of 
just the size.


---
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 issue #13569: [SPARK-15791] Fix NPE in ScalarSubquery

2016-06-09 Thread rxin
Github user rxin commented on the issue:

https://github.com/apache/spark/pull/13569
  
Merging in master/2.0.



---
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 #13569: [SPARK-15791] Fix NPE in ScalarSubquery

2016-06-09 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13569#discussion_r66563276
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala 
---
@@ -54,6 +54,10 @@ class SubquerySuite extends QueryTest with 
SharedSQLContext {
 t.createOrReplaceTempView("t")
   }
 
+  test("rdd deserialization does not crash [SPARK-15791]") {
+sql("select (select 1 as b) as b").rdd.count()
--- End diff --

+1 

although not that big of a deal, since other tests already cover some 
correctness. I'm going to merge this one to cut down the number of 2.0 blockers.



---
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 #13413: [SPARK-15663][SQL] SparkSession.catalog.listFunct...

2016-06-09 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/13413#discussion_r66563267
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -855,7 +855,8 @@ class SessionCatalog(
   .map { f => FunctionIdentifier(f, Some(dbName)) }
 val loadedFunctions = 
StringUtils.filterPattern(functionRegistry.listFunction(), pattern)
   .map { f => FunctionIdentifier(f) }
-dbFunctions ++ loadedFunctions
+val builtInFunctions = FunctionRegistry.builtin.listFunction().toSet
--- End diff --

Let's avoid of creating this set in every `listFunctions` call. 


---
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 #13413: [SPARK-15663][SQL] SparkSession.catalog.listFunct...

2016-06-09 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/13413#discussion_r66563161
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -1481,17 +1481,7 @@ def test_list_functions(self):
 spark.sql("CREATE DATABASE some_db")
 functions = dict((f.name, f) for f in 
spark.catalog.listFunctions())
 functionsDefault = dict((f.name, f) for f in 
spark.catalog.listFunctions("default"))
-self.assertTrue(len(functions) > 200)
-self.assertTrue("+" in functions)
-self.assertTrue("like" in functions)
-self.assertTrue("month" in functions)
-self.assertTrue("to_unix_timestamp" in functions)
-self.assertTrue("current_database" in functions)
-self.assertEquals(functions["+"], Function(
-name="+",
-description=None,
-className="org.apache.spark.sql.catalyst.expressions.Add",
-isTemporary=True))
+self.assertEquals(len(functions), 0)
--- End diff --

It will be still good to have some python tests.


---
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 issue #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to allow t...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13342
  
**[Test build #60274 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60274/consoleFull)**
 for PR 13342 at commit 
[`8cb7aa5`](https://github.com/apache/spark/commit/8cb7aa5593e0d845eaa92799f43792c5c9f1c02a).


---
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 #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to ...

2016-06-09 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/13342#discussion_r66562319
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -232,7 +234,7 @@ final class DataFrameWriter private[sql](df: DataFrame) 
{
* @since 1.4.0
*/
   @scala.annotation.varargs
-  def partitionBy(colNames: String*): DataFrameWriter = {
+  def partitionBy(colNames: String*): this.type = {
--- End diff --

Forgot to change it. No difference in byte codes but it will show 
`DataFrameWriter.this.type` in scala doc. I will update 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 issue #13571: [SPARK-15369][WIP][RFC][PySpark][SQL] Expose potential t...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13571
  
**[Test build #60273 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60273/consoleFull)**
 for PR 13571 at commit 
[`17880f2`](https://github.com/apache/spark/commit/17880f2d036033f48b878caba36e716daaf48fc3).


---
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 #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

2016-06-09 Thread frreiss
Github user frreiss commented on a diff in the pull request:

https://github.com/apache/spark/pull/13155#discussion_r66561815
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends 
Rule[LogicalPlan] {
   }
 
   /**
+   * Statically evaluate an expression containing zero or more 
placeholders, given a set
+   * of bindings for placeholder values.
+   */
+  private def evalExpr(expr : Expression, bindings : Map[Long, 
Option[Any]]) : Option[Any] = {
+val rewrittenExpr = expr transform {
+  case r @ AttributeReference(_, dataType, _, _) =>
+bindings(r.exprId.id) match {
+  case Some(v) => Literal.create(v, dataType)
+  case None => Literal.default(NullType)
+}
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate an expression containing one or more aggregates 
on an empty input.
+   */
+  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
+// AggregateExpressions are Unevaluable, so we need to replace all 
aggregates
+// in the expression with the value they would return for zero input 
tuples.
+val rewrittenExpr = expr transform {
+  case a @ AggregateExpression(aggFunc, _, _, resultId) =>
+aggFunc.defaultResult.getOrElse(Literal.default(NullType))
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate a scalar subquery on an empty input.
+   *
+   * WARNING: This method only covers subqueries that pass the 
checks under
+   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the 
checks in
+   * CheckAnalysis become less restrictive, this method will need to 
change.
+   */
+  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
+// Inputs to this method will start with a chain of zero or more 
SubqueryAlias
+// and Project operators, followed by an optional Filter, followed by 
an
+// Aggregate. Traverse the operators recursively.
+def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
+  lp match {
+case SubqueryAlias(_, child) => evalPlan(child)
+case Filter(condition, child) =>
+  val bindings = evalPlan(child)
+  if (bindings.size == 0) bindings
+  else {
+val exprResult = evalExpr(condition, bindings).getOrElse(false)
+  .asInstanceOf[Boolean]
+if (exprResult) bindings else Map()
+  }
+
+case Project(projectList, child) =>
+  val bindings = evalPlan(child)
+  if (bindings.size == 0) {
+bindings
+  } else {
+projectList.map(ne => (ne.exprId.id, evalExpr(ne, 
bindings))).toMap
+  }
+
+case Aggregate(_, aggExprs, _) =>
+  // Some of the expressions under the Aggregate node are the join 
columns
+  // for joining with the outer query block. Fill those 
expressions in with
+  // nulls and statically evaluate the remainder.
+  aggExprs.map(ne => ne match {
+case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
+case Alias(AttributeReference(_, _, _, _), _) => 
(ne.exprId.id, None)
+case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
+  }).toMap
+
+case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
+  }
+}
+
+val resultMap = evalPlan(plan)
+
+// By convention, the scalar subquery result is the leftmost field.
+resultMap(plan.output.head.exprId.id)
+  }
+
+  /**
+   * Split the plan for a scalar subquery into the parts above the 
Aggregate node
+   * (first part of returned value) and the parts below the Aggregate 
node, including
+   * the Aggregate (second part of returned value)
+   */
+  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], 
Aggregate] = {
+var topPart = List[LogicalPlan]()
+var bottomPart : LogicalPlan = plan
+while (! bottomPart.isInstanceOf[Aggregate]) {
+  topPart = bottomPart :: topPart
+  bottomPart = bottomPart.children.head
+}
+(topPart, bottomPart.asInstanceOf[Aggregate])
+  }
+
+  /**
+   * Rewrite the nodes above the Aggregate in a subquery so that they 
generate an
+   * auxiliary column "isFiltered"
+   * @param subqueryPlan plan before rewrite
+   * @param filteredId expression ID for the "isFiltered" column
+   */
+  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : 
ExprId) : LogicalPlan = {
+val 

[GitHub] spark issue #13589: [SPARK-15822][SPARK-15825][SQL] Fix SMJ Segfault/Invalid...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13589
  
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 issue #13589: [SPARK-15822][SPARK-15825][SQL] Fix SMJ Segfault/Invalid...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13589
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60268/
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 issue #13589: [SPARK-15822][SPARK-15825][SQL] Fix SMJ Segfault/Invalid...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13589
  
**[Test build #60268 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60268/consoleFull)**
 for PR 13589 at commit 
[`995c86a`](https://github.com/apache/spark/commit/995c86a09ae7eaf013abb618b65749b1e0f71eca).
 * 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 issue #13571: [SPARK-15369][WIP][RFC][PySpark][SQL] Expose potential t...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13571
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60269/
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



[GitHub] spark issue #13571: [SPARK-15369][WIP][RFC][PySpark][SQL] Expose potential t...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13571
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #13571: [SPARK-15369][WIP][RFC][PySpark][SQL] Expose potential t...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13571
  
**[Test build #60269 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60269/consoleFull)**
 for PR 13571 at commit 
[`8c056a8`](https://github.com/apache/spark/commit/8c056a8c4db2fd33623f4c0a8efd1b2f53b2147f).
 * This patch **fails PySpark unit 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 #13569: [SPARK-15791] Fix NPE in ScalarSubquery

2016-06-09 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/13569#discussion_r66561018
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala 
---
@@ -54,6 +54,10 @@ class SubquerySuite extends QueryTest with 
SharedSQLContext {
 t.createOrReplaceTempView("t")
   }
 
+  test("rdd deserialization does not crash [SPARK-15791]") {
+sql("select (select 1 as b) as b").rdd.count()
--- End diff --

Should we check the result?


---
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 #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

2016-06-09 Thread frreiss
Github user frreiss commented on a diff in the pull request:

https://github.com/apache/spark/pull/13155#discussion_r66561017
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends 
Rule[LogicalPlan] {
   }
 
   /**
+   * Statically evaluate an expression containing zero or more 
placeholders, given a set
+   * of bindings for placeholder values.
+   */
+  private def evalExpr(expr : Expression, bindings : Map[Long, 
Option[Any]]) : Option[Any] = {
+val rewrittenExpr = expr transform {
+  case r @ AttributeReference(_, dataType, _, _) =>
+bindings(r.exprId.id) match {
+  case Some(v) => Literal.create(v, dataType)
+  case None => Literal.default(NullType)
+}
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate an expression containing one or more aggregates 
on an empty input.
+   */
+  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
+// AggregateExpressions are Unevaluable, so we need to replace all 
aggregates
+// in the expression with the value they would return for zero input 
tuples.
+val rewrittenExpr = expr transform {
--- End diff --

Yes, that is true. Added a test case and an additional clause in that case 
statement in my local copy.


---
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 #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

2016-06-09 Thread frreiss
Github user frreiss commented on a diff in the pull request:

https://github.com/apache/spark/pull/13155#discussion_r66560947
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends 
Rule[LogicalPlan] {
   }
 
   /**
+   * Statically evaluate an expression containing zero or more 
placeholders, given a set
+   * of bindings for placeholder values.
+   */
+  private def evalExpr(expr : Expression, bindings : Map[Long, 
Option[Any]]) : Option[Any] = {
+val rewrittenExpr = expr transform {
+  case r @ AttributeReference(_, dataType, _, _) =>
+bindings(r.exprId.id) match {
+  case Some(v) => Literal.create(v, dataType)
+  case None => Literal.default(NullType)
+}
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate an expression containing one or more aggregates 
on an empty input.
+   */
+  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
+// AggregateExpressions are Unevaluable, so we need to replace all 
aggregates
+// in the expression with the value they would return for zero input 
tuples.
+val rewrittenExpr = expr transform {
+  case a @ AggregateExpression(aggFunc, _, _, resultId) =>
+aggFunc.defaultResult.getOrElse(Literal.default(NullType))
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate a scalar subquery on an empty input.
+   *
+   * WARNING: This method only covers subqueries that pass the 
checks under
+   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the 
checks in
+   * CheckAnalysis become less restrictive, this method will need to 
change.
+   */
+  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
+// Inputs to this method will start with a chain of zero or more 
SubqueryAlias
+// and Project operators, followed by an optional Filter, followed by 
an
+// Aggregate. Traverse the operators recursively.
+def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
+  lp match {
+case SubqueryAlias(_, child) => evalPlan(child)
+case Filter(condition, child) =>
+  val bindings = evalPlan(child)
+  if (bindings.size == 0) bindings
+  else {
+val exprResult = evalExpr(condition, bindings).getOrElse(false)
+  .asInstanceOf[Boolean]
+if (exprResult) bindings else Map()
+  }
+
+case Project(projectList, child) =>
+  val bindings = evalPlan(child)
+  if (bindings.size == 0) {
+bindings
+  } else {
+projectList.map(ne => (ne.exprId.id, evalExpr(ne, 
bindings))).toMap
+  }
+
+case Aggregate(_, aggExprs, _) =>
+  // Some of the expressions under the Aggregate node are the join 
columns
+  // for joining with the outer query block. Fill those 
expressions in with
+  // nulls and statically evaluate the remainder.
+  aggExprs.map(ne => ne match {
+case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
+case Alias(AttributeReference(_, _, _, _), _) => 
(ne.exprId.id, None)
+case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
+  }).toMap
+
+case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
+  }
+}
+
+val resultMap = evalPlan(plan)
+
+// By convention, the scalar subquery result is the leftmost field.
+resultMap(plan.output.head.exprId.id)
+  }
+
+  /**
+   * Split the plan for a scalar subquery into the parts above the 
Aggregate node
+   * (first part of returned value) and the parts below the Aggregate 
node, including
+   * the Aggregate (second part of returned value)
+   */
+  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], 
Aggregate] = {
+var topPart = List[LogicalPlan]()
+var bottomPart : LogicalPlan = plan
+while (! bottomPart.isInstanceOf[Aggregate]) {
+  topPart = bottomPart :: topPart
+  bottomPart = bottomPart.children.head
+}
+(topPart, bottomPart.asInstanceOf[Aggregate])
+  }
+
+  /**
+   * Rewrite the nodes above the Aggregate in a subquery so that they 
generate an
+   * auxiliary column "isFiltered"
+   * @param subqueryPlan plan before rewrite
+   * @param filteredId expression ID for the "isFiltered" column
+   */
+  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : 
ExprId) : LogicalPlan = {
+val 

[GitHub] spark pull request #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

2016-06-09 Thread frreiss
Github user frreiss commented on a diff in the pull request:

https://github.com/apache/spark/pull/13155#discussion_r66560868
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends 
Rule[LogicalPlan] {
   }
 
   /**
+   * Statically evaluate an expression containing zero or more 
placeholders, given a set
+   * of bindings for placeholder values.
+   */
+  private def evalExpr(expr : Expression, bindings : Map[Long, 
Option[Any]]) : Option[Any] = {
+val rewrittenExpr = expr transform {
+  case r @ AttributeReference(_, dataType, _, _) =>
+bindings(r.exprId.id) match {
+  case Some(v) => Literal.create(v, dataType)
+  case None => Literal.default(NullType)
+}
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate an expression containing one or more aggregates 
on an empty input.
+   */
+  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
+// AggregateExpressions are Unevaluable, so we need to replace all 
aggregates
+// in the expression with the value they would return for zero input 
tuples.
+val rewrittenExpr = expr transform {
+  case a @ AggregateExpression(aggFunc, _, _, resultId) =>
+aggFunc.defaultResult.getOrElse(Literal.default(NullType))
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate a scalar subquery on an empty input.
+   *
+   * WARNING: This method only covers subqueries that pass the 
checks under
+   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the 
checks in
+   * CheckAnalysis become less restrictive, this method will need to 
change.
+   */
+  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
+// Inputs to this method will start with a chain of zero or more 
SubqueryAlias
+// and Project operators, followed by an optional Filter, followed by 
an
+// Aggregate. Traverse the operators recursively.
+def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
+  lp match {
+case SubqueryAlias(_, child) => evalPlan(child)
+case Filter(condition, child) =>
+  val bindings = evalPlan(child)
+  if (bindings.size == 0) bindings
+  else {
+val exprResult = evalExpr(condition, bindings).getOrElse(false)
+  .asInstanceOf[Boolean]
+if (exprResult) bindings else Map()
+  }
+
+case Project(projectList, child) =>
+  val bindings = evalPlan(child)
+  if (bindings.size == 0) {
+bindings
+  } else {
+projectList.map(ne => (ne.exprId.id, evalExpr(ne, 
bindings))).toMap
+  }
+
+case Aggregate(_, aggExprs, _) =>
+  // Some of the expressions under the Aggregate node are the join 
columns
+  // for joining with the outer query block. Fill those 
expressions in with
+  // nulls and statically evaluate the remainder.
+  aggExprs.map(ne => ne match {
+case AttributeReference(_, _, _, _) => (ne.exprId.id, None)
+case Alias(AttributeReference(_, _, _, _), _) => 
(ne.exprId.id, None)
+case _ => (ne.exprId.id, evalAggOnZeroTups(ne))
+  }).toMap
+
+case _ => sys.error(s"Unexpected operator in scalar subquery: $lp")
+  }
+}
+
+val resultMap = evalPlan(plan)
+
+// By convention, the scalar subquery result is the leftmost field.
+resultMap(plan.output.head.exprId.id)
+  }
+
+  /**
+   * Split the plan for a scalar subquery into the parts above the 
Aggregate node
+   * (first part of returned value) and the parts below the Aggregate 
node, including
+   * the Aggregate (second part of returned value)
+   */
+  private def splitSubquery(plan : LogicalPlan) : Tuple2[Seq[LogicalPlan], 
Aggregate] = {
+var topPart = List[LogicalPlan]()
+var bottomPart : LogicalPlan = plan
+while (! bottomPart.isInstanceOf[Aggregate]) {
+  topPart = bottomPart :: topPart
+  bottomPart = bottomPart.children.head
+}
+(topPart, bottomPart.asInstanceOf[Aggregate])
+  }
+
+  /**
+   * Rewrite the nodes above the Aggregate in a subquery so that they 
generate an
+   * auxiliary column "isFiltered"
+   * @param subqueryPlan plan before rewrite
+   * @param filteredId expression ID for the "isFiltered" column
+   */
+  private def addIsFiltered(subqueryPlan : LogicalPlan, filteredId : 
ExprId) : LogicalPlan = {
+val 

[GitHub] spark issue #13371: [SPARK-15639][SQL] Try to push down filter at RowGroups ...

2016-06-09 Thread yhuai
Github user yhuai commented on the issue:

https://github.com/apache/spark/pull/13371
  
@viirya I took a look at parquet's code. Seems parquet only evaluate row 
group level filters when generating splits 
(https://github.com/apache/parquet-mr/blob/apache-parquet-1.7.0/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetInputFormat.java#L673).
 With FileSourceStrategy in Spark, I am not sure we will actually evaluate 
filter unneeded row groups as expected. Can you take a look? Also, it will be 
great if you can have a test to make sure that we actually can skip unneeded 
row groups. This test can be created as follows.

1. We first write a parquet file containing multiple row groups. Also, 
let's that there is a column `c` and those row groups have disjoint ranges of 
`c`'s values.
2. We write a query having a filter on `c` and we make sure that this query 
only need a subset of row groups.
3. We verify that we only create splits for the needed row groups.


---
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 issue #13591: [Minor] Replace all occurrences of None: Option[X] with ...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13591
  
**[Test build #60272 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60272/consoleFull)**
 for PR 13591 at commit 
[`11988c9`](https://github.com/apache/spark/commit/11988c94b9983af8926964b1c21f72bf8efaaa87).


---
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 #13591: [Minor] Replace all occurrences of None: Option[X...

2016-06-09 Thread techaddict
GitHub user techaddict opened a pull request:

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

[Minor] Replace all occurrences of None: Option[X] with Option.empty[X]

## What changes were proposed in this pull request?
Replace all occurrences of None: Option[X] with Option.empty[X]

## How was this patch tested?
Exisiting Tests

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

$ git pull https://github.com/techaddict/spark minor-7

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

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

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

This closes #13591


commit 11988c94b9983af8926964b1c21f72bf8efaaa87
Author: Sandeep Singh 
Date:   2016-06-10T04:12:10Z

[Minor] Replace all occurrences of None: Option[X] with Option.empty[X]




---
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 issue #13569: [SPARK-15791] Fix NPE in ScalarSubquery

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13569
  
**[Test build #3073 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3073/consoleFull)**
 for PR 13569 at commit 
[`7a94dc1`](https://github.com/apache/spark/commit/7a94dc143f14345466fcc4e344d9b40b3ded178d).
 * This patch passes all tests.
 * This patch **does not merge 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 issue #13586: [SPARK-15860] Metrics for codegen size and perf

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13586
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #13586: [SPARK-15860] Metrics for codegen size and perf

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13586
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60267/
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



[GitHub] spark issue #13586: [SPARK-15860] Metrics for codegen size and perf

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13586
  
**[Test build #60267 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60267/consoleFull)**
 for PR 13586 at commit 
[`ddd36cf`](https://github.com/apache/spark/commit/ddd36cfa05de2e6c9cbfbd31b70d66b8c32660a8).
 * This patch **fails Spark unit 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 issue #13572: [SPARK-15838] [SQL] Better Error Message When Having Dat...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13572
  
**[Test build #60271 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60271/consoleFull)**
 for PR 13572 at commit 
[`40a9bb6`](https://github.com/apache/spark/commit/40a9bb6df3e688102535adf7dc1af1145fccea5c).


---
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 issue #13541: [SPARK-15803][PYSPARK] Support with statement syntax for...

2016-06-09 Thread zjffdu
Github user zjffdu commented on the issue:

https://github.com/apache/spark/pull/13541
  
It's weird that I can pass the python test in local, but fails on github 
jenkins. And the error message not clear to me. @davies  Could you help take a 
look at it ? Thanks. Here's the output from jenkins.

```
Running PySpark tests

Running PySpark tests. Output is in 
/home/jenkins/workspace/SparkPullRequestBuilder@2/python/unit-tests.log
Will test against the following Python executables: ['python2.6', 
'python3.4', 'pypy']
Will test the following Python modules: ['pyspark-sql', 'pyspark-mllib', 
'pyspark-ml']

/home/jenkins/workspace/SparkPullRequestBuilder@2/python/pyspark/sql/context.py:457:
 DeprecationWarning: HiveContext is deprecated in Spark 2.0.0. Please use 
SparkSession.builder.enableHiveSupport().getOrCreate() instead.
  DeprecationWarning)

Running tests...
--
...Using Spark's default log4j profile: 
org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel).
Traceback (most recent call last):
  File "/usr/lib64/python2.6/runpy.py", line 122, in _run_module_as_main
"__main__", fname, loader, pkg_name)
  File "/usr/lib64/python2.6/runpy.py", line 34, in _run_code
exec code in run_globals
  File 
"/home/jenkins/workspace/SparkPullRequestBuilder@2/python/pyspark/sql/tests.py",
 line 1725, in 

unittest.main(testRunner=xmlrunner.XMLTestRunner(output='target/test-reports'))
  File "/usr/lib/python2.6/site-packages/unittest2/main.py", line 88, in 
__init__
self.runTests()
  File "/usr/lib/python2.6/site-packages/unittest2/main.py", line 237, in 
runTests
self.result = testRunner.run(self.test)
  File "/usr/lib/python2.6/site-packages/xmlrunner/runner.py", line 59, in 
run
test(result)
  File "/usr/lib/python2.6/site-packages/unittest2/suite.py", line 87, in 
__call__
return self.run(*args, **kwds)
  File "/usr/lib/python2.6/site-packages/unittest2/suite.py", line 126, in 
run
test(result)
  File "/usr/lib/python2.6/site-packages/unittest2/suite.py", line 87, in 
__call__
return self.run(*args, **kwds)
  File "/usr/lib/python2.6/site-packages/unittest2/suite.py", line 118, in 
run
self._handleClassSetUp(test, result)
  File "/usr/lib/python2.6/site-packages/unittest2/suite.py", line 173, in 
_handleClassSetUp
self._addClassOrModuleLevelException(result, e, errorName)
  File "/usr/lib/python2.6/site-packages/unittest2/suite.py", line 215, in 
_addClassOrModuleLevelException
result.addError(error, sys.exc_info())
  File "/usr/lib/python2.6/site-packages/xmlrunner/result.py", line 232, in 
addError
self._save_output_data()
  File "/usr/lib/python2.6/site-packages/xmlrunner/result.py", line 189, in 
_save_output_data
self._stdout_data = sys.stdout.getvalue()
AttributeError: 'file' object has no attribute 'getvalue'

Had test failures in pyspark.sql.tests with python2.6; see logs.
[error] running 
/home/jenkins/workspace/SparkPullRequestBuilder@2/python/run-tests 
--modules=pyspark-sql,pyspark-mllib,pyspark-ml --parallelism=4 ; received 
return code 255
Attempting to post to Github...
 > Post successful.
Build step 'Execute shell' marked build as failure
```


---
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 issue #13323: [SPARK-15555] [Mesos] Driver with --supervise option can...

2016-06-09 Thread tnachen
Github user tnachen commented on the issue:

https://github.com/apache/spark/pull/13323
  
Thanks @devaraj-kavali, this LGTM. @andrewor14 can you take a look?


---
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 issue #13541: [SPARK-15803][PYSPARK] Support with statement syntax for...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13541
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #13541: [SPARK-15803][PYSPARK] Support with statement syntax for...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13541
  
**[Test build #60270 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60270/consoleFull)**
 for PR 13541 at commit 
[`7cce183`](https://github.com/apache/spark/commit/7cce183de2a43bbd338d81919c75d9bbba8a751f).
 * This patch **fails PySpark unit 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 issue #13541: [SPARK-15803][PYSPARK] Support with statement syntax for...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13541
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60270/
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



[GitHub] spark pull request #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to ...

2016-06-09 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/13342#discussion_r66558301
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -232,7 +234,7 @@ final class DataFrameWriter private[sql](df: DataFrame) 
{
* @since 1.4.0
*/
   @scala.annotation.varargs
-  def partitionBy(colNames: String*): DataFrameWriter = {
+  def partitionBy(colNames: String*): this.type = {
--- End diff --

sorry i didnt notice this earlier but why is this suddenly `this.type` and 
not `DataFrameWriter[T]` like other methods. Alternatively, why are we using 
`this.type` everywhere?



---
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 #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

2016-06-09 Thread frreiss
Github user frreiss commented on a diff in the pull request:

https://github.com/apache/spark/pull/13155#discussion_r66558119
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends 
Rule[LogicalPlan] {
   }
 
   /**
+   * Statically evaluate an expression containing zero or more 
placeholders, given a set
+   * of bindings for placeholder values.
+   */
+  private def evalExpr(expr : Expression, bindings : Map[Long, 
Option[Any]]) : Option[Any] = {
+val rewrittenExpr = expr transform {
+  case r @ AttributeReference(_, dataType, _, _) =>
+bindings(r.exprId.id) match {
+  case Some(v) => Literal.create(v, dataType)
+  case None => Literal.default(NullType)
+}
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate an expression containing one or more aggregates 
on an empty input.
+   */
+  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
+// AggregateExpressions are Unevaluable, so we need to replace all 
aggregates
+// in the expression with the value they would return for zero input 
tuples.
+val rewrittenExpr = expr transform {
+  case a @ AggregateExpression(aggFunc, _, _, resultId) =>
+aggFunc.defaultResult.getOrElse(Literal.default(NullType))
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate a scalar subquery on an empty input.
+   *
+   * WARNING: This method only covers subqueries that pass the 
checks under
+   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the 
checks in
+   * CheckAnalysis become less restrictive, this method will need to 
change.
+   */
+  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
+// Inputs to this method will start with a chain of zero or more 
SubqueryAlias
+// and Project operators, followed by an optional Filter, followed by 
an
+// Aggregate. Traverse the operators recursively.
+def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
+  lp match {
+case SubqueryAlias(_, child) => evalPlan(child)
+case Filter(condition, child) =>
+  val bindings = evalPlan(child)
+  if (bindings.size == 0) bindings
--- End diff --

Fixed in my local copy.


---
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 #13155: [SPARK-15370] [SQL] Update RewriteCorrelatedScala...

2016-06-09 Thread frreiss
Github user frreiss commented on a diff in the pull request:

https://github.com/apache/spark/pull/13155#discussion_r66558125
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1695,16 +1695,176 @@ object RewriteCorrelatedScalarSubquery extends 
Rule[LogicalPlan] {
   }
 
   /**
+   * Statically evaluate an expression containing zero or more 
placeholders, given a set
+   * of bindings for placeholder values.
+   */
+  private def evalExpr(expr : Expression, bindings : Map[Long, 
Option[Any]]) : Option[Any] = {
+val rewrittenExpr = expr transform {
+  case r @ AttributeReference(_, dataType, _, _) =>
+bindings(r.exprId.id) match {
+  case Some(v) => Literal.create(v, dataType)
+  case None => Literal.default(NullType)
+}
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate an expression containing one or more aggregates 
on an empty input.
+   */
+  private def evalAggOnZeroTups(expr : Expression) : Option[Any] = {
+// AggregateExpressions are Unevaluable, so we need to replace all 
aggregates
+// in the expression with the value they would return for zero input 
tuples.
+val rewrittenExpr = expr transform {
+  case a @ AggregateExpression(aggFunc, _, _, resultId) =>
+aggFunc.defaultResult.getOrElse(Literal.default(NullType))
+}
+Option(rewrittenExpr.eval())
+  }
+
+  /**
+   * Statically evaluate a scalar subquery on an empty input.
+   *
+   * WARNING: This method only covers subqueries that pass the 
checks under
+   * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the 
checks in
+   * CheckAnalysis become less restrictive, this method will need to 
change.
+   */
+  private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = {
+// Inputs to this method will start with a chain of zero or more 
SubqueryAlias
+// and Project operators, followed by an optional Filter, followed by 
an
+// Aggregate. Traverse the operators recursively.
+def evalPlan(lp : LogicalPlan) : Map[Long, Option[Any]] = {
+  lp match {
+case SubqueryAlias(_, child) => evalPlan(child)
+case Filter(condition, child) =>
+  val bindings = evalPlan(child)
+  if (bindings.size == 0) bindings
+  else {
+val exprResult = evalExpr(condition, bindings).getOrElse(false)
+  .asInstanceOf[Boolean]
+if (exprResult) bindings else Map()
--- End diff --

Fixed in my local copy.


---
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 issue #13541: [SPARK-15803][PYSPARK] Support with statement syntax for...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13541
  
**[Test build #60270 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60270/consoleFull)**
 for PR 13541 at commit 
[`7cce183`](https://github.com/apache/spark/commit/7cce183de2a43bbd338d81919c75d9bbba8a751f).


---
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 issue #13576: [SPARK-15840][SQL] Add two missing options in documentat...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13576
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60265/
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 issue #13576: [SPARK-15840][SQL] Add two missing options in documentat...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13576
  
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 issue #13576: [SPARK-15840][SQL] Add two missing options in documentat...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13576
  
**[Test build #60265 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60265/consoleFull)**
 for PR 13576 at commit 
[`994b0a2`](https://github.com/apache/spark/commit/994b0a27a8f6749d2f4d54a5f1d0d716ac04d9de).
 * 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 issue #13590: SPARK-15858: Fix calculating error by tree stack over fl...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13590
  
Can one of the admins verify this patch?


---
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 #13590: SPARK-15858: Fix calculating error by tree stack ...

2016-06-09 Thread mhmoudr
GitHub user mhmoudr opened a pull request:

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

SPARK-15858: Fix calculating error by tree stack over flow problem an…

## What changes were proposed in this pull request?

Improving evaluateEachIteration function in mllib as it fails when trying 
to calculate error by tree for a model that has more than 500 trees

## How was this patch tested?

the batch tested on productions data set (2K rows x 2K features) training a 
gradient boosted model without validation with 1000 maxIteration settings, then 
trying to produce the error by tree, the new patch was able to perform the 
calculation within 30 seconds, while previously it was take hours then fail.


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

$ git pull https://github.com/mhmoudr/spark SPARK-15858

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

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

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

This closes #13590


commit a72dbdd0d60eec346d66bb9ab728643c9d4bad0f
Author: Mahmoud Rawas 
Date:   2016-06-10T01:27:21Z

SPARK-15858: Fix calculating error by tree stack over flow problem and over 
memory allocation issue for a model that have 2000+ trees.




---
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 issue #13572: [SPARK-15838] [SQL] CACHE TABLE AS SELECT should not rep...

2016-06-09 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/13572
  
Yeah, in Spark 1.6, we also silently drop the temporary table if the names 
are the same. Let me remove the related changes and update the title and JIRA


---
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 issue #13571: [SPARK-15369][WIP][RFC][PySpark][SQL] Expose potential t...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13571
  
**[Test build #60269 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60269/consoleFull)**
 for PR 13571 at commit 
[`8c056a8`](https://github.com/apache/spark/commit/8c056a8c4db2fd33623f4c0a8efd1b2f53b2147f).


---
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 issue #13588: SPARK-15858: Fix calculating error by tree stack over fl...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13588
  
Can one of the admins verify this patch?


---
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 issue #13589: [SPARK-15822][SPARK-15825][SQL] Fix SMJ Segfault/Invalid...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13589
  
**[Test build #60268 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60268/consoleFull)**
 for PR 13589 at commit 
[`995c86a`](https://github.com/apache/spark/commit/995c86a09ae7eaf013abb618b65749b1e0f71eca).


---
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 issue #13576: [SPARK-15840][SQL] Add two missing options in documentat...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13576
  
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 issue #13576: [SPARK-15840][SQL] Add two missing options in documentat...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13576
  
**[Test build #60264 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60264/consoleFull)**
 for PR 13576 at commit 
[`ada3858`](https://github.com/apache/spark/commit/ada38581e1d5ad0a839afe07f491718ced00efe3).
 * 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 issue #13576: [SPARK-15840][SQL] Add two missing options in documentat...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13576
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60264/
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 #13589: [SPARK-15822][SPARK-15825][SQL] Fix SMJ Segfault/...

2016-06-09 Thread hvanhovell
GitHub user hvanhovell opened a pull request:

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

[SPARK-15822][SPARK-15825][SQL] Fix SMJ Segfault/Invalid results

## What changes were proposed in this pull request?
I'll add desc later


## How was this patch tested?
TBD



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

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

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

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

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

This closes #13589


commit 995c86a09ae7eaf013abb618b65749b1e0f71eca
Author: Herman van Hovell 
Date:   2016-06-10T02:59:31Z

fix input row for comparing struct values




---
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 #13588: SPARK-15858: Fix calculating error by tree stack ...

2016-06-09 Thread mhmoudr
GitHub user mhmoudr opened a pull request:

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

SPARK-15858: Fix calculating error by tree stack over flow problem an…

## What changes were proposed in this pull request?

Improving evaluateEachIteration function in mllib as it fails when trying 
to calculate error by tree for a model that has more than 500 trees 

## How was this patch tested?

the batch tested on productions data set (2K rows x 2K features) training a 
gradient boosted model without validation with 1000 maxIteration settings, then 
trying to produce the error by tree, the new patch was able to perform the 
calculation within 30 seconds, while previously it was take hours then fail.



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

$ git pull https://github.com/mhmoudr/spark SPARK-15858.1.6

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

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

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

This closes #13588


commit 4726937bacd6ee43dd12b27e1746bc708e99c6da
Author: Mahmoud Rawas 
Date:   2016-06-10T01:27:21Z

SPARK-15858: Fix calculating error by tree stack over flow problem and over 
memory allocation issue for a model that have 2000+ trees.




---
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 issue #13587: [Documentation] fixed groupby aggregation example for py...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13587
  
Can one of the admins verify this patch?


---
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 #13587: [Documentation] fixed groupby aggregation example...

2016-06-09 Thread mortada
GitHub user mortada opened a pull request:

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

[Documentation] fixed groupby aggregation example for pyspark

## What changes were proposed in this pull request?

fixing documentation for the groupby/agg example in python

## How was this patch tested?

the existing example in the documentation dose not contain valid syntax 
(missing parenthesis) and is not using `Column` in the expression for `agg()`

after the fix here's how I tested it:

```
In [1]: from pyspark.sql import Row

In [2]: import pyspark.sql.functions as func

In [3]: %cpaste
Pasting code; enter '--' alone on the line to stop or use Ctrl-D.
:records = [{'age': 19, 'department': 1, 'expense': 100},
: {'age': 20, 'department': 1, 'expense': 200},
: {'age': 21, 'department': 2, 'expense': 300},
: {'age': 22, 'department': 2, 'expense': 300},
: {'age': 23, 'department': 3, 'expense': 300}]
:--

In [4]: df = sqlContext.createDataFrame([Row(**d) for d in records])

In [5]: df.groupBy("department").agg(df["department"], func.max("age"), 
func.sum("expense")).show()

+--+--+++
|department|department|max(age)|sum(expense)|
+--+--+++
| 1| 1|  20| 300|
| 2| 2|  22| 600|
| 3| 3|  23| 300|
+--+--+++


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

$ git pull https://github.com/mortada/spark groupby_agg_doc_fix

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

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

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

This closes #13587


commit 783415c9424cb6db1333aa5bc3ccd3cd1b227204
Author: Mortada Mehyar 
Date:   2016-06-10T02:34:16Z

[Documentation] fixed groupby aggregation example for pyspark




---
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 issue #13576: [SPARK-15840][SQL] Add two missing options in documentat...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13576
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60259/
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 issue #13576: [SPARK-15840][SQL] Add two missing options in documentat...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13576
  
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 issue #13586: [SPARK-15860] Metrics for codegen size and perf

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13586
  
**[Test build #60267 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60267/consoleFull)**
 for PR 13586 at commit 
[`ddd36cf`](https://github.com/apache/spark/commit/ddd36cfa05de2e6c9cbfbd31b70d66b8c32660a8).


---
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 issue #13576: [SPARK-15840][SQL] Add two missing options in documentat...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13576
  
**[Test build #60259 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60259/consoleFull)**
 for PR 13576 at commit 
[`573ba4e`](https://github.com/apache/spark/commit/573ba4e4de7f0e32f3851f03a2d989b42008757e).
 * 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 #13586: [SPARK-15860] Metrics for codegen size and perf

2016-06-09 Thread ericl
GitHub user ericl opened a pull request:

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

[SPARK-15860] Metrics for codegen size and perf

## What changes were proposed in this pull request?

Adds codahale metrics for the codegen source text size and how long it 
takes to compile. The size is particularly interesting, since the JVM does have 
hard limits on how large methods can get.

To simplify, I added the metrics under a statically-initialized source that 
is always registered with SparkEnv.

## How was this patch tested?

Unit tests


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

$ git pull https://github.com/ericl/spark spark-15860

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

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

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

This closes #13586


commit ddd36cfa05de2e6c9cbfbd31b70d66b8c32660a8
Author: Eric Liang 
Date:   2016-06-10T02:27:18Z

Thu Jun  9 19:27:18 PDT 2016




---
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 issue #13585: [SPARK-15859][SQL] Optimize the partition pruning within...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13585
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #13585: [SPARK-15859][SQL] Optimize the partition pruning within...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13585
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60263/
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



[GitHub] spark issue #13585: [SPARK-15859][SQL] Optimize the partition pruning within...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13585
  
**[Test build #60263 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60263/consoleFull)**
 for PR 13585 at commit 
[`08519f2`](https://github.com/apache/spark/commit/08519f2e7a3222cb791e6ce1b8af0c132ff16b29).
 * This patch **fails Spark unit 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 issue #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to allow t...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13342
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60260/
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 issue #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to allow t...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13342
  
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 issue #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to allow t...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13342
  
**[Test build #60260 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60260/consoleFull)**
 for PR 13342 at commit 
[`2f2e9b3`](https://github.com/apache/spark/commit/2f2e9b3f322a7b49de81b9ad15e98f50b6ff58b4).
 * 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 issue #13558: [SPARK-15820][PySpark][SQL]Add Catalog.refreshTable into...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13558
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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 issue #13558: [SPARK-15820][PySpark][SQL]Add Catalog.refreshTable into...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13558
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60266/
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



[GitHub] spark issue #13558: [SPARK-15820][PySpark][SQL]Add Catalog.refreshTable into...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13558
  
**[Test build #60266 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60266/consoleFull)**
 for PR 13558 at commit 
[`8d87c0f`](https://github.com/apache/spark/commit/8d87c0f2bd9140928915f835fd7d21b178422c69).
 * This patch **fails PySpark unit 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 issue #13569: [SPARK-15791] Fix NPE in ScalarSubquery

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13569
  
**[Test build #3073 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3073/consoleFull)**
 for PR 13569 at commit 
[`7a94dc1`](https://github.com/apache/spark/commit/7a94dc143f14345466fcc4e344d9b40b3ded178d).


---
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 issue #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to allow t...

2016-06-09 Thread tdas
Github user tdas commented on the issue:

https://github.com/apache/spark/pull/13342
  
LGTM, except a few minor nits.


---
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 issue #13580: Revert "[SPARK-14485][CORE] ignore task finished for exe...

2016-06-09 Thread rxin
Github user rxin commented on the issue:

https://github.com/apache/spark/pull/13580
  
LGTM -- seems like the lower risk option here, especially for 2.0.



---
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 #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to ...

2016-06-09 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/13342#discussion_r66552040
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala 
---
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.streaming.ContinuousQuery
+
+/**
+ * :: Experimental ::
+ * A class to consume data generated by a [[ContinuousQuery]]. Typically 
this is used to send the
+ * generated data to external systems. Each partition will use a new 
deserialized instance, so you
+ * usually should do all the initialization (e.g. opening a connection or 
initiating a transaction)
+ * in the open method
--- End diff --

nit: missing period.


---
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 #13582: [SPARK-15850][SQL] Remove function grouping in Sp...

2016-06-09 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 #13577: [Minor][Doc] Improve SQLContext Documentation and...

2016-06-09 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 issue #13582: [SPARK-15850][SQL] Remove function grouping in SparkSess...

2016-06-09 Thread hvanhovell
Github user hvanhovell commented on the issue:

https://github.com/apache/spark/pull/13582
  
LGTM - merging to master/2.0


---
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 issue #13558: [SPARK-15820][PySpark][SQL]Add Catalog.refreshTable into...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13558
  
**[Test build #60266 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60266/consoleFull)**
 for PR 13558 at commit 
[`8d87c0f`](https://github.com/apache/spark/commit/8d87c0f2bd9140928915f835fd7d21b178422c69).


---
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 #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to ...

2016-06-09 Thread tdas
Github user tdas commented on a diff in the pull request:

https://github.com/apache/spark/pull/13342#discussion_r66551851
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala 
---
@@ -68,9 +71,9 @@ import org.apache.spark.sql.streaming.ContinuousQuery
 abstract class ForeachWriter[T] extends Serializable {
 
   /**
-   * Called when starting to process one partition of new data in the 
executor side. `version` is
-   * for data deduplication. When recovering from a failure, some data may 
be processed twice. But
-   * it's guarantee that they will be opened with the same "version".
+   * Called when starting to process one partition of new data in the 
executor. The `version` is
+   * for data deduplication when there are failures. When recovering from 
a failure, some data may
+   * be generated multiple times but they will always have the same 
version.
*
* If this method finds this is a partition from a duplicated data set, 
it can return `false` to
--- End diff --

nit: finds using the `partitionId` and `version` that this partition has 
already been processed, it can return ...


---
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 issue #13558: [SPARK-15820][PySpark][SQL]Add Catalog.refreshTable into...

2016-06-09 Thread WeichenXu123
Github user WeichenXu123 commented on the issue:

https://github.com/apache/spark/pull/13558
  
Jenkins, test this please.


---
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 issue #13371: [SPARK-15639][SQL] Try to push down filter at RowGroups ...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13371
  
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 issue #13371: [SPARK-15639][SQL] Try to push down filter at RowGroups ...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13371
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60256/
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 issue #13371: [SPARK-15639][SQL] Try to push down filter at RowGroups ...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13371
  
**[Test build #60256 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60256/consoleFull)**
 for PR 13371 at commit 
[`077f7f8`](https://github.com/apache/spark/commit/077f7f8813a76d38c8a6d898ec54e401c91b6014).
 * 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 issue #13584: [SPARK-15509][ML][SparkR] R MLlib algorithms should supp...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13584
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60261/
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 issue #13584: [SPARK-15509][ML][SparkR] R MLlib algorithms should supp...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13584
  
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 issue #13584: [SPARK-15509][ML][SparkR] R MLlib algorithms should supp...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13584
  
**[Test build #60261 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60261/consoleFull)**
 for PR 13584 at commit 
[`43b2f8c`](https://github.com/apache/spark/commit/43b2f8c5fb9e0d74579b948b1d52cad4faa76b66).
 * 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 #13583: [SPARK-15853][SQL]HDFSMetadataLog.get should clos...

2016-06-09 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 issue #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to allow t...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13342
  
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 issue #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to allow t...

2016-06-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13342
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60254/
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 issue #13342: [SPARK-15593][SQL]Add DataFrameWriter.foreach to allow t...

2016-06-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13342
  
**[Test build #60254 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60254/consoleFull)**
 for PR 13342 at commit 
[`42286b0`](https://github.com/apache/spark/commit/42286b0a381cce2461d966d2dbad38b376a73b42).
 * 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



  1   2   3   4   5   >