[GitHub] spark pull request: [SPARK-12153][SPARK-7617][MLlib]add support of...

2016-01-30 Thread ygcao
Github user ygcao commented on a diff in the pull request:

https://github.com/apache/spark/pull/10152#discussion_r51356725
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala ---
@@ -289,17 +301,28 @@ class Word2Vec extends Serializable with Logging {
 val expTable = sc.broadcast(createExpTable())
 val bcVocab = sc.broadcast(vocab)
 val bcVocabHash = sc.broadcast(vocabHash)
-
-val sentences: RDD[Array[Int]] = words.mapPartitions { iter =>
+// each partition is a collection of sentences, will be translated 
into arrays of Index integer
+val sentences: RDD[Array[Int]] = dataset.mapPartitions { sentenceIter 
=>
   new Iterator[Array[Int]] {
-def hasNext: Boolean = iter.hasNext
+var wordIter: Iterator[String] = null
+
+def hasNext: Boolean = sentenceIter.hasNext || (wordIter != null 
&& wordIter.hasNext)
 
 def next(): Array[Int] = {
   val sentence = ArrayBuilder.make[Int]
   var sentenceLength = 0
-  while (iter.hasNext && sentenceLength < MAX_SENTENCE_LENGTH) {
-val word = bcVocabHash.value.get(iter.next())
-word match {
+  // do translation of each word into its index in the vocabulary,
--- End diff --

sorry, I can't do much about spark-perf thing, I even still didn't get time 
to figure out what's blocking me from running the whole test locally.
I(and my compiler) also don't aware of the existence of flatMapPartitions 
function, but I do made a version suppose to do whatever Sean suggested. Please 
review and help to do the perf-test for two latest versions.
BTW: I don't worry much about the perf difference since it shouldn't be 
much, even minutes(not quite possible) of difference for each partition, just 
mean possibly tens minutes of overall penalty, shouldn't matter much for a job 
runs hours. Of course, seeing the data is more convincing. My point is that, If 
diff is minor, we'd better optimize for readability.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12798] [SQL] generated BroadcastHashJoi...

2016-01-30 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/10989#discussion_r51356688
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala
 ---
@@ -81,6 +82,30 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite {
 benchmark.run()
   }
 
+  def testBroadcastHashJoin(values: Int): Unit = {
+val benchmark = new Benchmark("BroadcastHashJoin", values)
+
+val dim = broadcast(sqlContext.range(1 << 16).selectExpr("id as k", 
"cast(id as string) as v"))
+
+benchmark.addCase("BroadcastHashJoin w/o codegen") { iter =>
+  sqlContext.setConf("spark.sql.codegen.wholeStage", "false")
+  sqlContext.range(values).join(dim, (col("id") % 6) === 
col("k")).count()
+}
+benchmark.addCase(s"BroadcastHashJoin w codegen") { iter =>
+  sqlContext.setConf("spark.sql.codegen.wholeStage", "true")
+  sqlContext.range(values).join(dim, (col("id") % 6) === 
col("k")).count()
+}
+
+/*
+  Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz
+  BroadcastHashJoin: Avg Time(ms)Avg Rate(M/s)  
Relative Rate
+  
---
+  BroadcastHashJoin w/o codegen   3053.41 3.43 
1.00 X
+  BroadcastHashJoin w codegen 1028.4010.20 
2.97 X
--- End diff --

Since the dimension table is pretty small, overhead of broadcast is also 
low, when I ran it with larger range, the improvements did not change much, 
because looking up in BytesToBytes is the bottleneck. I will have another PR to 
improve the join with small dimension table.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51356658
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

@cloud-fan I will let `ResolveAggregateFunctions` handle the missing 
attribute resolution as long as the child of Sort is Aggregate.
```scala
// Skip sort with aggregate. This will be handled in 
ResolveAggregateFunctions
case sa @ Sort(_, _, child: Aggregate) => sa
```
When rewriting `ResolveSortReferences` in another PR, I will try to make 
the behaviors of both rules identical for resolving the missing attributes. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12798] [SQL] generated BroadcastHashJoi...

2016-01-30 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/10989#discussion_r51356558
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala
 ---
@@ -81,6 +82,30 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite {
 benchmark.run()
   }
 
+  def testBroadcastHashJoin(values: Int): Unit = {
+val benchmark = new Benchmark("BroadcastHashJoin", values)
+
+val dim = broadcast(sqlContext.range(1 << 16).selectExpr("id as k", 
"cast(id as string) as v"))
+
+benchmark.addCase("BroadcastHashJoin w/o codegen") { iter =>
+  sqlContext.setConf("spark.sql.codegen.wholeStage", "false")
+  sqlContext.range(values).join(dim, (col("id") % 6) === 
col("k")).count()
+}
+benchmark.addCase(s"BroadcastHashJoin w codegen") { iter =>
+  sqlContext.setConf("spark.sql.codegen.wholeStage", "true")
+  sqlContext.range(values).join(dim, (col("id") % 6) === 
col("k")).count()
+}
+
+/*
+  Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz
+  BroadcastHashJoin: Avg Time(ms)Avg Rate(M/s)  
Relative Rate
+  
---
+  BroadcastHashJoin w/o codegen   3053.41 3.43 
1.00 X
+  BroadcastHashJoin w codegen 1028.4010.20 
2.97 X
--- End diff --

can you also run a benchmark using a larger range so we amortize the 
broadcast overhead? i'm interested in seeing what the improvement is for the 
join part of the benchmark.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12798] [SQL] generated BroadcastHashJoi...

2016-01-30 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/10989#discussion_r51356463
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/BufferedRowIterator.java 
---
@@ -54,13 +54,27 @@ public void setInput(Iterator iter) {
   }
 
   /**
+   * Returns whether it should stop processing next row or not.
--- End diff --

what's "it"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12798] [SQL] generated BroadcastHashJoi...

2016-01-30 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/10989#discussion_r51356447
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/BufferedRowIterator.java 
---
@@ -31,22 +33,20 @@
  * TODO: replaced it by batched columnar format.
  */
 public class BufferedRowIterator {
-  protected InternalRow currentRow;
+  protected LinkedList currentRows = new LinkedList<>();
--- End diff --

orthogonal to this pr -- my first reaction to this is that maybe we should 
spend a week or two to convert all operators to a push-based model. Otherwise 
performance is going to suck big time for some operators.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12951] [SQL] support spilling in genera...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/10998#issuecomment-177420354
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/50458/
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: [SPARK-12951] [SQL] support spilling in genera...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/10998#issuecomment-177420351
  
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 pull request: [SPARK-13105] Reject NATURAL JOIN queries rath...

2016-01-30 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/10997#issuecomment-177419308
  
hm i thought about this more -- i actually think it is too risky to change 
the behavior here for 1.6.x. For example, there might be users whose queries 
are depending on the fact that natural is not a reserved keyword.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12951] [SQL] support spilling in genera...

2016-01-30 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/10998#issuecomment-177416239
  
Can you add some documentation in the code somewhere appropriate to explain 
the high level flow (e.g. when X happens, we switch to Y through Z).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12850] [SQL] Support Bucket Pruning (Pr...

2016-01-30 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/10942#discussion_r51356374
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala ---
@@ -59,6 +60,136 @@ class BucketedReadSuite extends QueryTest with 
SQLTestUtils with TestHiveSinglet
 }
   }
 
+  // To verify bucket pruning, we compare the contents of remaining 
buckets (before filtering)
+  // with the expectedAnswer.
+  private def checkPrunedAnswers(
+  bucketedDataFrame: DataFrame,
+  expectedAnswer: DataFrame): Unit = {
+val rdd = 
bucketedDataFrame.queryExecution.executedPlan.find(_.isInstanceOf[PhysicalRDD])
+assert(rdd.isDefined)
+checkAnswer(
+  expectedAnswer.orderBy(expectedAnswer.logicalPlan.output.map(attr => 
Column(attr)) : _*),
+  rdd.get.executeCollectPublic().sortBy(_.toString()))
+  }
+
+  test("read partitioning bucketed tables with bucket pruning filters") {
+val df = (10 until 50).map(i => (i % 5, i % 13 + 10, 
i.toString)).toDF("i", "j", "k")
+
+withTable("bucketed_table") {
+  // The number of buckets should be large enough to make sure each 
bucket contains
--- End diff --

I can make a try. : )


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12689][SQL] Migrate DDL parsing to the ...

2016-01-30 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13070][SQL] Better error message when P...

2016-01-30 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12689][SQL] Migrate DDL parsing to the ...

2016-01-30 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/10723#issuecomment-177415413
  
Thanks - merging this in master.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12850] [SQL] Support Bucket Pruning (Pr...

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

https://github.com/apache/spark/pull/10942#discussion_r51356299
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala ---
@@ -59,6 +60,136 @@ class BucketedReadSuite extends QueryTest with 
SQLTestUtils with TestHiveSinglet
 }
   }
 
+  // To verify bucket pruning, we compare the contents of remaining 
buckets (before filtering)
+  // with the expectedAnswer.
+  private def checkPrunedAnswers(
+  bucketedDataFrame: DataFrame,
+  expectedAnswer: DataFrame): Unit = {
+val rdd = 
bucketedDataFrame.queryExecution.executedPlan.find(_.isInstanceOf[PhysicalRDD])
+assert(rdd.isDefined)
+checkAnswer(
+  expectedAnswer.orderBy(expectedAnswer.logicalPlan.output.map(attr => 
Column(attr)) : _*),
+  rdd.get.executeCollectPublic().sortBy(_.toString()))
+  }
+
+  test("read partitioning bucketed tables with bucket pruning filters") {
+val df = (10 until 50).map(i => (i % 5, i % 13 + 10, 
i.toString)).toDF("i", "j", "k")
+
+withTable("bucketed_table") {
+  // The number of buckets should be large enough to make sure each 
bucket contains
--- End diff --

ah got it. But I think we should improve test instead of avoiding this 
problem, which makes people think bucket pruning doesn't work for more than 2 
bucket values in each bucket. How about we check each RDD partitions, make sure 
the should-be-pruned ones are empty, and also check answer at last.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13070][SQL] Better error message when P...

2016-01-30 Thread rxin
Github user rxin commented on the pull request:

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



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51356286
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

Unfortunately, it is not trivial. : ( So far, the current rule 
`ResolveSortReferences` only can handle the missing attributes. In this case, 
we have to push the aggregate function down to the underlying `Aggregate`. 
Thus, it does not work. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12951] [SQL] support spilling in genera...

2016-01-30 Thread davies
GitHub user davies opened a pull request:

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

[SPARK-12951] [SQL] support spilling in generated aggregate

This PR add spilling support for generated TungstenAggregate.

The changes will be covered by 
TungstenAggregationQueryWithControlledFallbackSuite

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

$ git pull https://github.com/davies/spark gen_spilling

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

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


commit 07c4119378402e1b245f53202923626e1e29b725
Author: Davies Liu 
Date:   2016-01-31T06:52:39Z

support spilling in generated aggregate




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51356255
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

So this seems that the rule in `ResolveAggregateFunctions` does not really 
resolve the missing attributes, we could keep that rule unchanged in this PR.

If it's not trivial to fix this, we could create another JIRA for that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-177414971
  
**[Test build #50457 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50457/consoleFull)**
 for PR 10527 at commit 
[`04a14cf`](https://github.com/apache/spark/commit/04a14cf072630fbe3619bf241ff3d10d383594a5).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51356217
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

```
select sum(product) over () from windowData where product > 3
group by product having avg(month) > 0
order by avg(area), product
```
This is not resolvable unless we reimplement the rule 
`ResolveAggregateFunctions`. So far, the rule assumes Sort and Aggregate are 
adjacent. If you read the plan, you will understand this is not true. This is 
not a simple change. Do you want me to do it in a separate JIRA?

```
'Sort ['avg('area) ASC,'product ASC], true
+- Project [_c0#61L]
   +- Project [_w0#62L,_c0#61L,_c0#61L]
  +- Window [_w0#62L], [(sum(_w0#62L),mode=Complete,isDistinct=false) 
windowspecdefinition(ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) 
AS _c0#61L]
 +- Project [_w0#62L]
+- Filter havingCondition#63: boolean
   +- Aggregate [product#2], [cast(((avg(cast(month#0 as 
bigint)),mode=Complete,isDistinct=false) > cast(0 as double)) as boolean) AS 
havingCondition#63,cast(product#2 as bigint) AS _w0#62L]
  +- Filter (product#2 > 3)
 +- Subquery windowdata
+- LogicalRDD [month#0,area#1,product#2], 
MapPartitionsRDD[1] at apply at Transformer.scala:22
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51356100
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

Let me add a test case to cover it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51356089
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

Can we make sure this query work:
```
 select  sum() over () from t where a > 0 group by a having avg(b) > 0 
order by avg(c), a 
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12850] [SQL] Support Bucket Pruning (Pr...

2016-01-30 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/10942#discussion_r51355982
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala ---
@@ -59,6 +60,136 @@ class BucketedReadSuite extends QueryTest with 
SQLTestUtils with TestHiveSinglet
 }
   }
 
+  // To verify bucket pruning, we compare the contents of remaining 
buckets (before filtering)
+  // with the expectedAnswer.
+  private def checkPrunedAnswers(
+  bucketedDataFrame: DataFrame,
+  expectedAnswer: DataFrame): Unit = {
+val rdd = 
bucketedDataFrame.queryExecution.executedPlan.find(_.isInstanceOf[PhysicalRDD])
+assert(rdd.isDefined)
+checkAnswer(
+  expectedAnswer.orderBy(expectedAnswer.logicalPlan.output.map(attr => 
Column(attr)) : _*),
+  rdd.get.executeCollectPublic().sortBy(_.toString()))
+  }
+
+  test("read partitioning bucketed tables with bucket pruning filters") {
+val df = (10 until 50).map(i => (i % 5, i % 13 + 10, 
i.toString)).toDF("i", "j", "k")
+
+withTable("bucketed_table") {
+  // The number of buckets should be large enough to make sure each 
bucket contains
--- End diff --

Yeah, you are right. The test cases will fail if the bucket contains both 
desired bucketing values and ineligible values at the same time.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13070][SQL] Better error message when P...

2016-01-30 Thread liancheng
Github user liancheng commented on the pull request:

https://github.com/apache/spark/pull/10979#issuecomment-177407724
  
LGTM, thanks for helping fixing this! Will get it merged once I get access 
to my laptop.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12850] [SQL] Support Bucket Pruning (Pr...

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

https://github.com/apache/spark/pull/10942#discussion_r51355960
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala ---
@@ -59,6 +60,136 @@ class BucketedReadSuite extends QueryTest with 
SQLTestUtils with TestHiveSinglet
 }
   }
 
+  // To verify bucket pruning, we compare the contents of remaining 
buckets (before filtering)
+  // with the expectedAnswer.
+  private def checkPrunedAnswers(
+  bucketedDataFrame: DataFrame,
+  expectedAnswer: DataFrame): Unit = {
+val rdd = 
bucketedDataFrame.queryExecution.executedPlan.find(_.isInstanceOf[PhysicalRDD])
+assert(rdd.isDefined)
+checkAnswer(
+  expectedAnswer.orderBy(expectedAnswer.logicalPlan.output.map(attr => 
Column(attr)) : _*),
+  rdd.get.executeCollectPublic().sortBy(_.toString()))
+  }
+
+  test("read partitioning bucketed tables with bucket pruning filters") {
+val df = (10 until 50).map(i => (i % 5, i % 13 + 10, 
i.toString)).toDF("i", "j", "k")
+
+withTable("bucketed_table") {
+  // The number of buckets should be large enough to make sure each 
bucket contains
--- End diff --

Sorry I don't understand this. Will test fail if we have more than 2 bucket 
values in each bucket?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12850] [SQL] Support Bucket Pruning (Pr...

2016-01-30 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/10942#discussion_r51355831
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala ---
@@ -59,6 +60,136 @@ class BucketedReadSuite extends QueryTest with 
SQLTestUtils with TestHiveSinglet
 }
   }
 
+  // To verify bucket pruning, we compare the contents of remaining 
buckets (before filtering)
+  // with the expectedAnswer.
+  private def checkPrunedAnswers(
+  bucketedDataFrame: DataFrame,
+  expectedAnswer: DataFrame): Unit = {
+val rdd = 
bucketedDataFrame.queryExecution.executedPlan.find(_.isInstanceOf[PhysicalRDD])
+assert(rdd.isDefined)
+checkAnswer(
+  expectedAnswer.orderBy(expectedAnswer.logicalPlan.output.map(attr => 
Column(attr)) : _*),
+  rdd.get.executeCollectPublic().sortBy(_.toString()))
+  }
+
+  test("read partitioning bucketed tables with bucket pruning filters") {
+val df = (10 until 50).map(i => (i % 5, i % 13 + 10, 
i.toString)).toDF("i", "j", "k")
+
+withTable("bucketed_table") {
+  // The number of buckets should be large enough to make sure each 
bucket contains
--- End diff --

This is just for simplifying verification of bucket pruning. 

In the current testing verification solution, we are checking if the rows 
in all the scanned buckets (**without applying any filter**) exactly match the 
expected/correct answers. You know, applying these filters could hide the bugs 
in bucket pruning. Thus, we want to skip all the filters when verifying bucket 
pruning. 

If more than one bucketing column values are hashed into the same bucket, 
the returned results could contain invalid rows that are in the same bucket. 
These invalid rows should be filtered out by the filters when running the whole 
query plan. 

Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-177402550
  
**[Test build #50456 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50456/consoleFull)**
 for PR 10527 at commit 
[`3a2510b`](https://github.com/apache/spark/commit/3a2510b4121cbee6343df72162dfff869ffee5dc).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-177399861
  
**[Test build #50457 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50457/consoleFull)**
 for PR 10527 at commit 
[`04a14cf`](https://github.com/apache/spark/commit/04a14cf072630fbe3619bf241ff3d10d383594a5).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

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

https://github.com/apache/spark/pull/10678#discussion_r51355714
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

`ResolveAggregateFunctions` can handle missing attributes that can be 
resolved in grandchild. If there are more complex cases, I think that rule can 
at least resolve aggregate functions and go back to this rule to complete 
resolution.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51355710
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

@cloud-fan Sure, let me change it. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51355706
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

@davies The missing attributes are also handled in 
`ResolveAggregateFunctions`. Thus it works. To answer your first question 
regarding `!s.resolved`, this is part of the algorithm design in the rule 
`ResolveAggregateFunctions`, as shown below: 
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L706-L708


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51355652
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

There could be missing attributes together with aggregate functions, will 
that work?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

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

https://github.com/apache/spark/pull/10678#discussion_r51355625
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

to be more clear, `ResolveSortReferences` tries to resolve attributes in 
`SortOrder`s, and `ResolveAggregateFunctions` tries to resolve aggregate 
functions in unexpected places(filter or sort), right?

So I think we should skip sort with aggregate functions here, i.e.
`case s: Sort if 
s.order.exists(ResolveAggregateFunctions.containsAggregate) => s`, and add 
comment to say this case should be handled in `ResolveAggregateFunctions`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/10678#issuecomment-177397295
  
LGTM, left two comments.

@marmbrus Could you take another pass on it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51355575
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,96 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  // Here, this rule only resolves the missing sort references if the 
child is not Aggregate
+  //   Another rule ResolveAggregateFunctions will resolve that case.
--- End diff --

The purpose of that rule is to resolve the aggregate functions (resolving 
expressions for Sort is kind of side effects), it's better have that in the 
`ResolveAggregateFunctions`. This also help to reduce the changes in this PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51355500
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,99 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  case s @ Sort(_, _, a: Aggregate) if a.resolved =>
--- End diff --

Do we need to check that `!s.resolved`, we have that in next case.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12988][SQL] Can't drop columns that con...

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

https://github.com/apache/spark/pull/10943#discussion_r51355492
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala ---
@@ -150,6 +153,17 @@ class DataFrame private[sql](
 }
   }
 
+  /**
+   * Resolves a column name. This is called when it is required to resolve 
a column by its
+   * name only and not as a column path..
+   */
+  private[sql] def resolveColName(colName: String, userSuppliedName: 
String): Boolean = {
--- End diff --

how about
```
private[sql] def indexOf(colName: String): Option[Int] = {
  val resolver = sqlContext.analyzer.resolver
  val index = queryExecution.analyzed.output.indexWhere(f => 
resolver(f.name, colName))
  if (index >= 0) Some(index) else None
}
```

then we can rewrite `withColumn` to:
```
indexOf(colName).map { index =>
  select(output.updated(index, col.as(colName)).map(Column(_)) : _*)
}.getOrElse {
  select(Column("*"), col.as(colName))
}
```

There may be better name for this, like `resolveToIndex`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10702#issuecomment-177396130
  
**[Test build #50455 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50455/consoleFull)**
 for PR 10702 at commit 
[`c0744d8`](https://github.com/apache/spark/commit/c0744d8a3c08756546925c9f82274f50d1d4affd).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12982][SQL] Add table name validation i...

2016-01-30 Thread jayadevanmurali
Github user jayadevanmurali commented on the pull request:

https://github.com/apache/spark/pull/10983#issuecomment-177392588
  
Thanks @hvanhovell , Got your point. I updated my code and repeat the 
steps. I was able to replicate this. Please check the steps

jayadevan@Satellite-L640:~/spark$ ./bin/spark-shell
NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes 
ahead of assembly.
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).
16/01/31 09:27:57 WARN NativeCodeLoader: Unable to load native-hadoop 
library for your platform... using builtin-java classes where applicable
16/01/31 09:27:57 WARN Utils: Your hostname, Satellite-L640 resolves to a 
loopback address: 127.0.1.1, but we couldn't find any external IP address!
16/01/31 09:27:57 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to 
another address
Spark context available as sc (master = local[*], app id = 
local-1454212680541).
SQL context available as sqlContext.
Welcome to
    __
 / __/__  ___ _/ /__
_\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 2.0.0-SNAPSHOT
  /_/
 
Using Scala version 2.11.7 (Java HotSpot(TM) 64-Bit Server VM, Java 
1.7.0_80)
Type in expressions to have them evaluated.
Type :help for more information.

scala> import org.apache.spark.sql.types.{StringType, StructField, 
StructType}
import org.apache.spark.sql.types.{StringType, StructField, StructType}

scala> import org.apache.spark.sql.{DataFrame, Row, SQLContext}
import org.apache.spark.sql.{DataFrame, Row, SQLContext}

scala> import org.apache.spark.{SparkContext, SparkConf}
import org.apache.spark.{SparkContext, SparkConf}

scala> val rows = List(Row("foo"), Row("bar"));
rows: List[org.apache.spark.sql.Row] = List([foo], [bar])

scala> val schema = StructType(Seq(StructField("col", StringType)));
schema: org.apache.spark.sql.types.StructType = 
StructType(StructField(col,StringType,true))

scala> val rdd = sc.parallelize(rows);
rdd: org.apache.spark.rdd.RDD[org.apache.spark.sql.Row] = 
ParallelCollectionRDD[0] at parallelize at :29

scala> val df = sqlContext.createDataFrame(rdd, schema)
df: org.apache.spark.sql.DataFrame = [col: string]

scala> df.registerTempTable("t~")

scala> df.sqlContext.dropTempTable("t~")
org.apache.spark.sql.AnalysisException: NoViableAltException(327@[209:20: ( 
DOT id2= identifier )?])
; line 1 pos 1
  at 
org.apache.spark.sql.catalyst.parser.ParseErrorReporter.throwError(ParseDriver.scala:158)
  at 
org.apache.spark.sql.catalyst.parser.ParseErrorReporter.throwError(ParseDriver.scala:147)
  at 
org.apache.spark.sql.catalyst.parser.ParseDriver$.parse(ParseDriver.scala:95)
  at 
org.apache.spark.sql.catalyst.parser.ParseDriver$.parseTableName(ParseDriver.scala:42)
  at 
org.apache.spark.sql.catalyst.CatalystQl.parseTableIdentifier(CatalystQl.scala:81)
  at org.apache.spark.sql.SQLContext.table(SQLContext.scala:811)
  at org.apache.spark.sql.SQLContext.dropTempTable(SQLContext.scala:738)
  ... 49 elided

So I will close this pull request and raise a new one.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2016-01-30 Thread vectorijk
Github user vectorijk commented on a diff in the pull request:

https://github.com/apache/spark/pull/10527#discussion_r51355228
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala
 ---
@@ -132,4 +132,49 @@ class MiscFunctionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
   }
 }
   }
+
+  test("aesEncrypt") {
+checkEvaluation(Base64(AesEncrypt(Literal("ABC".getBytes),
+  Literal("1234567890123456".getBytes))), "y6Ss+zCYObpCbgfWfyNWTw==")
+checkEvaluation(Base64(AesEncrypt(Literal("".getBytes),
+  Literal("1234567890123456".getBytes))), "BQGHoM3lqYcsurCRq3PlUw==")
+
+// input is null
+checkEvaluation(AesEncrypt(Literal.create(null, BinaryType),
+  Literal("1234567890123456".getBytes)), null)
+// key length (80 bits) is not one of the permitted values (128, 192 
or 256 bits)
+checkEvaluation(Base64(AesEncrypt(Literal("ABC".getBytes),
+  Literal("1234567890".getBytes))), null)
+// key is null
+checkEvaluation(Base64(AesEncrypt(Literal("ABC".getBytes),
+  Literal.create(null, BinaryType))), null)
+// both are null
+checkEvaluation(Base64(AesEncrypt(Literal.create(null, BinaryType),
+  Literal.create(null, BinaryType))), null)
+  }
+
+  test("aesDecrypt") {
+
checkEvaluation(AesDecrypt(UnBase64(Literal("y6Ss+zCYObpCbgfWfyNWTw==")),
+  Literal("1234567890123456".getBytes)), "ABC")
+
checkEvaluation(AesDecrypt(UnBase64(Literal("BQGHoM3lqYcsurCRq3PlUw==")),
+  Literal("1234567890123456".getBytes)), "")
+
+// input is null
+checkEvaluation(AesDecrypt(UnBase64(Literal.create(null, StringType)),
+  Literal("1234567890123456".getBytes)), null)
+// key length (80 bits) is not one of the permitted values (128, 192 
or 256 bits)
+
checkEvaluation(AesDecrypt(UnBase64(Literal("y6Ss+zCYObpCbgfWfyNWTw==")),
+  Literal("1234567890".getBytes)), null)
+// key is null
+
checkEvaluation(AesDecrypt(UnBase64(Literal("y6Ss+zCYObpCbgfWfyNWTw==")),
+  Literal.create(null, BinaryType)), null)
+// both are null
+checkEvaluation(AesDecrypt(UnBase64(Literal.create(null, StringType)),
+  Literal.create(null, BinaryType)), null)
+  }
+
+  ignore("aesEncryptWith256bitsKey") {
--- End diff --

will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10702#issuecomment-177386126
  
**[Test build #50454 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50454/consoleFull)**
 for PR 10702 at commit 
[`0b16353`](https://github.com/apache/spark/commit/0b16353a699fbcdea50e0f0213a405934447).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread dbtsai
Github user dbtsai commented on the pull request:

https://github.com/apache/spark/pull/10702#issuecomment-177385145
  
Commenting on your issues. 

Issue 1:
With `WeightedLeastSquares`, we have option to standardize the label and 
features separately. As a result, if the label is not standardized, even `yStd 
== 0`, the problem can be solved. 

As a result, in your case 4, when label is not standardized, and the 
features are standardized, this is not defined, so the users should get the 
result.

For case 3, can you elaborate why analytical solution exists even the label 
is standardized? 

Issue 2: 

In my opinion, even case 1, and case 2 are ill-defined since in GLMNET, the 
label is standardized by default, and GLMNET will not return any result at all. 
It just happens that without regularization, with/without standardization on 
labels will not change the solution, so we just treat them as if we don't 
standardize the label. This can explain your case 3.

Issue 3:

I think this is because your normal equation solver doesn't standardize the 
label, so the discrepancies occur. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-177383345
  
**[Test build #50456 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50456/consoleFull)**
 for PR 10527 at commit 
[`3a2510b`](https://github.com/apache/spark/commit/3a2510b4121cbee6343df72162dfff869ffee5dc).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

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

https://github.com/apache/spark/pull/10527#discussion_r51355182
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala
 ---
@@ -132,4 +132,49 @@ class MiscFunctionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
   }
 }
   }
+
+  test("aesEncrypt") {
+checkEvaluation(Base64(AesEncrypt(Literal("ABC".getBytes),
+  Literal("1234567890123456".getBytes))), "y6Ss+zCYObpCbgfWfyNWTw==")
+checkEvaluation(Base64(AesEncrypt(Literal("".getBytes),
+  Literal("1234567890123456".getBytes))), "BQGHoM3lqYcsurCRq3PlUw==")
+
+// input is null
+checkEvaluation(AesEncrypt(Literal.create(null, BinaryType),
+  Literal("1234567890123456".getBytes)), null)
+// key length (80 bits) is not one of the permitted values (128, 192 
or 256 bits)
+checkEvaluation(Base64(AesEncrypt(Literal("ABC".getBytes),
+  Literal("1234567890".getBytes))), null)
+// key is null
+checkEvaluation(Base64(AesEncrypt(Literal("ABC".getBytes),
+  Literal.create(null, BinaryType))), null)
+// both are null
+checkEvaluation(Base64(AesEncrypt(Literal.create(null, BinaryType),
+  Literal.create(null, BinaryType))), null)
+  }
+
+  test("aesDecrypt") {
+
checkEvaluation(AesDecrypt(UnBase64(Literal("y6Ss+zCYObpCbgfWfyNWTw==")),
+  Literal("1234567890123456".getBytes)), "ABC")
+
checkEvaluation(AesDecrypt(UnBase64(Literal("BQGHoM3lqYcsurCRq3PlUw==")),
+  Literal("1234567890123456".getBytes)), "")
+
+// input is null
+checkEvaluation(AesDecrypt(UnBase64(Literal.create(null, StringType)),
+  Literal("1234567890123456".getBytes)), null)
+// key length (80 bits) is not one of the permitted values (128, 192 
or 256 bits)
+
checkEvaluation(AesDecrypt(UnBase64(Literal("y6Ss+zCYObpCbgfWfyNWTw==")),
+  Literal("1234567890".getBytes)), null)
+// key is null
+
checkEvaluation(AesDecrypt(UnBase64(Literal("y6Ss+zCYObpCbgfWfyNWTw==")),
+  Literal.create(null, BinaryType)), null)
+// both are null
+checkEvaluation(AesDecrypt(UnBase64(Literal.create(null, StringType)),
+  Literal.create(null, BinaryType)), null)
+  }
+
+  ignore("aesEncryptWith256bitsKey") {
--- End diff --

please add a comment to explain why we ignore it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

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

https://github.com/apache/spark/pull/10527#discussion_r51355177
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
 ---
@@ -453,3 +455,119 @@ case class Murmur3Hash(children: Seq[Expression], 
seed: Int) extends Expression
 }
   }
 }
