[GitHub] spark pull request #20433: [SPARK-23264][SQL] Support interval values withou...

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

https://github.com/apache/spark/pull/20433#discussion_r175336924
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala ---
@@ -155,6 +155,7 @@ class QueryExecution(val sparkSession: SparkSession, 
val logical: LogicalPlan) {
   case (null, _) => "null"
   case (s: String, StringType) => "\"" + s + "\""
   case (decimal, DecimalType()) => decimal.toString
+  case (interval, CalendarIntervalType) => interval.toString
--- End diff --

ok, I'll try to add tests for that.


---

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



[GitHub] spark pull request #20433: [SPARK-23264][SQL] Support interval values withou...

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

https://github.com/apache/spark/pull/20433#discussion_r175336329
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala ---
@@ -155,6 +155,7 @@ class QueryExecution(val sparkSession: SparkSession, 
val logical: LogicalPlan) {
   case (null, _) => "null"
   case (s: String, StringType) => "\"" + s + "\""
   case (decimal, DecimalType()) => decimal.toString
+  case (interval, CalendarIntervalType) => interval.toString
--- End diff --

Do we have a test case to capture this change?


---

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



[GitHub] spark issue #20795: [SPARK-23486]cache the function name from the catalog fo...

2018-03-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/20795
  
Please ping me if this is ready to review. 


---

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



[GitHub] spark pull request #20774: [SPARK-23549][SQL] Cast to timestamp when compari...

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

https://github.com/apache/spark/pull/20774#discussion_r175335261
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -479,6 +479,15 @@ object SQLConf {
 .checkValues(HiveCaseSensitiveInferenceMode.values.map(_.toString))
 
.createWithDefault(HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString)
 
+  val HIVE_COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP =
+buildConf("spark.sql.hive.compareDateTimestampInTimestamp")
--- End diff --

`spark.sql.typeCoercion.compareDateTimestampInTimestamp`


---

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



[GitHub] spark issue #20687: [SPARK-23500][SQL] Fix complex type simplification rules...

2018-03-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/20687
  
@henryr Please try to add the test cases that matter in your opinion. I 
will also submit a follow-up PR to add more test cases after this PR is merged.


---

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



[GitHub] spark pull request #20774: [SPARK-23549][SQL] Cast to timestamp when compari...

2018-03-18 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/20774#discussion_r175335072
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -479,6 +479,15 @@ object SQLConf {
 .checkValues(HiveCaseSensitiveInferenceMode.values.map(_.toString))
 
.createWithDefault(HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString)
 
+  val HIVE_COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP =
+buildConf("spark.sql.hive.compareDateTimestampInTimestamp")
+  .doc("When true (default), compare Date with Timestamp after 
converting both sides to " +
+"Timestamp. This behavior is compatible with Hive 2.2 or later. 
See HIVE-15236. " +
+"When false, restore the behavior prior to Spark 2.4. Compare Date 
with Timestamp after " +
+"converting both sides to string.")
+.booleanConf
--- End diff --

perhaps mention this config will be removed in spark 3.0.

(on a related note we should look at those configs for backward 
compatibility and consider removing them in 3.0)


---

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



[GitHub] spark issue #20847: [SPARK-23644][CORE][UI][BACKPORT-2.3] Use absolute path ...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20847: [SPARK-23644][CORE][UI][BACKPORT-2.3] Use absolute path ...

2018-03-18 Thread jerryshao
Github user jerryshao commented on the issue:

https://github.com/apache/spark/pull/20847
  
@mgaido91 would you please create a JIRA to track all the UT failures. 
Hopes someone familiar with that part could fix them.


---

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



[GitHub] spark issue #20847: [SPARK-23644][CORE][UI][BACKPORT-2.3] Use absolute path ...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20847
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 

https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1599/
Test PASSed.


---

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



[GitHub] spark pull request #20774: [SPARK-23549][SQL] Cast to timestamp when compari...

2018-03-18 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/20774#discussion_r175334948
  
--- Diff: 
sql/core/src/test/resources/sql-tests/inputs/predicate-functions.sql ---
@@ -39,3 +43,4 @@ select 2.0 <= '2.2';
 select 0.5 <= '1.5';
 select to_date('2009-07-30 04:17:52') <= to_date('2009-07-30 04:17:52');
 select to_date('2009-07-30 04:17:52') <= '2009-07-30 04:17:52';
+select to_date('2017-03-01') <= to_timestamp('2017-03-01 00:00:01');
--- End diff --

+1 it is really confusing to look at the diff


---

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



[GitHub] spark issue #20847: [SPARK-23644][CORE][UI][BACKPORT-2.3] Use absolute path ...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark pull request #20816: [SPARK-21479][SQL] Outer join filter pushdown in ...

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

https://github.com/apache/spark/pull/20816#discussion_r175334935
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -669,11 +672,42 @@ object InferFiltersFromConstraints extends 
Rule[LogicalPlan] with PredicateHelpe
   val newConditionOpt = conditionOpt match {
 case Some(condition) =>
   val newFilters = additionalConstraints -- 
splitConjunctivePredicates(condition)
-  if (newFilters.nonEmpty) Option(And(newFilters.reduce(And), 
condition)) else None
+  if (newFilters.nonEmpty) Option(And(newFilters.reduce(And), 
condition)) else conditionOpt
 case None =>
   additionalConstraints.reduceOption(And)
   }
-  if (newConditionOpt.isDefined) Join(left, right, joinType, 
newConditionOpt) else join
+  // Infer filter for left/right outer joins
+  val newLeftOpt = joinType match {
+case RightOuter if newConditionOpt.isDefined =>
+  val rightConstraints = right.constraints.union(
+splitConjunctivePredicates(newConditionOpt.get).toSet)
+  val inferredConstraints = ExpressionSet(
+
QueryPlanConstraints.inferAdditionalConstraints(rightConstraints))
+  val leftConditions = inferredConstraints
--- End diff --

I see. I am proposing to add a new helper function `getRelevantConstraints 
` in the trait `QueryPlanConstraints`. I would suggest to ignore what we are 
doing for this specific case. That function could be called in the other cases 
in the future. 


---

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



[GitHub] spark issue #20847: [SPARK-23644][CORE][UI][BACKPORT-2.3] Use absolute path ...

2018-03-18 Thread jerryshao
Github user jerryshao commented on the issue:

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


---

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



[GitHub] spark pull request #20845: [SPARK-23708][CORE] Correct comment for function ...

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

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


---

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



[GitHub] spark issue #20845: [SPARK-23708][CORE] Correct comment for function addShut...

2018-03-18 Thread jerryshao
Github user jerryshao commented on the issue:

https://github.com/apache/spark/pull/20845
  
@caneGuy it is not necessary to create a JIRA for such tiny fix.


---

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



[GitHub] spark issue #20687: [SPARK-23500][SQL] Fix complex type simplification rules...

2018-03-18 Thread henryr
Github user henryr commented on the issue:

https://github.com/apache/spark/pull/20687
  
@gatorsmile thank you for the reviews! Are there specific test cases you'd 
like to see? I've checked correlated and uncorrelated subqueries, various 
flavours of join, aggregates with HAVING clauses, nested compound types, and so 
on. 



---

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



[GitHub] spark issue #20830: [SPARK-23691][PYTHON] Use sql_conf util in PySpark tests...

2018-03-18 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/20830
  
@BryanCutler, could I have the very first PR merged by you as a new fresh 
committer :-)?

I personally think it might be good to merge to branch-2.3 if it doesn't 
have conflicts. If it has, I think we are fine to get this into master only for 
now.


---

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



[GitHub] spark pull request #20774: [SPARK-23549][SQL] Cast to timestamp when compari...

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

https://github.com/apache/spark/pull/20774#discussion_r175333760
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -479,6 +479,15 @@ object SQLConf {
 .checkValues(HiveCaseSensitiveInferenceMode.values.map(_.toString))
 
.createWithDefault(HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString)
 
+  val HIVE_COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP =
+buildConf("spark.sql.hive.compareDateTimestampInTimestamp")
+  .doc("When true (default), compare Date with Timestamp after 
converting both sides to " +
+"Timestamp. This behavior is compatible with Hive 2.2 or later. 
See HIVE-15236. " +
+"When false, restore the behavior prior to Spark 2.4. Compare Date 
with Timestamp after " +
+"converting both sides to string.")
+.booleanConf
--- End diff --

internal()?


---

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



[GitHub] spark pull request #20774: [SPARK-23549][SQL] Cast to timestamp when compari...

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

https://github.com/apache/spark/pull/20774#discussion_r175333714
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -479,6 +479,15 @@ object SQLConf {
 .checkValues(HiveCaseSensitiveInferenceMode.values.map(_.toString))
 
.createWithDefault(HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString)
 
+  val HIVE_COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP =
+buildConf("spark.sql.hive.compareDateTimestampInTimestamp")
--- End diff --

It should not be under `hive`, because it is not only for hive


---

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



[GitHub] spark pull request #20798: [SPARK-23645][MINOR][DOCS][PYTHON] Add docs RE `p...

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

https://github.com/apache/spark/pull/20798#discussion_r175332547
  
--- Diff: docs/sql-programming-guide.md ---
@@ -1799,6 +1799,12 @@ different than a Pandas timestamp. It is recommended 
to use Pandas time series f
 working with timestamps in `pandas_udf`s to get the best performance, see
 [here](https://pandas.pydata.org/pandas-docs/stable/timeseries.html) for 
details.
 
+### Keyword Arguments and Related Structures
+
+Currently, for `pandas_udf` it is not possible to pass keyword arguments 
to a function. The wrapped
+function must also not be a `functools.partial` function object. Functions 
with a zero-length argument
+list are unsupported, but can be approximated via a single-argument udf 
which ignores the passed arg.
--- End diff --

Ah, hm .. I think we are fine to leave this note out of the SQL programing 
guide for now .. Arguably this seems rather a corner case (just given my 
monitoring mailing list and JIRAs so far). I personally have promoted guys to 
leave some notes about key points only.


---

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



[GitHub] spark pull request #20798: [SPARK-23645][MINOR][DOCS][PYTHON] Add docs RE `p...

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

https://github.com/apache/spark/pull/20798#discussion_r175332763
  
--- Diff: python/pyspark/sql/functions.py ---
@@ -2155,6 +2155,8 @@ def udf(f=None, returnType=StringType()):
 in boolean expressions and it ends up with being executed all 
internally. If the functions
 can fail on special rows, the workaround is to incorporate the 
condition into the functions.
 
+.. note:: The user-defined functions may not take keyword arguments.
--- End diff --

I think it's fine to say`may not take` -> `does not take`.


---

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



[GitHub] spark pull request #20774: [SPARK-23549][SQL] Cast to timestamp when compari...

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

https://github.com/apache/spark/pull/20774#discussion_r175332253
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
 ---
@@ -342,8 +349,8 @@ object TypeCoercion {
 p.makeCopy(Array(left, Cast(right, TimestampType)))
 
   case p @ BinaryComparison(left, right)
-if findCommonTypeForBinaryComparison(left.dataType, 
right.dataType).isDefined =>
-val commonType = findCommonTypeForBinaryComparison(left.dataType, 
right.dataType).get
+if findCommonTypeForBinaryComparison(left.dataType, 
right.dataType, conf).isDefined =>
--- End diff --

two more spaces before `if`


---

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



[GitHub] spark pull request #20774: [SPARK-23549][SQL] Cast to timestamp when compari...

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

https://github.com/apache/spark/pull/20774#discussion_r175332620
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
 ---
@@ -1251,18 +1251,40 @@ class TypeCoercionSuite extends AnalysisTest {
   }
 
   test("binary comparison with string promotion") {
-ruleTest(PromoteStrings,
-  GreaterThan(Literal("123"), Literal(1)),
-  GreaterThan(Cast(Literal("123"), IntegerType), Literal(1)))
-ruleTest(PromoteStrings,
-  LessThan(Literal(true), Literal("123")),
-  LessThan(Literal(true), Cast(Literal("123"), BooleanType)))
-ruleTest(PromoteStrings,
-  EqualTo(Literal(Array(1, 2)), Literal("123")),
-  EqualTo(Literal(Array(1, 2)), Literal("123")))
-ruleTest(PromoteStrings,
-  GreaterThan(Literal("1.5"), Literal(BigDecimal("0.5"))),
-  GreaterThan(Cast(Literal("1.5"), DoubleType), 
Cast(Literal(BigDecimal("0.5")), DoubleType)))
+val rule = TypeCoercion.PromoteStrings(conf)
+
+Seq("true", "false").foreach { flag =>
+  withSQLConf("spark.sql.hive.compareDateTimestampInTimestamp" -> 
flag) {
+ruleTest(rule,
+  GreaterThan(Literal("123"), Literal(1)),
+  GreaterThan(Cast(Literal("123"), IntegerType), Literal(1)))
+ruleTest(rule,
+  LessThan(Literal(true), Literal("123")),
+  LessThan(Literal(true), Cast(Literal("123"), BooleanType)))
+ruleTest(rule,
+  EqualTo(Literal(Array(1, 2)), Literal("123")),
+  EqualTo(Literal(Array(1, 2)), Literal("123")))
+ruleTest(rule,
+  GreaterThan(Literal("1.5"), Literal(BigDecimal("0.5"))),
+  GreaterThan(Cast(Literal("1.5"), DoubleType), 
Cast(Literal(BigDecimal("0.5")),
+DoubleType)))
+val date0301 = Literal(java.sql.Date.valueOf("2017-03-01"))
+val timestamp030100 = Literal(Timestamp.valueOf("2017-03-01 
00:00:00"))
+val timestamp030101 = Literal(Timestamp.valueOf("2017-03-01 
00:00:01"))
+if ("true".equals(flag)) {
+  // `Date` should be treated as timestamp at 00:00:00 See 
SPARK-23549
+  ruleTest(rule, EqualTo(date0301, timestamp030100),
+EqualTo(Cast(date0301, TimestampType), timestamp030100))
+  ruleTest(rule, LessThan(date0301, timestamp030101),
+LessThan(Cast(date0301, TimestampType), timestamp030101))
+} else {
+  ruleTest(rule, LessThan(date0301, timestamp030100),
+LessThan(Cast(date0301, StringType), Cast(timestamp030100, 
StringType)))
+  ruleTest(rule, LessThan(date0301, timestamp030101),
+LessThan(Cast(date0301, StringType), Cast(timestamp030101, 
StringType)))
+}
+  }
+}
--- End diff --

```Scala
  test("binary comparison with string promotion") {
val rule = TypeCoercion.PromoteStrings(conf)
ruleTest(rule,
  GreaterThan(Literal("123"), Literal(1)),
  GreaterThan(Cast(Literal("123"), IntegerType), Literal(1)))
ruleTest(rule,
  LessThan(Literal(true), Literal("123")),
  LessThan(Literal(true), Cast(Literal("123"), BooleanType)))
ruleTest(rule,
  EqualTo(Literal(Array(1, 2)), Literal("123")),
  EqualTo(Literal(Array(1, 2)), Literal("123")))
ruleTest(rule,
  GreaterThan(Literal("1.5"), Literal(BigDecimal("0.5"))),
  GreaterThan(Cast(Literal("1.5"), DoubleType), 
Cast(Literal(BigDecimal("0.5")),
DoubleType)))
Seq(true, false).foreach { convertToTS =>
  withSQLConf("spark.sql.hive.compareDateTimestampInTimestamp" -> 
convertToTS.toString) {
val date0301 = Literal(java.sql.Date.valueOf("2017-03-01"))
val timestamp030100 = Literal(Timestamp.valueOf("2017-03-01 
00:00:00"))
val timestamp030101 = Literal(Timestamp.valueOf("2017-03-01 
00:00:01"))
if (convertToTS) {
  // `Date` should be treated as timestamp at 00:00:00 See 
SPARK-23549
  ruleTest(rule, EqualTo(date0301, timestamp030100),
EqualTo(Cast(date0301, TimestampType), timestamp030100))
  ruleTest(rule, LessThan(date0301, timestamp030101),
LessThan(Cast(date0301, TimestampType), timestamp030101))
} else {
  ruleTest(rule, LessThan(date0301, timestamp030100),
LessThan(Cast(date0301, StringType), Cast(timestamp030100, 
StringType)))
  ruleTest(rule, LessThan(date0301, timestamp030101),
LessThan(Cast(date0301, StringType), Cast(timestamp030101, 
StringType)))
}
  }
}
 

[GitHub] spark pull request #20774: [SPARK-23549][SQL] Cast to timestamp when compari...

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

https://github.com/apache/spark/pull/20774#discussion_r175332237
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
 ---
@@ -125,29 +125,36 @@ object TypeCoercion {
   /**
* This function determines the target type of a comparison operator 
when one operand
* is a String and the other is not. It also handles when one op is a 
Date and the
-   * other is a Timestamp by making the target type to be String.
+   * other is a Timestamp by making the target type to be Timestamp as 
default
*/
-  val findCommonTypeForBinaryComparison: (DataType, DataType) => 
Option[DataType] = {
+  private def findCommonTypeForBinaryComparison(dt1: DataType, dt2: 
DataType, conf: SQLConf):
+  Option[DataType] = (dt1, dt2) match {
--- End diff --

```Scala
  private def findCommonTypeForBinaryComparison(
  dt1: DataType, dt2: DataType, conf: SQLConf): Option[DataType] = 
(dt1, dt2) match {
```


---

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



[GitHub] spark pull request #20774: [SPARK-23549][SQL] Cast to timestamp when compari...

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

https://github.com/apache/spark/pull/20774#discussion_r175332723
  
--- Diff: 
sql/core/src/test/resources/sql-tests/inputs/predicate-functions.sql ---
@@ -39,3 +43,4 @@ select 2.0 <= '2.2';
 select 0.5 <= '1.5';
 select to_date('2009-07-30 04:17:52') <= to_date('2009-07-30 04:17:52');
 select to_date('2009-07-30 04:17:52') <= '2009-07-30 04:17:52';
+select to_date('2017-03-01') <= to_timestamp('2017-03-01 00:00:01');
--- End diff --

A little hard to track the exact changes. Let us move all the new tests to 
the end. 


---

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



[GitHub] spark pull request #20774: [SPARK-23549][SQL] Cast to timestamp when compari...

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

https://github.com/apache/spark/pull/20774#discussion_r175332743
  
--- Diff: docs/sql-programming-guide.md ---
@@ -2289,6 +2289,13 @@ Spark SQL supports the vast majority of Hive 
features, such as:
   * `MAP<>`
   * `STRUCT<>`
 
+### Compare a DATE type with a TIMESTAMP type
--- End diff --

We also need to add it to migration guide. 


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20827
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 

https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1598/
Test PASSed.


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark pull request #20816: [SPARK-21479][SQL] Outer join filter pushdown in ...

2018-03-18 Thread maryannxue
Github user maryannxue commented on a diff in the pull request:

https://github.com/apache/spark/pull/20816#discussion_r175330576
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -669,11 +672,42 @@ object InferFiltersFromConstraints extends 
Rule[LogicalPlan] with PredicateHelpe
   val newConditionOpt = conditionOpt match {
 case Some(condition) =>
   val newFilters = additionalConstraints -- 
splitConjunctivePredicates(condition)
-  if (newFilters.nonEmpty) Option(And(newFilters.reduce(And), 
condition)) else None
+  if (newFilters.nonEmpty) Option(And(newFilters.reduce(And), 
condition)) else conditionOpt
 case None =>
   additionalConstraints.reduceOption(And)
   }
-  if (newConditionOpt.isDefined) Join(left, right, joinType, 
newConditionOpt) else join
+  // Infer filter for left/right outer joins
+  val newLeftOpt = joinType match {
+case RightOuter if newConditionOpt.isDefined =>
+  val rightConstraints = right.constraints.union(
+splitConjunctivePredicates(newConditionOpt.get).toSet)
+  val inferredConstraints = ExpressionSet(
+
QueryPlanConstraints.inferAdditionalConstraints(rightConstraints))
+  val leftConditions = inferredConstraints
--- End diff --

I think the `constructIsNotNullConstraints` logic does not deal with the 
"transitive" constraints so we not need to include it here. Instead the 
"isNotNull" deduction for inferred filters on the null-supplying side is 
guaranteed by 2 things here:
1) when getting constraints from the preserved side, 
`constructIsNotNullConstraints` has already been called and will be carried 
over by `inferAdditionalConstraints` to the null-supplying side;
2) the Filter matching part of `InferFiltersFromConstraints`.
That said, I'm good with the name `getRelevantConstraints` too.


---

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



[GitHub] spark pull request #20851: [SPARK-23727][SQL] Support for pushing down filte...

2018-03-18 Thread manku-timma
Github user manku-timma commented on a diff in the pull request:

https://github.com/apache/spark/pull/20851#discussion_r175327686
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
 ---
@@ -72,6 +82,15 @@ private[parquet] object ParquetFilters {
   (n: String, v: Any) => FilterApi.notEq(
 binaryColumn(n),
 Option(v).map(b => 
Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull)
+case DateType if SQLConf.get.parquetFilterPushDownDate =>
+  (n: String, v: Any) => {
+FilterApi.notEq(
+  intColumn(n),
+  Option(v).map { date =>
--- End diff --

These 3 lines are repeated several times in this change. Please refactor to 
a new method.


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/20856
  
What is the test case?


---

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



[GitHub] spark issue #20851: [SPARK-23727][SQL] Support for pushing down filters for ...

2018-03-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/20851
  
cc @michal-databricks @cloud-fan 


---

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



[GitHub] spark issue #20846: [SPARK-5498][SQL][FOLLOW] add schema to table partition

2018-03-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/20846
  
We do not allow users to change the table column type. Currently, only the 
column comments are allowed to change if users issue the command through Spark. 
However, users still can change it through Hive. Thus, nothing we can do from 
Spark side, right?


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread maropu
Github user maropu commented on the issue:

https://github.com/apache/spark/pull/20827
  
ok, I'll update soon.


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/20827
  
The fix looks good to me. Since it changes the external name, could you add 
it to the migration guide?


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/20827
  
cc @cloud-fan 


---

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



[GitHub] spark issue #20845: [SPARK-23708][CORE] Correct comment for function addShut...

2018-03-18 Thread caneGuy
Github user caneGuy commented on the issue:

https://github.com/apache/spark/pull/20845
  
@jiangxb1987 @Ngone51 Thanks! Any more comments?


---

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



[GitHub] spark pull request #20816: [SPARK-21479][SQL] Outer join filter pushdown in ...

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

https://github.com/apache/spark/pull/20816#discussion_r175324999
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -669,11 +672,42 @@ object InferFiltersFromConstraints extends 
Rule[LogicalPlan] with PredicateHelpe
   val newConditionOpt = conditionOpt match {
 case Some(condition) =>
   val newFilters = additionalConstraints -- 
splitConjunctivePredicates(condition)
-  if (newFilters.nonEmpty) Option(And(newFilters.reduce(And), 
condition)) else None
+  if (newFilters.nonEmpty) Option(And(newFilters.reduce(And), 
condition)) else conditionOpt
 case None =>
   additionalConstraints.reduceOption(And)
   }
-  if (newConditionOpt.isDefined) Join(left, right, joinType, 
newConditionOpt) else join
+  // Infer filter for left/right outer joins
+  val newLeftOpt = joinType match {
+case RightOuter if newConditionOpt.isDefined =>
+  val rightConstraints = right.constraints.union(
+splitConjunctivePredicates(newConditionOpt.get).toSet)
+  val inferredConstraints = ExpressionSet(
+
QueryPlanConstraints.inferAdditionalConstraints(rightConstraints))
+  val leftConditions = inferredConstraints
--- End diff --

We need to also union `constructIsNotNullConstraints`. Thus, let us use 
`getRelevantConstraints `


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread kiszk
Github user kiszk commented on the issue:

https://github.com/apache/spark/pull/20856
  
I think that this problem does not depend on data. Can you create a test 
case by modifying the query slightly with different column names?


---

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



[GitHub] spark pull request #20719: [SPARK-23568][ML] Use metadata numAttributes if a...

2018-03-18 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/20719#discussion_r175321399
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala 
---
@@ -170,6 +171,13 @@ private[evaluation] abstract class Silhouette {
   def overallScore(df: DataFrame, scoreColumn: Column): Double = {
 df.select(avg(scoreColumn)).collect()(0).getDouble(0)
   }
+
+  protected def getNumberOfFeatures(dataFrame: DataFrame, columnName: 
String): Int = {
+val group = 
AttributeGroup.fromStructField(dataFrame.schema(columnName))
+group.numAttributes.getOrElse {
--- End diff --

We should use `size` of `AttributeGroup` to determine its number of 
attributes. When it is -1, we can leverage `first` then.


---

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



[GitHub] spark pull request #20851: [SPARK-23727][SQL] Support for pushing down filte...

2018-03-18 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/20851#discussion_r175317152
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
 ---
@@ -50,6 +51,15 @@ private[parquet] object ParquetFilters {
   (n: String, v: Any) => FilterApi.eq(
 binaryColumn(n),
 Option(v).map(b => 
Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull)
+case DateType if SQLConf.get.parquetFilterPushDownDate =>
+  (n: String, v: Any) => {
+FilterApi.eq(
+  intColumn(n),
+  Option(v).map { date =>
+val days = date.asInstanceOf[java.sql.Date].getTime / (24 * 60 
* 60 * 1000)
+days.toInt.asInstanceOf[Integer]
--- End diff --

Use `DateTimeUtils.fromJavaDate` here and below?


---

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



[GitHub] spark pull request #20851: [SPARK-23727][SQL] Support for pushing down filte...

2018-03-18 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/20851#discussion_r175320790
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -353,6 +353,12 @@ object SQLConf {
 .booleanConf
 .createWithDefault(true)
 
+  val PARQUET_FILTER_PUSHDOWN_DATE_ENABLED = 
buildConf("spark.sql.parquet.filterPushdown.date")
+.doc("If true, enables Parquet filter push-down optimization for Date. 
" +
--- End diff --

Should be an internal conf, i.e., `.internal()`?


---

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



[GitHub] spark issue #20699: [SPARK-23544][SQL]Remove redundancy ShuffleExchange in t...

2018-03-18 Thread heary-cao
Github user heary-cao commented on the issue:

https://github.com/apache/spark/pull/20699
  
cc @hvanhovell , @cloud-fan 


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20856
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 

https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1597/
Test PASSed.


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20856
  
**[Test build #88362 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88362/testReport)**
 for PR 20856 at commit 
[`3981421`](https://github.com/apache/spark/commit/39814216026da32eee5aabf3886bbedd3b90ed08).


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread kiszk
Github user kiszk commented on the issue:

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


---

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



[GitHub] spark issue #20774: [SPARK-23549][SQL] Cast to timestamp when comparing time...

2018-03-18 Thread kiszk
Github user kiszk commented on the issue:

https://github.com/apache/spark/pull/20774
  
cc @gatorsmile


---

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



[GitHub] spark issue #20806: [SPARK-23661][SQL] Implement treeAggregate on Dataset AP...

2018-03-18 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/20806
  
@WeichenXu123 As the discussion with @cloud-fan at 
https://github.com/apache/spark/pull/20806#discussion_r174277864, I'd like to 
see some performance gain it, but I need to run benchmark to see if there is 
any.


---

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



[GitHub] spark pull request #20844: [SPARK-23707][SQL] Fresh 'initRange' name to avoi...

2018-03-18 Thread ConeyLiu
Github user ConeyLiu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20844#discussion_r175315224
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
 ---
@@ -396,9 +396,11 @@ case class RangeExec(range: 
org.apache.spark.sql.catalyst.plans.logical.Range)
 // The default size of a batch, which must be positive integer
 val batchSize = 1000
 
-val initRangeFuncName = ctx.addNewFunction("initRange",
+val initRange = ctx.freshName("initRange")
+
+val initRangeFuncName = ctx.addNewFunction(initRange,
   s"""
-| private void initRange(int idx) {
+| private void ${initRange}(int idx) {
--- End diff --

@cloud-fan thanks for reviewing. Both `BroadCastExchange` and 
`ShuffleExchange` don't support `CodegenSupport`, so there should be two 
`WholeStageCodegen`. 


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20827
  
**[Test build #88361 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88361/testReport)**
 for PR 20827 at commit 
[`697f34b`](https://github.com/apache/spark/commit/697f34bb159a14712f9a654811d26127b95a9e73).


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20827
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 

https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1596/
Test PASSed.


---

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



[GitHub] spark issue #20827: [SPARK-23666][SQL] Do not display exprIds of Alias in us...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20852: [SPARK-23728][BRANCH-2.3] Fix ML tests with expected exc...

2018-03-18 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/20852
  
Hi, @yanboliang and @jkbradley .
`branch-2.3` is broken. Could you review this PR?


---

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



[GitHub] spark issue #20686: [SPARK-22915][MLlib] Streaming tests for spark.ml.featur...

2018-03-18 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/20686
  
Thanks. For branch testing, I was confused with maven testing. Since you 
create a PR against `branch-2.3`, it looks okay.


---

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



[GitHub] spark issue #20742: [SPARK-23572][docs] Bring "security.md" up to date.

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20742: [SPARK-23572][docs] Bring "security.md" up to date.

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20742: [SPARK-23572][docs] Bring "security.md" up to date.

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20774: [SPARK-23549][SQL] Cast to timestamp when comparing time...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20774: [SPARK-23549][SQL] Cast to timestamp when comparing time...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20774: [SPARK-23549][SQL] Cast to timestamp when comparing time...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20851: [SPARK-23727][SQL] Support for pushing down filters for ...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20851: [SPARK-23727][SQL] Support for pushing down filters for ...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20851: [SPARK-23727][SQL] Support for pushing down filters for ...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20701: [SPARK-23528][ML] Add numIter to ClusteringSummary

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20701: [SPARK-23528][ML] Add numIter to ClusteringSummary

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20701
  
**[Test build #88355 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88355/testReport)**
 for PR 20701 at commit 
[`f6ee4a2`](https://github.com/apache/spark/commit/f6ee4a2b4bb2444d65ab0e26a141304b327bd998).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class KMeansModel (@Since(\"1.0.0\") val clusterCenters: 
Array[Vector],`


---

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



[GitHub] spark issue #20798: [SPARK-23645][MINOR][DOCS][PYTHON] Add docs RE `pandas_u...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20798: [SPARK-23645][MINOR][DOCS][PYTHON] Add docs RE `pandas_u...

2018-03-18 Thread mstewart141
Github user mstewart141 commented on the issue:

https://github.com/apache/spark/pull/20798
  
@HyukjinKwon thanks again. i've updated this PR to add documentation. I dug 
pretty deep into the bigger issue around kwargs/partial functions, and you can 
see what i did in the commit:

https://github.com/apache/spark/pull/20798/commits/969f9073ee06d2a5641f78247b75e30d9ad1679a

Basically, throughout the udf and arrow serialization code there is no 
notion of kwargs as supported, making it more challenging than I anticipated to 
wire everything together. Definitely not impossible, but not a small 
undertaking either.


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20854: [SPARK-23712][SQL] Interpreted UnsafeRowJoiner [WIP]

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20854: [SPARK-23712][SQL] Interpreted UnsafeRowJoiner [WIP]

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20851: [SPARK-23727][SQL] Support for pushing down filters for ...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20854: [SPARK-23712][SQL] Interpreted UnsafeRowJoiner [WIP]

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20851: [SPARK-23727][SQL] Support for pushing down filters for ...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20851: [SPARK-23727][SQL] Support for pushing down filters for ...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20798: [SPARK-23645][PYTHON] Allow python udfs to be called wit...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20798
  
**[Test build #88360 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88360/testReport)**
 for PR 20798 at commit 
[`65de58f`](https://github.com/apache/spark/commit/65de58f04c0e54ce13274a89e8aae1346dfa93be).


---

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



[GitHub] spark issue #18906: [SPARK-21692][PYSPARK][SQL] Add nullability support to P...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #18906: [SPARK-21692][PYSPARK][SQL] Add nullability support to P...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #18906: [SPARK-21692][PYSPARK][SQL] Add nullability support to P...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20856
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 

https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1595/
Test PASSed.


---

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



[GitHub] spark issue #20856: [SPARK-23731][SQL] FileSourceScanExec throws NullPointer...

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20856
  
**[Test build #88359 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88359/testReport)**
 for PR 20856 at commit 
[`3981421`](https://github.com/apache/spark/commit/39814216026da32eee5aabf3886bbedd3b90ed08).


---

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



[GitHub] spark pull request #20856: [SPARK-23731][SQL] FileSourceScanExec throws Null...

2018-03-18 Thread jaceklaskowski
GitHub user jaceklaskowski opened a pull request:

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

[SPARK-23731][SQL] FileSourceScanExec throws NullPointerException in 
subexpression elimination

## What changes were proposed in this pull request?

Avoids ("fixes") a NullPointerException in subexpression elimination for 
subqueries with FileSourceScanExec.

## How was this patch tested?

Local build. No new tests as I could not reproduce it other than using the 
query and data under NDA. Waiting for Jenkins.

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

$ git pull https://github.com/jaceklaskowski/spark 
SPARK-23731-FileSourceScanExec-throws-NPE

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

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


commit 39814216026da32eee5aabf3886bbedd3b90ed08
Author: Jacek Laskowski 
Date:   2018-03-18T17:12:32Z

[SPARK-23731][SQL] FileSourceScanExec throws NullPointerException in 
subexpression elimination




---

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



[GitHub] spark issue #17254: [SPARK-19917][SQL]qualified partition path stored in cat...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17254
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 

https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1594/
Test PASSed.


---

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



[GitHub] spark issue #17254: [SPARK-19917][SQL]qualified partition path stored in cat...

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/17254
  
Build finished. Test PASSed.


---

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



[GitHub] spark pull request #20855: [SPARK-23731][SQL] FileSourceScanExec throws Null...

2018-03-18 Thread jaceklaskowski
GitHub user jaceklaskowski opened a pull request:

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

[SPARK-23731][SQL] FileSourceScanExec throws NullPointerException in 
subexpression elimination

## What changes were proposed in this pull request?

Avoids (not necessarily fixes) a NullPointerException in subexpression 
elimination for subqueries with FileSourceScanExec.

## How was this patch tested?

Local build. No new tests as I could not reproduce it other than using the 
query and data under NDA. Waiting for Jenkins.

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

$ git pull https://github.com/jaceklaskowski/spark 
SPARK-23731-FileSourceScanExec-throws-NPE

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

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


commit 8ef323c572cee181e3bdbddeeb7119eda03d78f4
Author: Dongjoon Hyun 
Date:   2018-01-17T06:32:18Z

[SPARK-23072][SQL][TEST] Add a Unicode schema test for file-based data 
sources

## What changes were proposed in this pull request?

After [SPARK-20682](https://github.com/apache/spark/pull/19651), Apache 
Spark 2.3 is able to read ORC files with Unicode schema. Previously, it raises 
`org.apache.spark.sql.catalyst.parser.ParseException`.

This PR adds a Unicode schema test for CSV/JSON/ORC/Parquet file-based data 
sources. Note that TEXT data source only has [a single column with a fixed name 
'value'](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala#L71).

## How was this patch tested?

Pass the newly added test case.

Author: Dongjoon Hyun 

Closes #20266 from dongjoon-hyun/SPARK-23072.

(cherry picked from commit a0aedb0ded4183cc33b27e369df1cbf862779e26)
Signed-off-by: Wenchen Fan 

commit bfbc2d41b8a9278b347b6df2d516fe4679b41076
Author: Henry Robinson 
Date:   2018-01-17T08:01:41Z

[SPARK-23062][SQL] Improve EXCEPT documentation

## What changes were proposed in this pull request?

Make the default behavior of EXCEPT (i.e. EXCEPT DISTINCT) more
explicit in the documentation, and call out the change in behavior
from 1.x.

Author: Henry Robinson 

Closes #20254 from henryr/spark-23062.

(cherry picked from commit 1f3d933e0bd2b1e934a233ed699ad39295376e71)
Signed-off-by: gatorsmile 

commit cbb6bda437b0d2832496b5c45f8264e5527f1cce
Author: Dongjoon Hyun 
Date:   2018-01-17T13:53:36Z

[SPARK-21783][SQL] Turn on ORC filter push-down by default

## What changes were proposed in this pull request?

ORC filter push-down is disabled by default from the beginning, 
[SPARK-2883](https://github.com/apache/spark/commit/aa31e431fc09f0477f1c2351c6275769a31aca90#diff-41ef65b9ef5b518f77e2a03559893f4dR149
).

Now, Apache Spark starts to depend on Apache ORC 1.4.1. For Apache Spark 
2.3, this PR turns on ORC filter push-down by default like Parquet 
([SPARK-9207](https://issues.apache.org/jira/browse/SPARK-21783)) as a part of 
[SPARK-20901](https://issues.apache.org/jira/browse/SPARK-20901), "Feature 
parity for ORC with Parquet".

## How was this patch tested?

Pass the existing tests.

Author: Dongjoon Hyun 

Closes #20265 from dongjoon-hyun/SPARK-21783.

(cherry picked from commit 0f8a28617a0742d5a99debfbae91222c2e3b5cec)
Signed-off-by: Wenchen Fan 

commit aae73a21a42fa366a09c2be1a4b91308ef211beb
Author: Wang Gengliang 
Date:   2018-01-17T16:05:26Z

[SPARK-23079][SQL] Fix query constraints propagation with aliases

## What changes were proposed in this pull request?

Previously, PR #19201 fix the problem of non-converging constraints.
After that PR #19149 improve the loop and constraints is inferred only once.
So the problem of non-converging constraints is gone.

However, the case below will fail.

```

spark.range(5).write.saveAsTable("t")
val t = spark.read.table("t")
val left = t.withColumn("xid", $"id" + lit(1)).as("x")
val right = t.withColumnRenamed("id", "xid").as("y")
val df = left.join(right, "xid").filter("id = 3").toDF()
checkAnswer(df, Row(4, 3))

```

Because `aliasMap` replace all the aliased child. See the test case in PR 
for details.

This PR is to fix this bug by removing useless code for preventing 
non-converging constraints.
It can be also fixed with #20270, but this is much simpler and clean up the 
code.

## How was this patch 

[GitHub] spark pull request #20855: [SPARK-23731][SQL] FileSourceScanExec throws Null...

2018-03-18 Thread jaceklaskowski
Github user jaceklaskowski closed the pull request at:

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


---

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



[GitHub] spark issue #20742: [SPARK-23572][docs] Bring "security.md" up to date.

2018-03-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20742
  
**[Test build #88358 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88358/testReport)**
 for PR 20742 at commit 
[`53c1710`](https://github.com/apache/spark/commit/53c1710f54888714744b3f0934ceeb732ed88f81).


---

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



[GitHub] spark issue #20742: [SPARK-23572][docs] Bring "security.md" up to date.

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20742: [SPARK-23572][docs] Bring "security.md" up to date.

2018-03-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/20742
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 

https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1593/
Test PASSed.


---

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



[GitHub] spark issue #20742: [SPARK-23572][docs] Bring "security.md" up to date.

2018-03-18 Thread kiszk
Github user kiszk commented on the issue:

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


---

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



  1   2   >