[GitHub] spark pull request #17062: [SPARK-17495] [SQL] Support date, timestamp and i...

2017-03-03 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17062#discussion_r104282564
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala
 ---
@@ -169,6 +171,96 @@ class HashExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 // scalastyle:on nonascii
   }
 
+  test("hive-hash for date type") {
+def checkHiveHashForDateType(dateString: String, expected: Long): Unit 
= {
+  checkHiveHash(
+DateTimeUtils.stringToDate(UTF8String.fromString(dateString)).get,
+DateType,
+expected)
+}
+
+// basic case
+checkHiveHashForDateType("2017-01-01", 17167)
+
+// boundary cases
+checkHiveHashForDateType("-01-01", -719530)
+checkHiveHashForDateType("-12-31", 2932896)
+
+// epoch
+checkHiveHashForDateType("1970-01-01", 0)
+
+// before epoch
+checkHiveHashForDateType("1800-01-01", -62091)
+
+// Invalid input: bad date string. Hive returns 0 for such cases
+intercept[NoSuchElementException](checkHiveHashForDateType("0-0-0", 0))
+
intercept[NoSuchElementException](checkHiveHashForDateType("-1212-01-01", 0))
+
intercept[NoSuchElementException](checkHiveHashForDateType("2016-99-99", 0))
+
+// Invalid input: Empty string. Hive returns 0 for this case
+intercept[NoSuchElementException](checkHiveHashForDateType("", 0))
+
+// Invalid input: February 30th for a leap year. Hive supports this 
but Spark doesn't
+
intercept[NoSuchElementException](checkHiveHashForDateType("2016-02-30", 16861))
+  }
+
+  test("hive-hash for timestamp type") {
+def checkHiveHashForTimestampType(
+timestamp: String,
+expected: Long,
+timeZone: TimeZone = TimeZone.getTimeZone("UTC")): Unit = {
+  checkHiveHash(
+DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp), 
timeZone).get,
+TimestampType,
+expected)
+}
+
+// basic case
+checkHiveHashForTimestampType("2017-02-24 10:56:29", 1445725271)
+
+// with higher precision
+checkHiveHashForTimestampType("2017-02-24 10:56:29.11", 1353936655)
+
+// with different timezone
+checkHiveHashForTimestampType("2017-02-24 10:56:29", 1445732471,
+  TimeZone.getTimeZone("US/Pacific"))
+
+// boundary cases
+checkHiveHashForTimestampType("0001-01-01 00:00:00", 1645926784)
+checkHiveHashForTimestampType("-01-01 00:00:00", -1081818240)
+
+// epoch
+checkHiveHashForTimestampType("1970-01-01 00:00:00", 0)
+
+// before epoch
+checkHiveHashForTimestampType("1800-01-01 03:12:45", -267420885)
+
+// Invalid input: bad timestamp string. Hive returns 0 for such cases
+intercept[NoSuchElementException](checkHiveHashForTimestampType("0-0-0 
0:0:0", 0))
+
intercept[NoSuchElementException](checkHiveHashForTimestampType("-99-99-99 
99:99:45", 0))
+
intercept[NoSuchElementException](checkHiveHashForTimestampType("55-5-",
 0))
+
+// Invalid input: Empty string. Hive returns 0 for this case
+intercept[NoSuchElementException](checkHiveHashForTimestampType("", 0))
+
+// Invalid input: February 30th for a leap year. Hive supports this 
but Spark doesn't
+
intercept[NoSuchElementException](checkHiveHashForTimestampType("2016-02-30 
00:00:00", 0))
+
+// Invalid input: Hive accepts upto 9 decimal place precision but 
Spark uses upto 6
+
intercept[TestFailedException](checkHiveHashForTimestampType("2017-02-24 
10:56:29.", 0))
+  }
+
+  test("hive-hash for CalendarInterval type") {
+def checkHiveHashForTimestampType(interval: String, expected: Long): 
Unit = {
+  checkHiveHash(CalendarInterval.fromString(interval), 
CalendarIntervalType, expected)
+}
+
+checkHiveHashForTimestampType("interval 1 day", 3220073)
+checkHiveHashForTimestampType("interval 6 day 15 hour", 21202073)
+checkHiveHashForTimestampType("interval -23 day 56 hour -113 
minute 9898989 second",
+  -2128468593)
--- End diff --

Coud you add more test cases?

```
checkHiveHashForTimestampType("interval 0 day 0 hour 0 minute 0 
second", 23273)
checkHiveHashForTimestampType("interval 0 day 0 hour", 23273)
checkHiveHashForTimestampType("interval -1 day", 3220036)
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure 

[GitHub] spark issue #17125: [SPARK-19211][SQL] Explicitly prevent Insert into View o...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17125: [SPARK-19211][SQL] Explicitly prevent Insert into View o...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17125: [SPARK-19211][SQL] Explicitly prevent Insert into View o...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17125
  
**[Test build #73894 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73894/testReport)**
 for PR 17125 at commit 
[`68cee40`](https://github.com/apache/spark/commit/68cee40fc7a5378d009e3ddc54e80a5fb23531fc).
 * 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 #16842: [SPARK-19304] [Streaming] [Kinesis] fix kinesis s...

2017-03-03 Thread Gauravshah
Github user Gauravshah commented on a diff in the pull request:

https://github.com/apache/spark/pull/16842#discussion_r104282408
  
--- Diff: 
external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala
 ---
@@ -193,9 +201,10 @@ class KinesisSequenceRangeIterator(
   /**
* Get records starting from or after the given sequence number.
*/
-  private def getRecords(iteratorType: ShardIteratorType, seqNum: String): 
Iterator[Record] = {
+  private def getRecords(iteratorType: ShardIteratorType, seqNum: String,
--- End diff --

👍 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17138: [SPARK-17080] [SQL] join reorder

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17138#discussion_r104281655
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -288,6 +288,9 @@ case class Join(
 condition: Option[Expression])
   extends BinaryNode with PredicateHelper {
 
+  /** Whether this join node is ordered before. Set true after it has been 
ordered. */
+  var ordered: Boolean = false
--- End diff --

But it's a `transform`, i.e. after we reroder a, b, c, d to a new plan A J 
B J C J D, we will reorder a, b, c again. Is my understanding right?


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

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



[GitHub] spark pull request #17160: [SPARK-19701][SQL][PYTHON] Throws a correct excep...

2017-03-03 Thread binhnguyen1512
Github user binhnguyen1512 commented on a diff in the pull request:

https://github.com/apache/spark/pull/17160#discussion_r104281624
  
--- Diff: python/pyspark/sql/column.py ---
@@ -180,7 +180,9 @@ def __init__(self, jc):
 __ror__ = _bin_op("or")
 
 # container operators
-__contains__ = _bin_op("contains")
+def __contains__(self, item):
+raise ValueError("Cannot apply 'in' operator against a column: 
please use 'contains' "
+ "in a string column or 'array_contains' function 
for an array column.")
--- End diff --

Binhnguyen1512


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17160: [SPARK-19701][SQL][PYTHON] Throws a correct excep...

2017-03-03 Thread binhnguyen1512
Github user binhnguyen1512 commented on a diff in the pull request:

https://github.com/apache/spark/pull/17160#discussion_r104281627
  
--- Diff: python/pyspark/sql/column.py ---
@@ -180,7 +180,9 @@ def __init__(self, jc):
 __ror__ = _bin_op("or")
 
 # container operators
-__contains__ = _bin_op("contains")
+def __contains__(self, item):
+raise ValueError("Cannot apply 'in' operator against a column: 
please use 'contains' "
+ "in a string column or 'array_contains' function 
for an array column.")
--- End diff --

Binhnguyen1512


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17138: [SPARK-17080] [SQL] join reorder

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17138#discussion_r104281605
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala ---
@@ -60,6 +60,12 @@ trait CatalystConf {
* Enables CBO for estimation of plan statistics when set true.
*/
   def cboEnabled: Boolean
+
+  /** Enables join reorder in CBO. */
+  def joinReorderEnabled: Boolean
--- End diff --

`cboEnabled` controls all things related to stats computation, I think we 
may need a separate flag for join reorder only.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17138: [SPARK-17080] [SQL] join reorder

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17138#discussion_r104281601
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala ---
@@ -60,6 +60,12 @@ trait CatalystConf {
* Enables CBO for estimation of plan statistics when set true.
*/
   def cboEnabled: Boolean
+
+  /** Enables join reorder in CBO. */
+  def joinReorderEnabled: Boolean
--- End diff --

`cboEnabled` controls all things related to stats computation, I think we 
may need a separate flag for join reorder only.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17138: [SPARK-17080] [SQL] join reorder

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17138#discussion_r104281554
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala
 ---
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, 
Expression, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike}
+import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, 
Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+
+/**
+ * Cost-based join reorder.
+ * We may have several join reorder algorithms in the future. This class 
is the entry of these
+ * algorithms, and chooses which one to use.
+ */
+case class CostBasedJoinReorder(conf: CatalystConf) extends 
Rule[LogicalPlan] with PredicateHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+if (!conf.cboEnabled || !conf.joinReorderEnabled) {
+  plan
+} else {
+  plan transform {
+case p @ Project(projectList, j @ Join(_, _, _: InnerLike, _)) if 
!j.ordered =>
+  reorder(j, p.outputSet)
+case j @ Join(_, _, _: InnerLike, _) if !j.ordered =>
+  reorder(j, j.outputSet)
+  }
+}
+  }
+
+  def reorder(plan: LogicalPlan, output: AttributeSet): LogicalPlan = {
+val (items, conditions) = extractInnerJoins(plan)
+val result =
+  if (items.size > 2 && items.size <= conf.joinReorderDPThreshold && 
conditions.nonEmpty) {
+JoinReorderDP(conf, items, conditions, 
output).search().getOrElse(plan)
+  } else {
+plan
+  }
+// Set all inside joins ordered.
+setOrdered(result)
+result
+  }
+
+  /**
+   * Extract inner joinable items and join conditions.
+   * This method works for bushy trees and left/right deep trees.
+   */
+  def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], 
Set[Expression]) = plan match {
+case j @ Join(left, right, _: InnerLike, cond) =>
+  val (leftPlans, leftConditions) = extractInnerJoins(left)
+  val (rightPlans, rightConditions) = extractInnerJoins(right)
+  (leftPlans ++ rightPlans, 
cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
+leftConditions ++ rightConditions)
+case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
+  val (leftPlans, leftConditions) = extractInnerJoins(left)
+  val (rightPlans, rightConditions) = extractInnerJoins(right)
+  (leftPlans ++ rightPlans, 
cond.map(splitConjunctivePredicates).getOrElse(Nil).toSet ++
+leftConditions ++ rightConditions)
+case _ =>
+  (Seq(plan), Set())
+  }
+
+  def setOrdered(plan: LogicalPlan): Unit = plan match {
+case j @ Join(left, right, _: InnerLike, cond) =>
+  j.ordered = true
+  setOrdered(left)
+  setOrdered(right)
+case Project(_, j @ Join(left, right, _: InnerLike, cond)) =>
+  j.ordered = true
+  setOrdered(left)
+  setOrdered(right)
+case _ =>
+  }
+}
+
+/**
+ * Reorder the joins using a dynamic programming algorithm:
+ * First we put all items (basic joined nodes) into level 1, then we build 
all two-way joins
+ * at level 2 from plans at level 1 (single items), then build all 3-way 
joins from plans
+ * at previous levels (two-way joins and single items), then 4-way joins 
... etc, until we
+ * build all n-way joins and pick the best plan among them.
+ *
+ * When building m-way joins, we only keep the best plan (with the lowest 
cost) for the same set
+ * of m items. E.g., for 3-way joins, we keep only the best plan for items 
{A, B, C} among
+ * plans (A J B) J C, (A J C) J 

[GitHub] spark pull request #16696: [SPARK-19350] [SQL] Cardinality estimation of Lim...

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/16696#discussion_r104281417
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -773,14 +773,20 @@ case class LocalLimit(limitExpr: Expression, child: 
LogicalPlan) extends UnaryNo
   }
   override def computeStats(conf: CatalystConf): Statistics = {
 val limit = limitExpr.eval().asInstanceOf[Int]
-val sizeInBytes = if (limit == 0) {
+val childStats = child.stats(conf)
+if (limit == 0) {
   // sizeInBytes can't be zero, or sizeInBytes of BinaryNode will also 
be zero
   // (product of children).
-  1
+  Statistics(
+sizeInBytes = 1,
+rowCount = Some(0),
+isBroadcastable = childStats.isBroadcastable)
 } else {
-  (limit: Long) * output.map(a => a.dataType.defaultSize).sum
+  // The output row count of LocalLimit should be the sum of row count 
from each partition, but
+  // since the partition number is not available here, we just use 
statistics of the child
+  // except column stats, because we don't know the distribution after 
a limit operation
--- End diff --

A loose bound can lead to significant under-estimation. E.g. a > 50, after 
limit the actual range is [40, 60], while max and min in stats are still [0, 
60], then the filter factor will be 1/6 instead of 1/2.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17125: [SPARK-19211][SQL] Explicitly prevent Insert into...

2017-03-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17125#discussion_r104281325
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -604,7 +604,7 @@ class Analyzer(
 
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
   case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
-i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+i.copy(table = 
resolveRelation(EliminateSubqueryAliases(lookupTableFromCatalog(u
--- End diff --

so we will resolve something to `View` by doing 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 #17125: [SPARK-19211][SQL] Explicitly prevent Insert into...

2017-03-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17125#discussion_r104281300
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
@@ -1329,6 +1329,15 @@ class SparkSqlAstBuilder(conf: SQLConf) extends 
AstBuilder {
 if (ctx.identifierList != null) {
   operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx)
 } else {
+  // CREATE VIEW ... AS INSERT INTO is not allowed.
+  ctx.query.queryNoWith match {
+case s: SingleInsertQueryContext if s.insertInto != null =>
--- End diff --

hmm, a select query is `SingleInsertQueryContext`?


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

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



[GitHub] spark issue #16929: [SPARK-19595][SQL] Support json array in from_json

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16929
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73893/
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 #16696: [SPARK-19350] [SQL] Cardinality estimation of Lim...

2017-03-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/16696#discussion_r104281282
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -773,14 +773,20 @@ case class LocalLimit(limitExpr: Expression, child: 
LogicalPlan) extends UnaryNo
   }
   override def computeStats(conf: CatalystConf): Statistics = {
 val limit = limitExpr.eval().asInstanceOf[Int]
-val sizeInBytes = if (limit == 0) {
+val childStats = child.stats(conf)
+if (limit == 0) {
   // sizeInBytes can't be zero, or sizeInBytes of BinaryNode will also 
be zero
   // (product of children).
-  1
+  Statistics(
+sizeInBytes = 1,
+rowCount = Some(0),
+isBroadcastable = childStats.isBroadcastable)
 } else {
-  (limit: Long) * output.map(a => a.dataType.defaultSize).sum
+  // The output row count of LocalLimit should be the sum of row count 
from each partition, but
+  // since the partition number is not available here, we just use 
statistics of the child
+  // except column stats, because we don't know the distribution after 
a limit operation
--- End diff --

hmm, what's the strategy here? is a loose bound better than nothing?


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

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



[GitHub] spark issue #16929: [SPARK-19595][SQL] Support json array in from_json

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #16929: [SPARK-19595][SQL] Support json array in from_json

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16929
  
**[Test build #73893 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73893/testReport)**
 for PR 16929 at commit 
[`3d490e3`](https://github.com/apache/spark/commit/3d490e34136ec76deaed15ebe8d6e7e8aac96776).
 * 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 #17148: [SPARK-17075][SQL][followup] fix filter estimatio...

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17148#discussion_r104281195
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala
 ---
@@ -157,7 +157,7 @@ class FilterEstimationSuite extends 
StatsEstimationTestBase {
   Filter(IsNull(arInt), childStatsTestPlan(Seq(arInt), 10L)),
   ColumnStat(distinctCount = 0, min = None, max = None,
 nullCount = 0, avgLen = 4, maxLen = 4),
-  0)
+  rowCount = 0)
   }
 
   test("cint IS NOT NULL") {
--- End diff --

ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17148: [SPARK-17075][SQL][followup] fix filter estimatio...

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17148#discussion_r104281194
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala
 ---
@@ -249,17 +251,19 @@ case class FilterEstimation(plan: Filter, 
catalystConf: CatalystConf) extends Lo
   attr: Attribute,
   literal: Literal,
   update: Boolean): Option[Double] = {
+if (!colStatsMap.contains(attr)) {
+  logDebug("[CBO] No statistics for " + attr)
+  return None
--- End diff --

If we don't have stats, there's no need to go through the logic below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17052: [SPARK-19690][SS] Join a streaming DataFrame with...

2017-03-03 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/17052#discussion_r104281176
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
 ---
@@ -31,6 +31,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] 
with Logging {
 
   private var _analyzed: Boolean = false
 
+  private var _incremental: Boolean = false
--- End diff --

Adding it here will break `sameResult`, `equals` and other methods. Could 
you add a new parameter to the constructor of `LogicalRelation` and 
`LogicalRDD` instead?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17148: [SPARK-17075][SQL][followup] fix filter estimatio...

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17148#discussion_r104280865
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala
 ---
@@ -469,22 +487,28 @@ case class FilterEstimation(plan: Filter, 
catalystConf: CatalystConf) extends Lo
 val newValue = convertBoundValue(attr.dataType, literal.value)
 var newMax = colStat.max
 var newMin = colStat.min
+var newNdv = (ndv * percent).setScale(0, 
RoundingMode.HALF_UP).toBigInt()
+if (newNdv < 1) newNdv = 1
+
 op match {
-  case _: GreaterThan => newMin = newValue
-  case _: GreaterThanOrEqual => newMin = newValue
-  case _: LessThan => newMax = newValue
-  case _: LessThanOrEqual => newMax = newValue
+  case _: GreaterThan =>
+if (newNdv == 1) newMin = newMax else newMin = newValue
--- End diff --

if the new ndv is 1, then new max and new min must be equal.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17148: [SPARK-17075][SQL][followup] fix filter estimatio...

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17148#discussion_r104280853
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala
 ---
@@ -414,53 +422,63 @@ case class FilterEstimation(plan: Filter, 
catalystConf: CatalystConf) extends Lo
   literal: Literal,
   update: Boolean): Option[Double] = {
 
-var percent = 1.0
 val colStat = colStatsMap(attr)
-val statsRange =
-  Range(colStat.min, colStat.max, 
attr.dataType).asInstanceOf[NumericRange]
+val statsRange = Range(colStat.min, colStat.max, 
attr.dataType).asInstanceOf[NumericRange]
+val max = BigDecimal(statsRange.max)
+val min = BigDecimal(statsRange.min)
+val ndv = BigDecimal(colStat.distinctCount)
 
 // determine the overlapping degree between predicate range and 
column's range
-val literalValueBD = BigDecimal(literal.value.toString)
+val numericLiteral = if (literal.dataType.isInstanceOf[BooleanType]) {
+  if (literal.value.asInstanceOf[Boolean]) BigDecimal(1) else 
BigDecimal(0)
+} else {
+  BigDecimal(literal.value.toString)
+}
 val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
   case _: LessThan =>
-(literalValueBD <= statsRange.min, literalValueBD > statsRange.max)
+(numericLiteral <= min, numericLiteral > max)
   case _: LessThanOrEqual =>
-(literalValueBD < statsRange.min, literalValueBD >= statsRange.max)
+(numericLiteral < min, numericLiteral >= max)
   case _: GreaterThan =>
-(literalValueBD >= statsRange.max, literalValueBD < statsRange.min)
+(numericLiteral >= max, numericLiteral < min)
   case _: GreaterThanOrEqual =>
-(literalValueBD > statsRange.max, literalValueBD <= statsRange.min)
+(numericLiteral > max, numericLiteral <= min)
 }
 
+var percent = BigDecimal(1.0)
 if (noOverlap) {
   percent = 0.0
 } else if (completeOverlap) {
   percent = 1.0
 } else {
-  // this is partial overlap case
-  val literalDouble = literalValueBD.toDouble
-  val maxDouble = BigDecimal(statsRange.max).toDouble
-  val minDouble = BigDecimal(statsRange.min).toDouble
-
+  // This is the partial overlap case:
   // Without advanced statistics like histogram, we assume uniform 
data distribution.
   // We just prorate the adjusted range over the initial range to 
compute filter selectivity.
-  // For ease of computation, we convert all relevant numeric values 
to Double.
+  assert(max > min)
--- End diff --

it's in the partial overlap case, if max == min, it must be either no 
overlap or complete overlap for a binary expression. see here:
```
val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
  case _: LessThan =>
(numericLiteral <= min, numericLiteral > max)
  case _: LessThanOrEqual =>
(numericLiteral < min, numericLiteral >= max)
  case _: GreaterThan =>
(numericLiteral >= max, numericLiteral < min)
  case _: GreaterThanOrEqual =>
(numericLiteral > max, numericLiteral <= min)
}
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17148: [SPARK-17075][SQL][followup] fix filter estimatio...

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17148#discussion_r104280815
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala
 ---
@@ -323,7 +331,7 @@ case class FilterEstimation(plan: Filter, catalystConf: 
CatalystConf) extends Lo
 colStatsMap(attr) = newStats
   }
 
-  Some(1.0 / ndv.toDouble)
+  Some((1.0 / BigDecimal(ndv)).toDouble)
--- End diff --

ndv is a BigInt, its range is bigger than double, so `toDouble` is not safe 
here, while `1/ndv` is in (0, 1), so `toDouble` is safe


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #11601: [SPARK-13568] [ML] Create feature transformer to ...

2017-03-03 Thread hhbyyh
Github user hhbyyh commented on a diff in the pull request:

https://github.com/apache/spark/pull/11601#discussion_r104280679
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.ml.feature
+
+import org.apache.spark.{SparkException, SparkFunSuite}
+import org.apache.spark.ml.util.DefaultReadWriteTest
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.mllib.util.TestingUtils._
+import org.apache.spark.sql.{DataFrame, Row}
+
+class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with 
DefaultReadWriteTest {
+
+  test("Imputer for Double with default missing Value NaN") {
+val df = spark.createDataFrame( Seq(
+  (0, 1.0, 1.0, 1.0),
+  (1, 1.0, 1.0, 1.0),
+  (2, 3.0, 3.0, 3.0),
+  (3, 4.0, 4.0, 4.0),
+  (4, Double.NaN, 2.25, 1.0)
+)).toDF("id", "value", "expected_mean", "expected_median")
+val imputer = new 
Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
+ImputerSuite.iterateStrategyTest(imputer, df)
+  }
+
+  test("Imputer should handle NaNs when computing surrogate value, if 
missingValue is not NaN") {
+val df = spark.createDataFrame( Seq(
+  (0, 1.0, 1.0, 1.0),
+  (1, 3.0, 3.0, 3.0),
+  (2, Double.NaN, Double.NaN, Double.NaN),
+  (3, -1.0, 2.0, 3.0)
+)).toDF("id", "value", "expected_mean", "expected_median")
+val imputer = new 
Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
+  .setMissingValue(-1.0)
+ImputerSuite.iterateStrategyTest(imputer, df)
+  }
+
+  test("Imputer for Float with missing Value -1.0") {
+val df = spark.createDataFrame( Seq(
+  (0, 1.0F, 1.0F, 1.0F),
+  (1, 3.0F, 3.0F, 3.0F),
+  (2, 10.0F, 10.0F, 10.0F),
+  (3, 10.0F, 10.0F, 10.0F),
+  (4, -1.0F, 6.0F, 3.0F)
+)).toDF("id", "value", "expected_mean", "expected_median")
+val imputer = new 
Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
+  .setMissingValue(-1)
+ImputerSuite.iterateStrategyTest(imputer, df)
+  }
+
+  test("Imputer should impute null as well as 'missingValue'") {
+val df = spark.createDataFrame( Seq(
+  (0, 4.0, 4.0, 4.0),
+  (1, 10.0, 10.0, 10.0),
+  (2, 10.0, 10.0, 10.0),
+  (3, Double.NaN, 8.0, 10.0),
+  (4, -1.0, 8.0, 10.0)
+)).toDF("id", "value", "expected_mean", "expected_median")
+val df2 = df.selectExpr("*", "IF(value=-1.0, null, value) as 
nullable_value")
+val imputer = new 
Imputer().setInputCols(Array("nullable_value")).setOutputCols(Array("out"))
+ImputerSuite.iterateStrategyTest(imputer, df2)
+  }
+
+
+  test("Imputer throws exception when surrogate cannot be computed") {
+val df = spark.createDataFrame( Seq(
+  (0, Double.NaN, 1.0, 1.0),
+  (1, Double.NaN, 3.0, 3.0),
+  (2, Double.NaN, Double.NaN, Double.NaN)
+)).toDF("id", "value", "expected_mean", "expected_median")
+Seq("mean", "median").foreach { strategy =>
+  val imputer = new 
Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
+.setStrategy(strategy)
+  intercept[SparkException] {
+val model = imputer.fit(df)
+  }
+}
+  }
+
+  test("Imputer read/write") {
+val t = new Imputer()
+  .setInputCols(Array("myInputCol"))
+  .setOutputCols(Array("myOutputCol"))
+  .setMissingValue(-1.0)
+testDefaultReadWrite(t)
+  }
+
+  test("ImputerModel read/write") {
+val spark = this.spark
+import spark.implicits._
+val surrogateDF = Seq(1.234).toDF("myInputCol")
--- End diff --

this happens to be the correct column name for now.


---
If your project is set up for it, you can reply to this email and have 

[GitHub] spark issue #16981: [SPARK-19637][SQL] Add to_json in FunctionRegistry

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16981
  
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 #16696: [SPARK-19350] [SQL] Cardinality estimation of Lim...

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/16696#discussion_r104280684
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsConfSuite.scala
 ---
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.catalyst.statsEstimation
-
-import org.apache.spark.sql.catalyst.CatalystConf
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, 
AttributeReference}
-import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, 
LogicalPlan, Statistics}
-import org.apache.spark.sql.types.IntegerType
-
-
-class StatsConfSuite extends StatsEstimationTestBase {
--- End diff --

How to use `git mv` now? Do I need to revert to the unchanged version, and 
`git mv`, and then do all the changes all over again?


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

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



[GitHub] spark issue #16981: [SPARK-19637][SQL] Add to_json in FunctionRegistry

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #16981: [SPARK-19637][SQL] Add to_json in FunctionRegistry

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #17152: [SPARK-18389][SQL] Disallow cyclic view reference

2017-03-03 Thread jiangxb1987
Github user jiangxb1987 commented on the issue:

https://github.com/apache/spark/pull/17152
  
@gatorsmile Currently we don't perform recursive resolution over a 
temporary view, so perhaps that won't trigger a cyclic view reference. For 
example:
```
scala> spark.sql("CREATE TEMPORARY VIEW v1 AS SELECT * FROM tab")
res3: org.apache.spark.sql.DataFrame = []

scala> spark.sql("CREATE TEMPORARY VIEW v2 AS SELECT * FROM v1")
res4: org.apache.spark.sql.DataFrame = []

scala> spark.sql("ALTER VIEW v1 AS SELECT * FROM v2")
res5: org.apache.spark.sql.DataFrame = []

scala> spark.sql("SELECT * FROM v1")
res6: org.apache.spark.sql.DataFrame = [a: int, b: string]
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16696: [SPARK-19350] [SQL] Cardinality estimation of Lim...

2017-03-03 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/16696#discussion_r104280554
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -773,14 +773,20 @@ case class LocalLimit(limitExpr: Expression, child: 
LogicalPlan) extends UnaryNo
   }
   override def computeStats(conf: CatalystConf): Statistics = {
 val limit = limitExpr.eval().asInstanceOf[Int]
-val sizeInBytes = if (limit == 0) {
+val childStats = child.stats(conf)
+if (limit == 0) {
   // sizeInBytes can't be zero, or sizeInBytes of BinaryNode will also 
be zero
   // (product of children).
-  1
+  Statistics(
+sizeInBytes = 1,
+rowCount = Some(0),
+isBroadcastable = childStats.isBroadcastable)
 } else {
-  (limit: Long) * output.map(a => a.dataType.defaultSize).sum
+  // The output row count of LocalLimit should be the sum of row count 
from each partition, but
+  // since the partition number is not available here, we just use 
statistics of the child
+  // except column stats, because we don't know the distribution after 
a limit operation
--- End diff --

How can we make sure max/min values are still there after limit? Otherwise 
it will be a very loose bound of max/min.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17125: [SPARK-19211][SQL] Explicitly prevent Insert into...

2017-03-03 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/17125#discussion_r104280528
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -604,7 +604,7 @@ class Analyzer(
 
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
   case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
-i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+i.copy(table = 
resolveRelation(EliminateSubqueryAliases(lookupTableFromCatalog(u
--- End diff --

When we try to insert into a view, the logical plan that 
`lookupTableFromCatalog()` returns is not expanded, so we have to perform 
`resolveRelation()` over the node.


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

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



[GitHub] spark issue #17160: [SPARK-19701][SQL][PYTHON] Throws a correct exception fo...

2017-03-03 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/17160
  
what if we just remove `__contains__`, `__nonzero__ ` and `__bool__`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17125: [SPARK-19211][SQL] Explicitly prevent Insert into...

2017-03-03 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/17125#discussion_r104280480
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
@@ -1329,6 +1329,15 @@ class SparkSqlAstBuilder(conf: SQLConf) extends 
AstBuilder {
 if (ctx.identifierList != null) {
   operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx)
 } else {
+  // CREATE VIEW ... AS INSERT INTO is not allowed.
+  ctx.query.queryNoWith match {
+case s: SingleInsertQueryContext if s.insertInto != null =>
--- End diff --

For example, `CREATE VIEW v AS SELECT * FROM jt`.


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

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



[GitHub] spark issue #17125: [SPARK-19211][SQL] Explicitly prevent Insert into View o...

2017-03-03 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/17125
  
LGTM except 2 questions


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17125: [SPARK-19211][SQL] Explicitly prevent Insert into...

2017-03-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17125#discussion_r104280434
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -604,7 +604,7 @@ class Analyzer(
 
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
   case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
-i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+i.copy(table = 
resolveRelation(EliminateSubqueryAliases(lookupTableFromCatalog(u
--- End diff --

why this change?


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

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



[GitHub] spark pull request #17125: [SPARK-19211][SQL] Explicitly prevent Insert into...

2017-03-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17125#discussion_r104280410
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
@@ -1329,6 +1329,15 @@ class SparkSqlAstBuilder(conf: SQLConf) extends 
AstBuilder {
 if (ctx.identifierList != null) {
   operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx)
 } else {
+  // CREATE VIEW ... AS INSERT INTO is not allowed.
+  ctx.query.queryNoWith match {
+case s: SingleInsertQueryContext if s.insertInto != null =>
--- End diff --

when `s.insertInto` will be null?


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

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



[GitHub] spark issue #17125: [SPARK-19211][SQL] Explicitly prevent Insert into View o...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17125
  
**[Test build #73894 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73894/testReport)**
 for PR 17125 at commit 
[`68cee40`](https://github.com/apache/spark/commit/68cee40fc7a5378d009e3ddc54e80a5fb23531fc).


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

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



[GitHub] spark issue #17155: [SPARK-19815][SQL] Not orderable should be applied to ri...

2017-03-03 Thread zhzhan
Github user zhzhan commented on the issue:

https://github.com/apache/spark/pull/17155
  
@gatorsmile  Thanks for reviewing this. I am thinking the logic again. On 
the surface, the logic may be correct. Since in the join, the left and right 
key should be the same type. Will close the 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 #17155: [SPARK-19815][SQL] Not orderable should be applie...

2017-03-03 Thread zhzhan
Github user zhzhan closed the pull request at:

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


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

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



[GitHub] spark issue #17157: [SPARK-16845][SQL][BRANCH-2.0] `GeneratedClass$SpecificO...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17157: [SPARK-16845][SQL][BRANCH-2.0] `GeneratedClass$SpecificO...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17157: [SPARK-16845][SQL][BRANCH-2.0] `GeneratedClass$SpecificO...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17157
  
**[Test build #73887 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73887/consoleFull)**
 for PR 17157 at commit 
[`399d950`](https://github.com/apache/spark/commit/399d950d790ce5839bec9b59a1200ae0bf67fcf2).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark issue #17158: [SPARK-16845][SQL][BRANCH-1.6] `GeneratedClass$SpecificO...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17158: [SPARK-16845][SQL][BRANCH-1.6] `GeneratedClass$SpecificO...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17158: [SPARK-16845][SQL][BRANCH-1.6] `GeneratedClass$SpecificO...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17158
  
**[Test build #73890 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73890/consoleFull)**
 for PR 17158 at commit 
[`441aafc`](https://github.com/apache/spark/commit/441aafcb4abb603c03e30bfe1de7afec32b16873).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark issue #16854: [WIP][SPARK-15463][SQL] Add an API to load DataFrame fro...

2017-03-03 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/16854
  
Oh, no. It does not need to. 

I just meant to de-duplicate some logics by 
https://github.com/apache/spark/pull/16854#discussion_r103894945. Let me just 
remove that part and leave only code changes dedicated for this JIRA. It seems 
making reviewers confused. Let me clean up soon.


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

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



[GitHub] spark issue #16929: [SPARK-19595][SQL] Support json array in from_json

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #17120: [SPARK-19715][Structured Streaming] Option to Strip Path...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17120: [SPARK-19715][Structured Streaming] Option to Strip Path...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17120: [SPARK-19715][Structured Streaming] Option to Strip Path...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #17160: [SPARK-19701][SQL][PYTHON] Throws a correct exception fo...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17160: [SPARK-19701][SQL][PYTHON] Throws a correct exception fo...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17160: [SPARK-19701][SQL][PYTHON] Throws a correct exception fo...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #17120: [SPARK-19715][Structured Streaming] Option to Strip Path...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17120: [SPARK-19715][Structured Streaming] Option to Strip Path...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17120: [SPARK-19715][Structured Streaming] Option to Strip Path...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #15666: [SPARK-11421] [Core][Python][R] Added ability for addJar...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15666: [SPARK-11421] [Core][Python][R] Added ability for addJar...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15666: [SPARK-11421] [Core][Python][R] Added ability for addJar...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #17155: [SPARK-19815][SQL] Not orderable should be applied to ri...

2017-03-03 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/17155
  
Add a test case to 
https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala?


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

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



[GitHub] spark issue #16929: [SPARK-19595][SQL] Support json array in from_json

2017-03-03 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/16929
  
Thank you so much. Let me clean up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17160: [SPARK-19701][SQL][PYTHON] Throws a correct excep...

2017-03-03 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/17160#discussion_r104278452
  
--- Diff: python/pyspark/sql/column.py ---
@@ -180,7 +180,9 @@ def __init__(self, jc):
 __ror__ = _bin_op("or")
 
 # container operators
-__contains__ = _bin_op("contains")
+def __contains__(self, item):
+raise ValueError("Cannot apply 'in' operator against a column: 
please use 'contains' "
+ "in a string column or 'array_contains' function 
for an array column.")
--- End diff --

What I meant here is use

```python
>>> df = spark.range(1)
>>> df.select(df.id.contains(0)).show()
```
```
+---+
|contains(id, 0)|
+---+
|   true|
+---+
```

or

```python
>>> from pyspark.sql.functions import array_contains
>>> df = spark.createDataFrame([[[0]]], ["id"])
>>> df.select(array_contains(df.id, 0)).show()
```
```
+-+
|array_contains(id, 0)|
+-+
| true|
+-+
```


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

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



[GitHub] spark issue #16981: [SPARK-19637][SQL] Add to_json in FunctionRegistry

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16981
  
**[Test build #73892 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73892/testReport)**
 for PR 16981 at commit 
[`0468280`](https://github.com/apache/spark/commit/04682803c7ee3afd70a1897e5c5d702e8cc5e9c6).


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

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



[GitHub] spark issue #17160: [SPARK-19701][SQL][PYTHON] Throws a correct exception fo...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17160
  
**[Test build #73891 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73891/testReport)**
 for PR 17160 at commit 
[`509747c`](https://github.com/apache/spark/commit/509747ca82aecd07541cc5b0c06fd5f4c147c3cf).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16981: [SPARK-19637][SQL] Add to_json in FunctionRegistr...

2017-03-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/16981#discussion_r104278366
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -566,3 +586,17 @@ case class StructToJson(
 
   override def inputTypes: Seq[AbstractDataType] = StructType :: Nil
 }
+
+object StructToJson {
+
+  def convertToMapData(exp: Expression): Map[String, String] = exp match {
+case m: CreateMap
+if m.dataType.acceptsType(MapType(StringType, StringType, 
valueContainsNull = false)) =>
+  val arrayMap = m.eval().asInstanceOf[ArrayBasedMapData]
+  ArrayBasedMapData.toScalaMap(arrayMap).map { case (key, value) =>
+key.toString -> value.toString
+  }
+case _ =>
+  throw new AnalysisException("Must use a map() function for options")
--- End diff --

How about this?

https://github.com/apache/spark/pull/16981/files#diff-6626026091295ad8c0dfb66ecbcd04b1R601


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

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



[GitHub] spark issue #17160: [SPARK-19701][SQL][PYTHON] Throws a correct exception fo...

2017-03-03 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/17160
  
cc @cloud-fan, @davies and @holdenk.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17160: [SPARK-19701][SQL][PYTHON] Throws a correct excep...

2017-03-03 Thread HyukjinKwon
GitHub user HyukjinKwon opened a pull request:

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

[SPARK-19701][SQL][PYTHON] Throws a correct exception for 'in' operator 
against column

## What changes were proposed in this pull request?

This PR proposes to remove incorrect implementation that has been not 
executed so far (at least from Spark 1.5.2) for `in` operator and throw a 
correct exception rather than saying it is a bool. I tested the codes above in 
1.5.2, 1.6.3, 2.1.0 and in the master branch as below:

**1.5.2**

```python
>>> df = sqlContext.createDataFrame([[1]])
>>> 1 in df._1
Traceback (most recent call last):
  File "", line 1, in 
  File ".../spark-1.5.2-bin-hadoop2.6/python/pyspark/sql/column.py", line 
418, in __nonzero__
raise ValueError("Cannot convert column into bool: please use '&' for 
'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' 
for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**1.6.3**

```python
>>> 1 in sqlContext.range(1).id
Traceback (most recent call last):
  File "", line 1, in 
  File ".../spark-1.6.3-bin-hadoop2.6/python/pyspark/sql/column.py", line 
447, in __nonzero__
raise ValueError("Cannot convert column into bool: please use '&' for 
'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' 
for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**2.1.0**

```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
  File "", line 1, in 
  File ".../spark-2.1.0-bin-hadoop2.7/python/pyspark/sql/column.py", line 
426, in __nonzero__
raise ValueError("Cannot convert column into bool: please use '&' for 
'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' 
for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**Current Master**

```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
  File "", line 1, in 
  File ".../spark/python/pyspark/sql/column.py", line 452, in __nonzero__
raise ValueError("Cannot convert column into bool: please use '&' for 
'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' 
for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**After**

```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
  File "", line 1, in 
  File ".../spark/python/pyspark/sql/column.py", line 184, in __contains__
raise ValueError("Cannot apply 'in' operator against a column: please 
use 'contains' "
ValueError: Cannot apply 'in' operator against a column: please use 
'contains' in a string column or 'array_contains' function for an array column.
```

In more details,

It seems the implementation indented to support this

```python
1 in df.column
```

However, currently, it throws an exception as below:

```python
Traceback (most recent call last):
  File "", line 1, in 
  File ".../spark/python/pyspark/sql/column.py", line 426, in __nonzero__
raise ValueError("Cannot convert column into bool: please use '&' for 
'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' 
for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

What happens here is as below:

```python
class Column(object):
def __contains__(self, item):
print "I am contains"
return Column()
def __nonzero__(self):
raise Exception("I am nonzero.")

>>> 1 in Column()
I am contains
Traceback (most recent call last):
  File "", line 1, in 
  File "", line 6, in __nonzero__
Exception: I am nonzero.
```

It seems it calls `__contains__` first and then `__nonzero__` or `__bool__` 
is being called against `Column()` to make this a bool (or int to be specific).

It seems `__nonzero__` (for Python 2), `__bool__` (for Python 3) and 
`__contains__` forcing the the return into a bool unlike other operators. There 
are few references about this as below:


http://stackoverflow.com/questions/12244074/python-source-code-for-built-in-in-operator/12244378#12244378

http://stackoverflow.com/questions/38542543/functionality-of-python-in-vs-contains/38542777

It seems we can't overwrite `__nonzero__` or `__bool__` as a workaround to 
make this working because these force the return type as a bool as below:

```python
class Column(object):
def __contains__(self, item):
print "I am contains"
return Column()
def __nonzero__(self):

[GitHub] spark issue #17133: [SPARK-19793] Use clock.getTimeMillis when mark task as ...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17133: [SPARK-19793] Use clock.getTimeMillis when mark task as ...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17133: [SPARK-19793] Use clock.getTimeMillis when mark task as ...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #16929: [SPARK-19595][SQL] Support json array in from_json

2017-03-03 Thread brkyvz
Github user brkyvz commented on the issue:

https://github.com/apache/spark/pull/16929
  
@HyukjinKwon Implementation seems fine. Just left a cosmetic comment on 
your unit tests. Otherwise LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16929: [SPARK-19595][SQL] Support json array in from_jso...

2017-03-03 Thread brkyvz
Github user brkyvz commented on a diff in the pull request:

https://github.com/apache/spark/pull/16929#discussion_r104278223
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala
 ---
@@ -372,6 +372,58 @@ class JsonExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 )
   }
 
+  test("from_json - array") {
+val schema = ArrayType(StructType(StructField("a", IntegerType) :: 
Nil))
+
+// json array: `Array(Row(...), ...)`
+val jsonData1 = """[{"a": 1}, {"a": 2}]"""
+val expected =
+  InternalRow.fromSeq(1 :: Nil) ::
+  InternalRow.fromSeq(2 :: Nil) :: Nil
+checkEvaluation(JsonToStruct(
+  schema, Map.empty, Literal(jsonData1), gmtId), expected)
+
+// json object: `Array(Row(...))`
+val jsonData2 = """{"a": 1}"""
--- End diff --

I would make each example a separate test. This way it's easier to figure 
out what breaks later.

e.g.
  1. `from_json - input=array, schema=array, output=array`
  2. `from_json - input=object, schema=array, output=array of single object`
  3. `from_json - input=empty json array, schema=array, output=empty array`
...



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16929: [SPARK-19595][SQL] Support json array in from_jso...

2017-03-03 Thread brkyvz
Github user brkyvz commented on a diff in the pull request:

https://github.com/apache/spark/pull/16929#discussion_r104278176
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala
 ---
@@ -372,6 +372,58 @@ class JsonExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 )
   }
 
+  test("from_json - array") {
+val schema = ArrayType(StructType(StructField("a", IntegerType) :: 
Nil))
+
+// json array: `Array(Row(...), ...)`
+val jsonData1 = """[{"a": 1}, {"a": 2}]"""
+val expected =
+  InternalRow.fromSeq(1 :: Nil) ::
+  InternalRow.fromSeq(2 :: Nil) :: Nil
+checkEvaluation(JsonToStruct(
+  schema, Map.empty, Literal(jsonData1), gmtId), expected)
--- End diff --

could you put input and expected output in different rows for readability 
please


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

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



[GitHub] spark issue #16867: [SPARK-16929] Improve performance when check speculatabl...

2017-03-03 Thread jinxing64
Github user jinxing64 commented on the issue:

https://github.com/apache/spark/pull/16867
  
@squito 
Yes, some of machine learning jobs which do cartesian product in my cluster 
have over than 100k tasks in the `TaskSetManager`. 



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

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



[GitHub] spark issue #17159: [SPARK-19818][SparkR] union should check for name consis...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17159: [SPARK-19818][SparkR] union should check for name consis...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17159: [SPARK-19818][SparkR] union should check for name consis...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17159
  
**[Test build #73888 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73888/testReport)**
 for PR 17159 at commit 
[`7697806`](https://github.com/apache/spark/commit/769780697d81f91e911b5af516c24b8b4291f27d).
 * This patch **fails SparkR unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark pull request #16981: [SPARK-19637][SQL] Add to_json in FunctionRegistr...

2017-03-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/16981#discussion_r104277851
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -566,3 +586,17 @@ case class StructToJson(
 
   override def inputTypes: Seq[AbstractDataType] = StructType :: Nil
 }
+
+object StructToJson {
+
+  def convertToMapData(exp: Expression): Map[String, String] = exp match {
+case m: CreateMap
+if m.dataType.acceptsType(MapType(StringType, StringType, 
valueContainsNull = false)) =>
+  val arrayMap = m.eval().asInstanceOf[ArrayBasedMapData]
+  ArrayBasedMapData.toScalaMap(arrayMap).map { case (key, value) =>
+key.toString -> value.toString
+  }
+case _ =>
+  throw new AnalysisException("Must use a map() function for options")
--- End diff --

okay!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16981: [SPARK-19637][SQL] Add to_json in FunctionRegistr...

2017-03-03 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16981#discussion_r104277780
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -566,3 +586,17 @@ case class StructToJson(
 
   override def inputTypes: Seq[AbstractDataType] = StructType :: Nil
 }
+
+object StructToJson {
+
+  def convertToMapData(exp: Expression): Map[String, String] = exp match {
+case m: CreateMap
+if m.dataType.acceptsType(MapType(StringType, StringType, 
valueContainsNull = false)) =>
+  val arrayMap = m.eval().asInstanceOf[ArrayBasedMapData]
+  ArrayBasedMapData.toScalaMap(arrayMap).map { case (key, value) =>
+key.toString -> value.toString
+  }
+case _ =>
+  throw new AnalysisException("Must use a map() function for options")
--- End diff --

This message is misleading in the following case:
```
df2.selectExpr("to_json(a, map('a', 1))")
```


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

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



[GitHub] spark pull request #16981: [SPARK-19637][SQL] Add to_json in FunctionRegistr...

2017-03-03 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16981#discussion_r104277787
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 ---
@@ -566,3 +586,17 @@ case class StructToJson(
 
   override def inputTypes: Seq[AbstractDataType] = StructType :: Nil
 }
+
+object StructToJson {
+
+  def convertToMapData(exp: Expression): Map[String, String] = exp match {
+case m: CreateMap
+if m.dataType.acceptsType(MapType(StringType, StringType, 
valueContainsNull = false)) =>
+  val arrayMap = m.eval().asInstanceOf[ArrayBasedMapData]
+  ArrayBasedMapData.toScalaMap(arrayMap).map { case (key, value) =>
+key.toString -> value.toString
+  }
+case _ =>
+  throw new AnalysisException("Must use a map() function for options")
--- End diff --

Please also include the test case for this. 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 issue #17158: [SPARK-16845][SQL][BRANCH-1.6] `GeneratedClass$SpecificO...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17158
  
**[Test build #73890 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73890/consoleFull)**
 for PR 17158 at commit 
[`441aafc`](https://github.com/apache/spark/commit/441aafcb4abb603c03e30bfe1de7afec32b16873).


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

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



[GitHub] spark issue #16867: [SPARK-16929] Improve performance when check speculatabl...

2017-03-03 Thread jinxing64
Github user jinxing64 commented on the issue:

https://github.com/apache/spark/pull/16867
  
@kayousterhout  @squito 
Thanks a lot for your comments, really helpful :)
I really think median heap is a good idea. `slice` is `O(n)` and is not 
most efficient.
I'm doing implementation for `MedianHeap`, which provides `add`, `remove` 
and `getMedian` methods.
Yes, for both `TreeSet` and `MedianHeap`, when `handleSuccessfulTask`, we 
are doing insertion with `O(log n)`, I did a 
test(https://github.com/apache/spark/pull/17112) to measure the *sum of time* 
cost by `TreeSet.add`. There are 100k tasks in this test. Results are: 372ms, 
362ms, 458ms, 429ms, 363ms.  Thus I think the `O(log n)` time complexity will 
not be a problem.


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

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



[GitHub] spark issue #17158: [SPARK-16845][SQL][BRANCH-1.6] `GeneratedClass$SpecificO...

2017-03-03 Thread kiszk
Github user kiszk commented on the issue:

https://github.com/apache/spark/pull/17158
  
Jenkins, retest this please


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

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



[GitHub] spark issue #17156: [SPARK-19816][SQL][Tests] Fix an issue that DataFrameCal...

2017-03-03 Thread yhuai
Github user yhuai commented on the issue:

https://github.com/apache/spark/pull/17156
  
merged to branch-2.1


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

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



[GitHub] spark issue #17156: [SPARK-19816][SQL][Tests] Fix an issue that DataFrameCal...

2017-03-03 Thread yhuai
Github user yhuai commented on the issue:

https://github.com/apache/spark/pull/17156
  
Let's also merge this to branch-2.1.


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

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



[GitHub] spark issue #17112: [WIP] Measurement for SPARK-16929.

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17112: [WIP] Measurement for SPARK-16929.

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17112
  
**[Test build #73889 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73889/testReport)**
 for PR 17112 at commit 
[`61b96ff`](https://github.com/apache/spark/commit/61b96ff7190e34e12e567a99389459e826066964).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark issue #17112: [WIP] Measurement for SPARK-16929.

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17112
  
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 #17156: [SPARK-19816][SQL][Tests] Fix an issue that DataF...

2017-03-03 Thread asfgit
Github user asfgit closed the pull request at:

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


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

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



[GitHub] spark issue #17159: [SPARK-19818][SparkR] union should check for name consis...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/17159
  
**[Test build #73888 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73888/testReport)**
 for PR 17159 at commit 
[`7697806`](https://github.com/apache/spark/commit/769780697d81f91e911b5af516c24b8b4291f27d).


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

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



[GitHub] spark issue #17112: [WIP] Measurement for SPARK-16929.

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #17159: [SPARK-19818][SparkR] union should check for name consis...

2017-03-03 Thread actuaryzhang
Github user actuaryzhang commented on the issue:

https://github.com/apache/spark/pull/17159
  
The current implementation accepts data frames with different schemas. See 
issues below:
```
df <- createDataFrame(data.frame(name = c("Michael", "Andy", "Justin"), age 
= c(1, 30, 19)))
union(df, df[, c(2, 1)])
 name age
1 Michael 1.0
2Andy30.0
3  Justin19.0
4 1.0 Michael
```
@felixcheung 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #17159: [SPARK-19818][SparkR] union should check for name...

2017-03-03 Thread actuaryzhang
GitHub user actuaryzhang opened a pull request:

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

[SPARK-19818][SparkR] union should check for name consistency of input data 
frames

## What changes were proposed in this pull request?
Added checks for name consistency of input data frames in union. 

## How was this patch tested?
new test. 

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

$ git pull https://github.com/actuaryzhang/spark sparkRUnion

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

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


commit 769780697d81f91e911b5af516c24b8b4291f27d
Author: actuaryzhang 
Date:   2017-03-04T02:53:22Z

union checks for name consistency




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

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



[GitHub] spark issue #17156: [SPARK-19816][SQL][Tests] Fix an issue that DataFrameCal...

2017-03-03 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/17156
  
Thanks! Merging to 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 issue #17156: [SPARK-19816][SQL][Tests] Fix an issue that DataFrameCal...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17156: [SPARK-19816][SQL][Tests] Fix an issue that DataFrameCal...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17156: [SPARK-19816][SQL][Tests] Fix an issue that DataFrameCal...

2017-03-03 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #17158: [SPARK-16845][SQL][BRANCH-1.6] `GeneratedClass$SpecificO...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #17158: [SPARK-16845][SQL][BRANCH-1.6] `GeneratedClass$SpecificO...

2017-03-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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



  1   2   3   4   5   6   7   >