+
+/**
+ * A function that encrypts input using AES. Key lengths of 128, 192 or 
256 bits can be used.
+ * 192 and 256 bits keys can be used if Java Cryptography Extension (JCE) 
Unlimited Strength
+ * Jurisdiction Policy Files are installed. If either argument is NULL or 
the key length is
+ * not one of the permitted values, the return value is NULL.
+ */
+@ExpressionDescription(
+  usage = "_FUNC_(input, key) - Encrypts input using AES.",
+  extended = "> SELECT Base64(_FUNC_('ABC', '1234567890123456'));\n 
'y6Ss+zCYObpCbgfWfyNWTw=='")
+case class AesEncrypt(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def dataType: DataType = BinaryType
+  override def nullable: Boolean = true
+
+  override def inputTypes: Seq[DataType] = Seq(BinaryType, BinaryType)
+
+  protected override def nullSafeEval(input1: Any, input2: Any): Any = {
+val cipher = Cipher.getInstance("AES")
+val secretKey: SecretKeySpec = input2.asInstanceOf[Array[Byte]].length 
match {
+  case 16 | 24 | 32 =>
+new SecretKeySpec(input2.asInstanceOf[Array[Byte]], 0,
+  input2.asInstanceOf[Array[Byte]].length, "AES")
+  case _ => null
+}
+
+try {
+  cipher.init(Cipher.ENCRYPT_MODE, secretKey)
+  cipher.doFinal(input1.asInstanceOf[Array[Byte]], 0, 
input1.asInstanceOf[Array[Byte]].length)
+} catch {
+  case e: GeneralSecurityException => null
--- End diff --

I'm not quite sure how we should deal with errors, return null or throw 
exception? cc @marmbrus 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10702#issuecomment-177381602
  
**[Test build #50455 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50455/consoleFull)**
 for PR 10702 at commit 
[`c0744d8`](https://github.com/apache/spark/commit/c0744d8a3c08756546925c9f82274f50d1d4affd).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13105] Reject NATURAL JOIN queries rath...

2016-01-30 Thread cloud-fan
Github user cloud-fan commented on the pull request:

https://github.com/apache/spark/pull/10997#issuecomment-177381245
  
how about hive context? Should we update `HiveQl.scala` 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: [SPARK-12567][SQL] Add aes_{encrypt,decrypt} U...

2016-01-30 Thread vectorijk
Github user vectorijk commented on the pull request:

https://github.com/apache/spark/pull/10527#issuecomment-177379953
  
@cloud-fan Okay, addressed comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12850] [SQL] Support Bucket Pruning (Pr...

2016-01-30 Thread cloud-fan
Github user cloud-fan commented on the pull request:

https://github.com/apache/spark/pull/10942#issuecomment-177379925
  
cc @nongli @rxin @yhuai   to take another 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 pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread dbtsai
Github user dbtsai commented on a diff in the pull request:

https://github.com/apache/spark/pull/10702#discussion_r51355106
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala 
---
@@ -558,6 +575,47 @@ class LinearRegressionSuite
 }
   }
 
+  test("linear regression model with constant label") {
+/*
+   R code:
+   for (formula in c(b.const ~ . -1, b.const ~ .)) {
+ model <- lm(formula, data=df.const.label, weights=w)
+ print(as.vector(coef(model)))
+   }
+  [1] -9.221298  3.394343
+  [1] 17  0  0
+*/
+val expected = Seq(
+  Vectors.dense(0.0, -9.221298, 3.394343),
+  Vectors.dense(17.0, 0.0, 0.0))
+
+Seq("auto", "l-bfgs", "normal").foreach { solver =>
+  var idx = 0
+  for (fitIntercept <- Seq(false, true)) {
+val model = new LinearRegression()
+  .setFitIntercept(fitIntercept)
+  .setWeightCol("weight")
+  .setSolver(solver)
+  .fit(datasetWithWeightConstantLabel)
+val actual = Vectors.dense(model.intercept, model.coefficients(0), 
model.coefficients(1))
+assert(actual ~== expected(idx) absTol 1e-4)
+idx += 1
--- End diff --

In `LinearRegressionTrainingSummary`, you can get it from 
`objectiveHistory`. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12850] [SQL] Support Bucket Pruning (Pr...

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

https://github.com/apache/spark/pull/10942#discussion_r51355104
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala ---
@@ -59,6 +60,136 @@ class BucketedReadSuite extends QueryTest with 
SQLTestUtils with TestHiveSinglet
 }
   }
 
+  // To verify bucket pruning, we compare the contents of remaining 
buckets (before filtering)
+  // with the expectedAnswer.
+  private def checkPrunedAnswers(
+  bucketedDataFrame: DataFrame,
+  expectedAnswer: DataFrame): Unit = {
+val rdd = 
bucketedDataFrame.queryExecution.executedPlan.find(_.isInstanceOf[PhysicalRDD])
+assert(rdd.isDefined)
+checkAnswer(
+  expectedAnswer.orderBy(expectedAnswer.logicalPlan.output.map(attr => 
Column(attr)) : _*),
+  rdd.get.executeCollectPublic().sortBy(_.toString()))
+  }
+
+  test("read partitioning bucketed tables with bucket pruning filters") {
+val df = (10 until 50).map(i => (i % 5, i % 13 + 10, 
i.toString)).toDF("i", "j", "k")
+
+withTable("bucketed_table") {
+  // The number of buckets should be large enough to make sure each 
bucket contains
--- End diff --

why this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread iyounus
Github user iyounus commented on a diff in the pull request:

https://github.com/apache/spark/pull/10702#discussion_r51355081
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala 
---
@@ -558,6 +575,47 @@ class LinearRegressionSuite
 }
   }
 
+  test("linear regression model with constant label") {
+/*
+   R code:
+   for (formula in c(b.const ~ . -1, b.const ~ .)) {
+ model <- lm(formula, data=df.const.label, weights=w)
+ print(as.vector(coef(model)))
+   }
+  [1] -9.221298  3.394343
+  [1] 17  0  0
+*/
+val expected = Seq(
+  Vectors.dense(0.0, -9.221298, 3.394343),
+  Vectors.dense(17.0, 0.0, 0.0))
+
+Seq("auto", "l-bfgs", "normal").foreach { solver =>
+  var idx = 0
+  for (fitIntercept <- Seq(false, true)) {
+val model = new LinearRegression()
+  .setFitIntercept(fitIntercept)
+  .setWeightCol("weight")
+  .setSolver(solver)
+  .fit(datasetWithWeightConstantLabel)
+val actual = Vectors.dense(model.intercept, model.coefficients(0), 
model.coefficients(1))
+assert(actual ~== expected(idx) absTol 1e-4)
+idx += 1
--- End diff --

I'm not sure how to _check the size of lost history_. Could you please 
point me to some example?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13105] Reject NATURAL JOIN queries rath...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13105] Reject NATURAL JOIN queries rath...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13105] Reject NATURAL JOIN queries rath...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10997#issuecomment-177376511
  
**[Test build #50453 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50453/consoleFull)**
 for PR 10997 at commit 
[`21e42d8`](https://github.com/apache/spark/commit/21e42d893283a754f3d25758562864ceb57043d4).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10702#issuecomment-177376438
  
**[Test build #50454 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50454/consoleFull)**
 for PR 10702 at commit 
[`0b16353`](https://github.com/apache/spark/commit/0b16353a699fbcdea50e0f0213a405934447).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread dbtsai
Github user dbtsai commented on the pull request:

https://github.com/apache/spark/pull/10702#issuecomment-177376070
  
LGTM except minor comments. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread dbtsai
Github user dbtsai commented on a diff in the pull request:

https://github.com/apache/spark/pull/10702#discussion_r51354803
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala 
---
@@ -558,6 +575,47 @@ class LinearRegressionSuite
 }
   }
 
+  test("linear regression model with constant label") {
+/*
+   R code:
+   for (formula in c(b.const ~ . -1, b.const ~ .)) {
+ model <- lm(formula, data=df.const.label, weights=w)
+ print(as.vector(coef(model)))
+   }
+  [1] -9.221298  3.394343
+  [1] 17  0  0
+*/
+val expected = Seq(
+  Vectors.dense(0.0, -9.221298, 3.394343),
+  Vectors.dense(17.0, 0.0, 0.0))
+
+Seq("auto", "l-bfgs", "normal").foreach { solver =>
+  var idx = 0
+  for (fitIntercept <- Seq(false, true)) {
+val model = new LinearRegression()
+  .setFitIntercept(fitIntercept)
+  .setWeightCol("weight")
+  .setSolver(solver)
+  .fit(datasetWithWeightConstantLabel)
+val actual = Vectors.dense(model.intercept, model.coefficients(0), 
model.coefficients(1))
+assert(actual ~== expected(idx) absTol 1e-4)
+idx += 1
--- End diff --

When `fitInercept = true`, check the size of loss history is zero. (since 
the solution is returned without any optimization.) 

Will be nice to add one small test that `labelStd = 0` and `labelMean = 0` 
when `fitInercept = false`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread dbtsai
Github user dbtsai commented on a diff in the pull request:

https://github.com/apache/spark/pull/10702#discussion_r51354784
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala 
---
@@ -558,6 +575,47 @@ class LinearRegressionSuite
 }
   }
 
+  test("linear regression model with constant label") {
+/*
+   R code:
+   for (formula in c(b.const ~ . -1, b.const ~ .)) {
+ model <- lm(formula, data=df.const.label, weights=w)
+ print(as.vector(coef(model)))
+   }
+  [1] -9.221298  3.394343
+  [1] 17  0  0
+*/
+val expected = Seq(
+  Vectors.dense(0.0, -9.221298, 3.394343),
+  Vectors.dense(17.0, 0.0, 0.0))
+
+Seq("auto", "l-bfgs", "normal").foreach { solver =>
+  var idx = 0
+  for (fitIntercept <- Seq(false, true)) {
+val model = new LinearRegression()
+  .setFitIntercept(fitIntercept)
+  .setWeightCol("weight")
+  .setSolver(solver)
+  .fit(datasetWithWeightConstantLabel)
+val actual = Vectors.dense(model.intercept, model.coefficients(0), 
model.coefficients(1))
+assert(actual ~== expected(idx) absTol 1e-4)
+idx += 1
+  }
+}
+  }
+
+  test("regularized linear regression through origin with constant label") 
{
+// The problem is ill-defined if fitIntercept=false, regParam is 
non-zero and \
--- End diff --

seems you forget to remove `\` :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread dbtsai
Github user dbtsai commented on a diff in the pull request:

https://github.com/apache/spark/pull/10702#discussion_r51354776
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala ---
@@ -219,33 +219,44 @@ class LinearRegression @Since("1.3.0") 
(@Since("1.3.0") override val uid: String
 }
 
 val yMean = ySummarizer.mean(0)
-val yStd = math.sqrt(ySummarizer.variance(0))
-
-// If the yStd is zero, then the intercept is yMean with zero 
coefficient;
-// as a result, training is not needed.
-if (yStd == 0.0) {
-  logWarning(s"The standard deviation of the label is zero, so the 
coefficients will be " +
-s"zeros and the intercept will be the mean of the label; as a 
result, " +
-s"training is not needed.")
-  if (handlePersistence) instances.unpersist()
-  val coefficients = Vectors.sparse(numFeatures, Seq())
-  val intercept = yMean
-
-  val model = new LinearRegressionModel(uid, coefficients, intercept)
-  // Handle possible missing or invalid prediction columns
-  val (summaryModel, predictionColName) = 
model.findSummaryModelAndPredictionCol()
-
-  val trainingSummary = new LinearRegressionTrainingSummary(
-summaryModel.transform(dataset),
-predictionColName,
-$(labelCol),
-model,
-Array(0D),
-$(featuresCol),
-Array(0D))
-  return copyValues(model.setSummary(trainingSummary))
+val rawYStd = math.sqrt(ySummarizer.variance(0))
+if (rawYStd == 0.0) {
+  if ($(fitIntercept) || yMean==0.0) {
+// If the rawYStd is zero and fitIntercept=true, then the 
intercept is yMean with
+// zero coefficient; as a result, training is not needed.
+// Also, if yMean==0 and rawYStd==0, all the coefficients are zero 
regardless of
+// the fitIntercept
+logWarning(s"The standard deviation of the label is zero, so the 
coefficients will be " +
--- End diff --

Maybe you want to update the warning message for the second situation as 
well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread dbtsai
Github user dbtsai commented on a diff in the pull request:

https://github.com/apache/spark/pull/10702#discussion_r51354767
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala ---
@@ -219,33 +219,44 @@ class LinearRegression @Since("1.3.0") 
(@Since("1.3.0") override val uid: String
 }
 
 val yMean = ySummarizer.mean(0)
-val yStd = math.sqrt(ySummarizer.variance(0))
-
-// If the yStd is zero, then the intercept is yMean with zero 
coefficient;
-// as a result, training is not needed.
-if (yStd == 0.0) {
-  logWarning(s"The standard deviation of the label is zero, so the 
coefficients will be " +
-s"zeros and the intercept will be the mean of the label; as a 
result, " +
-s"training is not needed.")
-  if (handlePersistence) instances.unpersist()
-  val coefficients = Vectors.sparse(numFeatures, Seq())
-  val intercept = yMean
-
-  val model = new LinearRegressionModel(uid, coefficients, intercept)
-  // Handle possible missing or invalid prediction columns
-  val (summaryModel, predictionColName) = 
model.findSummaryModelAndPredictionCol()
-
-  val trainingSummary = new LinearRegressionTrainingSummary(
-summaryModel.transform(dataset),
-predictionColName,
-$(labelCol),
-model,
-Array(0D),
-$(featuresCol),
-Array(0D))
-  return copyValues(model.setSummary(trainingSummary))
+val rawYStd = math.sqrt(ySummarizer.variance(0))
+if (rawYStd == 0.0) {
+  if ($(fitIntercept) || yMean==0.0) {
+// If the rawYStd is zero and fitIntercept=true, then the 
intercept is yMean with
+// zero coefficient; as a result, training is not needed.
+// Also, if yMean==0 and rawYStd==0, all the coefficients are zero 
regardless of
+// the fitIntercept
+logWarning(s"The standard deviation of the label is zero, so the 
coefficients will be " +
+  s"zeros and the intercept will be the mean of the label; as a 
result, " +
+  s"training is not needed.")
+if (handlePersistence) instances.unpersist()
+val coefficients = Vectors.sparse(numFeatures, Seq())
+val intercept = yMean
+
+val model = new LinearRegressionModel(uid, coefficients, intercept)
+// Handle possible missing or invalid prediction columns
+val (summaryModel, predictionColName) = 
model.findSummaryModelAndPredictionCol()
+
+val trainingSummary = new LinearRegressionTrainingSummary(
+  summaryModel.transform(dataset),
+  predictionColName,
+  $(labelCol),
+  model,
+  Array(0D),
+  $(featuresCol),
+  Array(0D))
+return copyValues(model.setSummary(trainingSummary))
+  } else {
+require($(regParam) == 0.0, "The standard deviation of the label 
is zero. " +
+  "Model cannot be regularized.")
+logWarning(s"The standard deviation of the label is zero. " +
+  "Consider setting fitIntercept=true.")
+  }
 }
 
+// if y is constant (rawYStd is zero), then y cannot be scaled. In 
this case
+// setting yStd=1.0 ensures that y is not scaled anymore in l-bfgs 
algorithm.
+val yStd = if (rawYStd > 0) rawYStd else if (yMean != 0.0) 
math.abs(yMean) else 1.0
--- End diff --

`val yStd = if (rawYStd > 0) rawYStd else math.abs(yMean)' since you 
already check the condition before.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread iyounus
Github user iyounus commented on the pull request:

https://github.com/apache/spark/pull/10702#issuecomment-177375091
  
I've completed this PR. I think all the tests are there. Here, I'm going to 
document a couple of minor issues just for future reference.

__Issue 1__
For the case when `yStd = 0` and `fitIntercept = false`, we've four 
possibilities (`reParam: zero/non-zero` and `standardization: true/false`). 
Using `WeightedLeastSquares` (`normal` solver), I _can_ get the following 
results:
```
# data used for the following results
val df = sc.parallelize(Seq(
  (17.0, Vectors.dense(0.0, 5.0)),
  (17.0, Vectors.dense(1.0, 7.0)),
  (17.0, Vectors.dense(2.0, 11.0)),
  (17.0, Vectors.dense(3.0, 13.0))
), 2).toDF("label", "features")
```

```
# coefficients obtained from WeightedLeastSquares
(1) reg: 0.0, standardization: false
> 0.0 [-9.508474576271158,3.457627118644062]

(2) reg: 0.0, standardization: true
> 0.0 [-9.508474576271158,3.457627118644062]

(3) reg: 0.1, standardization: false
> 0.0 [-7.134240246406588,3.010780287474336]

(4) reg: 0.1, standardization: true
> 0.0 [-5.730337078651679,2.7219101123595495]
```
This is with `L2` regularization, and ignoring standardization of the label 
for the case (4). For the case (4), we throw an error because this is 
ill-defined, so the user never sees these results.

For case (3), even though the standardization is `false`, the label is 
still standardized because the `standardizeLable` is hardwired to be `true` 
when calling `WeightedLeastSquares` within `LinearRegression` class. Therefore, 
an error is thrown in this case too. Which, in my opinion, is not right thing 
to do because the analytical solution does exist for this case.

__Issue 2__
Again, for the case when `yStd = 0` and `fitIntercept = false`, I can get 
the following results using `l-bfgs` solver:

```
(1) reg: 0.0, standardization: false
> 0.0 [-9.508474576271176,3.4576271186440652]

(2) reg: 0.0, standardization: true
> 0.0 [-9.508474576271176,3.4576271186440652]

(3) reg: 0.1, standardization: false
> 0.0 [-9.327614273741196,3.423618722197146]

(4) reg: 0.1, standardization: true
> 0.0 [-9.08129403505256,3.374915377479131]
```

Here, results (1) and (2) are identical to what we get from 
`WeightedLeastSquares` as expected. Case (4) is ill-defined and we throw an 
error.

Now, for case (3), the numerical values are different as compared to 
`WeightedLeastSquares`. This is because we standardize label using `yMean`. 
Otherwise, the values obtained from `l-bfgs` are identical to 
`WeightedLeastSquares`. Note that the user will not see these values because an 
error is thrown for this case instead.

__Issue 3__
The normal equation with regression (Ridge Regression), gives significantly 
different results as compared to case (3) above. Here is my R code with results:
```
ridge_regression <- function(A, b, lambda, intercept=TRUE){
if (intercept) {
A = cbind(rep(1.0, length(b)), A)
I = diag(ncol(A))
I[1,1] = 0.0
} else {
I = diag(ncol(A))
}
R = chol( t(A) %*% A + lambda*I )
z = solve(t(R), t(A) %*% b)
w = solve(R, z)
return(w)
}
A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2)
b <- c(17, 17, 17, 17)
df <- as.data.frame(cbind(A, b))

ridge_regression(A, b, 0.1, intercept = FALSE)

[1,] -8.783272
[2,]  3.321237
```
In my opinion, when `standardization=flase`, the results from `normal` 
solver must match this. Even though the user doesn't see this case, it gives me 
less confidence in the implementation of normal equation, because it doesn't 
match this simple case. I also wrote about this at 
https://github.com/apache/spark/pull/10274.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12689][SQL] Migrate DDL parsing to the ...

2016-01-30 Thread viirya
Github user viirya commented on the pull request:

https://github.com/apache/spark/pull/10723#issuecomment-177364338
  
cc @rxin


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12988][SQL] Can't drop columns that con...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12988][SQL] Can't drop columns that con...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12988][SQL] Can't drop columns that con...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10943#issuecomment-177363640
  
**[Test build #50452 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50452/consoleFull)**
 for PR 10943 at commit 
[`8201994`](https://github.com/apache/spark/commit/82019947e9777a93ac4d137aed52e09a6434b56e).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13105] Reject NATURAL JOIN queries rath...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10997#issuecomment-177358778
  
**[Test build #50453 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50453/consoleFull)**
 for PR 10997 at commit 
[`21e42d8`](https://github.com/apache/spark/commit/21e42d893283a754f3d25758562864ceb57043d4).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13105] Reject NATURAL JOIN queries rath...

2016-01-30 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/10997#issuecomment-177357659
  
/cc @cloud-fan for review.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13105] Reject NATURAL JOIN queries rath...

2016-01-30 Thread JoshRosen
GitHub user JoshRosen opened a pull request:

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

[SPARK-13105] Reject NATURAL JOIN queries rather than returning wrong 
answers

In Spark 1.6 and earlier, Spark SQL does not support `NATURAL JOIN` 
queries. However, its SQL parser does not consider `NATURAL` to be a reserved 
word, which causes natural joins to be parsed as regular joins where the left 
table has been aliased. For instance,

```
SELECT * FROM foo NATURAL JOIN bar
```

gets interpreted as `foo JOIN bar` where `foo` is aliased to `natural`.

Rather than doing this, which leads to confusing / wrong results for users 
who expect NATURAL JOIN behavior, Spark should immediately reject these queries 
at analysis time and should provide an informative error message.

As a result, this patch is targeted at Spark 1.6 and earlier.

I chose to implement this check entirely within the parser in order to 
minimize the scope of the changes and to not introduce any new classes into the 
logical plan layer. I considered introducing a new `NaturalJoin` join type, 
parsing the query, then detecting and throwing an error from the analyzer but 
ended up rejecting this approach because I was concerned that adding a new 
class to a sealed trait would break compilation for third-party code which 
pattern-matches on Catalyst join types.

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

$ git pull https://github.com/JoshRosen/spark SPARK-13105

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

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


commit fc15a91de0e163c0ea5b7f7ca031aa62317ed301
Author: Josh Rosen 
Date:   2016-01-31T01:57:36Z

Add regression test for SPARK-13105

commit 21e42d893283a754f3d25758562864ceb57043d4
Author: Josh Rosen 
Date:   2016-01-31T01:57:44Z

Fix SPARK-13105 at the parser level.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10678#issuecomment-177353314
  
**[Test build #50449 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50449/consoleFull)**
 for PR 10678 at commit 
[`ba02f46`](https://github.com/apache/spark/commit/ba02f4695e4bfd07a9bef72f783bef3894d8191e).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-6847][Core][Streaming]Fix stack overflo...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-6847][Core][Streaming]Fix stack overflo...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-6847][Core][Streaming]Fix stack overflo...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10934#issuecomment-177353119
  
**[Test build #50448 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50448/consoleFull)**
 for PR 10934 at commit 
[`20e4509`](https://github.com/apache/spark/commit/20e45095506067f3f5195470e3a390cd4872e531).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12988][SQL] Can't drop columns that con...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10943#issuecomment-177352984
  
**[Test build #50452 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50452/consoleFull)**
 for PR 10943 at commit 
[`8201994`](https://github.com/apache/spark/commit/82019947e9777a93ac4d137aed52e09a6434b56e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12988][SQL] Can't drop columns that con...

2016-01-30 Thread dilipbiswal
Github user dilipbiswal commented on the pull request:

https://github.com/apache/spark/pull/10943#issuecomment-177351900
  
@cloud-fan Hi Wenchen, let me know if i have interpreted your suggestion 
correctly ? Please let me know if something is amiss. df.resolve() has many 
callers .. so i have not changed its name but have added a comment. Let me know 
if you want me to refactor it. Thanks..


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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: [ML][MINOR] Invalid MulticlassClassification r...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [ML][MINOR] Invalid MulticlassClassification r...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/10996#issuecomment-177350347
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/50451/
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: [ML][MINOR] Invalid MulticlassClassification r...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10996#issuecomment-177350319
  
**[Test build #50451 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50451/consoleFull)**
 for PR 10996 at commit 
[`41f5338`](https://github.com/apache/spark/commit/41f533825e080b47f2a31f1dc4cbac0adf39e40f).
 * 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: [ML][MINOR] Invalid MulticlassClassification r...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10996#issuecomment-177347951
  
**[Test build #50451 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50451/consoleFull)**
 for PR 10996 at commit 
[`41f5338`](https://github.com/apache/spark/commit/41f533825e080b47f2a31f1dc4cbac0adf39e40f).


---
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: [ML][MINOR] Invalid MulticlassClassification r...

2016-01-30 Thread Lewuathe
GitHub user Lewuathe opened a pull request:

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

[ML][MINOR] Invalid MulticlassClassification reference in ml-guide

In 
[ml-guide](https://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-cross-validation),
 there is invalid reference to `MulticlassClassificationEvaluator` apidoc.


https://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.ml.evaluation.MultiClassClassificationEvaluator

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

$ git pull https://github.com/Lewuathe/spark fix-typo-in-ml-guide

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

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


commit 41f533825e080b47f2a31f1dc4cbac0adf39e40f
Author: Lewuathe 
Date:   2016-01-31T00:23:17Z

[ML][MINOR] Invalid MulticlassClassification reference in ml-guide




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10702#issuecomment-177343128
  
**[Test build #50450 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50450/consoleFull)**
 for PR 10702 at commit 
[`e83b822`](https://github.com/apache/spark/commit/e83b8223846cc41942469fc4b78e9f0500239e0f).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [Spark-12732][ML] bug fix in linear regression...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10702#issuecomment-177337405
  
**[Test build #50450 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50450/consoleFull)**
 for PR 10702 at commit 
[`e83b822`](https://github.com/apache/spark/commit/e83b8223846cc41942469fc4b78e9f0500239e0f).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10678#issuecomment-177331561
  
**[Test build #50449 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50449/consoleFull)**
 for PR 10678 at commit 
[`ba02f46`](https://github.com/apache/spark/commit/ba02f4695e4bfd07a9bef72f783bef3894d8191e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13100] [SQL] improving the performance ...

2016-01-30 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-13100] [SQL] improving the performance ...

2016-01-30 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/10994#issuecomment-177328648
  
Thanks - merging this in.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-12705] [SPARK-10777] [SQL] Analyzer Rul...

2016-01-30 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/10678#discussion_r51352501
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -521,38 +522,96 @@ class Analyzer(
*/
   object ResolveSortReferences extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-  case s @ Sort(ordering, global, p @ Project(projectList, child))
-  if !s.resolved && p.resolved =>
-val (newOrdering, missing) = resolveAndFindMissing(ordering, p, 
child)
+  // Here, this rule only resolves the missing sort references if the 
child is not Aggregate
+  //   Another rule ResolveAggregateFunctions will resolve that case.
--- End diff --

@cloud-fan I kept the function implementation in the 
`ResolveAggregateFunctions`, but I called the function in 
`ResolveSortReferences`. Since the rule `ResolveAggregateFunctions` covers two 
cases (`filter` and `sort`), I am afraid the code readers might feel confused 
if we split them into two rules. This function call is public. I am not sure if 
this way is appropriate?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request: [SPARK-6847][Core][Streaming]Fix stack overflo...

2016-01-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10934#issuecomment-177327392
  
**[Test build #50448 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50448/consoleFull)**
 for PR 10934 at commit 
[`20e4509`](https://github.com/apache/spark/commit/20e45095506067f3f5195470e3a390cd4872e531).


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