[GitHub] spark pull request #20211: [SPARK-23011][PYTHON][SQL] Prepend missing groupi...

2018-01-09 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/20211#discussion_r160599447
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala
 ---
@@ -80,27 +84,77 @@ case class FlatMapGroupsInPandasExec(
 val sessionLocalTimeZone = conf.sessionLocalTimeZone
 val pandasRespectSessionTimeZone = conf.pandasRespectSessionTimeZone
 
-inputRDD.mapPartitionsInternal { iter =>
-  val grouped = if (groupingAttributes.isEmpty) {
-Iterator(iter)
-  } else {
+if (additionalGroupingAttributes.isEmpty) {
+  // Fast path if additional grouping attributes is empty
+
+  inputRDD.mapPartitionsInternal { iter =>
+val grouped = if (groupingAttributes.isEmpty) {
+  Iterator(iter)
+} else {
+  val groupedIter = GroupedIterator(iter, groupingAttributes, 
child.output)
+  val dropGrouping =
+
UnsafeProjection.create(child.output.drop(groupingAttributes.length), 
child.output)
+  groupedIter.map {
+case (_, groupedRowIter) => groupedRowIter.map(dropGrouping)
+  }
+}
+
+val context = TaskContext.get()
+
+val columnarBatchIter = new ArrowPythonRunner(
+  chainedFunc, bufferSize, reuseWorker,
+  PythonEvalType.SQL_PANDAS_GROUP_MAP_UDF, argOffsets, schema,
+  sessionLocalTimeZone, pandasRespectSessionTimeZone)
+  .compute(grouped, context.partitionId(), context)
+
+columnarBatchIter
+  .flatMap(_.rowIterator.asScala)
+  .map(UnsafeProjection.create(output, output))
+  }
+} else {
+  // If additionGroupingAttributes is not empty, join the grouping 
attributes with
+  // the udf output to get the final result
+
+  inputRDD.mapPartitionsInternal { iter =>
+assert(groupingAttributes.nonEmpty)
+
 val groupedIter = GroupedIterator(iter, groupingAttributes, 
child.output)
+
+val context = TaskContext.get()
+
+val queue = HybridRowQueue(context.taskMemoryManager(),
+  new File(Utils.getLocalDir(SparkEnv.get.conf)), 
additionalGroupingAttributes.length)
+context.addTaskCompletionListener { _ =>
+  queue.close()
+}
+val additionalGroupingProj = UnsafeProjection.create(
+  additionalGroupingAttributes, groupingAttributes)
 val dropGrouping =
   
UnsafeProjection.create(child.output.drop(groupingAttributes.length), 
child.output)
-groupedIter.map {
-  case (_, groupedRowIter) => groupedRowIter.map(dropGrouping)
+val grouped = groupedIter.map {
+  case (k, groupedRowIter) =>
+val additionalGrouping = additionalGroupingProj(k)
+queue.add(additionalGrouping)
+(additionalGrouping, groupedRowIter.map(dropGrouping))
--- End diff --

We can return only `groupedRowIter.map(dropGrouping)`.


---

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



[GitHub] spark pull request #20211: [SPARK-23011][PYTHON][SQL] Prepend missing groupi...

2018-01-09 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/20211#discussion_r160605967
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala ---
@@ -457,13 +458,26 @@ class RelationalGroupedDataset protected[sql](
 
 val groupingNamedExpressions = groupingExprs.map {
   case ne: NamedExpression => ne
-  case other => Alias(other, other.toString)()
+  case other => Alias(other, toPrettySQL(other))()
 }
 val groupingAttributes = groupingNamedExpressions.map(_.toAttribute)
 val child = df.logicalPlan
 val project = Project(groupingNamedExpressions ++ child.output, child)
-val output = expr.dataType.asInstanceOf[StructType].toAttributes
-val plan = FlatMapGroupsInPandas(groupingAttributes, expr, output, 
project)
+val udfOutput: Seq[Attribute] = 
expr.dataType.asInstanceOf[StructType].toAttributes
+val additionalGroupingAttributes = mutable.ArrayBuffer[Attribute]()
+
+for (attribute <- groupingAttributes) {
+  if (!udfOutput.map(_.name).contains(attribute.name)) {
--- End diff --

I'm wondering whether we should decide the additional grouping attributes 
by only their names?

For example from tests:

```python
result3 = df.groupby('id', 'v').apply(foo).sort('id', 'v').toPandas()
```

The column `v` in `result3` is not the actual grouping value, which is 
overwritten by the returned value from the UDF because the returned column name 
contains the name. I'm not sure it is the desired behavior.

  


---

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



[GitHub] spark pull request #20188: [SPARK-22993][ML] Clarify HasCheckpointInterval p...

2018-01-09 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark issue #20188: [SPARK-22993][ML] Clarify HasCheckpointInterval param do...

2018-01-09 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/20188
  
merged to master/2.3


---

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



[GitHub] spark pull request #20205: [SPARK-16060][SQL][follow-up] add a wrapper solut...

2018-01-09 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark issue #20215: [SPARK-23001] [SQL] Fix NullPointerException when DESC a...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20215
  
**[Test build #85909 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85909/testReport)**
 for PR 20215 at commit 
[`2fe541f`](https://github.com/apache/spark/commit/2fe541f4310e3f22bf637b7d102eeec23627386e).


---

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



[GitHub] spark issue #20205: [SPARK-16060][SQL][follow-up] add a wrapper solution for...

2018-01-09 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/20205
  
thanks, merging to master/2.3!


---

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



[GitHub] spark pull request #20174: [SPARK-22951][SQL] fix aggregation after dropDupl...

2018-01-09 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/20174#discussion_r160602821
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala ---
@@ -19,6 +19,8 @@ package org.apache.spark.sql
 
 import scala.util.Random
 
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.aggregate.Percentile
--- End diff --

Remove unused imports?


---

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



[GitHub] spark issue #18991: [SPARK-21783][SQL] Turn on ORC filter push-down by defau...

2018-01-09 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/18991
  
I'd expect orc has same test coverage as parquet, is it true?


---

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



[GitHub] spark pull request #20179: [SPARK-22982] Remove unsafe asynchronous close() ...

2018-01-09 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark issue #20179: [SPARK-22982] Remove unsafe asynchronous close() call fr...

2018-01-09 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/20179
  
thanks, merging to master/2.3!


---

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



[GitHub] spark issue #20174: [SPARK-22951][SQL] fix aggregation after dropDuplicates ...

2018-01-09 Thread liancheng
Github user liancheng commented on the issue:

https://github.com/apache/spark/pull/20174
  
@liufengdb I wrote a summary according to our offline discussion to explain 
the subtle change made in this PR. Please feel free to use it in the PR 
description if it looks good to you :)



Same as all the other SQL query engines, Spark SQL supports both grouping 
aggregation, e.g.:

```sql
SELECT count(*) FROM a_table GROUP BY key
```

and global aggregation, e.g.:

```sql
SELECT count(*) FROM a_table
```

Essentially, global aggregation is a special case of grouping aggregation 
where the whole table is treated as a single group. A key difference, though, 
exists in the case where a group contains zero rows:

- Grouping aggregation

  ```sql
  SELECT count(*) AS c FROM range(3) WHERE id < 0 GROUP BY id
  -- +---+
  -- | c |
  -- +---+
  -- +---+
  ```

  The above query returns zero rows.

- Global aggregation

  ```sql
  SELECT count(*) AS c FROM range(3) WHERE id < 0
  -- +---+
  -- | c |
  -- +---+
  -- | 0 |
  -- +---+
  ```

  The above query returns one row. To be more specific, global aggregation 
with zero input rows always return a single row with the initial aggregation 
state as the output.

To tell whether an `Aggregate` operator `A` is a global aggregation or not, 
Spark SQL simply checks the number of grouping keys, and `A` is a global 
aggregation if it has zero grouping keys.

However, this simple priciple drops the ball in the following case:

```scala
spark.emptyDataFrame.dropDuplicates().agg(count($"*") as "c").show()
// +---+
// | c |
// +---+
// | 1 |
// +---+
```

The reason is that:

1.  `df.dropDuplicates()` is roughly translated into something equivalent 
to:

```
val allColumns = df.columns.map { col }
df.groupBy(allColumns: _*).agg(allColumns.head, allColumns.tail: _*)
```

This translation is implemented in the rule 
`ReplaceDeduplicateWithAggregate`.

2.  `spark.emptyDataFrame` contains zero columns and zero rows.

Therefore, rule `ReplaceDeduplicateWithAggregate` translates 
`spark.emptyDataFrame.dropDuplicates()` into something equivalent to:

```scala
spark.emptyDataFrame.groupBy().agg(Map.empty[String, String])
```

Which confuses Spark SQL and gets recognized as a global aggregation 
because the aggregate operator contains no grouping keys. As a result, Spark 
SQL allocates a single row filled by the initial aggregation state and uses it 
as the output, and returns a wrong result.

To fix this issue, this PR tweaks `ReplaceDeduplicateWithAggregate` by 
appending a literal `1` to the grouping key list of the resulting `Aggregate` 
operator when the input plan contains zero output columns. In this way, 
`spark.emptyDataFrame.dropDuplicates()` is now translated into:

```scala
spark.emptyDataFrame.dropDuplicates()
=> spark.emptyDataFrame.groupBy(lit(1)).agg(Map.empty[String, String])
```

Which is now properly treated as a grouping aggregation and returns the 
correct answer.



---

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



[GitHub] spark issue #20205: [SPARK-16060][SQL][follow-up] add a wrapper solution for...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20205: [SPARK-16060][SQL][follow-up] add a wrapper solution for...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20205
  
Merged build finished. Test PASSed.


---

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



[GitHub] spark issue #20205: [SPARK-16060][SQL][follow-up] add a wrapper solution for...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20205
  
**[Test build #85901 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85901/testReport)**
 for PR 20205 at commit 
[`b78c6ec`](https://github.com/apache/spark/commit/b78c6ec6e7579f861ddc95ed12de647992de8d0e).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20174: [SPARK-22951][SQL] fix aggregation after dropDuplicates ...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20174
  
**[Test build #85908 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85908/testReport)**
 for PR 20174 at commit 
[`8eec4ce`](https://github.com/apache/spark/commit/8eec4ce6ddb2cf7ce6eafb9bd24ed7be845a6799).


---

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



[GitHub] spark issue #20174: [SPARK-22951][SQL] deduplicates on empty data frames sho...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20174
  
**[Test build #85907 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85907/testReport)**
 for PR 20174 at commit 
[`61ec4db`](https://github.com/apache/spark/commit/61ec4db9e1ee06b7a4ae99f5414f77290e45fc99).


---

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



[GitHub] spark issue #20207: [SPARK-23000] [TEST-HADOOP2.6] Fix Flaky test suite Data...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20207
  
Merged build finished. Test PASSed.


---

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



[GitHub] spark issue #20207: [SPARK-23000] [TEST-HADOOP2.6] Fix Flaky test suite Data...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20214
  
**[Test build #85905 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85905/testReport)**
 for PR 20214 at commit 
[`e393c63`](https://github.com/apache/spark/commit/e393c637b10353a1efaa799a5779ae319bb070f0).


---

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



[GitHub] spark issue #20174: [SPARK-22951][SQL] deduplicates on empty data frames sho...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20174
  
**[Test build #85906 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85906/testReport)**
 for PR 20174 at commit 
[`37a4a24`](https://github.com/apache/spark/commit/37a4a24a97c770f593ae7ff38ac2dc458a4ba686).


---

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



[GitHub] spark issue #20207: [SPARK-23000] [TEST-HADOOP2.6] Fix Flaky test suite Data...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20207
  
**[Test build #85898 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85898/testReport)**
 for PR 20207 at commit 
[`9039817`](https://github.com/apache/spark/commit/9039817d516f2cfb68f9caa41374098780fde3ca).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20013: [SPARK-20657][core] Speed up rendering of the stages pag...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20013: [SPARK-20657][core] Speed up rendering of the stages pag...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20013: [SPARK-20657][core] Speed up rendering of the stages pag...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20013
  
**[Test build #85899 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85899/testReport)**
 for PR 20013 at commit 
[`3d66505`](https://github.com/apache/spark/commit/3d6650589223cfd93c4ced9fb25246bcd88ca899).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20191: [SPARK-22997] Add additional defenses against use of fre...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20191: [SPARK-22997] Add additional defenses against use of fre...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20191
  
Merged build finished. Test PASSed.


---

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



[GitHub] spark issue #20191: [SPARK-22997] Add additional defenses against use of fre...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20191
  
**[Test build #85895 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85895/testReport)**
 for PR 20191 at commit 
[`a7f8c07`](https://github.com/apache/spark/commit/a7f8c07fb5158f39bbb6cc1f23cfb13a0d473536).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark pull request #20210: [SPARK-23009][PYTHON] Fix for non-str col names t...

2018-01-09 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark issue #20210: [SPARK-23009][PYTHON] Fix for non-str col names to creat...

2018-01-09 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/20210
  
Merged to master and branch-2.3.


---

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



[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20214
  
**[Test build #85900 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85900/testReport)**
 for PR 20214 at commit 
[`eb56aff`](https://github.com/apache/spark/commit/eb56aff74352a360d1d4b1273be23b670f3c958a).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20096: [SPARK-22908] Add kafka source and sink for continuous p...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20096: [SPARK-22908] Add kafka source and sink for continuous p...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20096: [SPARK-22908] Add kafka source and sink for continuous p...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20096
  
**[Test build #85892 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85892/testReport)**
 for PR 20096 at commit 
[`f94b53e`](https://github.com/apache/spark/commit/f94b53e3ab7e37fdcb9f34cf7d1313a4905fa341).
 * This patch **fails from timeout after a configured wait of \`250m\`**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20210: [SPARK-23009][PYTHON] Fix for non-str col names to creat...

2018-01-09 Thread ueshin
Github user ueshin commented on the issue:

https://github.com/apache/spark/pull/20210
  
LGTM.


---

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



[GitHub] spark issue #20215: [SPARK-23001] [SQL] Fix NullPointerException when DESC a...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20215: [SPARK-23001] [SQL] Fix NullPointerException when DESC a...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20215: [SPARK-23001] [SQL] Fix NullPointerException when DESC a...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20215
  
**[Test build #85903 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85903/testReport)**
 for PR 20215 at commit 
[`137fd7a`](https://github.com/apache/spark/commit/137fd7a268b23ca52ca190d5b630503e6dc73404).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20207: [SPARK-23000] [TEST-HADOOP2.6] Fix Flaky test suite Data...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20207
  
Merged build finished. Test PASSed.


---

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



[GitHub] spark issue #20096: [SPARK-22908] Add kafka source and sink for continuous p...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20096
  
**[Test build #85904 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85904/testReport)**
 for PR 20096 at commit 
[`f94b53e`](https://github.com/apache/spark/commit/f94b53e3ab7e37fdcb9f34cf7d1313a4905fa341).


---

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



[GitHub] spark issue #20207: [SPARK-23000] [TEST-HADOOP2.6] Fix Flaky test suite Data...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20207: [SPARK-23000] [TEST-HADOOP2.6] Fix Flaky test suite Data...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20207
  
**[Test build #85896 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85896/testReport)**
 for PR 20207 at commit 
[`f239b2b`](https://github.com/apache/spark/commit/f239b2b51f32addf82ad5454e86ecbb9bcbe65a2).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20096: [SPARK-22908] Add kafka source and sink for continuous p...

2018-01-09 Thread jose-torres
Github user jose-torres commented on the issue:

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


---

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



[GitHub] spark pull request #20213: [SPARK-23018][PYTHON] Fix createDataFrame from Pa...

2018-01-09 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark issue #20213: [SPARK-23018][PYTHON] Fix createDataFrame from Pandas ti...

2018-01-09 Thread ueshin
Github user ueshin commented on the issue:

https://github.com/apache/spark/pull/20213
  
Thanks! merging to master/2.3.


---

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



[GitHub] spark pull request #20213: [SPARK-23018][PYTHON] Fix createDataFrame from Pa...

2018-01-09 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20213#discussion_r160589209
  
--- Diff: python/pyspark/sql/session.py ---
@@ -459,21 +459,23 @@ def _convert_from_pandas(self, pdf, schema, timezone):
 # TODO: handle nested timestamps, such as 
ArrayType(TimestampType())?
 if isinstance(field.dataType, TimestampType):
 s = 
_check_series_convert_timestamps_tz_local(pdf[field.name], timezone)
-if not copied and s is not pdf[field.name]:
-# Copy once if the series is modified to 
prevent the original Pandas
-# DataFrame from being updated
-pdf = pdf.copy()
-copied = True
-pdf[field.name] = s
+if s is not pdf[field.name]:
+if not copied:
--- End diff --

Ah, sure. Makes sense. I rushed to read.


---

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



[GitHub] spark pull request #20213: [SPARK-23018][PYTHON] Fix createDataFrame from Pa...

2018-01-09 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/20213#discussion_r160588603
  
--- Diff: python/pyspark/sql/session.py ---
@@ -459,21 +459,23 @@ def _convert_from_pandas(self, pdf, schema, timezone):
 # TODO: handle nested timestamps, such as 
ArrayType(TimestampType())?
 if isinstance(field.dataType, TimestampType):
 s = 
_check_series_convert_timestamps_tz_local(pdf[field.name], timezone)
-if not copied and s is not pdf[field.name]:
-# Copy once if the series is modified to 
prevent the original Pandas
-# DataFrame from being updated
-pdf = pdf.copy()
-copied = True
-pdf[field.name] = s
+if s is not pdf[field.name]:
+if not copied:
--- End diff --

Looks like it was separated for assigning `pdf[field.name] = s` only if `s 
is not pdf[field.name]`.


---

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



[GitHub] spark pull request #20213: [SPARK-23018][PYTHON] Fix createDataFrame from Pa...

2018-01-09 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20213#discussion_r160588222
  
--- Diff: python/pyspark/sql/session.py ---
@@ -459,21 +459,23 @@ def _convert_from_pandas(self, pdf, schema, timezone):
 # TODO: handle nested timestamps, such as 
ArrayType(TimestampType())?
 if isinstance(field.dataType, TimestampType):
 s = 
_check_series_convert_timestamps_tz_local(pdf[field.name], timezone)
-if not copied and s is not pdf[field.name]:
-# Copy once if the series is modified to 
prevent the original Pandas
-# DataFrame from being updated
-pdf = pdf.copy()
-copied = True
-pdf[field.name] = s
+if s is not pdf[field.name]:
+if not copied:
--- End diff --

BTW, what's diff between:

```
if s is not pdf[field.name]:
if not copied:
```

vs

```
if not copied and s is not pdf[field.name]:
```

?


---

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



[GitHub] spark issue #20213: [SPARK-23018][PYTHON] Fix createDataFrame from Pandas ti...

2018-01-09 Thread ueshin
Github user ueshin commented on the issue:

https://github.com/apache/spark/pull/20213
  
LGTM.


---

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



[GitHub] spark pull request #20178: [Spark-22952][CORE] Deprecate stageAttemptId in f...

2018-01-09 Thread advancedxy
Github user advancedxy commented on a diff in the pull request:

https://github.com/apache/spark/pull/20178#discussion_r160587943
  
--- Diff: core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala ---
@@ -56,6 +56,8 @@ class StageInfo(
 completionTime = Some(System.currentTimeMillis)
   }
 
+  def attemptNumber(): Int = attemptId
--- End diff --

> But let's do it in a new PR, as it may need quite a lot of changes.

Lets's do that after Spark 2.3 release then? 

>However, it's also called attemptId in this class.

Yeah, (stage)attemptId is over a lot of places...


---

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



[GitHub] spark issue #20174: [SPARK-22951][SQL] deduplicates on empty data frames sho...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20174: [SPARK-22951][SQL] deduplicates on empty data frames sho...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20174: [SPARK-22951][SQL] deduplicates on empty data frames sho...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20174
  
**[Test build #85893 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85893/testReport)**
 for PR 20174 at commit 
[`01537ca`](https://github.com/apache/spark/commit/01537ca48ae377a590a8bacc96233dbff0dc35f5).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20199: [Spark-22967][TESTS]Fix VersionSuite's unit tests by cha...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20199: [Spark-22967][TESTS]Fix VersionSuite's unit tests by cha...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20199
  
**[Test build #85894 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85894/testReport)**
 for PR 20199 at commit 
[`04b6628`](https://github.com/apache/spark/commit/04b6628e310e55acc7b6449f2b48b4752d07a391).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20199: [Spark-22967][TESTS]Fix VersionSuite's unit tests by cha...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20199
  
Merged build finished. Test PASSed.


---

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



[GitHub] spark issue #18991: [SPARK-21783][SQL] Turn on ORC filter push-down by defau...

2018-01-09 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/18991
  
Any perf number ?


---

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



[GitHub] spark issue #19885: [SPARK-22587] Spark job fails if fs.defaultFS and applic...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19885
  
Merged build finished. Test PASSed.


---

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



[GitHub] spark issue #19885: [SPARK-22587] Spark job fails if fs.defaultFS and applic...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19885
  
**[Test build #85902 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85902/testReport)**
 for PR 19885 at commit 
[`8c5f029`](https://github.com/apache/spark/commit/8c5f029b0b2e4dbb9912db6ce44eb6ff0ec31f6c).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #19885: [SPARK-22587] Spark job fails if fs.defaultFS and applic...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark pull request #20211: [SPARK-23011][PYTHON][SQL] Prepend missing groupi...

2018-01-09 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20211#discussion_r160584966
  
--- Diff: python/pyspark/sql/group.py ---
@@ -233,6 +233,27 @@ def apply(self, udf):
 |  2| 1.1094003924504583|
 +---+---+
 
+Notes on grouping column:
--- End diff --

+1 for ^ 


---

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



[GitHub] spark issue #18991: [SPARK-21783][SQL] Turn on ORC filter push-down by defau...

2018-01-09 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/18991
  
I expect we can port more test cases to Spark, instead of relying on the 
external data sources.


---

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



[GitHub] spark issue #20215: [SPARK-23001] [SQL] Fix NullPointerException when DESC a...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20215
  
**[Test build #85903 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85903/testReport)**
 for PR 20215 at commit 
[`137fd7a`](https://github.com/apache/spark/commit/137fd7a268b23ca52ca190d5b630503e6dc73404).


---

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



[GitHub] spark pull request #20215: [SPARK-23001] [SQL] Fix NullPointerException when...

2018-01-09 Thread gatorsmile
GitHub user gatorsmile opened a pull request:

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

[SPARK-23001] [SQL] Fix NullPointerException when DESC a database with NULL 
description

## What changes were proposed in this pull request?
When users' DB description is NULL, users might hit `NullPointerException`. 
This PR is to fix the issue.

## How was this patch tested?
Added test cases

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

$ git pull https://github.com/gatorsmile/spark SPARK-23001

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

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


commit 137fd7a268b23ca52ca190d5b630503e6dc73404
Author: gatorsmile 
Date:   2018-01-10T03:48:06Z

fix




---

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



[GitHub] spark issue #20150: [SPARK-22956][SS] Bug fix for 2 streams union failover s...

2018-01-09 Thread xuanyuanking
Github user xuanyuanking commented on the issue:

https://github.com/apache/spark/pull/20150
  
Hi Shixiong, thanks a lot for your reply.
The full stack below can reproduce by running the added UT based on 
original code base.
```
Assert on query failed: : Query [id = 3421db21-652e-47af-9d54-2b74a222abed, 
runId = cd8d7c94-1286-44a5-b000-a8d870aef6fa] terminated with exception: 
Partition topic-0-0's offset was changed from 10 to 5, some data may have been 
missed. 
Some data may have been lost because they are not available in Kafka any 
more; either the
 data was aged out by Kafka or the topic may have been deleted before all 
the data in the
 topic was processed. If you don't want your streaming query to fail on 
such cases, set the
 source option "failOnDataLoss" to "false".


org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:295)

org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:189)

Caused by:  Partition topic-0-0's offset was changed from 10 to 5, 
some data may have been missed. 
Some data may have been lost because they are not available in Kafka any 
more; either the
 data was aged out by Kafka or the topic may have been deleted before all 
the data in the
 topic was processed. If you don't want your streaming query to fail on 
such cases, set the
 source option "failOnDataLoss" to "false".


org.apache.spark.sql.kafka010.KafkaSource.org$apache$spark$sql$kafka010$KafkaSource$$reportDataLoss(KafkaSource.scala:332)

org.apache.spark.sql.kafka010.KafkaSource$$anonfun$8.apply(KafkaSource.scala:291)

org.apache.spark.sql.kafka010.KafkaSource$$anonfun$8.apply(KafkaSource.scala:289)

scala.collection.TraversableLike$$anonfun$filterImpl$1.apply(TraversableLike.scala:248)

scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)

scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)

scala.collection.TraversableLike$class.filterImpl(TraversableLike.scala:247)

scala.collection.TraversableLike$class.filter(TraversableLike.scala:259)

scala.collection.AbstractTraversable.filter(Traversable.scala:104)

org.apache.spark.sql.kafka010.KafkaSource.getBatch(KafkaSource.scala:289)
```


---

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



[GitHub] spark pull request #20211: [SPARK-23011][PYTHON][SQL] Prepend missing groupi...

2018-01-09 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20211#discussion_r160583899
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3995,23 +3995,49 @@ def test_coerce(self):
 self.assertFramesEqual(expected, result)
 
 def test_complex_groupby(self):
+import pandas as pd
 from pyspark.sql.functions import pandas_udf, col, PandasUDFType
 df = self.data
+pdf = df.toPandas()
 
 @pandas_udf(
-'id long, v int, norm double',
+'v int, v2 double',
 PandasUDFType.GROUP_MAP
 )
-def normalize(pdf):
+def foo(pdf):
 v = pdf.v
-return pdf.assign(norm=(v - v.mean()) / v.std())
-
-result = df.groupby(col('id') % 2 == 
0).apply(normalize).sort('id', 'v').toPandas()
-pdf = df.toPandas()
-expected = pdf.groupby(pdf['id'] % 2 == 0).apply(normalize.func)
-expected = expected.sort_values(['id', 'v']).reset_index(drop=True)
-expected = expected.assign(norm=expected.norm.astype('float64'))
-self.assertFramesEqual(expected, result)
+return pd.DataFrame({'v': v + 1, 'v2': v - v.mean()})[:]
--- End diff --

Why should we copy here by the way?


---

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



[GitHub] spark issue #18991: [SPARK-21783][SQL][WIP] Turn on ORC filter push-down by ...

2018-01-09 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/18991
  
Ur, originally, it's not accepted by @gatorsmile due to lack of test cases.
So, Today, I reopen it for testing purpose.
Do you think we can enable it? I think we can.


---

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



[GitHub] spark issue #19885: [SPARK-22587] Spark job fails if fs.defaultFS and applic...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19885
  
**[Test build #85902 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85902/testReport)**
 for PR 19885 at commit 
[`8c5f029`](https://github.com/apache/spark/commit/8c5f029b0b2e4dbb9912db6ce44eb6ff0ec31f6c).


---

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



[GitHub] spark pull request #20171: [SPARK-22978] [PySpark] Register Vectorized UDFs ...

2018-01-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20171#discussion_r160583088
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3616,6 +3616,34 @@ def test_vectorized_udf_basic(self):
 bool_f(col('bool')))
 self.assertEquals(df.collect(), res.collect())
 
+def test_register_vectorized_udf_basic(self):
+from pyspark.sql.functions import pandas_udf
+from pyspark.rdd import PythonEvalType
+twoArgsPandasUDF = pandas_udf(lambda x: len(x), IntegerType())
+self.assertEqual(twoArgsPandasUDF.deterministic, True)
+self.assertEqual(twoArgsPandasUDF.evalType, 
PythonEvalType.SQL_PANDAS_SCALAR_UDF)
+newPandasUDF = self.spark.catalog.registerFunction(
+"twoArgsPandasUDF", twoArgsPandasUDF, IntegerType())
+self.assertEqual(newPandasUDF.deterministic, True)
+self.assertEqual(newPandasUDF.evalType, 
PythonEvalType.SQL_PANDAS_SCALAR_UDF)
+[row] = self.spark.sql("SELECT twoArgsPandasUDF('test')").collect()
+self.assertEqual(row[0], 4)
+
+def test_register_nondeterministic_vectorized_udf_basic(self):
+from pyspark.sql.functions import pandas_udf
+from pyspark.rdd import PythonEvalType
+import random
+randomPandasUDF = pandas_udf(
+lambda x: random.randint(6, 6) + x, 
StringType()).asNondeterministic()
--- End diff --

sounds good.


---

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



[GitHub] spark issue #18991: [SPARK-21783][SQL][WIP] Turn on ORC filter push-down by ...

2018-01-09 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/18991
  
why it's still WIP?


---

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



[GitHub] spark issue #19290: [SPARK-22063][R] Fixes lint check failures in R by lates...

2018-01-09 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/19290
  
argh, thanks for the reminder and the fix.
I knew calling internal method is going to bite us ... :(


---

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



[GitHub] spark issue #20206: [SPARK-19256][SQL] Remove ordering enforcement from `Fil...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20205: [SPARK-16060][SQL][follow-up] add a wrapper solution for...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20205
  
**[Test build #85901 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85901/testReport)**
 for PR 20205 at commit 
[`b78c6ec`](https://github.com/apache/spark/commit/b78c6ec6e7579f861ddc95ed12de647992de8d0e).


---

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



[GitHub] spark issue #20206: [SPARK-19256][SQL] Remove ordering enforcement from `Fil...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20206
  
**[Test build #85891 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85891/testReport)**
 for PR 20206 at commit 
[`1008b2e`](https://github.com/apache/spark/commit/1008b2efe8256fe95ae61ebba1ab673e0f397118).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20206: [SPARK-19256][SQL] Remove ordering enforcement from `Fil...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20214: [SPARK-23023][SQL] Cast field data to strings in showStr...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20214
  
**[Test build #85900 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85900/testReport)**
 for PR 20214 at commit 
[`eb56aff`](https://github.com/apache/spark/commit/eb56aff74352a360d1d4b1273be23b670f3c958a).


---

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



[GitHub] spark issue #19885: [SPARK-22587] Spark job fails if fs.defaultFS and applic...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #19885: [SPARK-22587] Spark job fails if fs.defaultFS and applic...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #19885: [SPARK-22587] Spark job fails if fs.defaultFS and applic...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19885
  
**[Test build #85897 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85897/testReport)**
 for PR 19885 at commit 
[`778e1ef`](https://github.com/apache/spark/commit/778e1ef903d46a46f3a389fc9b5bf8038ac7cb71).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark pull request #20205: [SPARK-16060][SQL][follow-up] add a wrapper solut...

2018-01-09 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/20205#discussion_r160581341
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java
 ---
@@ -196,17 +234,26 @@ public void initBatch(
* by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch 
columns.
*/
   private boolean nextBatch() throws IOException {
-for (WritableColumnVector vector : columnVectors) {
-  vector.reset();
-}
-columnarBatch.setNumRows(0);
--- End diff --

Yep. I meant keeping here since we return at line 390 and 240.


---

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



[GitHub] spark pull request #20214: [SPARK-23023][SQL] Cast field data to strings in ...

2018-01-09 Thread maropu
GitHub user maropu opened a pull request:

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

[SPARK-23023][SQL] Cast field data to strings in showString

## What changes were proposed in this pull request?
The current `Datset.showString` prints rows thru `RowEncoder` deserializers 
like;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
++
|a   |
++
|[WrappedArray(1, 2), WrappedArray(3), WrappedArray(4, 5, 6)]|
++
```
This result is incorrect because the correct one is;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
++
|a   |
++
|[[1, 2], [3], [4, 5, 6]]|
++
```
So, this pr fixed code in `showString` to cast field data to strings before 
printing.

## How was this patch tested?
Added tests in `DataFrameSuite`.


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

$ git pull https://github.com/maropu/spark SPARK-23023

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

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


commit eb56aff74352a360d1d4b1273be23b670f3c958a
Author: Takeshi Yamamuro 
Date:   2018-01-06T11:05:54Z

Cast data to strings in showString




---

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



[GitHub] spark issue #20013: [SPARK-20657][core] Speed up rendering of the stages pag...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20013
  
**[Test build #85899 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85899/testReport)**
 for PR 20013 at commit 
[`3d66505`](https://github.com/apache/spark/commit/3d6650589223cfd93c4ced9fb25246bcd88ca899).


---

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



[GitHub] spark pull request #20013: [SPARK-20657][core] Speed up rendering of the sta...

2018-01-09 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/20013#discussion_r160580420
  
--- Diff: core/src/main/scala/org/apache/spark/status/LiveEntity.scala ---
@@ -313,35 +316,68 @@ private class LiveExecutor(val executorId: String, 
_addTime: Long) extends LiveE
 
 }
 
-/** Metrics tracked per stage (both total and per executor). */
-private class MetricsTracker {
-  var executorRunTime = 0L
-  var executorCpuTime = 0L
-  var inputBytes = 0L
-  var inputRecords = 0L
-  var outputBytes = 0L
-  var outputRecords = 0L
-  var shuffleReadBytes = 0L
-  var shuffleReadRecords = 0L
-  var shuffleWriteBytes = 0L
-  var shuffleWriteRecords = 0L
-  var memoryBytesSpilled = 0L
-  var diskBytesSpilled = 0L
-
-  def update(delta: v1.TaskMetrics): Unit = {
-executorRunTime += delta.executorRunTime
-executorCpuTime += delta.executorCpuTime
-inputBytes += delta.inputMetrics.bytesRead
-inputRecords += delta.inputMetrics.recordsRead
-outputBytes += delta.outputMetrics.bytesWritten
-outputRecords += delta.outputMetrics.recordsWritten
-shuffleReadBytes += delta.shuffleReadMetrics.localBytesRead +
-  delta.shuffleReadMetrics.remoteBytesRead
-shuffleReadRecords += delta.shuffleReadMetrics.recordsRead
-shuffleWriteBytes += delta.shuffleWriteMetrics.bytesWritten
-shuffleWriteRecords += delta.shuffleWriteMetrics.recordsWritten
-memoryBytesSpilled += delta.memoryBytesSpilled
-diskBytesSpilled += delta.diskBytesSpilled
+private class MetricsTracker(
--- End diff --

ok, but the code becomes uglier because `v1.TaskMetrics` is kind of an 
annoying type to use.


---

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



[GitHub] spark issue #20207: [SPARK-23000] [TEST-HADOOP2.6] Fix Flaky test suite Data...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20207
  
**[Test build #85898 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85898/testReport)**
 for PR 20207 at commit 
[`9039817`](https://github.com/apache/spark/commit/9039817d516f2cfb68f9caa41374098780fde3ca).


---

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



[GitHub] spark pull request #20205: [SPARK-16060][SQL][follow-up] add a wrapper solut...

2018-01-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20205#discussion_r160579252
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java
 ---
@@ -196,17 +234,26 @@ public void initBatch(
* by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch 
columns.
*/
   private boolean nextBatch() throws IOException {
-for (WritableColumnVector vector : columnVectors) {
-  vector.reset();
-}
-columnarBatch.setNumRows(0);
--- End diff --

it's moved to 
https://github.com/apache/spark/pull/20205/files#diff-e594f7295e5408c01ace8175166313b6R253


---

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



[GitHub] spark issue #19885: [SPARK-22587] Spark job fails if fs.defaultFS and applic...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19885
  
**[Test build #85897 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85897/testReport)**
 for PR 19885 at commit 
[`778e1ef`](https://github.com/apache/spark/commit/778e1ef903d46a46f3a389fc9b5bf8038ac7cb71).


---

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



[GitHub] spark issue #20207: [SPARK-23000] [TEST-HADOOP2.6] Fix Flaky test suite Data...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20207
  
**[Test build #85896 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85896/testReport)**
 for PR 20207 at commit 
[`f239b2b`](https://github.com/apache/spark/commit/f239b2b51f32addf82ad5454e86ecbb9bcbe65a2).


---

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



[GitHub] spark issue #20207: [SPARK-23000] [TEST-HADOOP2.6] Fix Flaky test suite Data...

2018-01-09 Thread gatorsmile
Github user gatorsmile commented on the issue:

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


---

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



[GitHub] spark issue #20096: [SPARK-22908] Add kafka source and sink for continuous p...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20096: [SPARK-22908] Add kafka source and sink for continuous p...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20179: [SPARK-22982] Remove unsafe asynchronous close() call fr...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20179
  
Merged build finished. Test PASSed.


---

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



[GitHub] spark issue #20096: [SPARK-22908] Add kafka source and sink for continuous p...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20096
  
**[Test build #85888 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85888/testReport)**
 for PR 20096 at commit 
[`9158af2`](https://github.com/apache/spark/commit/9158af23dfff674641143b892f0f1093814035a3).
 * This patch **fails PySpark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20179: [SPARK-22982] Remove unsafe asynchronous close() call fr...

2018-01-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20179: [SPARK-22982] Remove unsafe asynchronous close() call fr...

2018-01-09 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20179
  
**[Test build #85885 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85885/testReport)**
 for PR 20179 at commit 
[`9e5f20e`](https://github.com/apache/spark/commit/9e5f20eef09fbd357c7cc8bb19eca7d4bf5f7170).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---

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



[GitHub] spark issue #20176: [SPARK-22981][SQL] Fix incorrect results of Casting Stru...

2018-01-09 Thread maropu
Github user maropu commented on the issue:

https://github.com/apache/spark/pull/20176
  
ok, I'll make a follow-up for `showString` later.


---

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



[GitHub] spark issue #20176: [SPARK-22981][SQL] Fix incorrect results of Casting Stru...

2018-01-09 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/20176
  
show binary as string and cast binary to string seems different to me, 
let's stick with what it is. BTW it's pretty dangerous to change the behavior 
of cast to be different with Hive.




---

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



[GitHub] spark pull request #20202: [MINOR] fix a typo in BroadcastJoinSuite

2018-01-09 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark issue #20202: [MINOR] fix a typo in BroadcastJoinSuite

2018-01-09 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/20202
  
thanks, merging to master/2.3!


---

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



  1   2   3   4   5   6   >