spark git commit: [SPARK-22939][PYSPARK] Support Spark UDF in registerFunction

2018-01-04 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 eb99b8ade -> 1f5e3540c


[SPARK-22939][PYSPARK] Support Spark UDF in registerFunction

## What changes were proposed in this pull request?
```Python
import random
from pyspark.sql.functions import udf
from pyspark.sql.types import IntegerType, StringType
random_udf = udf(lambda: int(random.random() * 100), 
IntegerType()).asNondeterministic()
spark.catalog.registerFunction("random_udf", random_udf, StringType())
spark.sql("SELECT random_udf()").collect()
```

We will get the following error.
```
Py4JError: An error occurred while calling o29.__getnewargs__. Trace:
py4j.Py4JException: Method __getnewargs__([]) does not exist
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:326)
at py4j.Gateway.invoke(Gateway.java:274)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:214)
at java.lang.Thread.run(Thread.java:745)
```

This PR is to support it.

## How was this patch tested?
WIP

Author: gatorsmile 

Closes #20137 from gatorsmile/registerFunction.

(cherry picked from commit 5aadbc929cb194e06dbd3bab054a161569289af5)
Signed-off-by: gatorsmile 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1f5e3540
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1f5e3540
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1f5e3540

Branch: refs/heads/branch-2.3
Commit: 1f5e3540c7535ceaea66ebd5ee2f598e8b3ba1a5
Parents: eb99b8a
Author: gatorsmile 
Authored: Thu Jan 4 21:07:31 2018 +0800
Committer: gatorsmile 
Committed: Thu Jan 4 21:07:56 2018 +0800

--
 python/pyspark/sql/catalog.py | 27 +
 python/pyspark/sql/context.py | 16 ++---
 python/pyspark/sql/tests.py   | 49 +++---
 python/pyspark/sql/udf.py | 21 ++--
 4 files changed, 84 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1f5e3540/python/pyspark/sql/catalog.py
--
diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py
index 659bc65..1566031 100644
--- a/python/pyspark/sql/catalog.py
+++ b/python/pyspark/sql/catalog.py
@@ -227,15 +227,15 @@ class Catalog(object):
 @ignore_unicode_prefix
 @since(2.0)
 def registerFunction(self, name, f, returnType=StringType()):
-"""Registers a python function (including lambda function) as a UDF
-so it can be used in SQL statements.
+"""Registers a Python function (including lambda function) or a 
:class:`UserDefinedFunction`
+as a UDF. The registered UDF can be used in SQL statement.
 
 In addition to a name and the function itself, the return type can be 
optionally specified.
 When the return type is not given it default to a string and 
conversion will automatically
 be done.  For any other return type, the produced object must match 
the specified type.
 
 :param name: name of the UDF
-:param f: python function
+:param f: a Python function, or a wrapped/native UserDefinedFunction
 :param returnType: a :class:`pyspark.sql.types.DataType` object
 :return: a wrapped :class:`UserDefinedFunction`
 
@@ -255,9 +255,26 @@ class Catalog(object):
 >>> _ = spark.udf.register("stringLengthInt", len, IntegerType())
 >>> spark.sql("SELECT stringLengthInt('test')").collect()
 [Row(stringLengthInt(test)=4)]
+
+>>> import random
+>>> from pyspark.sql.functions import udf
+>>> from pyspark.sql.types import IntegerType, StringType
+>>> random_udf = udf(lambda: random.randint(0, 100), 
IntegerType()).asNondeterministic()
+>>> newRandom_udf = spark.catalog.registerFunction("random_udf", 
random_udf, StringType())
+>>> spark.sql("SELECT random_udf()").collect()  # doctest: +SKIP
+[Row(random_udf()=u'82')]
+>>> spark.range(1).select(newRandom_udf()).collect()  # doctest: +SKIP
+[Row(random_udf()=u'62')]
 """
-udf = UserDefinedFunction(f, returnType=returnType, name=name,
-  evalType=PythonEvalType.SQL_BATCHED_UDF)
+
+# This is to check whether the input function is a wrapped/native 
UserDefinedFunction
+if hasattr(f, 'asNondeterministic'):
+udf = UserDefinedFunction(f.func, returnType=returnType, name=name,
+  

spark git commit: [SPARK-22939][PYSPARK] Support Spark UDF in registerFunction

2018-01-04 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master d5861aba9 -> 5aadbc929


[SPARK-22939][PYSPARK] Support Spark UDF in registerFunction

## What changes were proposed in this pull request?
```Python
import random
from pyspark.sql.functions import udf
from pyspark.sql.types import IntegerType, StringType
random_udf = udf(lambda: int(random.random() * 100), 
IntegerType()).asNondeterministic()
spark.catalog.registerFunction("random_udf", random_udf, StringType())
spark.sql("SELECT random_udf()").collect()
```

We will get the following error.
```
Py4JError: An error occurred while calling o29.__getnewargs__. Trace:
py4j.Py4JException: Method __getnewargs__([]) does not exist
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:326)
at py4j.Gateway.invoke(Gateway.java:274)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:214)
at java.lang.Thread.run(Thread.java:745)
```

This PR is to support it.

## How was this patch tested?
WIP

Author: gatorsmile 

Closes #20137 from gatorsmile/registerFunction.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5aadbc92
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5aadbc92
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5aadbc92

Branch: refs/heads/master
Commit: 5aadbc929cb194e06dbd3bab054a161569289af5
Parents: d5861ab
Author: gatorsmile 
Authored: Thu Jan 4 21:07:31 2018 +0800
Committer: gatorsmile 
Committed: Thu Jan 4 21:07:31 2018 +0800

--
 python/pyspark/sql/catalog.py | 27 +
 python/pyspark/sql/context.py | 16 ++---
 python/pyspark/sql/tests.py   | 49 +++---
 python/pyspark/sql/udf.py | 21 ++--
 4 files changed, 84 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5aadbc92/python/pyspark/sql/catalog.py
--
diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py
index 659bc65..1566031 100644
--- a/python/pyspark/sql/catalog.py
+++ b/python/pyspark/sql/catalog.py
@@ -227,15 +227,15 @@ class Catalog(object):
 @ignore_unicode_prefix
 @since(2.0)
 def registerFunction(self, name, f, returnType=StringType()):
-"""Registers a python function (including lambda function) as a UDF
-so it can be used in SQL statements.
+"""Registers a Python function (including lambda function) or a 
:class:`UserDefinedFunction`
+as a UDF. The registered UDF can be used in SQL statement.
 
 In addition to a name and the function itself, the return type can be 
optionally specified.
 When the return type is not given it default to a string and 
conversion will automatically
 be done.  For any other return type, the produced object must match 
the specified type.
 
 :param name: name of the UDF
-:param f: python function
+:param f: a Python function, or a wrapped/native UserDefinedFunction
 :param returnType: a :class:`pyspark.sql.types.DataType` object
 :return: a wrapped :class:`UserDefinedFunction`
 
@@ -255,9 +255,26 @@ class Catalog(object):
 >>> _ = spark.udf.register("stringLengthInt", len, IntegerType())
 >>> spark.sql("SELECT stringLengthInt('test')").collect()
 [Row(stringLengthInt(test)=4)]
+
+>>> import random
+>>> from pyspark.sql.functions import udf
+>>> from pyspark.sql.types import IntegerType, StringType
+>>> random_udf = udf(lambda: random.randint(0, 100), 
IntegerType()).asNondeterministic()
+>>> newRandom_udf = spark.catalog.registerFunction("random_udf", 
random_udf, StringType())
+>>> spark.sql("SELECT random_udf()").collect()  # doctest: +SKIP
+[Row(random_udf()=u'82')]
+>>> spark.range(1).select(newRandom_udf()).collect()  # doctest: +SKIP
+[Row(random_udf()=u'62')]
 """
-udf = UserDefinedFunction(f, returnType=returnType, name=name,
-  evalType=PythonEvalType.SQL_BATCHED_UDF)
+
+# This is to check whether the input function is a wrapped/native 
UserDefinedFunction
+if hasattr(f, 'asNondeterministic'):
+udf = UserDefinedFunction(f.func, returnType=returnType, name=name,
+  evalType=PythonEvalType.SQL_BATCHED_UDF,
+  deterministic=f.deterministic)
+else:
+  

spark git commit: [SPARK-22944][SQL] improve FoldablePropagation

2018-01-03 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 a51212b64 -> f51c8fde8


[SPARK-22944][SQL] improve FoldablePropagation

## What changes were proposed in this pull request?

`FoldablePropagation` is a little tricky as it needs to handle attributes that 
are miss-derived from children, e.g. outer join outputs. This rule does a kind 
of stop-able tree transform, to skip to apply this rule when hit a node which 
may have miss-derived attributes.

Logically we should be able to apply this rule above the unsupported nodes, by 
just treating the unsupported nodes as leaf nodes. This PR improves this rule 
to not stop the tree transformation, but reduce the foldable expressions that 
we want to propagate.

## How was this patch tested?

existing tests

Author: Wenchen Fan 

Closes #20139 from cloud-fan/foldable.

(cherry picked from commit 7d045c5f00e2c7c67011830e2169a4e130c3ace8)
Signed-off-by: gatorsmile 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f51c8fde
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f51c8fde
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f51c8fde

Branch: refs/heads/branch-2.3
Commit: f51c8fde8bf08705bacf8a93b5dba685ebbcec17
Parents: a51212b
Author: Wenchen Fan 
Authored: Thu Jan 4 13:14:52 2018 +0800
Committer: gatorsmile 
Committed: Thu Jan 4 13:15:14 2018 +0800

--
 .../sql/catalyst/optimizer/expressions.scala| 65 +++-
 .../optimizer/FoldablePropagationSuite.scala| 23 ++-
 2 files changed, 58 insertions(+), 30 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f51c8fde/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
index 7d830bb..1c0b7bd 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
@@ -506,18 +506,21 @@ object NullPropagation extends Rule[LogicalPlan] {
 
 
 /**
- * Propagate foldable expressions:
  * Replace attributes with aliases of the original foldable expressions if 
possible.
- * Other optimizations will take advantage of the propagated foldable 
expressions.
- *
+ * Other optimizations will take advantage of the propagated foldable 
expressions. For example,
+ * this rule can optimize
  * {{{
  *   SELECT 1.0 x, 'abc' y, Now() z ORDER BY x, y, 3
- *   ==>  SELECT 1.0 x, 'abc' y, Now() z ORDER BY 1.0, 'abc', Now()
  * }}}
+ * to
+ * {{{
+ *   SELECT 1.0 x, 'abc' y, Now() z ORDER BY 1.0, 'abc', Now()
+ * }}}
+ * and other rules can further optimize it and remove the ORDER BY operator.
  */
 object FoldablePropagation extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = {
-val foldableMap = AttributeMap(plan.flatMap {
+var foldableMap = AttributeMap(plan.flatMap {
   case Project(projectList, _) => projectList.collect {
 case a: Alias if a.child.foldable => (a.toAttribute, a)
   }
@@ -530,38 +533,44 @@ object FoldablePropagation extends Rule[LogicalPlan] {
 if (foldableMap.isEmpty) {
   plan
 } else {
-  var stop = false
   CleanupAliases(plan.transformUp {
-// A leaf node should not stop the folding process (note that we are 
traversing up the
-// tree, starting at the leaf nodes); so we are allowing it.
-case l: LeafNode =>
-  l
-
 // We can only propagate foldables for a subset of unary nodes.
-case u: UnaryNode if !stop && canPropagateFoldables(u) =>
+case u: UnaryNode if foldableMap.nonEmpty && canPropagateFoldables(u) 
=>
   u.transformExpressions(replaceFoldable)
 
-// Allow inner joins. We do not allow outer join, although its output 
attributes are
-// derived from its children, they are actually different attributes: 
the output of outer
-// join is not always picked from its children, but can also be null.
+// Join derives the output attributes from its child while they are 
actually not the
+// same attributes. For example, the output of outer join is not 
always picked from its
+// children, but can also be null. We should exclude these 
miss-derived attributes when
+// propagating the foldable expressions.
 // TODO(cloud-fan): It seems more reasonable to use new attributes as 
the output attributes
 // of outer join.
-case j @ Join(_, _, Inner, _) if 

spark git commit: [SPARK-22944][SQL] improve FoldablePropagation

2018-01-03 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master b29702913 -> 7d045c5f0


[SPARK-22944][SQL] improve FoldablePropagation

## What changes were proposed in this pull request?

`FoldablePropagation` is a little tricky as it needs to handle attributes that 
are miss-derived from children, e.g. outer join outputs. This rule does a kind 
of stop-able tree transform, to skip to apply this rule when hit a node which 
may have miss-derived attributes.

Logically we should be able to apply this rule above the unsupported nodes, by 
just treating the unsupported nodes as leaf nodes. This PR improves this rule 
to not stop the tree transformation, but reduce the foldable expressions that 
we want to propagate.

## How was this patch tested?

existing tests

Author: Wenchen Fan 

Closes #20139 from cloud-fan/foldable.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7d045c5f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7d045c5f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7d045c5f

Branch: refs/heads/master
Commit: 7d045c5f00e2c7c67011830e2169a4e130c3ace8
Parents: b297029
Author: Wenchen Fan 
Authored: Thu Jan 4 13:14:52 2018 +0800
Committer: gatorsmile 
Committed: Thu Jan 4 13:14:52 2018 +0800

--
 .../sql/catalyst/optimizer/expressions.scala| 65 +++-
 .../optimizer/FoldablePropagationSuite.scala| 23 ++-
 2 files changed, 58 insertions(+), 30 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7d045c5f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
index 7d830bb..1c0b7bd 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
@@ -506,18 +506,21 @@ object NullPropagation extends Rule[LogicalPlan] {
 
 
 /**
- * Propagate foldable expressions:
  * Replace attributes with aliases of the original foldable expressions if 
possible.
- * Other optimizations will take advantage of the propagated foldable 
expressions.
- *
+ * Other optimizations will take advantage of the propagated foldable 
expressions. For example,
+ * this rule can optimize
  * {{{
  *   SELECT 1.0 x, 'abc' y, Now() z ORDER BY x, y, 3
- *   ==>  SELECT 1.0 x, 'abc' y, Now() z ORDER BY 1.0, 'abc', Now()
  * }}}
+ * to
+ * {{{
+ *   SELECT 1.0 x, 'abc' y, Now() z ORDER BY 1.0, 'abc', Now()
+ * }}}
+ * and other rules can further optimize it and remove the ORDER BY operator.
  */
 object FoldablePropagation extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = {
-val foldableMap = AttributeMap(plan.flatMap {
+var foldableMap = AttributeMap(plan.flatMap {
   case Project(projectList, _) => projectList.collect {
 case a: Alias if a.child.foldable => (a.toAttribute, a)
   }
@@ -530,38 +533,44 @@ object FoldablePropagation extends Rule[LogicalPlan] {
 if (foldableMap.isEmpty) {
   plan
 } else {
-  var stop = false
   CleanupAliases(plan.transformUp {
-// A leaf node should not stop the folding process (note that we are 
traversing up the
-// tree, starting at the leaf nodes); so we are allowing it.
-case l: LeafNode =>
-  l
-
 // We can only propagate foldables for a subset of unary nodes.
-case u: UnaryNode if !stop && canPropagateFoldables(u) =>
+case u: UnaryNode if foldableMap.nonEmpty && canPropagateFoldables(u) 
=>
   u.transformExpressions(replaceFoldable)
 
-// Allow inner joins. We do not allow outer join, although its output 
attributes are
-// derived from its children, they are actually different attributes: 
the output of outer
-// join is not always picked from its children, but can also be null.
+// Join derives the output attributes from its child while they are 
actually not the
+// same attributes. For example, the output of outer join is not 
always picked from its
+// children, but can also be null. We should exclude these 
miss-derived attributes when
+// propagating the foldable expressions.
 // TODO(cloud-fan): It seems more reasonable to use new attributes as 
the output attributes
 // of outer join.
-case j @ Join(_, _, Inner, _) if !stop =>
-  j.transformExpressions(replaceFoldable)
-
-// We can fold the projections an expand holds. However 

[1/2] spark git commit: [SPARK-20960][SQL] make ColumnVector public

2018-01-03 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 79f7263da -> a51212b64


http://git-wip-us.apache.org/repos/asf/spark/blob/a51212b6/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java 
b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java
new file mode 100644
index 000..3c6656d
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java
@@ -0,0 +1,205 @@
+/*
+ * 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.vectorized;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.types.*;
+import org.apache.spark.unsafe.types.CalendarInterval;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * Row abstraction in {@link ColumnVector}.
+ */
+public final class ColumnarRow extends InternalRow {
+  // The data for this row.
+  // E.g. the value of 3rd int field is `data.getChildColumn(3).getInt(rowId)`.
+  private final ColumnVector data;
+  private final int rowId;
+  private final int numFields;
+
+  public ColumnarRow(ColumnVector data, int rowId) {
+assert (data.dataType() instanceof StructType);
+this.data = data;
+this.rowId = rowId;
+this.numFields = ((StructType) data.dataType()).size();
+  }
+
+  @Override
+  public int numFields() { return numFields; }
+
+  /**
+   * Revisit this. This is expensive. This is currently only used in test 
paths.
+   */
+  @Override
+  public InternalRow copy() {
+GenericInternalRow row = new GenericInternalRow(numFields);
+for (int i = 0; i < numFields(); i++) {
+  if (isNullAt(i)) {
+row.setNullAt(i);
+  } else {
+DataType dt = data.getChildColumn(i).dataType();
+if (dt instanceof BooleanType) {
+  row.setBoolean(i, getBoolean(i));
+} else if (dt instanceof ByteType) {
+  row.setByte(i, getByte(i));
+} else if (dt instanceof ShortType) {
+  row.setShort(i, getShort(i));
+} else if (dt instanceof IntegerType) {
+  row.setInt(i, getInt(i));
+} else if (dt instanceof LongType) {
+  row.setLong(i, getLong(i));
+} else if (dt instanceof FloatType) {
+  row.setFloat(i, getFloat(i));
+} else if (dt instanceof DoubleType) {
+  row.setDouble(i, getDouble(i));
+} else if (dt instanceof StringType) {
+  row.update(i, getUTF8String(i).copy());
+} else if (dt instanceof BinaryType) {
+  row.update(i, getBinary(i));
+} else if (dt instanceof DecimalType) {
+  DecimalType t = (DecimalType)dt;
+  row.setDecimal(i, getDecimal(i, t.precision(), t.scale()), 
t.precision());
+} else if (dt instanceof DateType) {
+  row.setInt(i, getInt(i));
+} else if (dt instanceof TimestampType) {
+  row.setLong(i, getLong(i));
+} else {
+  throw new RuntimeException("Not implemented. " + dt);
+}
+  }
+}
+return row;
+  }
+
+  @Override
+  public boolean anyNull() {
+throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isNullAt(int ordinal) { return 
data.getChildColumn(ordinal).isNullAt(rowId); }
+
+  @Override
+  public boolean getBoolean(int ordinal) { return 
data.getChildColumn(ordinal).getBoolean(rowId); }
+
+  @Override
+  public byte getByte(int ordinal) { return 
data.getChildColumn(ordinal).getByte(rowId); }
+
+  @Override
+  public short getShort(int ordinal) { return 
data.getChildColumn(ordinal).getShort(rowId); }
+
+  @Override
+  public int getInt(int ordinal) { return 
data.getChildColumn(ordinal).getInt(rowId); }
+
+  @Override
+  public long getLong(int ordinal) { return 
data.getChildColumn(ordinal).getLong(rowId); }
+
+  @Override
+  public float getFloat(int ordinal) { return 
data.getChildColumn(ordinal).getFloat(rowId); }
+
+  @Override
+  public double getDouble(int ordinal) { return 

[2/2] spark git commit: [SPARK-20960][SQL] make ColumnVector public

2018-01-03 Thread lixiao
[SPARK-20960][SQL] make ColumnVector public

## What changes were proposed in this pull request?

move `ColumnVector` and related classes to `org.apache.spark.sql.vectorized`, 
and improve the document.

## How was this patch tested?

existing tests.

Author: Wenchen Fan 

Closes #20116 from cloud-fan/column-vector.

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


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a51212b6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a51212b6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a51212b6

Branch: refs/heads/branch-2.3
Commit: a51212b642f05f28447b80aa29f5482de2c27f58
Parents: 79f7263
Author: Wenchen Fan 
Authored: Thu Jan 4 07:28:53 2018 +0800
Committer: gatorsmile 
Committed: Thu Jan 4 07:29:33 2018 +0800

--
 .../parquet/VectorizedParquetRecordReader.java  |   7 +-
 .../execution/vectorized/ArrowColumnVector.java | 620 ---
 .../sql/execution/vectorized/ColumnVector.java  | 208 ---
 .../execution/vectorized/ColumnVectorUtils.java |   2 +
 .../sql/execution/vectorized/ColumnarArray.java | 202 --
 .../sql/execution/vectorized/ColumnarBatch.java | 149 -
 .../sql/execution/vectorized/ColumnarRow.java   | 206 --
 .../vectorized/MutableColumnarRow.java  |   4 +
 .../vectorized/WritableColumnVector.java|   7 +-
 .../spark/sql/vectorized/ArrowColumnVector.java | 562 +
 .../spark/sql/vectorized/ColumnVector.java  | 215 +++
 .../spark/sql/vectorized/ColumnarArray.java | 201 ++
 .../spark/sql/vectorized/ColumnarBatch.java | 129 
 .../spark/sql/vectorized/ColumnarRow.java   | 205 ++
 .../spark/sql/execution/ColumnarBatchScan.scala |   4 +-
 .../execution/aggregate/HashAggregateExec.scala |   2 +-
 .../aggregate/VectorizedHashMapGenerator.scala  |   3 +-
 .../sql/execution/arrow/ArrowConverters.scala   |   2 +-
 .../columnar/InMemoryTableScanExec.scala|   1 +
 .../sql/execution/datasources/FileScanRDD.scala |   2 +-
 .../execution/python/ArrowPythonRunner.scala|   2 +-
 .../sql/execution/arrow/ArrowWriterSuite.scala  |   2 +-
 .../vectorized/ArrowColumnVectorSuite.scala |   1 +
 .../vectorized/ColumnVectorSuite.scala  |   2 +-
 .../vectorized/ColumnarBatchSuite.scala |   6 +-
 25 files changed, 1341 insertions(+), 1403 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a51212b6/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java
index 6c157e8..cd745b1 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java
@@ -31,10 +31,10 @@ import org.apache.parquet.schema.Type;
 import org.apache.spark.memory.MemoryMode;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
-import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
 import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
 import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
 import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
 import org.apache.spark.sql.types.StructField;
 import org.apache.spark.sql.types.StructType;
 
@@ -248,7 +248,10 @@ public class VectorizedParquetRecordReader extends 
SpecificParquetRecordReaderBa
* Advances to the next batch of rows. Returns false if there are no more.
*/
   public boolean nextBatch() throws IOException {
-columnarBatch.reset();
+for (WritableColumnVector vector : columnVectors) {
+  vector.reset();
+}
+columnarBatch.setNumRows(0);
 if (rowsReturned >= totalRowCount) return false;
 checkEndOfRowGroup();
 

http://git-wip-us.apache.org/repos/asf/spark/blob/a51212b6/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
deleted file mode 100644

[2/2] spark git commit: [SPARK-20960][SQL] make ColumnVector public

2018-01-03 Thread lixiao
[SPARK-20960][SQL] make ColumnVector public

## What changes were proposed in this pull request?

move `ColumnVector` and related classes to `org.apache.spark.sql.vectorized`, 
and improve the document.

## How was this patch tested?

existing tests.

Author: Wenchen Fan 

Closes #20116 from cloud-fan/column-vector.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b2970291
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b2970291
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b2970291

Branch: refs/heads/master
Commit: b297029130735316e1ac1144dee44761a12bfba7
Parents: 9a2b65a
Author: Wenchen Fan 
Authored: Thu Jan 4 07:28:53 2018 +0800
Committer: gatorsmile 
Committed: Thu Jan 4 07:28:53 2018 +0800

--
 .../parquet/VectorizedParquetRecordReader.java  |   7 +-
 .../execution/vectorized/ArrowColumnVector.java | 620 ---
 .../sql/execution/vectorized/ColumnVector.java  | 208 ---
 .../execution/vectorized/ColumnVectorUtils.java |   2 +
 .../sql/execution/vectorized/ColumnarArray.java | 202 --
 .../sql/execution/vectorized/ColumnarBatch.java | 149 -
 .../sql/execution/vectorized/ColumnarRow.java   | 206 --
 .../vectorized/MutableColumnarRow.java  |   4 +
 .../vectorized/WritableColumnVector.java|   7 +-
 .../spark/sql/vectorized/ArrowColumnVector.java | 562 +
 .../spark/sql/vectorized/ColumnVector.java  | 215 +++
 .../spark/sql/vectorized/ColumnarArray.java | 201 ++
 .../spark/sql/vectorized/ColumnarBatch.java | 129 
 .../spark/sql/vectorized/ColumnarRow.java   | 205 ++
 .../spark/sql/execution/ColumnarBatchScan.scala |   4 +-
 .../execution/aggregate/HashAggregateExec.scala |   2 +-
 .../aggregate/VectorizedHashMapGenerator.scala  |   3 +-
 .../sql/execution/arrow/ArrowConverters.scala   |   2 +-
 .../columnar/InMemoryTableScanExec.scala|   1 +
 .../sql/execution/datasources/FileScanRDD.scala |   2 +-
 .../execution/python/ArrowPythonRunner.scala|   2 +-
 .../sql/execution/arrow/ArrowWriterSuite.scala  |   2 +-
 .../vectorized/ArrowColumnVectorSuite.scala |   1 +
 .../vectorized/ColumnVectorSuite.scala  |   2 +-
 .../vectorized/ColumnarBatchSuite.scala |   6 +-
 25 files changed, 1341 insertions(+), 1403 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b2970291/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java
index 6c157e8..cd745b1 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java
@@ -31,10 +31,10 @@ import org.apache.parquet.schema.Type;
 import org.apache.spark.memory.MemoryMode;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils;
-import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
 import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
 import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
 import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
 import org.apache.spark.sql.types.StructField;
 import org.apache.spark.sql.types.StructType;
 
@@ -248,7 +248,10 @@ public class VectorizedParquetRecordReader extends 
SpecificParquetRecordReaderBa
* Advances to the next batch of rows. Returns false if there are no more.
*/
   public boolean nextBatch() throws IOException {
-columnarBatch.reset();
+for (WritableColumnVector vector : columnVectors) {
+  vector.reset();
+}
+columnarBatch.setNumRows(0);
 if (rowsReturned >= totalRowCount) return false;
 checkEndOfRowGroup();
 

http://git-wip-us.apache.org/repos/asf/spark/blob/b2970291/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
deleted file mode 100644
index af5673e..000
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
+++ 

[1/2] spark git commit: [SPARK-20960][SQL] make ColumnVector public

2018-01-03 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 9a2b65a3c -> b29702913


http://git-wip-us.apache.org/repos/asf/spark/blob/b2970291/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java 
b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java
new file mode 100644
index 000..3c6656d
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java
@@ -0,0 +1,205 @@
+/*
+ * 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.vectorized;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.types.*;
+import org.apache.spark.unsafe.types.CalendarInterval;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * Row abstraction in {@link ColumnVector}.
+ */
+public final class ColumnarRow extends InternalRow {
+  // The data for this row.
+  // E.g. the value of 3rd int field is `data.getChildColumn(3).getInt(rowId)`.
+  private final ColumnVector data;
+  private final int rowId;
+  private final int numFields;
+
+  public ColumnarRow(ColumnVector data, int rowId) {
+assert (data.dataType() instanceof StructType);
+this.data = data;
+this.rowId = rowId;
+this.numFields = ((StructType) data.dataType()).size();
+  }
+
+  @Override
+  public int numFields() { return numFields; }
+
+  /**
+   * Revisit this. This is expensive. This is currently only used in test 
paths.
+   */
+  @Override
+  public InternalRow copy() {
+GenericInternalRow row = new GenericInternalRow(numFields);
+for (int i = 0; i < numFields(); i++) {
+  if (isNullAt(i)) {
+row.setNullAt(i);
+  } else {
+DataType dt = data.getChildColumn(i).dataType();
+if (dt instanceof BooleanType) {
+  row.setBoolean(i, getBoolean(i));
+} else if (dt instanceof ByteType) {
+  row.setByte(i, getByte(i));
+} else if (dt instanceof ShortType) {
+  row.setShort(i, getShort(i));
+} else if (dt instanceof IntegerType) {
+  row.setInt(i, getInt(i));
+} else if (dt instanceof LongType) {
+  row.setLong(i, getLong(i));
+} else if (dt instanceof FloatType) {
+  row.setFloat(i, getFloat(i));
+} else if (dt instanceof DoubleType) {
+  row.setDouble(i, getDouble(i));
+} else if (dt instanceof StringType) {
+  row.update(i, getUTF8String(i).copy());
+} else if (dt instanceof BinaryType) {
+  row.update(i, getBinary(i));
+} else if (dt instanceof DecimalType) {
+  DecimalType t = (DecimalType)dt;
+  row.setDecimal(i, getDecimal(i, t.precision(), t.scale()), 
t.precision());
+} else if (dt instanceof DateType) {
+  row.setInt(i, getInt(i));
+} else if (dt instanceof TimestampType) {
+  row.setLong(i, getLong(i));
+} else {
+  throw new RuntimeException("Not implemented. " + dt);
+}
+  }
+}
+return row;
+  }
+
+  @Override
+  public boolean anyNull() {
+throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isNullAt(int ordinal) { return 
data.getChildColumn(ordinal).isNullAt(rowId); }
+
+  @Override
+  public boolean getBoolean(int ordinal) { return 
data.getChildColumn(ordinal).getBoolean(rowId); }
+
+  @Override
+  public byte getByte(int ordinal) { return 
data.getChildColumn(ordinal).getByte(rowId); }
+
+  @Override
+  public short getShort(int ordinal) { return 
data.getChildColumn(ordinal).getShort(rowId); }
+
+  @Override
+  public int getInt(int ordinal) { return 
data.getChildColumn(ordinal).getInt(rowId); }
+
+  @Override
+  public long getLong(int ordinal) { return 
data.getChildColumn(ordinal).getLong(rowId); }
+
+  @Override
+  public float getFloat(int ordinal) { return 
data.getChildColumn(ordinal).getFloat(rowId); }
+
+  @Override
+  public double getDouble(int ordinal) { return 

spark git commit: [SPARK-22932][SQL] Refactor AnalysisContext

2018-01-03 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 b96248862 -> 27c949d67


[SPARK-22932][SQL] Refactor AnalysisContext

## What changes were proposed in this pull request?
Add a `reset` function to ensure the state in `AnalysisContext ` is per-query.

## How was this patch tested?
The existing test cases

Author: gatorsmile 

Closes #20127 from gatorsmile/refactorAnalysisContext.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/27c949d6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/27c949d6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/27c949d6

Branch: refs/heads/branch-2.3
Commit: 27c949d673e45fdbbae0f2c08969b9d51222dd8d
Parents: b962488
Author: gatorsmile 
Authored: Tue Jan 2 09:19:18 2018 +0800
Committer: gatorsmile 
Committed: Thu Jan 4 00:59:56 2018 +0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 25 
 1 file changed, 20 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/27c949d6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 6d294d4..35b3511 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -52,6 +52,7 @@ object SimpleAnalyzer extends Analyzer(
 /**
  * Provides a way to keep state during the analysis, this enables us to 
decouple the concerns
  * of analysis environment from the catalog.
+ * The state that is kept here is per-query.
  *
  * Note this is thread local.
  *
@@ -70,6 +71,8 @@ object AnalysisContext {
   }
 
   def get: AnalysisContext = value.get()
+  def reset(): Unit = value.remove()
+
   private def set(context: AnalysisContext): Unit = value.set(context)
 
   def withAnalysisContext[A](database: Option[String])(f: => A): A = {
@@ -95,6 +98,17 @@ class Analyzer(
 this(catalog, conf, conf.optimizerMaxIterations)
   }
 
+  override def execute(plan: LogicalPlan): LogicalPlan = {
+AnalysisContext.reset()
+try {
+  executeSameContext(plan)
+} finally {
+  AnalysisContext.reset()
+}
+  }
+
+  private def executeSameContext(plan: LogicalPlan): LogicalPlan = 
super.execute(plan)
+
   def resolver: Resolver = conf.resolver
 
   protected val fixedPoint = FixedPoint(maxIterations)
@@ -176,7 +190,7 @@ class Analyzer(
   case With(child, relations) =>
 substituteCTE(child, relations.foldLeft(Seq.empty[(String, 
LogicalPlan)]) {
   case (resolved, (name, relation)) =>
-resolved :+ name -> execute(substituteCTE(relation, resolved))
+resolved :+ name -> executeSameContext(substituteCTE(relation, 
resolved))
 })
   case other => other
 }
@@ -600,7 +614,7 @@ class Analyzer(
   "avoid errors. Increase the value of 
spark.sql.view.maxNestedViewDepth to work " +
   "aroud this.")
   }
-  execute(child)
+  executeSameContext(child)
 }
 view.copy(child = newChild)
   case p @ SubqueryAlias(_, view: View) =>
@@ -1269,7 +1283,7 @@ class Analyzer(
   do {
 // Try to resolve the subquery plan using the regular analyzer.
 previous = current
-current = execute(current)
+current = executeSameContext(current)
 
 // Use the outer references to resolve the subquery plan if it isn't 
resolved yet.
 val i = plans.iterator
@@ -1392,7 +1406,7 @@ class Analyzer(
   grouping,
   Alias(cond, "havingCondition")() :: Nil,
   child)
-  val resolvedOperator = execute(aggregatedCondition)
+  val resolvedOperator = executeSameContext(aggregatedCondition)
   def resolvedAggregateFilter =
 resolvedOperator
   .asInstanceOf[Aggregate]
@@ -1450,7 +1464,8 @@ class Analyzer(
   val aliasedOrdering =
 unresolvedSortOrders.map(o => Alias(o.child, "aggOrder")())
   val aggregatedOrdering = aggregate.copy(aggregateExpressions = 
aliasedOrdering)
-  val resolvedAggregate: Aggregate = 
execute(aggregatedOrdering).asInstanceOf[Aggregate]
+  val resolvedAggregate: Aggregate =
+executeSameContext(aggregatedOrdering).asInstanceOf[Aggregate]
   val resolvedAliasedOrdering: Seq[Alias] =
 resolvedAggregate.aggregateExpressions.asInstanceOf[Seq[Alias]]
 



spark git commit: [SPARK-20236][SQL] dynamic partition overwrite

2018-01-03 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 a05e85ecb -> b96248862


[SPARK-20236][SQL] dynamic partition overwrite

## What changes were proposed in this pull request?

When overwriting a partitioned table with dynamic partition columns, the 
behavior is different between data source and hive tables.

data source table: delete all partition directories that match the static 
partition values provided in the insert statement.

hive table: only delete partition directories which have data written into it

This PR adds a new config to make users be able to choose hive's behavior.

## How was this patch tested?

new tests

Author: Wenchen Fan 

Closes #18714 from cloud-fan/overwrite-partition.

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


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b9624886
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b9624886
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b9624886

Branch: refs/heads/branch-2.3
Commit: b96248862589bae1ddcdb14ce4c802789a001306
Parents: a05e85e
Author: Wenchen Fan 
Authored: Wed Jan 3 22:18:13 2018 +0800
Committer: gatorsmile 
Committed: Wed Jan 3 22:19:02 2018 +0800

--
 .../spark/internal/io/FileCommitProtocol.scala  | 25 +--
 .../io/HadoopMapReduceCommitProtocol.scala  | 75 +++
 .../org/apache/spark/sql/internal/SQLConf.scala | 21 ++
 .../InsertIntoHadoopFsRelationCommand.scala | 20 -
 .../SQLHadoopMapReduceCommitProtocol.scala  | 10 ++-
 .../apache/spark/sql/sources/InsertSuite.scala  | 78 
 6 files changed, 200 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b9624886/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala 
b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
index 50f51e1..6d0059b 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
@@ -28,8 +28,9 @@ import org.apache.spark.util.Utils
  *
  * 1. Implementations must be serializable, as the committer instance 
instantiated on the driver
  *will be used for tasks on executors.
- * 2. Implementations should have a constructor with 2 arguments:
- *  (jobId: String, path: String)
+ * 2. Implementations should have a constructor with 2 or 3 arguments:
+ *  (jobId: String, path: String) or
+ *  (jobId: String, path: String, dynamicPartitionOverwrite: Boolean)
  * 3. A committer should not be reused across multiple Spark jobs.
  *
  * The proper call sequence is:
@@ -139,10 +140,22 @@ object FileCommitProtocol {
   /**
* Instantiates a FileCommitProtocol using the given className.
*/
-  def instantiate(className: String, jobId: String, outputPath: String)
-: FileCommitProtocol = {
+  def instantiate(
+  className: String,
+  jobId: String,
+  outputPath: String,
+  dynamicPartitionOverwrite: Boolean = false): FileCommitProtocol = {
 val clazz = 
Utils.classForName(className).asInstanceOf[Class[FileCommitProtocol]]
-val ctor = clazz.getDeclaredConstructor(classOf[String], classOf[String])
-ctor.newInstance(jobId, outputPath)
+// First try the constructor with arguments (jobId: String, outputPath: 
String,
+// dynamicPartitionOverwrite: Boolean).
+// If that doesn't exist, try the one with (jobId: string, outputPath: 
String).
+try {
+  val ctor = clazz.getDeclaredConstructor(classOf[String], 
classOf[String], classOf[Boolean])
+  ctor.newInstance(jobId, outputPath, 
dynamicPartitionOverwrite.asInstanceOf[java.lang.Boolean])
+} catch {
+  case _: NoSuchMethodException =>
+val ctor = clazz.getDeclaredConstructor(classOf[String], 
classOf[String])
+ctor.newInstance(jobId, outputPath)
+}
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/b9624886/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
 
b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
index 95c99d2..6d20ef1 100644
--- 
a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
+++ 
b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
@@ 

spark git commit: [SPARK-20236][SQL] dynamic partition overwrite

2018-01-03 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 1a87a1609 -> a66fe36ce


[SPARK-20236][SQL] dynamic partition overwrite

## What changes were proposed in this pull request?

When overwriting a partitioned table with dynamic partition columns, the 
behavior is different between data source and hive tables.

data source table: delete all partition directories that match the static 
partition values provided in the insert statement.

hive table: only delete partition directories which have data written into it

This PR adds a new config to make users be able to choose hive's behavior.

## How was this patch tested?

new tests

Author: Wenchen Fan 

Closes #18714 from cloud-fan/overwrite-partition.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a66fe36c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a66fe36c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a66fe36c

Branch: refs/heads/master
Commit: a66fe36cee9363b01ee70e469f1c968f633c5713
Parents: 1a87a16
Author: Wenchen Fan 
Authored: Wed Jan 3 22:18:13 2018 +0800
Committer: gatorsmile 
Committed: Wed Jan 3 22:18:13 2018 +0800

--
 .../spark/internal/io/FileCommitProtocol.scala  | 25 +--
 .../io/HadoopMapReduceCommitProtocol.scala  | 75 +++
 .../org/apache/spark/sql/internal/SQLConf.scala | 21 ++
 .../InsertIntoHadoopFsRelationCommand.scala | 20 -
 .../SQLHadoopMapReduceCommitProtocol.scala  | 10 ++-
 .../apache/spark/sql/sources/InsertSuite.scala  | 78 
 6 files changed, 200 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a66fe36c/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala 
b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
index 50f51e1..6d0059b 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
@@ -28,8 +28,9 @@ import org.apache.spark.util.Utils
  *
  * 1. Implementations must be serializable, as the committer instance 
instantiated on the driver
  *will be used for tasks on executors.
- * 2. Implementations should have a constructor with 2 arguments:
- *  (jobId: String, path: String)
+ * 2. Implementations should have a constructor with 2 or 3 arguments:
+ *  (jobId: String, path: String) or
+ *  (jobId: String, path: String, dynamicPartitionOverwrite: Boolean)
  * 3. A committer should not be reused across multiple Spark jobs.
  *
  * The proper call sequence is:
@@ -139,10 +140,22 @@ object FileCommitProtocol {
   /**
* Instantiates a FileCommitProtocol using the given className.
*/
-  def instantiate(className: String, jobId: String, outputPath: String)
-: FileCommitProtocol = {
+  def instantiate(
+  className: String,
+  jobId: String,
+  outputPath: String,
+  dynamicPartitionOverwrite: Boolean = false): FileCommitProtocol = {
 val clazz = 
Utils.classForName(className).asInstanceOf[Class[FileCommitProtocol]]
-val ctor = clazz.getDeclaredConstructor(classOf[String], classOf[String])
-ctor.newInstance(jobId, outputPath)
+// First try the constructor with arguments (jobId: String, outputPath: 
String,
+// dynamicPartitionOverwrite: Boolean).
+// If that doesn't exist, try the one with (jobId: string, outputPath: 
String).
+try {
+  val ctor = clazz.getDeclaredConstructor(classOf[String], 
classOf[String], classOf[Boolean])
+  ctor.newInstance(jobId, outputPath, 
dynamicPartitionOverwrite.asInstanceOf[java.lang.Boolean])
+} catch {
+  case _: NoSuchMethodException =>
+val ctor = clazz.getDeclaredConstructor(classOf[String], 
classOf[String])
+ctor.newInstance(jobId, outputPath)
+}
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/a66fe36c/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
 
b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
index 95c99d2..6d20ef1 100644
--- 
a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
+++ 
b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
@@ -39,8 +39,19 @@ import org.apache.spark.mapred.SparkHadoopMapRedUtil
  *
  * @param jobId the job's or stage's id
  * @param 

spark git commit: [SPARK-22934][SQL] Make optional clauses order insensitive for CREATE TABLE SQL statement

2018-01-03 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 b96a21324 -> a05e85ecb


[SPARK-22934][SQL] Make optional clauses order insensitive for CREATE TABLE SQL 
statement

## What changes were proposed in this pull request?
Currently, our CREATE TABLE syntax require the EXACT order of clauses. It is 
pretty hard to remember the exact order. Thus, this PR is to make optional 
clauses order insensitive for `CREATE TABLE` SQL statement.

```
CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name
[(col_name1 col_type1 [COMMENT col_comment1], ...)]
USING datasource
[OPTIONS (key1=val1, key2=val2, ...)]
[PARTITIONED BY (col_name1, col_name2, ...)]
[CLUSTERED BY (col_name3, col_name4, ...) INTO num_buckets BUCKETS]
[LOCATION path]
[COMMENT table_comment]
[TBLPROPERTIES (key1=val1, key2=val2, ...)]
[AS select_statement]
```

The proposal is to make the following clauses order insensitive.
```
[OPTIONS (key1=val1, key2=val2, ...)]
[PARTITIONED BY (col_name1, col_name2, ...)]
[CLUSTERED BY (col_name3, col_name4, ...) INTO num_buckets BUCKETS]
[LOCATION path]
[COMMENT table_comment]
[TBLPROPERTIES (key1=val1, key2=val2, ...)]
```

The same idea is also applicable to Create Hive Table.
```
CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name
[(col_name1[:] col_type1 [COMMENT col_comment1], ...)]
[COMMENT table_comment]
[PARTITIONED BY (col_name2[:] col_type2 [COMMENT col_comment2], ...)]
[ROW FORMAT row_format]
[STORED AS file_format]
[LOCATION path]
[TBLPROPERTIES (key1=val1, key2=val2, ...)]
[AS select_statement]
```

The proposal is to make the following clauses order insensitive.
```
[COMMENT table_comment]
[PARTITIONED BY (col_name2[:] col_type2 [COMMENT col_comment2], ...)]
[ROW FORMAT row_format]
[STORED AS file_format]
[LOCATION path]
[TBLPROPERTIES (key1=val1, key2=val2, ...)]
```

## How was this patch tested?
Added test cases

Author: gatorsmile 

Closes #20133 from gatorsmile/createDataSourceTableDDL.

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


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a05e85ec
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a05e85ec
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a05e85ec

Branch: refs/heads/branch-2.3
Commit: a05e85ecb76091567a26a3a14ad0879b4728addc
Parents: b96a213
Author: gatorsmile 
Authored: Wed Jan 3 22:09:30 2018 +0800
Committer: gatorsmile 
Committed: Wed Jan 3 22:10:06 2018 +0800

--
 .../apache/spark/sql/catalyst/parser/SqlBase.g4 |  24 +-
 .../spark/sql/catalyst/parser/ParserUtils.scala |   9 +
 .../spark/sql/execution/SparkSqlParser.scala|  81 ---
 .../sql/execution/command/DDLParserSuite.scala  | 220 +++
 .../spark/sql/execution/command/DDLSuite.scala  |   2 +-
 .../spark/sql/hive/execution/HiveDDLSuite.scala |  13 +-
 .../sql/hive/execution/SQLQuerySuite.scala  | 124 ++-
 7 files changed, 335 insertions(+), 138 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a05e85ec/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
--
diff --git 
a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 
b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
index 6fe995f..6daf01d 100644
--- 
a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
+++ 
b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
@@ -73,18 +73,22 @@ statement
 | ALTER DATABASE identifier SET DBPROPERTIES tablePropertyList 
#setDatabaseProperties
 | DROP DATABASE (IF EXISTS)? identifier (RESTRICT | CASCADE)?  
#dropDatabase
 | createTableHeader ('(' colTypeList ')')? tableProvider
-(OPTIONS options=tablePropertyList)?
-(PARTITIONED BY partitionColumnNames=identifierList)?
-bucketSpec? locationSpec?
-(COMMENT comment=STRING)?
-(TBLPROPERTIES tableProps=tablePropertyList)?
+((OPTIONS options=tablePropertyList) |
+(PARTITIONED BY partitionColumnNames=identifierList) |
+bucketSpec |
+locationSpec |
+(COMMENT comment=STRING) |
+(TBLPROPERTIES tableProps=tablePropertyList))*
 (AS? query)?   
#createTable
 | createTableHeader ('(' columns=colTypeList ')')?
-(COMMENT comment=STRING)?
-(PARTITIONED BY '(' partitionColumns=colTypeList ')')?
-bucketSpec? skewSpec?
-rowFormat? 

spark git commit: [SPARK-22934][SQL] Make optional clauses order insensitive for CREATE TABLE SQL statement

2018-01-03 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 247a08939 -> 1a87a1609


[SPARK-22934][SQL] Make optional clauses order insensitive for CREATE TABLE SQL 
statement

## What changes were proposed in this pull request?
Currently, our CREATE TABLE syntax require the EXACT order of clauses. It is 
pretty hard to remember the exact order. Thus, this PR is to make optional 
clauses order insensitive for `CREATE TABLE` SQL statement.

```
CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name
[(col_name1 col_type1 [COMMENT col_comment1], ...)]
USING datasource
[OPTIONS (key1=val1, key2=val2, ...)]
[PARTITIONED BY (col_name1, col_name2, ...)]
[CLUSTERED BY (col_name3, col_name4, ...) INTO num_buckets BUCKETS]
[LOCATION path]
[COMMENT table_comment]
[TBLPROPERTIES (key1=val1, key2=val2, ...)]
[AS select_statement]
```

The proposal is to make the following clauses order insensitive.
```
[OPTIONS (key1=val1, key2=val2, ...)]
[PARTITIONED BY (col_name1, col_name2, ...)]
[CLUSTERED BY (col_name3, col_name4, ...) INTO num_buckets BUCKETS]
[LOCATION path]
[COMMENT table_comment]
[TBLPROPERTIES (key1=val1, key2=val2, ...)]
```

The same idea is also applicable to Create Hive Table.
```
CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name
[(col_name1[:] col_type1 [COMMENT col_comment1], ...)]
[COMMENT table_comment]
[PARTITIONED BY (col_name2[:] col_type2 [COMMENT col_comment2], ...)]
[ROW FORMAT row_format]
[STORED AS file_format]
[LOCATION path]
[TBLPROPERTIES (key1=val1, key2=val2, ...)]
[AS select_statement]
```

The proposal is to make the following clauses order insensitive.
```
[COMMENT table_comment]
[PARTITIONED BY (col_name2[:] col_type2 [COMMENT col_comment2], ...)]
[ROW FORMAT row_format]
[STORED AS file_format]
[LOCATION path]
[TBLPROPERTIES (key1=val1, key2=val2, ...)]
```

## How was this patch tested?
Added test cases

Author: gatorsmile 

Closes #20133 from gatorsmile/createDataSourceTableDDL.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1a87a160
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1a87a160
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1a87a160

Branch: refs/heads/master
Commit: 1a87a1609c4d2c9027a2cf669ea3337b89f61fb6
Parents: 247a089
Author: gatorsmile 
Authored: Wed Jan 3 22:09:30 2018 +0800
Committer: gatorsmile 
Committed: Wed Jan 3 22:09:30 2018 +0800

--
 .../apache/spark/sql/catalyst/parser/SqlBase.g4 |  24 +-
 .../spark/sql/catalyst/parser/ParserUtils.scala |   9 +
 .../spark/sql/execution/SparkSqlParser.scala|  81 ---
 .../sql/execution/command/DDLParserSuite.scala  | 220 +++
 .../spark/sql/execution/command/DDLSuite.scala  |   2 +-
 .../spark/sql/hive/execution/HiveDDLSuite.scala |  13 +-
 .../sql/hive/execution/SQLQuerySuite.scala  | 124 ++-
 7 files changed, 335 insertions(+), 138 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1a87a160/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
--
diff --git 
a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 
b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
index 6fe995f..6daf01d 100644
--- 
a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
+++ 
b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
@@ -73,18 +73,22 @@ statement
 | ALTER DATABASE identifier SET DBPROPERTIES tablePropertyList 
#setDatabaseProperties
 | DROP DATABASE (IF EXISTS)? identifier (RESTRICT | CASCADE)?  
#dropDatabase
 | createTableHeader ('(' colTypeList ')')? tableProvider
-(OPTIONS options=tablePropertyList)?
-(PARTITIONED BY partitionColumnNames=identifierList)?
-bucketSpec? locationSpec?
-(COMMENT comment=STRING)?
-(TBLPROPERTIES tableProps=tablePropertyList)?
+((OPTIONS options=tablePropertyList) |
+(PARTITIONED BY partitionColumnNames=identifierList) |
+bucketSpec |
+locationSpec |
+(COMMENT comment=STRING) |
+(TBLPROPERTIES tableProps=tablePropertyList))*
 (AS? query)?   
#createTable
 | createTableHeader ('(' columns=colTypeList ')')?
-(COMMENT comment=STRING)?
-(PARTITIONED BY '(' partitionColumns=colTypeList ')')?
-bucketSpec? skewSpec?
-rowFormat?  createFileFormat? locationSpec?
-(TBLPROPERTIES tablePropertyList)?
+((COMMENT comment=STRING) |
+

spark git commit: [SPARK-22932][SQL] Refactor AnalysisContext

2018-01-01 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master e734a4b9c -> e0c090f22


[SPARK-22932][SQL] Refactor AnalysisContext

## What changes were proposed in this pull request?
Add a `reset` function to ensure the state in `AnalysisContext ` is per-query.

## How was this patch tested?
The existing test cases

Author: gatorsmile 

Closes #20127 from gatorsmile/refactorAnalysisContext.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e0c090f2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e0c090f2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e0c090f2

Branch: refs/heads/master
Commit: e0c090f227e9b64e595b47d4d1f96f8a2fff5bf7
Parents: e734a4b
Author: gatorsmile 
Authored: Tue Jan 2 09:19:18 2018 +0800
Committer: gatorsmile 
Committed: Tue Jan 2 09:19:18 2018 +0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 25 
 1 file changed, 20 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e0c090f2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 6d294d4..35b3511 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -52,6 +52,7 @@ object SimpleAnalyzer extends Analyzer(
 /**
  * Provides a way to keep state during the analysis, this enables us to 
decouple the concerns
  * of analysis environment from the catalog.
+ * The state that is kept here is per-query.
  *
  * Note this is thread local.
  *
@@ -70,6 +71,8 @@ object AnalysisContext {
   }
 
   def get: AnalysisContext = value.get()
+  def reset(): Unit = value.remove()
+
   private def set(context: AnalysisContext): Unit = value.set(context)
 
   def withAnalysisContext[A](database: Option[String])(f: => A): A = {
@@ -95,6 +98,17 @@ class Analyzer(
 this(catalog, conf, conf.optimizerMaxIterations)
   }
 
+  override def execute(plan: LogicalPlan): LogicalPlan = {
+AnalysisContext.reset()
+try {
+  executeSameContext(plan)
+} finally {
+  AnalysisContext.reset()
+}
+  }
+
+  private def executeSameContext(plan: LogicalPlan): LogicalPlan = 
super.execute(plan)
+
   def resolver: Resolver = conf.resolver
 
   protected val fixedPoint = FixedPoint(maxIterations)
@@ -176,7 +190,7 @@ class Analyzer(
   case With(child, relations) =>
 substituteCTE(child, relations.foldLeft(Seq.empty[(String, 
LogicalPlan)]) {
   case (resolved, (name, relation)) =>
-resolved :+ name -> execute(substituteCTE(relation, resolved))
+resolved :+ name -> executeSameContext(substituteCTE(relation, 
resolved))
 })
   case other => other
 }
@@ -600,7 +614,7 @@ class Analyzer(
   "avoid errors. Increase the value of 
spark.sql.view.maxNestedViewDepth to work " +
   "aroud this.")
   }
-  execute(child)
+  executeSameContext(child)
 }
 view.copy(child = newChild)
   case p @ SubqueryAlias(_, view: View) =>
@@ -1269,7 +1283,7 @@ class Analyzer(
   do {
 // Try to resolve the subquery plan using the regular analyzer.
 previous = current
-current = execute(current)
+current = executeSameContext(current)
 
 // Use the outer references to resolve the subquery plan if it isn't 
resolved yet.
 val i = plans.iterator
@@ -1392,7 +1406,7 @@ class Analyzer(
   grouping,
   Alias(cond, "havingCondition")() :: Nil,
   child)
-  val resolvedOperator = execute(aggregatedCondition)
+  val resolvedOperator = executeSameContext(aggregatedCondition)
   def resolvedAggregateFilter =
 resolvedOperator
   .asInstanceOf[Aggregate]
@@ -1450,7 +1464,8 @@ class Analyzer(
   val aliasedOrdering =
 unresolvedSortOrders.map(o => Alias(o.child, "aggOrder")())
   val aggregatedOrdering = aggregate.copy(aggregateExpressions = 
aliasedOrdering)
-  val resolvedAggregate: Aggregate = 
execute(aggregatedOrdering).asInstanceOf[Aggregate]
+  val resolvedAggregate: Aggregate =
+executeSameContext(aggregatedOrdering).asInstanceOf[Aggregate]
   val resolvedAliasedOrdering: Seq[Alias] =
 resolvedAggregate.aggregateExpressions.asInstanceOf[Seq[Alias]]
 



spark git commit: [SPARK-22895][SQL] Push down the deterministic predicates that are after the first non-deterministic

2017-12-30 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master ee3af15fe -> cfbe11e81


[SPARK-22895][SQL] Push down the deterministic predicates that are after the 
first non-deterministic

## What changes were proposed in this pull request?
Currently, we do not guarantee an order evaluation of conjuncts in either 
Filter or Join operator. This is also true to the mainstream RDBMS vendors like 
DB2 and MS SQL Server. Thus, we should also push down the deterministic 
predicates that are after the first non-deterministic, if possible.

## How was this patch tested?
Updated the existing test cases.

Author: gatorsmile 

Closes #20069 from gatorsmile/morePushDown.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cfbe11e8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cfbe11e8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cfbe11e8

Branch: refs/heads/master
Commit: cfbe11e8164c04cd7d388e4faeded21a9331dac4
Parents: ee3af15
Author: gatorsmile 
Authored: Sun Dec 31 15:06:54 2017 +0800
Committer: gatorsmile 
Committed: Sun Dec 31 15:06:54 2017 +0800

--
 docs/sql-programming-guide.md   |  1 +
 .../sql/catalyst/optimizer/Optimizer.scala  | 40 
 .../optimizer/FilterPushdownSuite.scala | 33 
 .../v2/PushDownOperatorsToDataSource.scala  | 10 ++---
 .../execution/python/ExtractPythonUDFs.scala|  6 +--
 .../StreamingSymmetricHashJoinHelper.scala  |  5 +--
 .../python/BatchEvalPythonExecSuite.scala   | 10 +++--
 .../StreamingSymmetricHashJoinHelperSuite.scala | 14 +++
 8 files changed, 54 insertions(+), 65 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cfbe11e8/docs/sql-programming-guide.md
--
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 4b5f56c..dc3e384 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1636,6 +1636,7 @@ options.
 
   - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when 
the referenced columns only include the internal corrupt record column (named 
`_corrupt_record` by default). For example, 
`spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()`
 and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. 
Instead, you can cache or save the parsed results and then send the same query. 
For example, `val df = spark.read.schema(schema).json(file).cache()` and then 
`df.filter($"_corrupt_record".isNotNull).count()`.
   - The `percentile_approx` function previously accepted numeric type input 
and output double type results. Now it supports date type, timestamp type and 
numeric types as input types. The result type is also changed to be the same as 
the input type, which is more reasonable for percentiles.
+  - Since Spark 2.3, the Join/Filter's deterministic predicates that are after 
the first non-deterministic predicates are also pushed down/through the child 
operators, if possible. In prior Spark versions, these filters are not eligible 
for predicate pushdown.
   - Partition column inference previously found incorrect common type for 
different inferred types, for example, previously it ended up with double type 
as the common type for double type and date type. Now it finds the correct 
common type for such conflicts. The conflict resolution follows the table below:
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/cfbe11e8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index eeb1b13..0d4b02c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -805,15 +805,15 @@ object PushDownPredicate extends Rule[LogicalPlan] with 
PredicateHelper {
 
   // For each filter, expand the alias and check if the filter can be 
evaluated using
   // attributes produced by the aggregate operator's child operator.
-  val (candidates, containingNonDeterministic) =
-splitConjunctivePredicates(condition).span(_.deterministic)
+  val (candidates, nonDeterministic) =
+splitConjunctivePredicates(condition).partition(_.deterministic)
 
   val (pushDown, rest) = candidates.partition { cond =>
 val replaced = replaceAlias(cond, aliasMap)
 

spark git commit: [SPARK-22363][SQL][TEST] Add unit test for Window spilling

2017-12-30 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master ea0a5eef2 -> ee3af15fe


[SPARK-22363][SQL][TEST] Add unit test for Window spilling

## What changes were proposed in this pull request?

There is already test using window spilling, but the test coverage is not ideal.

In this PR the already existing test was fixed and additional cases added.

## How was this patch tested?

Automated: Pass the Jenkins.

Author: Gabor Somogyi 

Closes #20022 from gaborgsomogyi/SPARK-22363.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ee3af15f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ee3af15f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ee3af15f

Branch: refs/heads/master
Commit: ee3af15fea18356a9223d61cfe6aaa98ab4dc733
Parents: ea0a5ee
Author: Gabor Somogyi 
Authored: Sun Dec 31 14:47:23 2017 +0800
Committer: gatorsmile 
Committed: Sun Dec 31 14:47:23 2017 +0800

--
 .../sql/DataFrameWindowFunctionsSuite.scala | 44 +++-
 1 file changed, 42 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ee3af15f/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
index ea725af..01c988e 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql
 
 import java.sql.{Date, Timestamp}
 
+import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled}
 import org.apache.spark.sql.expressions.{MutableAggregationBuffer, 
UserDefinedAggregateFunction, Window}
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.internal.SQLConf
@@ -518,9 +519,46 @@ class DataFrameWindowFunctionsSuite extends QueryTest with 
SharedSQLContext {
   Seq(Row(3, "1", null, 3.0, 4.0, 3.0), Row(5, "1", false, 4.0, 5.0, 5.0)))
   }
 
+  test("Window spill with less than the inMemoryThreshold") {
+val df = Seq((1, "1"), (2, "2"), (1, "3"), (2, "4")).toDF("key", "value")
+val window = Window.partitionBy($"key").orderBy($"value")
+
+withSQLConf(SQLConf.WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "2",
+  SQLConf.WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "2") {
+  assertNotSpilled(sparkContext, "select") {
+df.select($"key", sum("value").over(window)).collect()
+  }
+}
+  }
+
+  test("Window spill with more than the inMemoryThreshold but less than the 
spillThreshold") {
+val df = Seq((1, "1"), (2, "2"), (1, "3"), (2, "4")).toDF("key", "value")
+val window = Window.partitionBy($"key").orderBy($"value")
+
+withSQLConf(SQLConf.WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "1",
+  SQLConf.WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "2") {
+  assertNotSpilled(sparkContext, "select") {
+df.select($"key", sum("value").over(window)).collect()
+  }
+}
+  }
+
+  test("Window spill with more than the inMemoryThreshold and spillThreshold") 
{
+val df = Seq((1, "1"), (2, "2"), (1, "3"), (2, "4")).toDF("key", "value")
+val window = Window.partitionBy($"key").orderBy($"value")
+
+withSQLConf(SQLConf.WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "1",
+  SQLConf.WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "1") {
+  assertSpilled(sparkContext, "select") {
+df.select($"key", sum("value").over(window)).collect()
+  }
+}
+  }
+
   test("SPARK-21258: complex object in combination with spilling") {
 // Make sure we trigger the spilling path.
-withSQLConf(SQLConf.WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "17") {
+withSQLConf(SQLConf.WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "1",
+  SQLConf.WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "17") {
   val sampleSchema = new StructType().
 add("f0", StringType).
 add("f1", LongType).
@@ -558,7 +596,9 @@ class DataFrameWindowFunctionsSuite extends QueryTest with 
SharedSQLContext {
 
   import testImplicits._
 
-  spark.read.schema(sampleSchema).json(input.toDS()).select(c0, 
c1).foreach { _ => () }
+  assertSpilled(sparkContext, "select") {
+spark.read.schema(sampleSchema).json(input.toDS()).select(c0, 
c1).foreach { _ => () }
+  }
 }
   }
 }


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



spark git commit: [TEST][MINOR] remove redundant `EliminateSubqueryAliases` in test code

2017-12-30 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 14c4a62c1 -> 234d9435d


[TEST][MINOR] remove redundant `EliminateSubqueryAliases` in test code

## What changes were proposed in this pull request?

The `analyze` method in `implicit class DslLogicalPlan` already includes 
`EliminateSubqueryAliases`. So there's no need to call 
`EliminateSubqueryAliases` again after calling `analyze` in some test code.

## How was this patch tested?

Existing tests.

Author: Zhenhua Wang 

Closes #20122 from wzhfy/redundant_code.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/234d9435
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/234d9435
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/234d9435

Branch: refs/heads/master
Commit: 234d9435d47ad9a9cfb0f349f1fd20062c25b8dd
Parents: 14c4a62
Author: Zhenhua Wang 
Authored: Sat Dec 30 20:48:39 2017 +0800
Committer: gatorsmile 
Committed: Sat Dec 30 20:48:39 2017 +0800

--
 .../sql/catalyst/optimizer/ColumnPruningSuite.scala   |  4 ++--
 .../sql/catalyst/optimizer/FilterPushdownSuite.scala  | 14 +++---
 .../catalyst/optimizer/JoinOptimizationSuite.scala|  3 +--
 3 files changed, 10 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/234d9435/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala
index 9f0f7e1..3f41f4b 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala
@@ -256,7 +256,7 @@ class ColumnPruningSuite extends PlanTest {
   x.select('a)
 .sortBy(SortOrder('a, Ascending)).analyze
 
-comparePlans(optimized, analysis.EliminateSubqueryAliases(correctAnswer))
+comparePlans(optimized, correctAnswer)
 
 // push down invalid
 val originalQuery1 = {
@@ -271,7 +271,7 @@ class ColumnPruningSuite extends PlanTest {
 .sortBy(SortOrder('a, Ascending))
 .select('b).analyze
 
-comparePlans(optimized1, analysis.EliminateSubqueryAliases(correctAnswer1))
+comparePlans(optimized1, correctAnswer1)
   }
 
   test("Column pruning on Window with useless aggregate functions") {

http://git-wip-us.apache.org/repos/asf/spark/blob/234d9435/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
index 4a23179..a9c2306 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
@@ -504,7 +504,7 @@ class FilterPushdownSuite extends PlanTest {
 }
 val optimized = Optimize.execute(originalQuery.analyze)
 
-comparePlans(analysis.EliminateSubqueryAliases(originalQuery.analyze), 
optimized)
+comparePlans(originalQuery.analyze, optimized)
   }
 
   test("joins: conjunctive predicates") {
@@ -523,7 +523,7 @@ class FilterPushdownSuite extends PlanTest {
   left.join(right, condition = Some("x.b".attr === "y.b".attr))
 .analyze
 
-comparePlans(optimized, analysis.EliminateSubqueryAliases(correctAnswer))
+comparePlans(optimized, correctAnswer)
   }
 
   test("joins: conjunctive predicates #2") {
@@ -542,7 +542,7 @@ class FilterPushdownSuite extends PlanTest {
   left.join(right, condition = Some("x.b".attr === "y.b".attr))
 .analyze
 
-comparePlans(optimized, analysis.EliminateSubqueryAliases(correctAnswer))
+comparePlans(optimized, correctAnswer)
   }
 
   test("joins: conjunctive predicates #3") {
@@ -566,7 +566,7 @@ class FilterPushdownSuite extends PlanTest {
   condition = Some("z.a".attr === "x.b".attr))
 .analyze
 
-comparePlans(optimized, analysis.EliminateSubqueryAliases(correctAnswer))
+comparePlans(optimized, correctAnswer)
   }
 
   test("joins: push down where clause into left anti join") {
@@ -581,7 +581,7 @@ class FilterPushdownSuite extends PlanTest {
   x.where("x.a".attr > 10)
 .join(y, LeftAnti, Some("x.b".attr === "y.b".attr))
 .analyze
-comparePlans(optimized, 

spark git commit: [SPARK-22771][SQL] Concatenate binary inputs into a binary output

2017-12-29 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 2ea17afb6 -> f2b3525c1


[SPARK-22771][SQL] Concatenate binary inputs into a binary output

## What changes were proposed in this pull request?
This pr modified `concat` to concat binary inputs into a single binary output.
`concat` in the current master always output data as a string. But, in some 
databases (e.g., PostgreSQL), if all inputs are binary, `concat` also outputs 
binary.

## How was this patch tested?
Added tests in `SQLQueryTestSuite` and `TypeCoercionSuite`.

Author: Takeshi Yamamuro 

Closes #19977 from maropu/SPARK-22771.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f2b3525c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f2b3525c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f2b3525c

Branch: refs/heads/master
Commit: f2b3525c17d660cf6f082bbafea8632615b4f58e
Parents: 2ea17af
Author: Takeshi Yamamuro 
Authored: Sat Dec 30 14:09:56 2017 +0800
Committer: gatorsmile 
Committed: Sat Dec 30 14:09:56 2017 +0800

--
 R/pkg/R/functions.R |   3 +-
 .../apache/spark/unsafe/types/ByteArray.java|  25 ++
 docs/sql-programming-guide.md   |   2 +
 python/pyspark/sql/functions.py |   3 +-
 .../spark/sql/catalyst/analysis/Analyzer.scala  |   2 +-
 .../sql/catalyst/analysis/TypeCoercion.scala|  26 +-
 .../expressions/stringExpressions.scala |  52 +++-
 .../sql/catalyst/optimizer/expressions.scala|  15 +-
 .../org/apache/spark/sql/internal/SQLConf.scala |   8 +
 .../catalyst/analysis/TypeCoercionSuite.scala   |  54 +
 .../optimizer/CombineConcatsSuite.scala |  14 +-
 .../scala/org/apache/spark/sql/functions.scala  |   3 +-
 .../sql-tests/inputs/string-functions.sql   |  23 ++
 .../inputs/typeCoercion/native/concat.sql   |  93 
 .../sql-tests/results/string-functions.sql.out  |  45 +++-
 .../results/typeCoercion/native/concat.sql.out  | 239 +++
 16 files changed, 587 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f2b3525c/R/pkg/R/functions.R
--
diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R
index fff230d..55365a4 100644
--- a/R/pkg/R/functions.R
+++ b/R/pkg/R/functions.R
@@ -2133,7 +2133,8 @@ setMethod("countDistinct",
   })
 
 #' @details
-#' \code{concat}: Concatenates multiple input string columns together into a 
single string column.
+#' \code{concat}: Concatenates multiple input columns together into a single 
column.
+#' If all inputs are binary, concat returns an output as binary. Otherwise, it 
returns as string.
 #'
 #' @rdname column_string_functions
 #' @aliases concat concat,Column-method

http://git-wip-us.apache.org/repos/asf/spark/blob/f2b3525c/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
--
diff --git 
a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java 
b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
index 7ced13d..c03caf0 100644
--- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
@@ -74,4 +74,29 @@ public final class ByteArray {
 }
 return Arrays.copyOfRange(bytes, start, end);
   }
+
+  public static byte[] concat(byte[]... inputs) {
+// Compute the total length of the result
+int totalLength = 0;
+for (int i = 0; i < inputs.length; i++) {
+  if (inputs[i] != null) {
+totalLength += inputs[i].length;
+  } else {
+return null;
+  }
+}
+
+// Allocate a new byte array, and copy the inputs one by one into it
+final byte[] result = new byte[totalLength];
+int offset = 0;
+for (int i = 0; i < inputs.length; i++) {
+  int len = inputs[i].length;
+  Platform.copyMemory(
+inputs[i], Platform.BYTE_ARRAY_OFFSET,
+result, Platform.BYTE_ARRAY_OFFSET + offset,
+len);
+  offset += len;
+}
+return result;
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f2b3525c/docs/sql-programming-guide.md
--
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index f02f462..4b5f56c 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1780,6 +1780,8 @@ options.
  
  - Since Spark 2.3, when either broadcast hash join or broadcast nested loop 
join is applicable, we prefer to broadcasting the table that is explicitly 
specified in a broadcast hint. 

spark git commit: [SPARK-22916][SQL] shouldn't bias towards build right if user does not specify

2017-12-29 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 224375c55 -> cc30ef800


[SPARK-22916][SQL] shouldn't bias towards build right if user does not specify

## What changes were proposed in this pull request?

When there are no broadcast hints, the current spark strategies will prefer to 
building the right side, without considering the sizes of the two tables. This 
patch added the logic to consider the sizes of the two tables for the build 
side. To make the logic clear, the build side is determined by two steps:

1. If there are broadcast hints, the build side is determined by 
`broadcastSideByHints`;
2. If there are no broadcast hints, the build side is determined by 
`broadcastSideBySizes`;
3. If the broadcast is disabled by the config, it falls back to the next cases.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, 
manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, 
remove this)

Please review http://spark.apache.org/contributing.html before opening a pull 
request.

Author: Feng Liu 

Closes #20099 from liufengdb/fix-spark-strategies.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cc30ef80
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cc30ef80
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cc30ef80

Branch: refs/heads/master
Commit: cc30ef8009b82c71a4b8e9caba82ed141761ab85
Parents: 224375c
Author: Feng Liu 
Authored: Fri Dec 29 18:48:47 2017 +0800
Committer: gatorsmile 
Committed: Fri Dec 29 18:48:47 2017 +0800

--
 .../spark/sql/execution/SparkStrategies.scala   | 75 
 .../execution/joins/BroadcastJoinSuite.scala| 75 +++-
 .../sql/execution/metric/SQLMetricsSuite.scala  | 15 ++--
 3 files changed, 116 insertions(+), 49 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cc30ef80/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 6b3f301..0ed7c2f 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -158,45 +158,65 @@ abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
   def smallerSide =
 if (right.stats.sizeInBytes <= left.stats.sizeInBytes) BuildRight else 
BuildLeft
 
-  val buildRight = canBuildRight && right.stats.hints.broadcast
-  val buildLeft = canBuildLeft && left.stats.hints.broadcast
-
-  if (buildRight && buildLeft) {
+  if (canBuildRight && canBuildLeft) {
 // Broadcast smaller side base on its estimated physical size
 // if both sides have broadcast hint
 smallerSide
-  } else if (buildRight) {
+  } else if (canBuildRight) {
 BuildRight
-  } else if (buildLeft) {
+  } else if (canBuildLeft) {
 BuildLeft
-  } else if (canBuildRight && canBuildLeft) {
+  } else {
 // for the last default broadcast nested loop join
 smallerSide
-  } else {
-throw new AnalysisException("Can not decide which side to broadcast 
for this join")
   }
 }
 
+private def canBroadcastByHints(joinType: JoinType, left: LogicalPlan, 
right: LogicalPlan)
+  : Boolean = {
+  val buildLeft = canBuildLeft(joinType) && left.stats.hints.broadcast
+  val buildRight = canBuildRight(joinType) && right.stats.hints.broadcast
+  buildLeft || buildRight
+}
+
+private def broadcastSideByHints(joinType: JoinType, left: LogicalPlan, 
right: LogicalPlan)
+  : BuildSide = {
+  val buildLeft = canBuildLeft(joinType) && left.stats.hints.broadcast
+  val buildRight = canBuildRight(joinType) && right.stats.hints.broadcast
+  broadcastSide(buildLeft, buildRight, left, right)
+}
+
+private def canBroadcastBySizes(joinType: JoinType, left: LogicalPlan, 
right: LogicalPlan)
+  : Boolean = {
+  val buildLeft = canBuildLeft(joinType) && canBroadcast(left)
+  val buildRight = canBuildRight(joinType) && canBroadcast(right)
+  buildLeft || buildRight
+}
+
+private def broadcastSideBySizes(joinType: JoinType, left: LogicalPlan, 
right: LogicalPlan)
+  : BuildSide = {
+  val buildLeft = canBuildLeft(joinType) && canBroadcast(left)
+  val buildRight = canBuildRight(joinType) && canBroadcast(right)
+  broadcastSide(buildLeft, buildRight, left, right)
+}
+
 def 

spark git commit: [SPARK-22891][SQL] Make hive client creation thread safe

2017-12-28 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 796e48c60 -> 67ea11ea0


[SPARK-22891][SQL] Make hive client creation thread safe

## What changes were proposed in this pull request?

This is to walk around the hive issue: 
https://issues.apache.org/jira/browse/HIVE-11935

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, 
manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, 
remove this)

Please review http://spark.apache.org/contributing.html before opening a pull 
request.

Author: Feng Liu 

Closes #20109 from liufengdb/synchronized.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/67ea11ea
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/67ea11ea
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/67ea11ea

Branch: refs/heads/master
Commit: 67ea11ea0fd83a86c2b7546c7cb16e6e50c88cc6
Parents: 796e48c
Author: Feng Liu 
Authored: Fri Dec 29 15:05:47 2017 +0800
Committer: gatorsmile 
Committed: Fri Dec 29 15:05:47 2017 +0800

--
 .../org/apache/spark/sql/hive/client/IsolatedClientLoader.scala| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/67ea11ea/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
index 930f0dd..7a76fd3 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
@@ -248,7 +248,7 @@ private[hive] class IsolatedClientLoader(
   }
 
   /** The isolated client interface to Hive. */
-  private[hive] def createClient(): HiveClient = {
+  private[hive] def createClient(): HiveClient = synchronized {
 if (!isolationOn) {
   return new HiveClientImpl(version, sparkConf, hadoopConf, config, 
baseClassLoader, this)
 }


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



spark git commit: [SPARK-22818][SQL] csv escape of quote escape

2017-12-28 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master cfcd74668 -> ffe6fd77a


[SPARK-22818][SQL] csv escape of quote escape

## What changes were proposed in this pull request?

Escape of escape should be considered when using the UniVocity csv 
encoding/decoding library.

Ref: 
https://github.com/uniVocity/univocity-parsers#escaping-quote-escape-characters

One option is added for reading and writing CSV: `escapeQuoteEscaping`

## How was this patch tested?

Unit test added.

Author: soonmok-kwon 

Closes #20004 from ep1804/SPARK-22818.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ffe6fd77
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ffe6fd77
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ffe6fd77

Branch: refs/heads/master
Commit: ffe6fd77a42043db8ebaf43d98059dcd28a53f1e
Parents: cfcd746
Author: soonmok-kwon 
Authored: Fri Dec 29 07:30:06 2017 +0800
Committer: gatorsmile 
Committed: Fri Dec 29 07:30:06 2017 +0800

--
 python/pyspark/sql/readwriter.py| 33 +---
 python/pyspark/sql/streaming.py | 17 ++
 .../org/apache/spark/sql/DataFrameReader.scala  | 11 ---
 .../org/apache/spark/sql/DataFrameWriter.scala  |  9 --
 .../execution/datasources/csv/CSVOptions.scala  | 10 ++
 .../spark/sql/streaming/DataStreamReader.scala  | 11 ---
 .../execution/datasources/csv/CSVSuite.scala| 31 ++
 7 files changed, 94 insertions(+), 28 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ffe6fd77/python/pyspark/sql/readwriter.py
--
diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py
index 4e58bfb..49af1bc 100644
--- a/python/pyspark/sql/readwriter.py
+++ b/python/pyspark/sql/readwriter.py
@@ -333,7 +333,7 @@ class DataFrameReader(OptionUtils):
 ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, 
positiveInf=None,
 negativeInf=None, dateFormat=None, timestampFormat=None, 
maxColumns=None,
 maxCharsPerColumn=None, maxMalformedLogPerPartition=None, 
mode=None,
-columnNameOfCorruptRecord=None, multiLine=None):
+columnNameOfCorruptRecord=None, multiLine=None, 
charToEscapeQuoteEscaping=None):
 """Loads a CSV file and returns the result as a  :class:`DataFrame`.
 
 This function will go through the input once to determine the input 
schema if
@@ -344,17 +344,17 @@ class DataFrameReader(OptionUtils):
  or RDD of Strings storing CSV rows.
 :param schema: an optional :class:`pyspark.sql.types.StructType` for 
the input schema
or a DDL-formatted string (For example ``col0 INT, col1 
DOUBLE``).
-:param sep: sets the single character as a separator for each field 
and value.
+:param sep: sets a single character as a separator for each field and 
value.
 If None is set, it uses the default value, ``,``.
 :param encoding: decodes the CSV files by the given encoding type. If 
None is set,
  it uses the default value, ``UTF-8``.
-:param quote: sets the single character used for escaping quoted 
values where the
+:param quote: sets a single character used for escaping quoted values 
where the
   separator can be part of the value. If None is set, it 
uses the default
   value, ``"``. If you would like to turn off quotations, 
you need to set an
   empty string.
-:param escape: sets the single character used for escaping quotes 
inside an already
+:param escape: sets a single character used for escaping quotes inside 
an already
quoted value. If None is set, it uses the default 
value, ``\``.
-:param comment: sets the single character used for skipping lines 
beginning with this
+:param comment: sets a single character used for skipping lines 
beginning with this
 character. By default (None), it is disabled.
 :param header: uses the first line as names of columns. If None is 
set, it uses the
default value, ``false``.
@@ -410,6 +410,10 @@ class DataFrameReader(OptionUtils):
   
``spark.sql.columnNameOfCorruptRecord``.
 :param multiLine: parse records, which may span multiple lines. If 
None is
   set, it uses the default value, ``false``.
+:param charToEscapeQuoteEscaping: sets a single character used for 
escaping the escape for
+  

spark git commit: [SPARK-22890][TEST] Basic tests for DateTimeOperations

2017-12-28 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 9c21ece35 -> 613b71a12


[SPARK-22890][TEST] Basic tests for DateTimeOperations

## What changes were proposed in this pull request?

Test Coverage for `DateTimeOperations`, this is a Sub-tasks for 
[SPARK-22722](https://issues.apache.org/jira/browse/SPARK-22722).

## How was this patch tested?

N/A

Author: Yuming Wang 

Closes #20061 from wangyum/SPARK-22890.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/613b71a1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/613b71a1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/613b71a1

Branch: refs/heads/master
Commit: 613b71a1230723ed556239b8b387722043c67252
Parents: 9c21ece
Author: Yuming Wang 
Authored: Fri Dec 29 06:58:38 2017 +0800
Committer: gatorsmile 
Committed: Fri Dec 29 06:58:38 2017 +0800

--
 .../typeCoercion/native/dateTimeOperations.sql  |  60 
 .../native/dateTimeOperations.sql.out   | 349 +++
 .../org/apache/spark/sql/SQLQuerySuite.scala|  13 +-
 3 files changed, 410 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/613b71a1/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/dateTimeOperations.sql
--
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/dateTimeOperations.sql
 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/dateTimeOperations.sql
new file mode 100644
index 000..1e98221
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/dateTimeOperations.sql
@@ -0,0 +1,60 @@
+--
+--   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.
+--
+
+CREATE TEMPORARY VIEW t AS SELECT 1;
+
+select cast(1 as tinyint) + interval 2 day;
+select cast(1 as smallint) + interval 2 day;
+select cast(1 as int) + interval 2 day;
+select cast(1 as bigint) + interval 2 day;
+select cast(1 as float) + interval 2 day;
+select cast(1 as double) + interval 2 day;
+select cast(1 as decimal(10, 0)) + interval 2 day;
+select cast('2017-12-11' as string) + interval 2 day;
+select cast('2017-12-11 09:30:00' as string) + interval 2 day;
+select cast('1' as binary) + interval 2 day;
+select cast(1 as boolean) + interval 2 day;
+select cast('2017-12-11 09:30:00.0' as timestamp) + interval 2 day;
+select cast('2017-12-11 09:30:00' as date) + interval 2 day;
+
+select interval 2 day + cast(1 as tinyint);
+select interval 2 day + cast(1 as smallint);
+select interval 2 day + cast(1 as int);
+select interval 2 day + cast(1 as bigint);
+select interval 2 day + cast(1 as float);
+select interval 2 day + cast(1 as double);
+select interval 2 day + cast(1 as decimal(10, 0));
+select interval 2 day + cast('2017-12-11' as string);
+select interval 2 day + cast('2017-12-11 09:30:00' as string);
+select interval 2 day + cast('1' as binary);
+select interval 2 day + cast(1 as boolean);
+select interval 2 day + cast('2017-12-11 09:30:00.0' as timestamp);
+select interval 2 day + cast('2017-12-11 09:30:00' as date);
+
+select cast(1 as tinyint) - interval 2 day;
+select cast(1 as smallint) - interval 2 day;
+select cast(1 as int) - interval 2 day;
+select cast(1 as bigint) - interval 2 day;
+select cast(1 as float) - interval 2 day;
+select cast(1 as double) - interval 2 day;
+select cast(1 as decimal(10, 0)) - interval 2 day;
+select cast('2017-12-11' as string) - interval 2 day;
+select cast('2017-12-11 09:30:00' as string) - interval 2 day;
+select cast('1' as binary) - interval 2 day;
+select cast(1 as boolean) - interval 2 day;
+select cast('2017-12-11 09:30:00.0' as timestamp) - interval 2 day;
+select cast('2017-12-11 09:30:00' as date) - interval 2 day;

http://git-wip-us.apache.org/repos/asf/spark/blob/613b71a1/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out

spark git commit: [SPARK-20392][SQL][FOLLOWUP] should not add extra AnalysisBarrier

2017-12-28 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 1eebfbe19 -> 755f2f518


[SPARK-20392][SQL][FOLLOWUP] should not add extra AnalysisBarrier

## What changes were proposed in this pull request?

I found this problem while auditing the analyzer code. It's dangerous to 
introduce extra `AnalysisBarrer` during analysis, as the plan inside it will 
bypass all analysis afterward, which may not be expected. We should only 
preserve `AnalysisBarrer` but not introduce new ones.

## How was this patch tested?

existing tests

Author: Wenchen Fan 

Closes #20094 from cloud-fan/barrier.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/755f2f51
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/755f2f51
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/755f2f51

Branch: refs/heads/master
Commit: 755f2f5189a08597fddc90b7f9df4ad0ec6bd2ef
Parents: 1eebfbe
Author: Wenchen Fan 
Authored: Thu Dec 28 21:33:03 2017 +0800
Committer: gatorsmile 
Committed: Thu Dec 28 21:33:03 2017 +0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 191 ---
 .../sql/hive/execution/SQLQuerySuite.scala  |   2 +-
 2 files changed, 84 insertions(+), 109 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/755f2f51/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 10b237f..7f2128e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -17,7 +17,6 @@
 
 package org.apache.spark.sql.catalyst.analysis
 
-import scala.annotation.tailrec
 import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.sql.AnalysisException
@@ -665,14 +664,18 @@ class Analyzer(
  * Generate a new logical plan for the right child with different 
expression IDs
  * for all conflicting attributes.
  */
-private def dedupRight (left: LogicalPlan, originalRight: LogicalPlan): 
LogicalPlan = {
-  // Remove analysis barrier if any.
-  val right = EliminateBarriers(originalRight)
+private def dedupRight (left: LogicalPlan, right: LogicalPlan): 
LogicalPlan = {
   val conflictingAttributes = left.outputSet.intersect(right.outputSet)
   logDebug(s"Conflicting attributes ${conflictingAttributes.mkString(",")} 
" +
 s"between $left and $right")
 
   right.collect {
+// For `AnalysisBarrier`, recursively de-duplicate its child.
+case oldVersion: AnalysisBarrier
+if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty 
=>
+  val newVersion = dedupRight(left, oldVersion.child)
+  (oldVersion, AnalysisBarrier(newVersion))
+
 // Handle base relations that might appear more than once.
 case oldVersion: MultiInstanceRelation
 if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty 
=>
@@ -710,10 +713,10 @@ class Analyzer(
* that this rule cannot handle. When that is the case, there must 
be another rule
* that resolves these conflicts. Otherwise, the analysis will fail.
*/
-  originalRight
+  right
 case Some((oldRelation, newRelation)) =>
   val attributeRewrites = 
AttributeMap(oldRelation.output.zip(newRelation.output))
-  val newRight = right transformUp {
+  right transformUp {
 case r if r == oldRelation => newRelation
   } transformUp {
 case other => other transformExpressions {
@@ -723,7 +726,6 @@ class Analyzer(
 s.withNewPlan(dedupOuterReferencesInSubquery(s.plan, 
attributeRewrites))
 }
   }
-  AnalysisBarrier(newRight)
   }
 }
 
@@ -958,7 +960,8 @@ class Analyzer(
   protected[sql] def resolveExpression(
   expr: Expression,
   plan: LogicalPlan,
-  throws: Boolean = false) = {
+  throws: Boolean = false): Expression = {
+if (expr.resolved) return expr
 // Resolve expression in one round.
 // If throws == false or the desired attribute doesn't exist
 // (like try to resolve `a.b` but `a` doesn't exist), fail and return the 
origin one.
@@ -1079,100 +1082,74 @@ class Analyzer(
   case sa @ Sort(_, _, AnalysisBarrier(child: Aggregate)) => sa
   case sa @ Sort(_, _, child: Aggregate) => sa
 
-  case s @ Sort(order, _, originalChild) if !s.resolved && 
originalChild.resolved =>
-   

spark git commit: [SPARK-22904][SQL] Add tests for decimal operations and string casts

2017-12-27 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master b8bfce51a -> 774715d5c


[SPARK-22904][SQL] Add tests for decimal operations and string casts

## What changes were proposed in this pull request?

Test coverage for arithmetic operations leading to:

 1. Precision loss
 2. Overflow

Moreover, tests for casting bad string to other input types and for using bad 
string as operators of some functions.

## How was this patch tested?

added tests

Author: Marco Gaido 

Closes #20084 from mgaido91/SPARK-22904.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/774715d5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/774715d5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/774715d5

Branch: refs/heads/master
Commit: 774715d5c73ab6d410208fa1675cd11166e03165
Parents: b8bfce5
Author: Marco Gaido 
Authored: Wed Dec 27 23:53:10 2017 +0800
Committer: gatorsmile 
Committed: Wed Dec 27 23:53:10 2017 +0800

--
 .../native/decimalArithmeticOperations.sql  |  33 +++
 .../native/stringCastAndExpressions.sql |  57 
 .../native/decimalArithmeticOperations.sql.out  |  82 ++
 .../native/stringCastAndExpressions.sql.out | 261 +++
 4 files changed, 433 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/774715d5/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalArithmeticOperations.sql
--
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalArithmeticOperations.sql
 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalArithmeticOperations.sql
new file mode 100644
index 000..c8e108a
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalArithmeticOperations.sql
@@ -0,0 +1,33 @@
+--
+--   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.
+--
+
+CREATE TEMPORARY VIEW t AS SELECT 1.0 as a, 0.0 as b;
+
+-- division, remainder and pmod by 0 return NULL
+select a / b from t;
+select a % b from t;
+select pmod(a, b) from t;
+
+-- arithmetic operations causing an overflow return NULL
+select (5e36 + 0.1) + 5e36;
+select (-4e36 - 0.1) - 7e36;
+select 12345678901234567890.0 * 12345678901234567890.0;
+select 1e35 / 0.1;
+
+-- arithmetic operations causing a precision loss return NULL
+select 123456789123456789.1234567890 * 1.123456789123456789;
+select 0.001 / 9876543210987654321098765432109876543.2

http://git-wip-us.apache.org/repos/asf/spark/blob/774715d5/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/stringCastAndExpressions.sql
--
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/stringCastAndExpressions.sql
 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/stringCastAndExpressions.sql
new file mode 100644
index 000..f17adb5
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/stringCastAndExpressions.sql
@@ -0,0 +1,57 @@
+--
+--   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.
+--
+
+CREATE TEMPORARY VIEW t AS 

spark git commit: [SPARK-22894][SQL] DateTimeOperations should accept SQL like string type

2017-12-26 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 9348e6842 -> 91d1b300d


[SPARK-22894][SQL] DateTimeOperations should accept SQL like string type

## What changes were proposed in this pull request?

`DateTimeOperations` accept 
[`StringType`](https://github.com/apache/spark/blob/ae998ec2b5548b7028d741da4813473dde1ad81e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala#L669),
  but:

```
spark-sql> SELECT '2017-12-24' + interval 2 months 2 seconds;
Error in query: cannot resolve '(CAST('2017-12-24' AS DOUBLE) + interval 2 
months 2 seconds)' due to data type mismatch: differing types in 
'(CAST('2017-12-24' AS DOUBLE) + interval 2 months 2 seconds)' (double and 
calendarinterval).; line 1 pos 7;
'Project [unresolvedalias((cast(2017-12-24 as double) + interval 2 months 2 
seconds), None)]
+- OneRowRelation
spark-sql>
```

After this PR:
```
spark-sql> SELECT '2017-12-24' + interval 2 months 2 seconds;
2018-02-24 00:00:02
Time taken: 0.2 seconds, Fetched 1 row(s)

```

## How was this patch tested?

unit tests

Author: Yuming Wang 

Closes #20067 from wangyum/SPARK-22894.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/91d1b300
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/91d1b300
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/91d1b300

Branch: refs/heads/master
Commit: 91d1b300d467cc91948f71e87b7afe1b9027e60f
Parents: 9348e68
Author: Yuming Wang 
Authored: Tue Dec 26 09:40:41 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 26 09:40:41 2017 -0800

--
 .../spark/sql/catalyst/analysis/TypeCoercion.scala |  6 --
 .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 13 -
 2 files changed, 16 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/91d1b300/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
index 2f306f5..1c4be54 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
@@ -324,9 +324,11 @@ object TypeCoercion {
   // Skip nodes who's children have not been resolved yet.
   case e if !e.childrenResolved => e
 
-  case a @ BinaryArithmetic(left @ StringType(), right) =>
+  case a @ BinaryArithmetic(left @ StringType(), right)
+if right.dataType != CalendarIntervalType =>
 a.makeCopy(Array(Cast(left, DoubleType), right))
-  case a @ BinaryArithmetic(left, right @ StringType()) =>
+  case a @ BinaryArithmetic(left, right @ StringType())
+if left.dataType != CalendarIntervalType =>
 a.makeCopy(Array(left, Cast(right, DoubleType)))
 
   // For equality between string and timestamp we cast the string to a 
timestamp

http://git-wip-us.apache.org/repos/asf/spark/blob/91d1b300/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
--
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 5e07728..1972dec 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql
 import java.io.File
 import java.math.MathContext
 import java.net.{MalformedURLException, URL}
-import java.sql.Timestamp
+import java.sql.{Date, Timestamp}
 import java.util.concurrent.atomic.AtomicBoolean
 
 import org.apache.spark.{AccumulatorSuite, SparkException}
@@ -2760,6 +2760,17 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
 }
   }
 
+  test("SPARK-22894: DateTimeOperations should accept SQL like string type") {
+val date = "2017-12-24"
+val str = sql(s"SELECT CAST('$date' as STRING) + interval 2 months 2 
seconds")
+val dt = sql(s"SELECT CAST('$date' as DATE) + interval 2 months 2 seconds")
+val ts = sql(s"SELECT CAST('$date' as TIMESTAMP) + interval 2 months 2 
seconds")
+
+checkAnswer(str, Row("2018-02-24 00:00:02") :: Nil)
+checkAnswer(dt, Row(Date.valueOf("2018-02-24")) :: Nil)
+checkAnswer(ts, Row(Timestamp.valueOf("2018-02-24 00:00:02")) :: Nil)
+  }
+
   // Only New OrcFileFormat supports this
   
Seq(classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat].getCanonicalName,
   

spark git commit: [SPARK-22833][EXAMPLE] Improvement SparkHive Scala Examples

2017-12-26 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master ff48b1b33 -> 9348e6842


[SPARK-22833][EXAMPLE] Improvement SparkHive Scala Examples

## What changes were proposed in this pull request?
Some improvements:
1. Point out we are using both Spark SQ native syntax and HQL syntax in the 
example
2. Avoid using the same table name with temp view, to not confuse users.
3. Create the external hive table with a directory that already has data, which 
is a more common use case.
4. Remove the usage of `spark.sql.parquet.writeLegacyFormat`. This config was 
introduced by https://github.com/apache/spark/pull/8566 and has nothing to do 
with Hive.
5. Remove `repartition` and `coalesce` example. These 2 are not Hive specific, 
we should put them in a different example file. BTW they can't accurately 
control the number of output files, `spark.sql.files.maxRecordsPerFile` also 
controls it.

## How was this patch tested?

N/A

Author: Wenchen Fan 

Closes #20081 from cloud-fan/minor.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9348e684
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9348e684
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9348e684

Branch: refs/heads/master
Commit: 9348e684208465a8f75c893bdeaa30fc42c0cb5f
Parents: ff48b1b
Author: Wenchen Fan 
Authored: Tue Dec 26 09:37:39 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 26 09:37:39 2017 -0800

--
 .../examples/sql/hive/SparkHiveExample.scala| 75 
 .../org/apache/spark/sql/internal/SQLConf.scala |  4 +-
 2 files changed, 46 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9348e684/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala
--
diff --git 
a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala
 
b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala
index b193bd5..70fb5b2 100644
--- 
a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala
+++ 
b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala
@@ -102,40 +102,53 @@ object SparkHiveExample {
 // |  5| val_5|  5| val_5|
 // ...
 
-// Create Hive managed table with Parquet
-sql("CREATE TABLE records(key int, value string) STORED AS PARQUET")
-// Save DataFrame to Hive managed table as Parquet format
-val hiveTableDF = sql("SELECT * FROM records")
-
hiveTableDF.write.mode(SaveMode.Overwrite).saveAsTable("database_name.records")
-// Create External Hive table with Parquet
-sql("CREATE EXTERNAL TABLE records(key int, value string) " +
-  "STORED AS PARQUET LOCATION '/user/hive/warehouse/'")
-// to make Hive Parquet format compatible with Spark Parquet format
-spark.sqlContext.setConf("spark.sql.parquet.writeLegacyFormat", "true")
-
-// Multiple Parquet files could be created accordingly to volume of data 
under directory given.
-val hiveExternalTableLocation = 
"/user/hive/warehouse/database_name.db/records"
-
-// Save DataFrame to Hive External table as compatible Parquet format
-
hiveTableDF.write.mode(SaveMode.Overwrite).parquet(hiveExternalTableLocation)
-
-// Turn on flag for Dynamic Partitioning
-spark.sqlContext.setConf("hive.exec.dynamic.partition", "true")
-spark.sqlContext.setConf("hive.exec.dynamic.partition.mode", "nonstrict")
-
-// You can create partitions in Hive table, so downstream queries run much 
faster.
-hiveTableDF.write.mode(SaveMode.Overwrite).partitionBy("key")
-  .parquet(hiveExternalTableLocation)
+// Create a Hive managed Parquet table, with HQL syntax instead of the 
Spark SQL native syntax
+// `USING hive`
+sql("CREATE TABLE hive_records(key int, value string) STORED AS PARQUET")
+// Save DataFrame to the Hive managed table
+val df = spark.table("src")
+df.write.mode(SaveMode.Overwrite).saveAsTable("hive_records")
+// After insertion, the Hive managed table has data now
+sql("SELECT * FROM hive_records").show()
+// +---+---+
+// |key|  value|
+// +---+---+
+// |238|val_238|
+// | 86| val_86|
+// |311|val_311|
+// ...
 
-// Reduce number of files for each partition by repartition
-hiveTableDF.repartition($"key").write.mode(SaveMode.Overwrite)
-  .partitionBy("key").parquet(hiveExternalTableLocation)
+// Prepare a Parquet data directory
+val dataDir = "/tmp/parquet_data"
+spark.range(10).write.parquet(dataDir)
+// Create a Hive external Parquet table
+sql(s"CREATE EXTERNAL TABLE hive_ints(key int) STORED AS PARQUET 

spark git commit: [SPARK-22893][SQL][HOTFIX] Fix a error message of VersionsSuite

2017-12-25 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 12d20dd75 -> be03d3ad7


[SPARK-22893][SQL][HOTFIX] Fix a error message of VersionsSuite

## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/20064 breaks Jenkins tests because it 
missed to update one error message for Hive 0.12 and Hive 0.13. This PR fixes 
that.
- 
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/3924/
- 
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/3977/
- 
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/4226/
- 
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.6/4260/

## How was this patch tested?

Pass the Jenkins without failure.

Author: Dongjoon Hyun 

Closes #20079 from dongjoon-hyun/SPARK-22893.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/be03d3ad
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/be03d3ad
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/be03d3ad

Branch: refs/heads/master
Commit: be03d3ad793dfe8f3ec33074d4ae95f5adb86ee4
Parents: 12d20dd
Author: Dongjoon Hyun 
Authored: Mon Dec 25 16:17:39 2017 -0800
Committer: gatorsmile 
Committed: Mon Dec 25 16:17:39 2017 -0800

--
 .../scala/org/apache/spark/sql/hive/client/VersionsSuite.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/be03d3ad/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
index 9d15dab..94473a0 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
@@ -773,7 +773,7 @@ class VersionsSuite extends SparkFunSuite with Logging {
""".stripMargin
   )
 
-  val errorMsg = "data type mismatch: cannot cast DecimalType(2,1) to 
BinaryType"
+  val errorMsg = "data type mismatch: cannot cast decimal(2,1) to 
binary"
 
   if (isPartitioned) {
 val insertStmt = s"INSERT OVERWRITE TABLE $tableName partition 
(ds='a') SELECT 1.3"


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



[2/2] spark git commit: [SPARK-22893][SQL] Unified the data type mismatch message

2017-12-25 Thread lixiao
[SPARK-22893][SQL] Unified the data type mismatch message

## What changes were proposed in this pull request?

We should use `dataType.simpleString` to unified the data type mismatch message:
Before:
```
spark-sql> select cast(1 as binary);
Error in query: cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: 
cannot cast IntegerType to BinaryType; line 1 pos 7;
```
After:
```
park-sql> select cast(1 as binary);
Error in query: cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: 
cannot cast int to binary; line 1 pos 7;
```

## How was this patch tested?

Exist test.

Author: Yuming Wang 

Closes #20064 from wangyum/SPARK-22893.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/33ae2437
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/33ae2437
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/33ae2437

Branch: refs/heads/master
Commit: 33ae2437ba4e634b510b0f96e914ad1ef4ccafd8
Parents: fba0313
Author: Yuming Wang 
Authored: Mon Dec 25 01:14:09 2017 -0800
Committer: gatorsmile 
Committed: Mon Dec 25 01:14:09 2017 -0800

--
 .../spark/sql/catalyst/expressions/Cast.scala   |   2 +-
 .../aggregate/ApproximatePercentile.scala   |   4 +-
 .../expressions/conditionalExpressions.scala|   3 +-
 .../sql/catalyst/expressions/generators.scala   |  10 +-
 .../sql/catalyst/expressions/predicates.scala   |   2 +-
 .../expressions/windowExpressions.scala |   8 +-
 .../native/binaryComparison.sql.out |  48 ++--
 .../typeCoercion/native/inConversion.sql.out| 280 +--
 .../native/windowFrameCoercion.sql.out  |   8 +-
 .../resources/sql-tests/results/window.sql.out  |   4 +-
 .../org/apache/spark/sql/DatasetSuite.scala |   2 +-
 .../spark/sql/GeneratorFunctionSuite.scala  |   4 +-
 12 files changed, 189 insertions(+), 186 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/33ae2437/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index 5279d41..274d881 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -181,7 +181,7 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
   TypeCheckResult.TypeCheckSuccess
 } else {
   TypeCheckResult.TypeCheckFailure(
-s"cannot cast ${child.dataType} to $dataType")
+s"cannot cast ${child.dataType.simpleString} to 
${dataType.simpleString}")
 }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/33ae2437/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
index 7facb9d..149ac26 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
@@ -132,7 +132,7 @@ case class ApproximatePercentile(
 case TimestampType => value.asInstanceOf[Long].toDouble
 case n: NumericType => 
n.numeric.toDouble(value.asInstanceOf[n.InternalType])
 case other: DataType =>
-  throw new UnsupportedOperationException(s"Unexpected data type 
$other")
+  throw new UnsupportedOperationException(s"Unexpected data type 
${other.simpleString}")
   }
   buffer.add(doubleValue)
 }
@@ -157,7 +157,7 @@ case class ApproximatePercentile(
   case DoubleType => doubleResult
   case _: DecimalType => doubleResult.map(Decimal(_))
   case other: DataType =>
-throw new UnsupportedOperationException(s"Unexpected data type $other")
+throw new UnsupportedOperationException(s"Unexpected data type 
${other.simpleString}")
 }
 if (result.length == 0) {
   null

http://git-wip-us.apache.org/repos/asf/spark/blob/33ae2437/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala
--
diff --git 

[1/2] spark git commit: [SPARK-22893][SQL] Unified the data type mismatch message

2017-12-25 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master fba03133d -> 33ae2437b


http://git-wip-us.apache.org/repos/asf/spark/blob/33ae2437/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out
--
diff --git 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out
index bf8ddee..875ccc1 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out
@@ -80,7 +80,7 @@ SELECT cast(1 as tinyint) in (cast('1' as binary)) FROM t
 struct<>
 -- !query 9 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST(1 AS TINYINT) IN (CAST('1' AS BINARY)))' due to data 
type mismatch: Arguments must be same type but were: ByteType != BinaryType; 
line 1 pos 26
+cannot resolve '(CAST(1 AS TINYINT) IN (CAST('1' AS BINARY)))' due to data 
type mismatch: Arguments must be same type but were: tinyint != binary; line 1 
pos 26
 
 
 -- !query 10
@@ -89,7 +89,7 @@ SELECT cast(1 as tinyint) in (cast(1 as boolean)) FROM t
 struct<>
 -- !query 10 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST(1 AS TINYINT) IN (CAST(1 AS BOOLEAN)))' due to data type 
mismatch: Arguments must be same type but were: ByteType != BooleanType; line 1 
pos 26
+cannot resolve '(CAST(1 AS TINYINT) IN (CAST(1 AS BOOLEAN)))' due to data type 
mismatch: Arguments must be same type but were: tinyint != boolean; line 1 pos 
26
 
 
 -- !query 11
@@ -98,7 +98,7 @@ SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00.0' as 
timestamp)) FROM t
 struct<>
 -- !query 11 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST(1 AS TINYINT) IN (CAST('2017-12-11 09:30:00.0' AS 
TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: 
ByteType != TimestampType; line 1 pos 26
+cannot resolve '(CAST(1 AS TINYINT) IN (CAST('2017-12-11 09:30:00.0' AS 
TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: 
tinyint != timestamp; line 1 pos 26
 
 
 -- !query 12
@@ -107,7 +107,7 @@ SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00' as 
date)) FROM t
 struct<>
 -- !query 12 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST(1 AS TINYINT) IN (CAST('2017-12-11 09:30:00' AS DATE)))' 
due to data type mismatch: Arguments must be same type but were: ByteType != 
DateType; line 1 pos 26
+cannot resolve '(CAST(1 AS TINYINT) IN (CAST('2017-12-11 09:30:00' AS DATE)))' 
due to data type mismatch: Arguments must be same type but were: tinyint != 
date; line 1 pos 26
 
 
 -- !query 13
@@ -180,7 +180,7 @@ SELECT cast(1 as smallint) in (cast('1' as binary)) FROM t
 struct<>
 -- !query 21 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST(1 AS SMALLINT) IN (CAST('1' AS BINARY)))' due to data 
type mismatch: Arguments must be same type but were: ShortType != BinaryType; 
line 1 pos 27
+cannot resolve '(CAST(1 AS SMALLINT) IN (CAST('1' AS BINARY)))' due to data 
type mismatch: Arguments must be same type but were: smallint != binary; line 1 
pos 27
 
 
 -- !query 22
@@ -189,7 +189,7 @@ SELECT cast(1 as smallint) in (cast(1 as boolean)) FROM t
 struct<>
 -- !query 22 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST(1 AS SMALLINT) IN (CAST(1 AS BOOLEAN)))' due to data 
type mismatch: Arguments must be same type but were: ShortType != BooleanType; 
line 1 pos 27
+cannot resolve '(CAST(1 AS SMALLINT) IN (CAST(1 AS BOOLEAN)))' due to data 
type mismatch: Arguments must be same type but were: smallint != boolean; line 
1 pos 27
 
 
 -- !query 23
@@ -198,7 +198,7 @@ SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00.0' 
as timestamp)) FROM
 struct<>
 -- !query 23 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST(1 AS SMALLINT) IN (CAST('2017-12-11 09:30:00.0' AS 
TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: 
ShortType != TimestampType; line 1 pos 27
+cannot resolve '(CAST(1 AS SMALLINT) IN (CAST('2017-12-11 09:30:00.0' AS 
TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: 
smallint != timestamp; line 1 pos 27
 
 
 -- !query 24
@@ -207,7 +207,7 @@ SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00' 
as date)) FROM t
 struct<>
 -- !query 24 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST(1 AS SMALLINT) IN (CAST('2017-12-11 09:30:00' AS 
DATE)))' due to data type mismatch: Arguments must be same type but were: 
ShortType != DateType; line 1 pos 27
+cannot resolve '(CAST(1 AS SMALLINT) IN (CAST('2017-12-11 09:30:00' AS 
DATE)))' due to data type mismatch: Arguments must be same type but were: 
smallint != date; line 1 pos 27
 
 
 -- !query 25
@@ -280,7 +280,7 @@ SELECT cast(1 as 

spark git commit: [SPARK-22862] Docs on lazy elimination of columns missing from an encoder

2017-12-21 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.1 0f6862106 -> 1df8020e1


[SPARK-22862] Docs on lazy elimination of columns missing from an encoder

This behavior has confused some users, so lets clarify it.

Author: Michael Armbrust 

Closes #20048 from marmbrus/datasetAsDocs.

(cherry picked from commit 8df1da396f64bb7fe76d73cd01498fdf3b8ed964)
Signed-off-by: gatorsmile 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1df8020e
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1df8020e
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1df8020e

Branch: refs/heads/branch-2.1
Commit: 1df8020e1e1bd136b2374471584e110c4bc0d948
Parents: 0f68621
Author: Michael Armbrust 
Authored: Thu Dec 21 21:38:16 2017 -0800
Committer: gatorsmile 
Committed: Thu Dec 21 21:38:56 2017 -0800

--
 sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 
 1 file changed, 4 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1df8020e/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index ac34e0b..ecced7c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -368,6 +368,10 @@ class Dataset[T] private[sql](
* If the schema of the Dataset does not match the desired `U` type, you can 
use `select`
* along with `alias` or `as` to rearrange or rename as required.
*
+   * Note that `as[]` only changes the view of the data that is passed into 
typed operations,
+   * such as `map()`, and does not eagerly project away any columns that are 
not present in
+   * the specified class.
+   *
* @group basic
* @since 1.6.0
*/


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



spark git commit: [SPARK-22862] Docs on lazy elimination of columns missing from an encoder

2017-12-21 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.2 1e4cca02f -> 1cf3e3a26


[SPARK-22862] Docs on lazy elimination of columns missing from an encoder

This behavior has confused some users, so lets clarify it.

Author: Michael Armbrust 

Closes #20048 from marmbrus/datasetAsDocs.

(cherry picked from commit 8df1da396f64bb7fe76d73cd01498fdf3b8ed964)
Signed-off-by: gatorsmile 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1cf3e3a2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1cf3e3a2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1cf3e3a2

Branch: refs/heads/branch-2.2
Commit: 1cf3e3a26961d306eb17b7629d8742a4df45f339
Parents: 1e4cca0
Author: Michael Armbrust 
Authored: Thu Dec 21 21:38:16 2017 -0800
Committer: gatorsmile 
Committed: Thu Dec 21 21:38:34 2017 -0800

--
 sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 
 1 file changed, 4 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1cf3e3a2/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 1acbad9..fb700a4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -368,6 +368,10 @@ class Dataset[T] private[sql](
* If the schema of the Dataset does not match the desired `U` type, you can 
use `select`
* along with `alias` or `as` to rearrange or rename as required.
*
+   * Note that `as[]` only changes the view of the data that is passed into 
typed operations,
+   * such as `map()`, and does not eagerly project away any columns that are 
not present in
+   * the specified class.
+   *
* @group basic
* @since 1.6.0
*/


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



spark git commit: [SPARK-22862] Docs on lazy elimination of columns missing from an encoder

2017-12-21 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 22e1849bc -> 8df1da396


[SPARK-22862] Docs on lazy elimination of columns missing from an encoder

This behavior has confused some users, so lets clarify it.

Author: Michael Armbrust 

Closes #20048 from marmbrus/datasetAsDocs.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8df1da39
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8df1da39
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8df1da39

Branch: refs/heads/master
Commit: 8df1da396f64bb7fe76d73cd01498fdf3b8ed964
Parents: 22e1849
Author: Michael Armbrust 
Authored: Thu Dec 21 21:38:16 2017 -0800
Committer: gatorsmile 
Committed: Thu Dec 21 21:38:16 2017 -0800

--
 sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 
 1 file changed, 4 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8df1da39/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index ef00562..209b800 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -401,6 +401,10 @@ class Dataset[T] private[sql](
* If the schema of the Dataset does not match the desired `U` type, you can 
use `select`
* along with `alias` or `as` to rearrange or rename as required.
*
+   * Note that `as[]` only changes the view of the data that is passed into 
typed operations,
+   * such as `map()`, and does not eagerly project away any columns that are 
not present in
+   * the specified class.
+   *
* @group basic
* @since 1.6.0
*/


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



spark git commit: [SPARK-22042][FOLLOW-UP][SQL] ReorderJoinPredicates can break when child's partitioning is not decided

2017-12-21 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 4e107fdb7 -> fe65361b0


[SPARK-22042][FOLLOW-UP][SQL] ReorderJoinPredicates can break when child's 
partitioning is not decided

## What changes were proposed in this pull request?

This is a followup PR of https://github.com/apache/spark/pull/19257 where 
gatorsmile had left couple comments wrt code style.

## How was this patch tested?

Doesn't change any functionality. Will depend on build to see if no checkstyle 
rules are violated.

Author: Tejas Patil 

Closes #20041 from tejasapatil/followup_19257.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fe65361b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fe65361b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fe65361b

Branch: refs/heads/master
Commit: fe65361b0579777c360dee1d7f633f28df0c6aeb
Parents: 4e107fd
Author: Tejas Patil 
Authored: Thu Dec 21 09:22:08 2017 -0800
Committer: gatorsmile 
Committed: Thu Dec 21 09:22:08 2017 -0800

--
 .../execution/exchange/EnsureRequirements.scala | 82 ++--
 .../spark/sql/sources/BucketedReadSuite.scala   |  4 +-
 2 files changed, 44 insertions(+), 42 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/fe65361b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
index 82f0b9f..c8e236b 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
@@ -252,54 +252,56 @@ case class EnsureRequirements(conf: SQLConf) extends 
Rule[SparkPlan] {
 operator.withNewChildren(children)
   }
 
-  /**
-   * When the physical operators are created for JOIN, the ordering of join 
keys is based on order
-   * in which the join keys appear in the user query. That might not match 
with the output
-   * partitioning of the join node's children (thus leading to extra sort / 
shuffle being
-   * introduced). This rule will change the ordering of the join keys to match 
with the
-   * partitioning of the join nodes' children.
-   */
-  def reorderJoinPredicates(plan: SparkPlan): SparkPlan = {
-def reorderJoinKeys(
-leftKeys: Seq[Expression],
-rightKeys: Seq[Expression],
-leftPartitioning: Partitioning,
-rightPartitioning: Partitioning): (Seq[Expression], Seq[Expression]) = 
{
-
-  def reorder(expectedOrderOfKeys: Seq[Expression],
-  currentOrderOfKeys: Seq[Expression]): (Seq[Expression], 
Seq[Expression]) = {
-val leftKeysBuffer = ArrayBuffer[Expression]()
-val rightKeysBuffer = ArrayBuffer[Expression]()
+  private def reorder(
+  leftKeys: Seq[Expression],
+  rightKeys: Seq[Expression],
+  expectedOrderOfKeys: Seq[Expression],
+  currentOrderOfKeys: Seq[Expression]): (Seq[Expression], Seq[Expression]) 
= {
+val leftKeysBuffer = ArrayBuffer[Expression]()
+val rightKeysBuffer = ArrayBuffer[Expression]()
 
-expectedOrderOfKeys.foreach(expression => {
-  val index = currentOrderOfKeys.indexWhere(e => 
e.semanticEquals(expression))
-  leftKeysBuffer.append(leftKeys(index))
-  rightKeysBuffer.append(rightKeys(index))
-})
-(leftKeysBuffer, rightKeysBuffer)
-  }
+expectedOrderOfKeys.foreach(expression => {
+  val index = currentOrderOfKeys.indexWhere(e => 
e.semanticEquals(expression))
+  leftKeysBuffer.append(leftKeys(index))
+  rightKeysBuffer.append(rightKeys(index))
+})
+(leftKeysBuffer, rightKeysBuffer)
+  }
 
-  if (leftKeys.forall(_.deterministic) && 
rightKeys.forall(_.deterministic)) {
-leftPartitioning match {
-  case HashPartitioning(leftExpressions, _)
-if leftExpressions.length == leftKeys.length &&
-  leftKeys.forall(x => 
leftExpressions.exists(_.semanticEquals(x))) =>
-reorder(leftExpressions, leftKeys)
+  private def reorderJoinKeys(
+  leftKeys: Seq[Expression],
+  rightKeys: Seq[Expression],
+  leftPartitioning: Partitioning,
+  rightPartitioning: Partitioning): (Seq[Expression], Seq[Expression]) = {
+if (leftKeys.forall(_.deterministic) && rightKeys.forall(_.deterministic)) 
{
+  leftPartitioning match {
+case HashPartitioning(leftExpressions, _)
+  if leftExpressions.length == leftKeys.length &&
+leftKeys.forall(x => 

[3/3] spark git commit: [SPARK-22822][TEST] Basic tests for WindowFrameCoercion and DecimalPrecision

2017-12-21 Thread lixiao
[SPARK-22822][TEST] Basic tests for WindowFrameCoercion and DecimalPrecision

## What changes were proposed in this pull request?

Test Coverage for `WindowFrameCoercion` and `DecimalPrecision`, this is a 
Sub-tasks for [SPARK-22722](https://issues.apache.org/jira/browse/SPARK-22722).

## How was this patch tested?

N/A

Author: Yuming Wang 

Closes #20008 from wangyum/SPARK-22822.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4e107fdb
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4e107fdb
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4e107fdb

Branch: refs/heads/master
Commit: 4e107fdb7463a67d9c77c4a3434dfe70c72982f4
Parents: d3a1d95
Author: Yuming Wang 
Authored: Thu Dec 21 09:18:27 2017 -0800
Committer: gatorsmile 
Committed: Thu Dec 21 09:18:27 2017 -0800

--
 .../expressions/windowExpressions.scala |4 +-
 .../typeCoercion/native/decimalPrecision.sql| 1448 +++
 .../typeCoercion/native/windowFrameCoercion.sql |   44 +
 .../native/decimalPrecision.sql.out | 9514 ++
 .../native/windowFrameCoercion.sql.out  |  206 +
 .../resources/sql-tests/results/window.sql.out  |2 +-
 6 files changed, 11215 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4e107fdb/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
index e11e3a1..220cc4f 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
@@ -251,8 +251,8 @@ case class SpecifiedWindowFrame(
   TypeCheckFailure(s"Window frame $location bound '$e' is not a literal.")
 case e: Expression if !frameType.inputType.acceptsType(e.dataType) =>
   TypeCheckFailure(
-s"The data type of the $location bound '${e.dataType} does not match " 
+
-  s"the expected data type '${frameType.inputType}'.")
+s"The data type of the $location bound '${e.dataType}' does not match 
" +
+  s"the expected data type '${frameType.inputType.simpleString}'.")
 case _ => TypeCheckSuccess
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/4e107fdb/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalPrecision.sql
--
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalPrecision.sql
 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalPrecision.sql
new file mode 100644
index 000..8b04864
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalPrecision.sql
@@ -0,0 +1,1448 @@
+--
+--   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.
+--
+
+CREATE TEMPORARY VIEW t AS SELECT 1;
+
+SELECT cast(1 as tinyint) + cast(1 as decimal(3, 0)) FROM t;
+SELECT cast(1 as tinyint) + cast(1 as decimal(5, 0)) FROM t;
+SELECT cast(1 as tinyint) + cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as tinyint) + cast(1 as decimal(20, 0)) FROM t;
+
+SELECT cast(1 as smallint) + cast(1 as decimal(3, 0)) FROM t;
+SELECT cast(1 as smallint) + cast(1 as decimal(5, 0)) FROM t;
+SELECT cast(1 as smallint) + cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as smallint) + cast(1 as decimal(20, 0)) FROM t;
+
+SELECT cast(1 as int) + cast(1 as decimal(3, 0)) FROM t;
+SELECT cast(1 as int) + cast(1 as decimal(5, 0)) FROM t;
+SELECT cast(1 as int) + cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as int) + cast(1 as decimal(20, 0)) FROM t;
+
+SELECT cast(1 as bigint) + cast(1 as decimal(3, 0)) FROM t;

[2/3] spark git commit: [SPARK-22822][TEST] Basic tests for WindowFrameCoercion and DecimalPrecision

2017-12-21 Thread lixiao
http://git-wip-us.apache.org/repos/asf/spark/blob/4e107fdb/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out
--
diff --git 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out
new file mode 100644
index 000..ebc8201
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out
@@ -0,0 +1,9514 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 1145
+
+
+-- !query 0
+CREATE TEMPORARY VIEW t AS SELECT 1
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+SELECT cast(1 as tinyint) + cast(1 as decimal(3, 0)) FROM t
+-- !query 1 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) + CAST(1 AS 
DECIMAL(3,0))):decimal(4,0)>
+-- !query 1 output
+2
+
+
+-- !query 2
+SELECT cast(1 as tinyint) + cast(1 as decimal(5, 0)) FROM t
+-- !query 2 schema
+struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(6,0)) + 
CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(6,0))):decimal(6,0)>
+-- !query 2 output
+2
+
+
+-- !query 3
+SELECT cast(1 as tinyint) + cast(1 as decimal(10, 0)) FROM t
+-- !query 3 schema
+struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(11,0)) + 
CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)>
+-- !query 3 output
+2
+
+
+-- !query 4
+SELECT cast(1 as tinyint) + cast(1 as decimal(20, 0)) FROM t
+-- !query 4 schema
+struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(21,0)) + 
CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)>
+-- !query 4 output
+2
+
+
+-- !query 5
+SELECT cast(1 as smallint) + cast(1 as decimal(3, 0)) FROM t
+-- !query 5 schema
+struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(6,0)) + 
CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(6,0))):decimal(6,0)>
+-- !query 5 output
+2
+
+
+-- !query 6
+SELECT cast(1 as smallint) + cast(1 as decimal(5, 0)) FROM t
+-- !query 6 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) + CAST(1 AS 
DECIMAL(5,0))):decimal(6,0)>
+-- !query 6 output
+2
+
+
+-- !query 7
+SELECT cast(1 as smallint) + cast(1 as decimal(10, 0)) FROM t
+-- !query 7 schema
+struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(11,0)) + 
CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)>
+-- !query 7 output
+2
+
+
+-- !query 8
+SELECT cast(1 as smallint) + cast(1 as decimal(20, 0)) FROM t
+-- !query 8 schema
+struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(21,0)) + 
CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)>
+-- !query 8 output
+2
+
+
+-- !query 9
+SELECT cast(1 as int) + cast(1 as decimal(3, 0)) FROM t
+-- !query 9 schema
+struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0)) + 
CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)>
+-- !query 9 output
+2
+
+
+-- !query 10
+SELECT cast(1 as int) + cast(1 as decimal(5, 0)) FROM t
+-- !query 10 schema
+struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0)) + 
CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)>
+-- !query 10 output
+2
+
+
+-- !query 11
+SELECT cast(1 as int) + cast(1 as decimal(10, 0)) FROM t
+-- !query 11 schema
+struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) + CAST(1 AS 
DECIMAL(10,0))):decimal(11,0)>
+-- !query 11 output
+2
+
+
+-- !query 12
+SELECT cast(1 as int) + cast(1 as decimal(20, 0)) FROM t
+-- !query 12 schema
+struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(21,0)) + 
CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)>
+-- !query 12 output
+2
+
+
+-- !query 13
+SELECT cast(1 as bigint) + cast(1 as decimal(3, 0)) FROM t
+-- !query 13 schema
+struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) + 
CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(21,0))):decimal(21,0)>
+-- !query 13 output
+2
+
+
+-- !query 14
+SELECT cast(1 as bigint) + cast(1 as decimal(5, 0)) FROM t
+-- !query 14 schema
+struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) + 
CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(21,0))):decimal(21,0)>
+-- !query 14 output
+2
+
+
+-- !query 15
+SELECT cast(1 as bigint) + cast(1 as decimal(10, 0)) FROM t
+-- !query 15 schema
+struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) + 
CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)>
+-- !query 15 output
+2
+
+
+-- !query 16
+SELECT cast(1 as bigint) + cast(1 as decimal(20, 0)) FROM t
+-- !query 16 schema
+struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) + CAST(1 AS 
DECIMAL(20,0))):decimal(21,0)>
+-- !query 16 output
+2
+
+
+-- !query 17
+SELECT cast(1 as float) + cast(1 as decimal(3, 0)) FROM t
+-- !query 17 schema
+struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) + CAST(CAST(1 AS DECIMAL(3,0)) AS 
DOUBLE)):double>
+-- !query 17 output
+2.0
+
+
+-- !query 18

[1/3] spark git commit: [SPARK-22822][TEST] Basic tests for WindowFrameCoercion and DecimalPrecision

2017-12-21 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master d3a1d9527 -> 4e107fdb7


http://git-wip-us.apache.org/repos/asf/spark/blob/4e107fdb/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out
--
diff --git 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out
new file mode 100644
index 000..5dd257b
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out
@@ -0,0 +1,206 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 25
+
+
+-- !query 0
+CREATE TEMPORARY VIEW t AS SELECT 1
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint)) FROM t
+-- !query 1 schema
+struct
+-- !query 1 output
+1
+
+
+-- !query 2
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint)) FROM t
+-- !query 2 schema
+struct
+-- !query 2 output
+1
+
+
+-- !query 3
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int)) FROM t
+-- !query 3 schema
+struct
+-- !query 3 output
+1
+
+
+-- !query 4
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint)) FROM t
+-- !query 4 schema
+struct
+-- !query 4 output
+1
+
+
+-- !query 5
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float)) FROM t
+-- !query 5 schema
+struct
+-- !query 5 output
+1
+
+
+-- !query 6
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double)) FROM t
+-- !query 6 schema
+struct
+-- !query 6 output
+1
+
+
+-- !query 7
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0))) FROM t
+-- !query 7 schema
+struct
+-- !query 7 output
+1
+
+
+-- !query 8
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string)) FROM t
+-- !query 8 schema
+struct
+-- !query 8 output
+1
+
+
+-- !query 9
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary)) FROM t
+-- !query 9 schema
+struct
+-- !query 9 output
+1
+
+
+-- !query 10
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean)) FROM t
+-- !query 10 schema
+struct
+-- !query 10 output
+1
+
+
+-- !query 11
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as 
timestamp)) FROM t
+-- !query 11 schema
+struct
+-- !query 11 output
+1
+
+
+-- !query 12
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as 
date)) FROM t
+-- !query 12 schema
+struct
+-- !query 12 output
+1
+
+
+-- !query 13
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint) DESC RANGE 
BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t
+-- !query 13 schema
+struct
+-- !query 13 output
+1
+
+
+-- !query 14
+SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint) DESC RANGE 
BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t
+-- !query 14 schema
+struct
+-- !query 14 output
+1
+
+
+-- !query 15
+SELECT COUNT(*) OVER (PARTITION BY 1 

spark git commit: [SPARK-22849] ivy.retrieve pattern should also consider `classifier`

2017-12-20 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master d762d110d -> c89b43118


[SPARK-22849] ivy.retrieve pattern should also consider `classifier`

## What changes were proposed in this pull request?
In the previous PR 
https://github.com/apache/spark/pull/5755#discussion_r157848354, we dropped 
`(-[classifier])` from the retrieval pattern. We should add it back; otherwise,
> If this pattern for instance doesn't has the [type] or [classifier] token, 
> Ivy will download the source/javadoc artifacts to the same file as the 
> regular jar.

## How was this patch tested?
The existing tests

Author: gatorsmile 

Closes #20037 from gatorsmile/addClassifier.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c89b4311
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c89b4311
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c89b4311

Branch: refs/heads/master
Commit: c89b43118347677f122db190c9033394c15cee30
Parents: d762d11
Author: gatorsmile 
Authored: Wed Dec 20 11:19:57 2017 -0800
Committer: gatorsmile 
Committed: Wed Dec 20 11:19:57 2017 -0800

--
 core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c89b4311/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
--
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala 
b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index ab834bb..cbe1f2c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -1271,7 +1271,7 @@ private[spark] object SparkSubmitUtils {
 // retrieve all resolved dependencies
 ivy.retrieve(rr.getModuleDescriptor.getModuleRevisionId,
   packagesDirectory.getAbsolutePath + File.separator +
-"[organization]_[artifact]-[revision].[ext]",
+"[organization]_[artifact]-[revision](-[classifier]).[ext]",
   retrieveOptions.setConfs(Array(ivyConfName)))
 resolveDependencyPaths(rr.getArtifacts.toArray, packagesDirectory)
   } finally {


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



spark git commit: [SPARK-22649][PYTHON][SQL] Adding localCheckpoint to Dataset API

2017-12-19 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 6e36d8d56 -> 13268a58f


[SPARK-22649][PYTHON][SQL] Adding localCheckpoint to Dataset API

## What changes were proposed in this pull request?

This change adds local checkpoint support to datasets and respective bind from 
Python Dataframe API.

If reliability requirements can be lowered to favor performance, as in cases of 
further quick transformations followed by a reliable save, localCheckpoints() 
fit very well.
Furthermore, at the moment Reliable checkpoints still incur double computation 
(see #9428)
In general it makes the API more complete as well.

## How was this patch tested?

Python land quick use case:

```python
>>> from time import sleep
>>> from pyspark.sql import types as T
>>> from pyspark.sql import functions as F

>>> def f(x):
sleep(1)
return x*2
   ...:

>>> df1 = spark.range(30, numPartitions=6)
>>> df2 = df1.select(F.udf(f, T.LongType())("id"))

>>> %time _ = df2.collect()
CPU times: user 7.79 ms, sys: 5.84 ms, total: 13.6 ms
Wall time: 12.2 s

>>> %time df3 = df2.localCheckpoint()
CPU times: user 2.38 ms, sys: 2.3 ms, total: 4.68 ms
Wall time: 10.3 s

>>> %time _ = df3.collect()
CPU times: user 5.09 ms, sys: 410 µs, total: 5.5 ms
Wall time: 148 ms

>>> sc.setCheckpointDir(".")
>>> %time df3 = df2.checkpoint()
CPU times: user 4.04 ms, sys: 1.63 ms, total: 5.67 ms
Wall time: 20.3 s
```

Author: Fernando Pereira 

Closes #19805 from ferdonline/feature_dataset_localCheckpoint.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/13268a58
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/13268a58
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/13268a58

Branch: refs/heads/master
Commit: 13268a58f8f67bf994f0ad5076419774c45daeeb
Parents: 6e36d8d
Author: Fernando Pereira 
Authored: Tue Dec 19 20:47:12 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 19 20:47:12 2017 -0800

--
 python/pyspark/sql/dataframe.py |  14 +++
 .../scala/org/apache/spark/sql/Dataset.scala|  49 -
 .../org/apache/spark/sql/DatasetSuite.scala | 107 +++
 3 files changed, 121 insertions(+), 49 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/13268a58/python/pyspark/sql/dataframe.py
--
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 9864dc9..75395a7 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -368,6 +368,20 @@ class DataFrame(object):
 jdf = self._jdf.checkpoint(eager)
 return DataFrame(jdf, self.sql_ctx)
 
+@since(2.3)
+def localCheckpoint(self, eager=True):
+"""Returns a locally checkpointed version of this Dataset. 
Checkpointing can be used to
+truncate the logical plan of this DataFrame, which is especially 
useful in iterative
+algorithms where the plan may grow exponentially. Local checkpoints 
are stored in the
+executors using the caching subsystem and therefore they are not 
reliable.
+
+:param eager: Whether to checkpoint this DataFrame immediately
+
+.. note:: Experimental
+"""
+jdf = self._jdf.localCheckpoint(eager)
+return DataFrame(jdf, self.sql_ctx)
+
 @since(2.1)
 def withWatermark(self, eventTime, delayThreshold):
 """Defines an event time watermark for this :class:`DataFrame`. A 
watermark tracks a point

http://git-wip-us.apache.org/repos/asf/spark/blob/13268a58/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index c34cf0a..ef00562 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -527,7 +527,7 @@ class Dataset[T] private[sql](
*/
   @Experimental
   @InterfaceStability.Evolving
-  def checkpoint(): Dataset[T] = checkpoint(eager = true)
+  def checkpoint(): Dataset[T] = checkpoint(eager = true, reliableCheckpoint = 
true)
 
   /**
* Returns a checkpointed version of this Dataset. Checkpointing can be used 
to truncate the
@@ -540,9 +540,52 @@ class Dataset[T] private[sql](
*/
   @Experimental
   @InterfaceStability.Evolving
-  def checkpoint(eager: Boolean): Dataset[T] = {
+  def checkpoint(eager: Boolean): Dataset[T] = checkpoint(eager = eager, 
reliableCheckpoint = true)
+
+  /**
+   * Eagerly locally checkpoints a Dataset and return the new Dataset. 
Checkpointing can be
+   * used to truncate the logical plan 

spark git commit: [SPARK-22829] Add new built-in function date_trunc()

2017-12-19 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 3a7494dfe -> 6e36d8d56


[SPARK-22829] Add new built-in function date_trunc()

## What changes were proposed in this pull request?

Adding date_trunc() as a built-in function.
`date_trunc` is common in other databases, but Spark or Hive does not have 
support for this. `date_trunc` is commonly used by data scientists and business 
intelligence application such as Superset 
(https://github.com/apache/incubator-superset).
We do have `trunc` but this only works with 'MONTH' and 'YEAR' level on the 
DateType input.

date_trunc() in other databases:
AWS Redshift: http://docs.aws.amazon.com/redshift/latest/dg/r_DATE_TRUNC.html
PostgreSQL: https://www.postgresql.org/docs/9.1/static/functions-datetime.html
Presto: https://prestodb.io/docs/current/functions/datetime.html

## How was this patch tested?

Unit tests

(Please explain how this patch was tested. E.g. unit tests, integration tests, 
manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, 
remove this)

Please review http://spark.apache.org/contributing.html before opening a pull 
request.

Author: Youngbin Kim 

Closes #20015 from youngbink/date_trunc.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6e36d8d5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6e36d8d5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6e36d8d5

Branch: refs/heads/master
Commit: 6e36d8d56279a2c5c92c8df8e89ee99b514817e7
Parents: 3a7494d
Author: Youngbin Kim 
Authored: Tue Dec 19 20:22:33 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 19 20:22:33 2017 -0800

--
 python/pyspark/sql/functions.py |  20 ++-
 .../catalyst/analysis/FunctionRegistry.scala|   1 +
 .../expressions/datetimeExpressions.scala   | 170 ++-
 .../spark/sql/catalyst/util/DateTimeUtils.scala | 102 +--
 .../expressions/DateExpressionsSuite.scala  |  73 +++-
 .../sql/catalyst/util/DateTimeUtilsSuite.scala  |  70 
 .../scala/org/apache/spark/sql/functions.scala  |  15 ++
 .../apache/spark/sql/DateFunctionsSuite.scala   |  46 +
 8 files changed, 445 insertions(+), 52 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6e36d8d5/python/pyspark/sql/functions.py
--
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 4e0fadd..5453005 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -1099,7 +1099,7 @@ def trunc(date, format):
 """
 Returns date truncated to the unit specified by the format.
 
-:param format: 'year', '', 'yy' or 'month', 'mon', 'mm'
+:param format: 'year', '', 'yy' or 'month', 'mon', 'mm'
 
 >>> df = spark.createDataFrame([('1997-02-28',)], ['d'])
 >>> df.select(trunc(df.d, 'year').alias('year')).collect()
@@ -,6 +,24 @@ def trunc(date, format):
 return Column(sc._jvm.functions.trunc(_to_java_column(date), format))
 
 
+@since(2.3)
+def date_trunc(format, timestamp):
+"""
+Returns timestamp truncated to the unit specified by the format.
+
+:param format: 'year', '', 'yy', 'month', 'mon', 'mm',
+'day', 'dd', 'hour', 'minute', 'second', 'week', 'quarter'
+
+>>> df = spark.createDataFrame([('1997-02-28 05:02:11',)], ['t'])
+>>> df.select(date_trunc('year', df.t).alias('year')).collect()
+[Row(year=datetime.datetime(1997, 1, 1, 0, 0))]
+>>> df.select(date_trunc('mon', df.t).alias('month')).collect()
+[Row(month=datetime.datetime(1997, 2, 1, 0, 0))]
+"""
+sc = SparkContext._active_spark_context
+return Column(sc._jvm.functions.date_trunc(format, 
_to_java_column(timestamp)))
+
+
 @since(1.5)
 def next_day(date, dayOfWeek):
 """

http://git-wip-us.apache.org/repos/asf/spark/blob/6e36d8d5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index 11538bd..5ddb398 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -392,6 +392,7 @@ object FunctionRegistry {
 expression[ToUnixTimestamp]("to_unix_timestamp"),
 expression[ToUTCTimestamp]("to_utc_timestamp"),
 expression[TruncDate]("trunc"),
+expression[TruncTimestamp]("date_trunc"),
 

[2/2] spark git commit: [SPARK-22821][TEST] Basic tests for WidenSetOperationTypes, BooleanEquality, StackCoercion and Division

2017-12-19 Thread lixiao
[SPARK-22821][TEST] Basic tests for WidenSetOperationTypes, BooleanEquality, 
StackCoercion and Division

## What changes were proposed in this pull request?

Test Coverage for `WidenSetOperationTypes`, `BooleanEquality`, `StackCoercion`  
and `Division`, this is a Sub-tasks for 
[SPARK-22722](https://issues.apache.org/jira/browse/SPARK-22722).

## How was this patch tested?
N/A

Author: Yuming Wang 

Closes #20006 from wangyum/SPARK-22821.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6129ffa1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6129ffa1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6129ffa1

Branch: refs/heads/master
Commit: 6129ffa11ea62437a25844455e87a1e4c21b030f
Parents: ef10f45
Author: Yuming Wang 
Authored: Tue Dec 19 11:56:22 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 19 11:56:22 2017 -0800

--
 .../typeCoercion/native/booleanEquality.sql |  122 ++
 .../inputs/typeCoercion/native/division.sql |  174 +++
 .../native/widenSetOperationTypes.sql   |  175 +++
 .../typeCoercion/native/booleanEquality.sql.out |  802 +++
 .../typeCoercion/native/division.sql.out| 1242 +
 .../native/widenSetOperationTypes.sql.out   | 1305 ++
 6 files changed, 3820 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6129ffa1/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/booleanEquality.sql
--
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/booleanEquality.sql
 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/booleanEquality.sql
new file mode 100644
index 000..442f235
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/booleanEquality.sql
@@ -0,0 +1,122 @@
+--
+--   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.
+--
+
+CREATE TEMPORARY VIEW t AS SELECT 1;
+
+SELECT true = cast(1 as tinyint) FROM t;
+SELECT true = cast(1 as smallint) FROM t;
+SELECT true = cast(1 as int) FROM t;
+SELECT true = cast(1 as bigint) FROM t;
+SELECT true = cast(1 as float) FROM t;
+SELECT true = cast(1 as double) FROM t;
+SELECT true = cast(1 as decimal(10, 0)) FROM t;
+SELECT true = cast(1 as string) FROM t;
+SELECT true = cast('1' as binary) FROM t;
+SELECT true = cast(1 as boolean) FROM t;
+SELECT true = cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT true = cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT true <=> cast(1 as tinyint) FROM t;
+SELECT true <=> cast(1 as smallint) FROM t;
+SELECT true <=> cast(1 as int) FROM t;
+SELECT true <=> cast(1 as bigint) FROM t;
+SELECT true <=> cast(1 as float) FROM t;
+SELECT true <=> cast(1 as double) FROM t;
+SELECT true <=> cast(1 as decimal(10, 0)) FROM t;
+SELECT true <=> cast(1 as string) FROM t;
+SELECT true <=> cast('1' as binary) FROM t;
+SELECT true <=> cast(1 as boolean) FROM t;
+SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as tinyint) = true FROM t;
+SELECT cast(1 as smallint) = true FROM t;
+SELECT cast(1 as int) = true FROM t;
+SELECT cast(1 as bigint) = true FROM t;
+SELECT cast(1 as float) = true FROM t;
+SELECT cast(1 as double) = true FROM t;
+SELECT cast(1 as decimal(10, 0)) = true FROM t;
+SELECT cast(1 as string) = true FROM t;
+SELECT cast('1' as binary) = true FROM t;
+SELECT cast(1 as boolean) = true FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) = true FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) = true FROM t;
+
+SELECT cast(1 as tinyint) <=> true FROM t;
+SELECT cast(1 as smallint) <=> true FROM t;
+SELECT cast(1 as int) <=> true FROM t;
+SELECT cast(1 as bigint) <=> true FROM t;
+SELECT cast(1 as float) <=> true FROM t;
+SELECT cast(1 as double) <=> true FROM t;
+SELECT cast(1 as decimal(10, 0)) <=> true FROM t;
+SELECT 

[1/2] spark git commit: [SPARK-22821][TEST] Basic tests for WidenSetOperationTypes, BooleanEquality, StackCoercion and Division

2017-12-19 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master ef10f452e -> 6129ffa11


http://git-wip-us.apache.org/repos/asf/spark/blob/6129ffa1/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out
--
diff --git 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out
new file mode 100644
index 000..017e0fe
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out
@@ -0,0 +1,1242 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 145
+
+
+-- !query 0
+CREATE TEMPORARY VIEW t AS SELECT 1
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+SELECT cast(1 as tinyint) / cast(1 as tinyint) FROM t
+-- !query 1 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS 
DOUBLE)):double>
+-- !query 1 output
+1.0
+
+
+-- !query 2
+SELECT cast(1 as tinyint) / cast(1 as smallint) FROM t
+-- !query 2 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS 
DOUBLE)):double>
+-- !query 2 output
+1.0
+
+
+-- !query 3
+SELECT cast(1 as tinyint) / cast(1 as int) FROM t
+-- !query 3 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS INT) AS 
DOUBLE)):double>
+-- !query 3 output
+1.0
+
+
+-- !query 4
+SELECT cast(1 as tinyint) / cast(1 as bigint) FROM t
+-- !query 4 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS 
DOUBLE)):double>
+-- !query 4 output
+1.0
+
+
+-- !query 5
+SELECT cast(1 as tinyint) / cast(1 as float) FROM t
+-- !query 5 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS 
DOUBLE)):double>
+-- !query 5 output
+1.0
+
+
+-- !query 6
+SELECT cast(1 as tinyint) / cast(1 as double) FROM t
+-- !query 6 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS 
DOUBLE)):double>
+-- !query 6 output
+1.0
+
+
+-- !query 7
+SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t
+-- !query 7 schema
+struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) / 
CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)>
+-- !query 7 output
+1
+
+
+-- !query 8
+SELECT cast(1 as tinyint) / cast(1 as string) FROM t
+-- !query 8 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(CAST(1 AS STRING) AS 
DOUBLE) AS DOUBLE)):double>
+-- !query 8 output
+1.0
+
+
+-- !query 9
+SELECT cast(1 as tinyint) / cast('1' as binary) FROM t
+-- !query 9 schema
+struct<>
+-- !query 9 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) / CAST('1' AS BINARY))' due to data type 
mismatch: differing types in '(CAST(1 AS TINYINT) / CAST('1' AS BINARY))' 
(tinyint and binary).; line 1 pos 7
+
+
+-- !query 10
+SELECT cast(1 as tinyint) / cast(1 as boolean) FROM t
+-- !query 10 schema
+struct<>
+-- !query 10 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) / CAST(1 AS BOOLEAN))' due to data type 
mismatch: differing types in '(CAST(1 AS TINYINT) / CAST(1 AS BOOLEAN))' 
(tinyint and boolean).; line 1 pos 7
+
+
+-- !query 11
+SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 11 schema
+struct<>
+-- !query 11 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) / CAST('2017-12-11 09:30:00.0' AS 
TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) 
/ CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (tinyint and timestamp).; line 1 
pos 7
+
+
+-- !query 12
+SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 12 schema
+struct<>
+-- !query 12 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) / CAST('2017-12-11 09:30:00' AS DATE))' 
due to data type mismatch: differing types in '(CAST(1 AS TINYINT) / 
CAST('2017-12-11 09:30:00' AS DATE))' (tinyint and date).; line 1 pos 7
+
+
+-- !query 13
+SELECT cast(1 as smallint) / cast(1 as tinyint) FROM t
+-- !query 13 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS 
DOUBLE)):double>
+-- !query 13 output
+1.0
+
+
+-- !query 14
+SELECT cast(1 as smallint) / cast(1 as smallint) FROM t
+-- !query 14 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS 
DOUBLE)):double>
+-- !query 14 output
+1.0
+
+
+-- !query 15
+SELECT cast(1 as smallint) / cast(1 as int) FROM t
+-- !query 15 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS INT) AS 
DOUBLE)):double>
+-- !query 15 output
+1.0
+
+
+-- !query 16
+SELECT cast(1 as smallint) / cast(1 as bigint) FROM t
+-- !query 16 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS 
DOUBLE)):double>
+-- !query 16 output
+1.0
+
+
+-- !query 17
+SELECT cast(1 as smallint) / cast(1 as float) FROM 

spark git commit: [SPARK-21652][SQL][FOLLOW-UP] Fix rule conflict caused by InferFiltersFromConstraints

2017-12-19 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master ee56fc343 -> ef10f452e


[SPARK-21652][SQL][FOLLOW-UP] Fix rule conflict caused by 
InferFiltersFromConstraints

## What changes were proposed in this pull request?
The optimizer rule `InferFiltersFromConstraints` could trigger our batch  
`Operator Optimizations` exceeds the max iteration limit (i.e., 100) so that 
the final plan might not be properly optimized. The rule 
`InferFiltersFromConstraints` could conflict with the other Filter/Join 
predicate reduction rules. Thus, we need to separate 
`InferFiltersFromConstraints` from the other rules.

This PR is to separate `InferFiltersFromConstraints ` from the main batch 
`Operator Optimizations` .

## How was this patch tested?
The existing test cases.

Author: gatorsmile 

Closes #19149 from gatorsmile/inferFilterRule.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ef10f452
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ef10f452
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ef10f452

Branch: refs/heads/master
Commit: ef10f452e62c77d0434e80f7266f6685eb1bcb2c
Parents: ee56fc3
Author: gatorsmile 
Authored: Tue Dec 19 09:05:47 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 19 09:05:47 2017 -0800

--
 .../sql/catalyst/optimizer/Optimizer.scala  | 115 +++
 1 file changed, 64 insertions(+), 51 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ef10f452/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 5acadf8..6a4d1e9 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -47,7 +47,62 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
   protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations)
 
   def batches: Seq[Batch] = {
-Batch("Eliminate Distinct", Once, EliminateDistinct) ::
+val operatorOptimizationRuleSet =
+  Seq(
+// Operator push down
+PushProjectionThroughUnion,
+ReorderJoin,
+EliminateOuterJoin,
+PushPredicateThroughJoin,
+PushDownPredicate,
+LimitPushDown,
+ColumnPruning,
+InferFiltersFromConstraints,
+// Operator combine
+CollapseRepartition,
+CollapseProject,
+CollapseWindow,
+CombineFilters,
+CombineLimits,
+CombineUnions,
+// Constant folding and strength reduction
+NullPropagation,
+ConstantPropagation,
+FoldablePropagation,
+OptimizeIn,
+ConstantFolding,
+ReorderAssociativeOperator,
+LikeSimplification,
+BooleanSimplification,
+SimplifyConditionals,
+RemoveDispensableExpressions,
+SimplifyBinaryComparison,
+PruneFilters,
+EliminateSorts,
+SimplifyCasts,
+SimplifyCaseConversionExpressions,
+RewriteCorrelatedScalarSubquery,
+EliminateSerialization,
+RemoveRedundantAliases,
+RemoveRedundantProject,
+SimplifyCreateStructOps,
+SimplifyCreateArrayOps,
+SimplifyCreateMapOps,
+CombineConcats) ++
+extendedOperatorOptimizationRules
+
+val operatorOptimizationBatch: Seq[Batch] = {
+  val rulesWithoutInferFiltersFromConstraints =
+operatorOptimizationRuleSet.filterNot(_ == InferFiltersFromConstraints)
+  Batch("Operator Optimization before Inferring Filters", fixedPoint,
+rulesWithoutInferFiltersFromConstraints: _*) ::
+  Batch("Infer Filters", Once,
+InferFiltersFromConstraints) ::
+  Batch("Operator Optimization after Inferring Filters", fixedPoint,
+rulesWithoutInferFiltersFromConstraints: _*) :: Nil
+}
+
+(Batch("Eliminate Distinct", Once, EliminateDistinct) ::
 // Technically some of the rules in Finish Analysis are not optimizer 
rules and belong more
 // in the analyzer, because they are needed for correctness (e.g. 
ComputeCurrentTime).
 // However, because we also use the analyzer to canonicalized queries (for 
view definition),
@@ -81,68 +136,26 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
   ReplaceDistinctWithAggregate) ::
 Batch("Aggregate", fixedPoint,
   RemoveLiteralFromGroupExpressions,
-  RemoveRepetitionFromGroupExpressions) ::
-Batch("Operator 

[3/3] spark git commit: [SPARK-22816][TEST] Basic tests for PromoteStrings and InConversion

2017-12-17 Thread lixiao
[SPARK-22816][TEST] Basic tests for PromoteStrings and InConversion

## What changes were proposed in this pull request?

Test Coverage for `PromoteStrings` and `InConversion`, this is a Sub-tasks for 
[SPARK-22722](https://issues.apache.org/jira/browse/SPARK-22722).

## How was this patch tested?

N/A

Author: Yuming Wang 

Closes #20001 from wangyum/SPARK-22816.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7f6d10a7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7f6d10a7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7f6d10a7

Branch: refs/heads/master
Commit: 7f6d10a7376594cf7e5225a05fa9e58b2011ad3d
Parents: 77988a9
Author: Yuming Wang 
Authored: Sun Dec 17 09:15:10 2017 -0800
Committer: gatorsmile 
Committed: Sun Dec 17 09:15:10 2017 -0800

--
 .../inputs/typeCoercion/native/inConversion.sql |  330 +++
 .../typeCoercion/native/promoteStrings.sql  |  364 +++
 .../typeCoercion/native/inConversion.sql.out| 2454 +
 .../typeCoercion/native/promoteStrings.sql.out  | 2578 ++
 4 files changed, 5726 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7f6d10a7/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/inConversion.sql
--
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/inConversion.sql
 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/inConversion.sql
new file mode 100644
index 000..39dbe72
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/inConversion.sql
@@ -0,0 +1,330 @@
+--
+--   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.
+--
+
+CREATE TEMPORARY VIEW t AS SELECT 1;
+
+SELECT cast(1 as tinyint) in (cast(1 as tinyint)) FROM t;
+SELECT cast(1 as tinyint) in (cast(1 as smallint)) FROM t;
+SELECT cast(1 as tinyint) in (cast(1 as int)) FROM t;
+SELECT cast(1 as tinyint) in (cast(1 as bigint)) FROM t;
+SELECT cast(1 as tinyint) in (cast(1 as float)) FROM t;
+SELECT cast(1 as tinyint) in (cast(1 as double)) FROM t;
+SELECT cast(1 as tinyint) in (cast(1 as decimal(10, 0))) FROM t;
+SELECT cast(1 as tinyint) in (cast(1 as string)) FROM t;
+SELECT cast(1 as tinyint) in (cast('1' as binary)) FROM t;
+SELECT cast(1 as tinyint) in (cast(1 as boolean)) FROM t;
+SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM 
t;
+SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00' as date)) FROM t;
+
+SELECT cast(1 as smallint) in (cast(1 as tinyint)) FROM t;
+SELECT cast(1 as smallint) in (cast(1 as smallint)) FROM t;
+SELECT cast(1 as smallint) in (cast(1 as int)) FROM t;
+SELECT cast(1 as smallint) in (cast(1 as bigint)) FROM t;
+SELECT cast(1 as smallint) in (cast(1 as float)) FROM t;
+SELECT cast(1 as smallint) in (cast(1 as double)) FROM t;
+SELECT cast(1 as smallint) in (cast(1 as decimal(10, 0))) FROM t;
+SELECT cast(1 as smallint) in (cast(1 as string)) FROM t;
+SELECT cast(1 as smallint) in (cast('1' as binary)) FROM t;
+SELECT cast(1 as smallint) in (cast(1 as boolean)) FROM t;
+SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00.0' as timestamp)) 
FROM t;
+SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00' as date)) FROM t;
+
+SELECT cast(1 as int) in (cast(1 as tinyint)) FROM t;
+SELECT cast(1 as int) in (cast(1 as smallint)) FROM t;
+SELECT cast(1 as int) in (cast(1 as int)) FROM t;
+SELECT cast(1 as int) in (cast(1 as bigint)) FROM t;
+SELECT cast(1 as int) in (cast(1 as float)) FROM t;
+SELECT cast(1 as int) in (cast(1 as double)) FROM t;
+SELECT cast(1 as int) in (cast(1 as decimal(10, 0))) FROM t;
+SELECT cast(1 as int) in (cast(1 as string)) FROM t;
+SELECT cast(1 as int) in (cast('1' as binary)) FROM t;
+SELECT cast(1 as int) in (cast(1 as boolean)) FROM t;
+SELECT cast(1 as int) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t;
+SELECT cast(1 as int) in (cast('2017-12-11 09:30:00' as 

[1/3] spark git commit: [SPARK-22816][TEST] Basic tests for PromoteStrings and InConversion

2017-12-17 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 77988a9d0 -> 7f6d10a73


http://git-wip-us.apache.org/repos/asf/spark/blob/7f6d10a7/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out
--
diff --git 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out
new file mode 100644
index 000..0beb1f6
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out
@@ -0,0 +1,2578 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 316
+
+
+-- !query 0
+CREATE TEMPORARY VIEW t AS SELECT 1
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+SELECT '1' + cast(1 as tinyint) FROM t
+-- !query 1 schema
+struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS TINYINT) AS DOUBLE)):double>
+-- !query 1 output
+2.0
+
+
+-- !query 2
+SELECT '1' + cast(1 as smallint)FROM t
+-- !query 2 schema
+struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double>
+-- !query 2 output
+2.0
+
+
+-- !query 3
+SELECT '1' + cast(1 as int) FROM t
+-- !query 3 schema
+struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS INT) AS DOUBLE)):double>
+-- !query 3 output
+2.0
+
+
+-- !query 4
+SELECT '1' + cast(1 as bigint)  FROM t
+-- !query 4 schema
+struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS BIGINT) AS DOUBLE)):double>
+-- !query 4 output
+2.0
+
+
+-- !query 5
+SELECT '1' + cast(1 as float)   FROM t
+-- !query 5 schema
+struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS FLOAT) AS DOUBLE)):double>
+-- !query 5 output
+2.0
+
+
+-- !query 6
+SELECT '1' + cast(1 as double)  FROM t
+-- !query 6 schema
+struct<(CAST(1 AS DOUBLE) + CAST(1 AS DOUBLE)):double>
+-- !query 6 output
+2.0
+
+
+-- !query 7
+SELECT '1' + cast(1 as decimal(10, 0))  FROM t
+-- !query 7 schema
+struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double>
+-- !query 7 output
+2.0
+
+
+-- !query 8
+SELECT '1' + '1'FROM t
+-- !query 8 schema
+struct<(CAST(1 AS DOUBLE) + CAST(1 AS DOUBLE)):double>
+-- !query 8 output
+2.0
+
+
+-- !query 9
+SELECT '1' + cast('1' as binary)FROM t
+-- !query 9 schema
+struct<>
+-- !query 9 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS DOUBLE) + CAST('1' AS BINARY))' due to data type 
mismatch: differing types in '(CAST('1' AS DOUBLE) + CAST('1' AS BINARY))' 
(double and binary).; line 1 pos 7
+
+
+-- !query 10
+SELECT '1' + cast(1 as boolean) FROM t
+-- !query 10 schema
+struct<>
+-- !query 10 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS DOUBLE) + CAST(1 AS BOOLEAN))' due to data type 
mismatch: differing types in '(CAST('1' AS DOUBLE) + CAST(1 AS BOOLEAN))' 
(double and boolean).; line 1 pos 7
+
+
+-- !query 11
+SELECT '1' + cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 11 schema
+struct<>
+-- !query 11 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS DOUBLE) + CAST('2017-12-11 09:30:00.0' AS 
TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS 
DOUBLE) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; 
line 1 pos 7
+
+
+-- !query 12
+SELECT '1' + cast('2017-12-11 09:30:00' as date)FROM t
+-- !query 12 schema
+struct<>
+-- !query 12 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS DOUBLE) + CAST('2017-12-11 09:30:00' AS DATE))' 
due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) + 
CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7
+
+
+-- !query 13
+SELECT '1' - cast(1 as tinyint) FROM t
+-- !query 13 schema
+struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS TINYINT) AS DOUBLE)):double>
+-- !query 13 output
+0.0
+
+
+-- !query 14
+SELECT '1' - cast(1 as smallint)FROM t
+-- !query 14 schema
+struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double>
+-- !query 14 output
+0.0
+
+
+-- !query 15
+SELECT '1' - cast(1 as int) FROM t
+-- !query 15 schema
+struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS INT) AS DOUBLE)):double>
+-- !query 15 output
+0.0
+
+
+-- !query 16
+SELECT '1' - cast(1 as bigint)  FROM t
+-- !query 16 schema
+struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS BIGINT) AS DOUBLE)):double>
+-- !query 16 output
+0.0
+
+
+-- !query 17
+SELECT '1' - cast(1 as float)   FROM t
+-- !query 17 schema
+struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS FLOAT) AS DOUBLE)):double>
+-- !query 17 output
+0.0
+
+
+-- !query 18
+SELECT '1' - cast(1 as double) 

[2/3] spark git commit: [SPARK-22816][TEST] Basic tests for PromoteStrings and InConversion

2017-12-17 Thread lixiao
http://git-wip-us.apache.org/repos/asf/spark/blob/7f6d10a7/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out
--
diff --git 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out
new file mode 100644
index 000..bf8ddee
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out
@@ -0,0 +1,2454 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 289
+
+
+-- !query 0
+CREATE TEMPORARY VIEW t AS SELECT 1
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+SELECT cast(1 as tinyint) in (cast(1 as tinyint)) FROM t
+-- !query 1 schema
+struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT))):boolean>
+-- !query 1 output
+true
+
+
+-- !query 2
+SELECT cast(1 as tinyint) in (cast(1 as smallint)) FROM t
+-- !query 2 schema
+struct<(CAST(CAST(1 AS TINYINT) AS SMALLINT) IN (CAST(CAST(1 AS SMALLINT) AS 
SMALLINT))):boolean>
+-- !query 2 output
+true
+
+
+-- !query 3
+SELECT cast(1 as tinyint) in (cast(1 as int)) FROM t
+-- !query 3 schema
+struct<(CAST(CAST(1 AS TINYINT) AS INT) IN (CAST(CAST(1 AS INT) AS 
INT))):boolean>
+-- !query 3 output
+true
+
+
+-- !query 4
+SELECT cast(1 as tinyint) in (cast(1 as bigint)) FROM t
+-- !query 4 schema
+struct<(CAST(CAST(1 AS TINYINT) AS BIGINT) IN (CAST(CAST(1 AS BIGINT) AS 
BIGINT))):boolean>
+-- !query 4 output
+true
+
+
+-- !query 5
+SELECT cast(1 as tinyint) in (cast(1 as float)) FROM t
+-- !query 5 schema
+struct<(CAST(CAST(1 AS TINYINT) AS FLOAT) IN (CAST(CAST(1 AS FLOAT) AS 
FLOAT))):boolean>
+-- !query 5 output
+true
+
+
+-- !query 6
+SELECT cast(1 as tinyint) in (cast(1 as double)) FROM t
+-- !query 6 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS 
DOUBLE))):boolean>
+-- !query 6 output
+true
+
+
+-- !query 7
+SELECT cast(1 as tinyint) in (cast(1 as decimal(10, 0))) FROM t
+-- !query 7 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS 
DECIMAL(10,0)) AS DECIMAL(10,0:boolean>
+-- !query 7 output
+true
+
+
+-- !query 8
+SELECT cast(1 as tinyint) in (cast(1 as string)) FROM t
+-- !query 8 schema
+struct<(CAST(CAST(1 AS TINYINT) AS STRING) IN (CAST(CAST(1 AS STRING) AS 
STRING))):boolean>
+-- !query 8 output
+true
+
+
+-- !query 9
+SELECT cast(1 as tinyint) in (cast('1' as binary)) FROM t
+-- !query 9 schema
+struct<>
+-- !query 9 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) IN (CAST('1' AS BINARY)))' due to data 
type mismatch: Arguments must be same type but were: ByteType != BinaryType; 
line 1 pos 26
+
+
+-- !query 10
+SELECT cast(1 as tinyint) in (cast(1 as boolean)) FROM t
+-- !query 10 schema
+struct<>
+-- !query 10 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) IN (CAST(1 AS BOOLEAN)))' due to data type 
mismatch: Arguments must be same type but were: ByteType != BooleanType; line 1 
pos 26
+
+
+-- !query 11
+SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM 
t
+-- !query 11 schema
+struct<>
+-- !query 11 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) IN (CAST('2017-12-11 09:30:00.0' AS 
TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: 
ByteType != TimestampType; line 1 pos 26
+
+
+-- !query 12
+SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00' as date)) FROM t
+-- !query 12 schema
+struct<>
+-- !query 12 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) IN (CAST('2017-12-11 09:30:00' AS DATE)))' 
due to data type mismatch: Arguments must be same type but were: ByteType != 
DateType; line 1 pos 26
+
+
+-- !query 13
+SELECT cast(1 as smallint) in (cast(1 as tinyint)) FROM t
+-- !query 13 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS SMALLINT) IN (CAST(CAST(1 AS TINYINT) AS 
SMALLINT))):boolean>
+-- !query 13 output
+true
+
+
+-- !query 14
+SELECT cast(1 as smallint) in (cast(1 as smallint)) FROM t
+-- !query 14 schema
+struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT))):boolean>
+-- !query 14 output
+true
+
+
+-- !query 15
+SELECT cast(1 as smallint) in (cast(1 as int)) FROM t
+-- !query 15 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS INT) IN (CAST(CAST(1 AS INT) AS 
INT))):boolean>
+-- !query 15 output
+true
+
+
+-- !query 16
+SELECT cast(1 as smallint) in (cast(1 as bigint)) FROM t
+-- !query 16 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS BIGINT) IN (CAST(CAST(1 AS BIGINT) AS 
BIGINT))):boolean>
+-- !query 16 output
+true
+
+
+-- !query 17
+SELECT cast(1 as smallint) in (cast(1 as float)) FROM t
+-- !query 17 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS FLOAT) IN (CAST(CAST(1 AS FLOAT) AS 
FLOAT))):boolean>
+-- !query 17 output
+true
+
+
+-- !query 18
+SELECT cast(1 as smallint) in 

[2/2] spark git commit: [SPARK-22762][TEST] Basic tests for IfCoercion and CaseWhenCoercion

2017-12-15 Thread lixiao
[SPARK-22762][TEST] Basic tests for IfCoercion and CaseWhenCoercion

## What changes were proposed in this pull request?

Basic tests for IfCoercion and CaseWhenCoercion

## How was this patch tested?

N/A

Author: Yuming Wang 

Closes #19949 from wangyum/SPARK-22762.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/46776234
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/46776234
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/46776234

Branch: refs/heads/master
Commit: 46776234a49742e94c64897322500582d7393d35
Parents: 9fafa82
Author: Yuming Wang 
Authored: Fri Dec 15 09:58:31 2017 -0800
Committer: gatorsmile 
Committed: Fri Dec 15 09:58:31 2017 -0800

--
 .../typeCoercion/native/caseWhenCoercion.sql|  174 +++
 .../inputs/typeCoercion/native/ifCoercion.sql   |  174 +++
 .../native/caseWhenCoercion.sql.out | 1232 ++
 .../typeCoercion/native/ifCoercion.sql.out  | 1232 ++
 4 files changed, 2812 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/46776234/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/caseWhenCoercion.sql
--
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/caseWhenCoercion.sql
 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/caseWhenCoercion.sql
new file mode 100644
index 000..a780529
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/caseWhenCoercion.sql
@@ -0,0 +1,174 @@
+--
+--   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.
+--
+
+CREATE TEMPORARY VIEW t AS SELECT 1;
+
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as tinyint) END FROM 
t;
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as smallint) END 
FROM t;
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as int) END FROM t;
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as bigint) END FROM 
t;
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as float) END FROM t;
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as double) END FROM 
t;
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as decimal(10, 0)) 
END FROM t;
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as string) END FROM 
t;
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2' as binary) END 
FROM t;
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as boolean) END FROM 
t;
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 
09:30:00.0' as timestamp) END FROM t;
+SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00' 
as date) END FROM t;
+
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as tinyint) END 
FROM t;
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as smallint) END 
FROM t;
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as int) END FROM t;
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as bigint) END FROM 
t;
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as float) END FROM 
t;
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as double) END FROM 
t;
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as decimal(10, 0)) 
END FROM t;
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as string) END FROM 
t;
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2' as binary) END 
FROM t;
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as boolean) END 
FROM t;
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 
09:30:00.0' as timestamp) END FROM t;
+SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00' 
as date) END FROM t;
+
+SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as tinyint) END FROM t;
+SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as smallint) END FROM t;
+SELECT CASE 

[1/2] spark git commit: [SPARK-22762][TEST] Basic tests for IfCoercion and CaseWhenCoercion

2017-12-15 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 9fafa8209 -> 46776234a


http://git-wip-us.apache.org/repos/asf/spark/blob/46776234/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out
--
diff --git 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out
new file mode 100644
index 000..7097027
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out
@@ -0,0 +1,1232 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 145
+
+
+-- !query 0
+CREATE TEMPORARY VIEW t AS SELECT 1
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+SELECT IF(true, cast(1 as tinyint), cast(2 as tinyint)) FROM t
+-- !query 1 schema
+struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS TINYINT))):tinyint>
+-- !query 1 output
+1
+
+
+-- !query 2
+SELECT IF(true, cast(1 as tinyint), cast(2 as smallint)) FROM t
+-- !query 2 schema
+struct<(IF(true, CAST(CAST(1 AS TINYINT) AS SMALLINT), CAST(2 AS 
SMALLINT))):smallint>
+-- !query 2 output
+1
+
+
+-- !query 3
+SELECT IF(true, cast(1 as tinyint), cast(2 as int)) FROM t
+-- !query 3 schema
+struct<(IF(true, CAST(CAST(1 AS TINYINT) AS INT), CAST(2 AS INT))):int>
+-- !query 3 output
+1
+
+
+-- !query 4
+SELECT IF(true, cast(1 as tinyint), cast(2 as bigint)) FROM t
+-- !query 4 schema
+struct<(IF(true, CAST(CAST(1 AS TINYINT) AS BIGINT), CAST(2 AS 
BIGINT))):bigint>
+-- !query 4 output
+1
+
+
+-- !query 5
+SELECT IF(true, cast(1 as tinyint), cast(2 as float)) FROM t
+-- !query 5 schema
+struct<(IF(true, CAST(CAST(1 AS TINYINT) AS FLOAT), CAST(2 AS FLOAT))):float>
+-- !query 5 output
+1.0
+
+
+-- !query 6
+SELECT IF(true, cast(1 as tinyint), cast(2 as double)) FROM t
+-- !query 6 schema
+struct<(IF(true, CAST(CAST(1 AS TINYINT) AS DOUBLE), CAST(2 AS 
DOUBLE))):double>
+-- !query 6 output
+1.0
+
+
+-- !query 7
+SELECT IF(true, cast(1 as tinyint), cast(2 as decimal(10, 0))) FROM t
+-- !query 7 schema
+struct<(IF(true, CAST(CAST(1 AS TINYINT) AS DECIMAL(10,0)), CAST(2 AS 
DECIMAL(10,0:decimal(10,0)>
+-- !query 7 output
+1
+
+
+-- !query 8
+SELECT IF(true, cast(1 as tinyint), cast(2 as string)) FROM t
+-- !query 8 schema
+struct<(IF(true, CAST(CAST(1 AS TINYINT) AS STRING), CAST(2 AS 
STRING))):string>
+-- !query 8 output
+1
+
+
+-- !query 9
+SELECT IF(true, cast(1 as tinyint), cast('2' as binary)) FROM t
+-- !query 9 schema
+struct<>
+-- !query 9 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(IF(true, CAST(1 AS TINYINT), CAST('2' AS BINARY)))' due to 
data type mismatch: differing types in '(IF(true, CAST(1 AS TINYINT), CAST('2' 
AS BINARY)))' (tinyint and binary).; line 1 pos 7
+
+
+-- !query 10
+SELECT IF(true, cast(1 as tinyint), cast(2 as boolean)) FROM t
+-- !query 10 schema
+struct<>
+-- !query 10 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(IF(true, CAST(1 AS TINYINT), CAST(2 AS BOOLEAN)))' due to 
data type mismatch: differing types in '(IF(true, CAST(1 AS TINYINT), CAST(2 AS 
BOOLEAN)))' (tinyint and boolean).; line 1 pos 7
+
+
+-- !query 11
+SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as 
timestamp)) FROM t
+-- !query 11 schema
+struct<>
+-- !query 11 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(IF(true, CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00.0' AS 
TIMESTAMP)))' due to data type mismatch: differing types in '(IF(true, CAST(1 
AS TINYINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (tinyint and 
timestamp).; line 1 pos 7
+
+
+-- !query 12
+SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t
+-- !query 12 schema
+struct<>
+-- !query 12 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(IF(true, CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00' AS 
DATE)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS 
TINYINT), CAST('2017-12-11 09:30:00' AS DATE)))' (tinyint and date).; line 1 
pos 7
+
+
+-- !query 13
+SELECT IF(true, cast(1 as smallint), cast(2 as tinyint)) FROM t
+-- !query 13 schema
+struct<(IF(true, CAST(1 AS SMALLINT), CAST(CAST(2 AS TINYINT) AS 
SMALLINT))):smallint>
+-- !query 13 output
+1
+
+
+-- !query 14
+SELECT IF(true, cast(1 as smallint), cast(2 as smallint)) FROM t
+-- !query 14 schema
+struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS SMALLINT))):smallint>
+-- !query 14 output
+1
+
+
+-- !query 15
+SELECT IF(true, cast(1 as smallint), cast(2 as int)) FROM t
+-- !query 15 schema
+struct<(IF(true, CAST(CAST(1 AS SMALLINT) AS INT), CAST(2 AS INT))):int>
+-- !query 15 output
+1
+
+
+-- !query 16
+SELECT IF(true, cast(1 as smallint), cast(2 as bigint)) FROM t
+-- !query 16 schema
+struct<(IF(true, CAST(CAST(1 AS SMALLINT) AS BIGINT), CAST(2 AS 
BIGINT))):bigint>
+-- !query 16 output
+1
+
+
+-- !query 17
+SELECT 

spark git commit: [SPARK-22800][TEST][SQL] Add a SSB query suite

2017-12-15 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master e58f27567 -> 9fafa8209


[SPARK-22800][TEST][SQL] Add a SSB query suite

## What changes were proposed in this pull request?
Add a test suite to ensure all the [SSB (Star Schema 
Benchmark)](https://www.cs.umb.edu/~poneil/StarSchemaB.PDF) queries can be 
successfully analyzed, optimized and compiled without hitting the max iteration 
threshold.

## How was this patch tested?
Added `SSBQuerySuite`.

Author: Takeshi Yamamuro 

Closes #19990 from maropu/SPARK-22800.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9fafa820
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9fafa820
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9fafa820

Branch: refs/heads/master
Commit: 9fafa8209c51adc2a22b89aedf9af7b5e29e0059
Parents: e58f275
Author: Takeshi Yamamuro 
Authored: Fri Dec 15 09:56:22 2017 -0800
Committer: gatorsmile 
Committed: Fri Dec 15 09:56:22 2017 -0800

--
 sql/core/src/test/resources/ssb/1.1.sql |  6 ++
 sql/core/src/test/resources/ssb/1.2.sql |  6 ++
 sql/core/src/test/resources/ssb/1.3.sql |  6 ++
 sql/core/src/test/resources/ssb/2.1.sql |  9 ++
 sql/core/src/test/resources/ssb/2.2.sql |  9 ++
 sql/core/src/test/resources/ssb/2.3.sql |  9 ++
 sql/core/src/test/resources/ssb/3.1.sql | 10 +++
 sql/core/src/test/resources/ssb/3.2.sql | 10 +++
 sql/core/src/test/resources/ssb/3.3.sql | 12 +++
 sql/core/src/test/resources/ssb/3.4.sql | 12 +++
 sql/core/src/test/resources/ssb/4.1.sql | 11 +++
 sql/core/src/test/resources/ssb/4.2.sql | 12 +++
 sql/core/src/test/resources/ssb/4.3.sql | 12 +++
 .../org/apache/spark/sql/SSBQuerySuite.scala| 87 
 .../org/apache/spark/sql/TPCHQuerySuite.scala   |  2 -
 15 files changed, 211 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9fafa820/sql/core/src/test/resources/ssb/1.1.sql
--
diff --git a/sql/core/src/test/resources/ssb/1.1.sql 
b/sql/core/src/test/resources/ssb/1.1.sql
new file mode 100644
index 000..62da302
--- /dev/null
+++ b/sql/core/src/test/resources/ssb/1.1.sql
@@ -0,0 +1,6 @@
+select sum(lo_extendedprice*lo_discount) as revenue
+   from lineorder, date
+   where lo_orderdate = d_datekey
+   and d_year = 1993
+   and lo_discount between 1 and 3
+   and lo_quantity < 25

http://git-wip-us.apache.org/repos/asf/spark/blob/9fafa820/sql/core/src/test/resources/ssb/1.2.sql
--
diff --git a/sql/core/src/test/resources/ssb/1.2.sql 
b/sql/core/src/test/resources/ssb/1.2.sql
new file mode 100644
index 000..1657bfd
--- /dev/null
+++ b/sql/core/src/test/resources/ssb/1.2.sql
@@ -0,0 +1,6 @@
+select sum(lo_extendedprice*lo_discount) as revenue
+   from lineorder, date
+   where lo_orderdate = d_datekey
+   and d_yearmonthnum = 199401
+   and lo_discount between 4 and 6
+   and lo_quantity between 26 and 35

http://git-wip-us.apache.org/repos/asf/spark/blob/9fafa820/sql/core/src/test/resources/ssb/1.3.sql
--
diff --git a/sql/core/src/test/resources/ssb/1.3.sql 
b/sql/core/src/test/resources/ssb/1.3.sql
new file mode 100644
index 000..e9bbf51
--- /dev/null
+++ b/sql/core/src/test/resources/ssb/1.3.sql
@@ -0,0 +1,6 @@
+select sum(lo_extendedprice*lo_discount) as revenue
+   from lineorder, date
+   where lo_orderdate = d_datekey
+   and d_weeknuminyear = 6 and d_year = 1994
+   and lo_discount between 5 and 7
+   and lo_quantity between 36 and 40

http://git-wip-us.apache.org/repos/asf/spark/blob/9fafa820/sql/core/src/test/resources/ssb/2.1.sql
--
diff --git a/sql/core/src/test/resources/ssb/2.1.sql 
b/sql/core/src/test/resources/ssb/2.1.sql
new file mode 100644
index 000..00d4027
--- /dev/null
+++ b/sql/core/src/test/resources/ssb/2.1.sql
@@ -0,0 +1,9 @@
+select sum(lo_revenue), d_year, p_brand1
+   from lineorder, date, part, supplier
+   where lo_orderdate = d_datekey
+   and lo_partkey = p_partkey
+   and lo_suppkey = s_suppkey
+   and p_category = 'MFGR#12'
+   and s_region = 'AMERICA'
+   group by d_year, p_brand1
+   order by d_year, p_brand1

http://git-wip-us.apache.org/repos/asf/spark/blob/9fafa820/sql/core/src/test/resources/ssb/2.2.sql

[spark] Git Push Summary

2017-12-15 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/revert19961 [deleted] e58f27567

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



spark git commit: Revert "[SPARK-22496][SQL] thrift server adds operation logs"

2017-12-15 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 3775dd31e -> e58f27567


Revert "[SPARK-22496][SQL] thrift server adds operation logs"

This reverts commit 0ea2d8c12e49e30df6bbfa57d74134b25f96a196.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e58f2756
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e58f2756
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e58f2756

Branch: refs/heads/master
Commit: e58f275678fb4f904124a4a2a1762f04c835eb0e
Parents: 3775dd3
Author: gatorsmile 
Authored: Fri Dec 15 09:46:15 2017 -0800
Committer: gatorsmile 
Committed: Fri Dec 15 09:46:15 2017 -0800

--
 .../cli/operation/ExecuteStatementOperation.java   | 13 -
 .../hive/service/cli/operation/SQLOperation.java   | 12 
 .../thriftserver/SparkExecuteStatementOperation.scala  |  1 -
 3 files changed, 12 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e58f2756/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
--
diff --git 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
index 6740d3b..3f2de10 100644
--- 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
+++ 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
@@ -23,7 +23,6 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.ql.processors.CommandProcessor;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
-import org.apache.hadoop.hive.ql.session.OperationLog;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationType;
 import org.apache.hive.service.cli.session.HiveSession;
@@ -68,16 +67,4 @@ public abstract class ExecuteStatementOperation extends 
Operation {
   this.confOverlay = confOverlay;
 }
   }
-
-  protected void registerCurrentOperationLog() {
-if (isOperationLogEnabled) {
-  if (operationLog == null) {
-LOG.warn("Failed to get current OperationLog object of Operation: " +
-  getHandle().getHandleIdentifier());
-isOperationLogEnabled = false;
-return;
-  }
-  OperationLog.setCurrentOperationLog(operationLog);
-}
-  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/e58f2756/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
--
diff --git 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
index fd9108e..5014ced 100644
--- 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -274,6 +274,18 @@ public class SQLOperation extends 
ExecuteStatementOperation {
 }
   }
 
+  private void registerCurrentOperationLog() {
+if (isOperationLogEnabled) {
+  if (operationLog == null) {
+LOG.warn("Failed to get current OperationLog object of Operation: " +
+getHandle().getHandleIdentifier());
+isOperationLogEnabled = false;
+return;
+  }
+  OperationLog.setCurrentOperationLog(operationLog);
+}
+  }
+
   private void cleanup(OperationState state) throws HiveSQLException {
 setState(state);
 if (shouldRunAsync()) {

http://git-wip-us.apache.org/repos/asf/spark/blob/e58f2756/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
index 664bc20..f5191fa 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
@@ -170,7 +170,6 @@ private[hive] class SparkExecuteStatementOperation(
 override def run(): Unit = {
   val doAsAction = new PrivilegedExceptionAction[Unit]() {
 override def run(): 

spark git commit: Revert "[SPARK-22496][SQL] thrift server adds operation logs"

2017-12-15 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/revert19961 [created] e58f27567


Revert "[SPARK-22496][SQL] thrift server adds operation logs"

This reverts commit 0ea2d8c12e49e30df6bbfa57d74134b25f96a196.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e58f2756
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e58f2756
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e58f2756

Branch: refs/heads/revert19961
Commit: e58f275678fb4f904124a4a2a1762f04c835eb0e
Parents: 3775dd3
Author: gatorsmile 
Authored: Fri Dec 15 09:46:15 2017 -0800
Committer: gatorsmile 
Committed: Fri Dec 15 09:46:15 2017 -0800

--
 .../cli/operation/ExecuteStatementOperation.java   | 13 -
 .../hive/service/cli/operation/SQLOperation.java   | 12 
 .../thriftserver/SparkExecuteStatementOperation.scala  |  1 -
 3 files changed, 12 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e58f2756/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
--
diff --git 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
index 6740d3b..3f2de10 100644
--- 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
+++ 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
@@ -23,7 +23,6 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.ql.processors.CommandProcessor;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
-import org.apache.hadoop.hive.ql.session.OperationLog;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationType;
 import org.apache.hive.service.cli.session.HiveSession;
@@ -68,16 +67,4 @@ public abstract class ExecuteStatementOperation extends 
Operation {
   this.confOverlay = confOverlay;
 }
   }
-
-  protected void registerCurrentOperationLog() {
-if (isOperationLogEnabled) {
-  if (operationLog == null) {
-LOG.warn("Failed to get current OperationLog object of Operation: " +
-  getHandle().getHandleIdentifier());
-isOperationLogEnabled = false;
-return;
-  }
-  OperationLog.setCurrentOperationLog(operationLog);
-}
-  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/e58f2756/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
--
diff --git 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
index fd9108e..5014ced 100644
--- 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -274,6 +274,18 @@ public class SQLOperation extends 
ExecuteStatementOperation {
 }
   }
 
+  private void registerCurrentOperationLog() {
+if (isOperationLogEnabled) {
+  if (operationLog == null) {
+LOG.warn("Failed to get current OperationLog object of Operation: " +
+getHandle().getHandleIdentifier());
+isOperationLogEnabled = false;
+return;
+  }
+  OperationLog.setCurrentOperationLog(operationLog);
+}
+  }
+
   private void cleanup(OperationState state) throws HiveSQLException {
 setState(state);
 if (shouldRunAsync()) {

http://git-wip-us.apache.org/repos/asf/spark/blob/e58f2756/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
index 664bc20..f5191fa 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
@@ -170,7 +170,6 @@ private[hive] class SparkExecuteStatementOperation(
 override def run(): Unit = {
   val doAsAction = new PrivilegedExceptionAction[Unit]() {
 override def 

spark git commit: [SPARK-22753][SQL] Get rid of dataSource.writeAndRead

2017-12-14 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 3fea5c4f1 -> 3775dd31e


[SPARK-22753][SQL] Get rid of dataSource.writeAndRead

## What changes were proposed in this pull request?

As the discussion in https://github.com/apache/spark/pull/16481 and 
https://github.com/apache/spark/pull/18975#discussion_r155454606
Currently the BaseRelation returned by `dataSource.writeAndRead` only used in 
`CreateDataSourceTableAsSelect`, planForWriting and writeAndRead has some 
common code paths.
In this patch I removed the writeAndRead function and added the getRelation 
function which only use in `CreateDataSourceTableAsSelectCommand` while saving 
data to non-existing table.

## How was this patch tested?

Existing UT

Author: Yuanjian Li 

Closes #19941 from xuanyuanking/SPARK-22753.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3775dd31
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3775dd31
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3775dd31

Branch: refs/heads/master
Commit: 3775dd31ee86c32b6161ca99d8fd5cfd7c1a758e
Parents: 3fea5c4
Author: Yuanjian Li 
Authored: Thu Dec 14 23:11:13 2017 -0800
Committer: gatorsmile 
Committed: Thu Dec 14 23:11:13 2017 -0800

--
 .../sql/execution/command/InsertIntoDataSourceDirCommand.scala| 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3775dd31/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala
index 9e35190..1dc24b3 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala
@@ -67,8 +67,7 @@ case class InsertIntoDataSourceDirCommand(
 
 val saveMode = if (overwrite) SaveMode.Overwrite else 
SaveMode.ErrorIfExists
 try {
-  
sparkSession.sessionState.executePlan(dataSource.planForWriting(saveMode, 
query))
-  dataSource.writeAndRead(saveMode, query)
+  
sparkSession.sessionState.executePlan(dataSource.planForWriting(saveMode, 
query)).toRdd
 } catch {
   case ex: AnalysisException =>
 logError(s"Failed to write to directory " + 
storage.locationUri.toString, ex)


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



spark git commit: [SPARK-22787][TEST][SQL] Add a TPC-H query suite

2017-12-14 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 0ea2d8c12 -> 3fea5c4f1


[SPARK-22787][TEST][SQL] Add a TPC-H query suite

## What changes were proposed in this pull request?
Add a test suite to ensure all the TPC-H queries can be successfully analyzed, 
optimized and compiled without hitting the max iteration threshold.

## How was this patch tested?
N/A

Author: gatorsmile 

Closes #19982 from gatorsmile/testTPCH.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3fea5c4f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3fea5c4f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3fea5c4f

Branch: refs/heads/master
Commit: 3fea5c4f19cb5369ff8bbeca80768a8aadb463f5
Parents: 0ea2d8c
Author: gatorsmile 
Authored: Thu Dec 14 22:56:57 2017 -0800
Committer: gatorsmile 
Committed: Thu Dec 14 22:56:57 2017 -0800

--
 .../datasources/text/TextFileFormat.scala   |   7 +-
 sql/core/src/test/resources/tpch/q1.sql |  23 
 sql/core/src/test/resources/tpch/q10.sql|  34 ++
 sql/core/src/test/resources/tpch/q11.sql|  29 +
 sql/core/src/test/resources/tpch/q12.sql|  30 +
 sql/core/src/test/resources/tpch/q13.sql|  22 
 sql/core/src/test/resources/tpch/q14.sql|  15 +++
 sql/core/src/test/resources/tpch/q15.sql|  35 ++
 sql/core/src/test/resources/tpch/q16.sql|  32 ++
 sql/core/src/test/resources/tpch/q17.sql|  19 
 sql/core/src/test/resources/tpch/q18.sql|  35 ++
 sql/core/src/test/resources/tpch/q19.sql|  37 +++
 sql/core/src/test/resources/tpch/q2.sql |  46 
 sql/core/src/test/resources/tpch/q20.sql|  39 +++
 sql/core/src/test/resources/tpch/q21.sql|  42 +++
 sql/core/src/test/resources/tpch/q22.sql|  39 +++
 sql/core/src/test/resources/tpch/q3.sql |  25 +
 sql/core/src/test/resources/tpch/q4.sql |  23 
 sql/core/src/test/resources/tpch/q5.sql |  26 +
 sql/core/src/test/resources/tpch/q6.sql |  11 ++
 sql/core/src/test/resources/tpch/q7.sql |  41 +++
 sql/core/src/test/resources/tpch/q8.sql |  39 +++
 sql/core/src/test/resources/tpch/q9.sql |  34 ++
 .../apache/spark/sql/BenchmarkQueryTest.scala   |  78 +
 .../org/apache/spark/sql/TPCDSQuerySuite.scala  |  58 +-
 .../org/apache/spark/sql/TPCHQuerySuite.scala   | 110 +++
 26 files changed, 872 insertions(+), 57 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3fea5c4f/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
index 8a6ab30..c661e9b 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
@@ -116,9 +116,10 @@ class TextFileFormat extends TextBasedFileFormat with 
DataSourceRegister {
 readToUnsafeMem(broadcastedHadoopConf, requiredSchema, 
textOptions.wholeText)
   }
 
-  private def readToUnsafeMem(conf: Broadcast[SerializableConfiguration],
-  requiredSchema: StructType, wholeTextMode: Boolean):
-  (PartitionedFile) => Iterator[UnsafeRow] = {
+  private def readToUnsafeMem(
+  conf: Broadcast[SerializableConfiguration],
+  requiredSchema: StructType,
+  wholeTextMode: Boolean): (PartitionedFile) => Iterator[UnsafeRow] = {
 
 (file: PartitionedFile) => {
   val confValue = conf.value.value

http://git-wip-us.apache.org/repos/asf/spark/blob/3fea5c4f/sql/core/src/test/resources/tpch/q1.sql
--
diff --git a/sql/core/src/test/resources/tpch/q1.sql 
b/sql/core/src/test/resources/tpch/q1.sql
new file mode 100644
index 000..73eb8d8
--- /dev/null
+++ b/sql/core/src/test/resources/tpch/q1.sql
@@ -0,0 +1,23 @@
+-- using default substitutions
+
+select
+   l_returnflag,
+   l_linestatus,
+   sum(l_quantity) as sum_qty,
+   sum(l_extendedprice) as sum_base_price,
+   sum(l_extendedprice * (1 - l_discount)) as sum_disc_price,
+   sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge,
+   avg(l_quantity) as avg_qty,
+   avg(l_extendedprice) as avg_price,
+   avg(l_discount) as avg_disc,
+   count(*) as count_order
+from
+   lineitem
+where
+   l_shipdate <= date 

spark git commit: [SPARK-22496][SQL] thrift server adds operation logs

2017-12-14 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 59daf91b7 -> 0ea2d8c12


[SPARK-22496][SQL] thrift server adds operation logs

## What changes were proposed in this pull request?
since hive 2.0+  upgrades log4j to log4j2,a lot of 
[changes](https://issues.apache.org/jira/browse/HIVE-11304) are made working on 
it.
as spark is not to ready to update its inner hive version(1.2.1) , so I manage 
to make little changes.
the function registerCurrentOperationLog  is moved from SQLOperstion to its 
parent class ExecuteStatementOperation so spark can use it.

## How was this patch tested?
manual test

Closes #19721 from ChenjunZou/operation-log.

Author: zouchenjun 

Closes #19961 from ChenjunZou/spark-22496.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0ea2d8c1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0ea2d8c1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0ea2d8c1

Branch: refs/heads/master
Commit: 0ea2d8c12e49e30df6bbfa57d74134b25f96a196
Parents: 59daf91
Author: zouchenjun 
Authored: Thu Dec 14 15:37:26 2017 -0800
Committer: gatorsmile 
Committed: Thu Dec 14 15:37:26 2017 -0800

--
 .../cli/operation/ExecuteStatementOperation.java   | 13 +
 .../hive/service/cli/operation/SQLOperation.java   | 12 
 .../thriftserver/SparkExecuteStatementOperation.scala  |  1 +
 3 files changed, 14 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0ea2d8c1/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
--
diff --git 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
index 3f2de10..6740d3b 100644
--- 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
+++ 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
@@ -23,6 +23,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.ql.processors.CommandProcessor;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
+import org.apache.hadoop.hive.ql.session.OperationLog;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationType;
 import org.apache.hive.service.cli.session.HiveSession;
@@ -67,4 +68,16 @@ public abstract class ExecuteStatementOperation extends 
Operation {
   this.confOverlay = confOverlay;
 }
   }
+
+  protected void registerCurrentOperationLog() {
+if (isOperationLogEnabled) {
+  if (operationLog == null) {
+LOG.warn("Failed to get current OperationLog object of Operation: " +
+  getHandle().getHandleIdentifier());
+isOperationLogEnabled = false;
+return;
+  }
+  OperationLog.setCurrentOperationLog(operationLog);
+}
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0ea2d8c1/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
--
diff --git 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
index 5014ced..fd9108e 100644
--- 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -274,18 +274,6 @@ public class SQLOperation extends 
ExecuteStatementOperation {
 }
   }
 
-  private void registerCurrentOperationLog() {
-if (isOperationLogEnabled) {
-  if (operationLog == null) {
-LOG.warn("Failed to get current OperationLog object of Operation: " +
-getHandle().getHandleIdentifier());
-isOperationLogEnabled = false;
-return;
-  }
-  OperationLog.setCurrentOperationLog(operationLog);
-}
-  }
-
   private void cleanup(OperationState state) throws HiveSQLException {
 setState(state);
 if (shouldRunAsync()) {

http://git-wip-us.apache.org/repos/asf/spark/blob/0ea2d8c1/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
 

spark git commit: [SPARK-16496][SQL] Add wholetext as option for reading text in SQL.

2017-12-14 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 606ae491e -> 40de176c9


[SPARK-16496][SQL] Add wholetext as option for reading text in SQL.

## What changes were proposed in this pull request?

In multiple text analysis problems, it is not often desirable for the rows to 
be split by "\n". There exists a wholeText reader for RDD API, and this JIRA 
just adds the same support for Dataset API.
## How was this patch tested?

Added relevant new tests for both scala and Java APIs

Author: Prashant Sharma 
Author: Prashant Sharma 

Closes #14151 from ScrapCodes/SPARK-16496/wholetext.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/40de176c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/40de176c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/40de176c

Branch: refs/heads/master
Commit: 40de176c93c5aa05bcbb1328721118b6b46ba51d
Parents: 606ae49
Author: Prashant Sharma 
Authored: Thu Dec 14 11:19:34 2017 -0800
Committer: gatorsmile 
Committed: Thu Dec 14 11:19:34 2017 -0800

--
 python/pyspark/sql/readwriter.py|   7 +-
 .../org/apache/spark/sql/DataFrameReader.scala  |  16 ++-
 .../datasources/HadoopFileWholeTextReader.scala |  57 ++
 .../datasources/text/TextFileFormat.scala   |  31 +-
 .../datasources/text/TextOptions.scala  |   7 ++
 .../execution/datasources/text/TextSuite.scala  |   5 +-
 .../datasources/text/WholeTextFileSuite.scala   | 108 +++
 7 files changed, 221 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/40de176c/python/pyspark/sql/readwriter.py
--
diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py
index 1ad974e..4e58bfb 100644
--- a/python/pyspark/sql/readwriter.py
+++ b/python/pyspark/sql/readwriter.py
@@ -304,7 +304,7 @@ class DataFrameReader(OptionUtils):
 
 @ignore_unicode_prefix
 @since(1.6)
-def text(self, paths):
+def text(self, paths, wholetext=False):
 """
 Loads text files and returns a :class:`DataFrame` whose schema starts 
with a
 string column named "value", and followed by partitioned columns if 
there
@@ -313,11 +313,16 @@ class DataFrameReader(OptionUtils):
 Each line in the text file is a new row in the resulting DataFrame.
 
 :param paths: string, or list of strings, for input path(s).
+:param wholetext: if true, read each file from input path(s) as a 
single row.
 
 >>> df = spark.read.text('python/test_support/sql/text-test.txt')
 >>> df.collect()
 [Row(value=u'hello'), Row(value=u'this')]
+>>> df = spark.read.text('python/test_support/sql/text-test.txt', 
wholetext=True)
+>>> df.collect()
+[Row(value=u'hello\\nthis')]
 """
+self._set_opts(wholetext=wholetext)
 if isinstance(paths, basestring):
 paths = [paths]
 return 
self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(paths)))

http://git-wip-us.apache.org/repos/asf/spark/blob/40de176c/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
index ea1cf66..39fec8f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
@@ -646,7 +646,14 @@ class DataFrameReader private[sql](sparkSession: 
SparkSession) extends Logging {
* Loads text files and returns a `DataFrame` whose schema starts with a 
string column named
* "value", and followed by partitioned columns if there are any.
*
-   * Each line in the text files is a new row in the resulting DataFrame. For 
example:
+   * You can set the following text-specific option(s) for reading text files:
+   * 
+   * `wholetext` ( default `false`): If true, read a file as a single row 
and not split by "\n".
+   * 
+   * 
+   * By default, each line in the text files is a new row in the resulting 
DataFrame.
+   *
+   * Usage example:
* {{{
*   // Scala:
*   spark.read.text("/path/to/spark/README.md")
@@ -678,7 +685,12 @@ class DataFrameReader private[sql](sparkSession: 
SparkSession) extends Logging {
* If the directory structure of the text files contains partitioning 
information, those are
* ignored in the resulting Dataset. To include partitioning information as 
columns, use `text`.
*
-   * Each line in the text files is a new element in the resulting 

spark git commit: [SPARK-22779][SQL] Resolve default values for fallback configs.

2017-12-13 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master f8c7c1f21 -> c3dd2a26d


[SPARK-22779][SQL] Resolve default values for fallback configs.

SQLConf allows some callers to define a custom default value for
configs, and that complicates a little bit the handling of fallback
config entries, since most of the default value resolution is
hidden by the config code.

This change peaks into the internals of these fallback configs
to figure out the correct default value, and also returns the
current human-readable default when showing the default value
(e.g. through "set -v").

Author: Marcelo Vanzin 

Closes #19974 from vanzin/SPARK-22779.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c3dd2a26
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c3dd2a26
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c3dd2a26

Branch: refs/heads/master
Commit: c3dd2a26deaadf508b4e163eab2c0544cd922540
Parents: f8c7c1f
Author: Marcelo Vanzin 
Authored: Wed Dec 13 22:46:20 2017 -0800
Committer: gatorsmile 
Committed: Wed Dec 13 22:46:20 2017 -0800

--
 .../spark/internal/config/ConfigEntry.scala |  8 --
 .../org/apache/spark/sql/internal/SQLConf.scala | 16 ---
 .../spark/sql/internal/SQLConfSuite.scala   | 30 
 3 files changed, 47 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c3dd2a26/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala 
b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala
index f119028..ede3ace 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala
@@ -139,7 +139,7 @@ private[spark] class OptionalConfigEntry[T](
 s => Some(rawValueConverter(s)),
 v => v.map(rawStringConverter).orNull, doc, isPublic) {
 
-  override def defaultValueString: String = ""
+  override def defaultValueString: String = ConfigEntry.UNDEFINED
 
   override def readFrom(reader: ConfigReader): Option[T] = {
 readString(reader).map(rawValueConverter)
@@ -149,12 +149,12 @@ private[spark] class OptionalConfigEntry[T](
 /**
  * A config entry whose default value is defined by another config entry.
  */
-private class FallbackConfigEntry[T] (
+private[spark] class FallbackConfigEntry[T] (
 key: String,
 alternatives: List[String],
 doc: String,
 isPublic: Boolean,
-private[config] val fallback: ConfigEntry[T])
+val fallback: ConfigEntry[T])
   extends ConfigEntry[T](key, alternatives,
 fallback.valueConverter, fallback.stringConverter, doc, isPublic) {
 
@@ -167,6 +167,8 @@ private class FallbackConfigEntry[T] (
 
 private[spark] object ConfigEntry {
 
+  val UNDEFINED = ""
+
   private val knownConfigs = new 
java.util.concurrent.ConcurrentHashMap[String, ConfigEntry[_]]()
 
   def registerEntry(entry: ConfigEntry[_]): Unit = {

http://git-wip-us.apache.org/repos/asf/spark/blob/c3dd2a26/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 1121444..cf7e3eb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -1379,7 +1379,7 @@ class SQLConf extends Serializable with Logging {
 Option(settings.get(key)).
   orElse {
 // Try to use the default value
-Option(sqlConfEntries.get(key)).map(_.defaultValueString)
+Option(sqlConfEntries.get(key)).map { e => 
e.stringConverter(e.readFrom(reader)) }
   }.
   getOrElse(throw new NoSuchElementException(key))
   }
@@ -1417,14 +1417,21 @@ class SQLConf extends Serializable with Logging {
* not set yet, return `defaultValue`.
*/
   def getConfString(key: String, defaultValue: String): String = {
-if (defaultValue != null && defaultValue != "") {
+if (defaultValue != null && defaultValue != ConfigEntry.UNDEFINED) {
   val entry = sqlConfEntries.get(key)
   if (entry != null) {
 // Only verify configs in the SQLConf object
 entry.valueConverter(defaultValue)
   }
 }
-Option(settings.get(key)).getOrElse(defaultValue)
+Option(settings.get(key)).getOrElse {
+  // If the key is not set, need to check whether the config entry is 
registered and is
+  // a 

spark git commit: [SPARK-22600][SQL][FOLLOW-UP] Fix a compilation error in TPCDS q75/q77

2017-12-13 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master a83e8e6c2 -> ef9299965


[SPARK-22600][SQL][FOLLOW-UP] Fix a compilation error in TPCDS q75/q77

## What changes were proposed in this pull request?
This pr fixed a compilation error of TPCDS `q75`/`q77`  caused by #19813;
```
  java.util.concurrent.ExecutionException: 
org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 
371, Column 16: failed to compile: 
org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 
371, Column 16: Expression "bhj_matched" is not an rvalue
  at 
com.google.common.util.concurrent.AbstractFuture$Sync.getValue(AbstractFuture.java:306)
  at 
com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:293)
  at 
com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116)
  at 
com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:135)
```

## How was this patch tested?
Manually checked `q75`/`q77` can be properly compiled

Author: Takeshi Yamamuro 

Closes #19969 from maropu/SPARK-22600-FOLLOWUP.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ef929996
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ef929996
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ef929996

Branch: refs/heads/master
Commit: ef92999653f0e2a47752379a867647445d849aab
Parents: a83e8e6
Author: Takeshi Yamamuro 
Authored: Wed Dec 13 15:55:16 2017 -0800
Committer: gatorsmile 
Committed: Wed Dec 13 15:55:16 2017 -0800

--
 .../spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala | 1 -
 .../apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala   | 2 +-
 2 files changed, 1 insertion(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ef929996/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala
index a2dda48..807cb94 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/ExpressionCodegen.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.expressions.codegen
 import scala.collection.mutable
 
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.types.DataType
 
 /**
  * Defines util methods used in expression code generation.

http://git-wip-us.apache.org/repos/asf/spark/blob/ef929996/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
index c96ed6e..634014a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
@@ -192,7 +192,7 @@ case class BroadcastHashJoinExec(
   |  $value = ${ev.value};
   |}
  """.stripMargin
-ExprCode(code, isNull, value)
+ExprCode(code, isNull, value, inputRow = matched)
   }
 }
   }


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



spark git commit: [SPARK-22772][SQL] Use splitExpressionsWithCurrentInputs to split codes in elt

2017-12-13 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 0bdb4e516 -> ba0e79f57


[SPARK-22772][SQL] Use splitExpressionsWithCurrentInputs to split codes in elt

## What changes were proposed in this pull request?

In SPARK-22550 which fixes 64KB JVM bytecode limit problem with elt, 
`buildCodeBlocks` is used to split codes. However, we should use 
`splitExpressionsWithCurrentInputs` because it considers both normal and 
wholestage codgen (it is not supported yet, so it simply doesn't split the 
codes).

## How was this patch tested?

Existing tests.

Author: Liang-Chi Hsieh 

Closes #19964 from viirya/SPARK-22772.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ba0e79f5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ba0e79f5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ba0e79f5

Branch: refs/heads/master
Commit: ba0e79f57caa279773fb014b7883ee5d69dd0a68
Parents: 0bdb4e5
Author: Liang-Chi Hsieh 
Authored: Wed Dec 13 13:54:16 2017 -0800
Committer: gatorsmile 
Committed: Wed Dec 13 13:54:16 2017 -0800

--
 .../expressions/codegen/CodeGenerator.scala |  2 +-
 .../expressions/stringExpressions.scala | 81 ++--
 2 files changed, 43 insertions(+), 40 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ba0e79f5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index 257c3f1..b1d9311 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -878,7 +878,7 @@ class CodegenContext {
*
* @param expressions the codes to evaluate expressions.
*/
-  def buildCodeBlocks(expressions: Seq[String]): Seq[String] = {
+  private def buildCodeBlocks(expressions: Seq[String]): Seq[String] = {
 val blocks = new ArrayBuffer[String]()
 val blockBuilder = new StringBuilder()
 var length = 0

http://git-wip-us.apache.org/repos/asf/spark/blob/ba0e79f5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
index 47f0b57..8c4d2fd 100755
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
@@ -289,53 +289,56 @@ case class Elt(children: Seq[Expression])
 val index = indexExpr.genCode(ctx)
 val strings = stringExprs.map(_.genCode(ctx))
 val indexVal = ctx.freshName("index")
+val indexMatched = ctx.freshName("eltIndexMatched")
+
 val stringVal = ctx.freshName("stringVal")
+ctx.addMutableState(ctx.javaType(dataType), stringVal)
+
 val assignStringValue = strings.zipWithIndex.map { case (eval, index) =>
   s"""
-case ${index + 1}:
-  ${eval.code}
-  $stringVal = ${eval.isNull} ? null : ${eval.value};
-  break;
-  """
+ |if ($indexVal == ${index + 1}) {
+ |  ${eval.code}
+ |  $stringVal = ${eval.isNull} ? null : ${eval.value};
+ |  $indexMatched = true;
+ |  continue;
+ |}
+  """.stripMargin
 }
 
-val cases = ctx.buildCodeBlocks(assignStringValue)
-val codes = if (cases.length == 1) {
-  s"""
-UTF8String $stringVal = null;
-switch ($indexVal) {
-  ${cases.head}
-}
-   """
-} else {
-  var prevFunc = "null"
-  for (c <- cases.reverse) {
-val funcName = ctx.freshName("eltFunc")
-val funcBody = s"""
- private UTF8String $funcName(InternalRow ${ctx.INPUT_ROW}, int 
$indexVal) {
-   UTF8String $stringVal = null;
-   switch ($indexVal) {
- $c
- default:
-   return $prevFunc;
-   }
-   return $stringVal;
- }
-"""
-val fullFuncName = ctx.addNewFunction(funcName, funcBody)
-prevFunc = s"$fullFuncName(${ctx.INPUT_ROW}, $indexVal)"
-  }
-  s"UTF8String $stringVal = $prevFunc;"
-}
+val codes = ctx.splitExpressionsWithCurrentInputs(
+  expressions 

spark git commit: [SPARK-22763][CORE] SHS: Ignore unknown events and parse through the file

2017-12-13 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master c5a4701ac -> 1abcbed67


[SPARK-22763][CORE] SHS: Ignore unknown events and parse through the file

## What changes were proposed in this pull request?

While spark code changes, there are new events in event log: #19649
And we used to maintain a whitelist to avoid exceptions: #15663
Currently Spark history server will stop parsing on unknown events or 
unrecognized properties. We may still see part of the UI data.
For better compatibility, we can ignore unknown events and parse through the 
log file.

## How was this patch tested?
Unit test

Author: Wang Gengliang 

Closes #19953 from gengliangwang/ReplayListenerBus.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1abcbed6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1abcbed6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1abcbed6

Branch: refs/heads/master
Commit: 1abcbed678c2bc4f05640db2791fd2d84267d740
Parents: c5a4701
Author: Wang Gengliang 
Authored: Wed Dec 13 11:54:22 2017 -0800
Committer: gatorsmile 
Committed: Wed Dec 13 11:54:22 2017 -0800

--
 .../spark/scheduler/ReplayListenerBus.scala | 37 ++--
 .../spark/scheduler/ReplayListenerSuite.scala   | 29 +++
 2 files changed, 47 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1abcbed6/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala 
b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
index 26a6a3e..c9cd662 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
@@ -69,6 +69,8 @@ private[spark] class ReplayListenerBus extends 
SparkListenerBus with Logging {
   eventsFilter: ReplayEventsFilter): Unit = {
 var currentLine: String = null
 var lineNumber: Int = 0
+val unrecognizedEvents = new scala.collection.mutable.HashSet[String]
+val unrecognizedProperties = new scala.collection.mutable.HashSet[String]
 
 try {
   val lineEntries = lines
@@ -84,16 +86,22 @@ private[spark] class ReplayListenerBus extends 
SparkListenerBus with Logging {
 
   postToAll(JsonProtocol.sparkEventFromJson(parse(currentLine)))
 } catch {
-  case e: ClassNotFoundException if 
KNOWN_REMOVED_CLASSES.contains(e.getMessage) =>
-// Ignore events generated by Structured Streaming in Spark 2.0.0 
and 2.0.1.
-// It's safe since no place uses them.
-logWarning(s"Dropped incompatible Structured Streaming log: 
$currentLine")
-  case e: UnrecognizedPropertyException if e.getMessage != null && 
e.getMessage.startsWith(
-"Unrecognized field \"queryStatus\" " +
-  "(class org.apache.spark.sql.streaming.StreamingQueryListener$") 
=>
-// Ignore events generated by Structured Streaming in Spark 2.0.2
-// It's safe since no place uses them.
-logWarning(s"Dropped incompatible Structured Streaming log: 
$currentLine")
+  case e: ClassNotFoundException =>
+// Ignore unknown events, parse through the event log file.
+// To avoid spamming, warnings are only displayed once for each 
unknown event.
+if (!unrecognizedEvents.contains(e.getMessage)) {
+  logWarning(s"Drop unrecognized event: ${e.getMessage}")
+  unrecognizedEvents.add(e.getMessage)
+}
+logDebug(s"Drop incompatible event log: $currentLine")
+  case e: UnrecognizedPropertyException =>
+// Ignore unrecognized properties, parse through the event log 
file.
+// To avoid spamming, warnings are only displayed once for each 
unrecognized property.
+if (!unrecognizedProperties.contains(e.getMessage)) {
+  logWarning(s"Drop unrecognized property: ${e.getMessage}")
+  unrecognizedProperties.add(e.getMessage)
+}
+logDebug(s"Drop incompatible event log: $currentLine")
   case jpe: JsonParseException =>
 // We can only ignore exception from last line of the file that 
might be truncated
 // the last entry may not be the very last line in the event log, 
but we treat it
@@ -125,13 +133,4 @@ private[spark] object ReplayListenerBus {
 
   // utility filter that selects all event logs during replay
   val SELECT_ALL_FILTER: ReplayEventsFilter = { (eventString: String) => true }
-
-  /**
-   * Classes that were removed. Structured 

spark git commit: Revert "[SPARK-21417][SQL] Infer join conditions using propagated constraints"

2017-12-13 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 8eb5609d8 -> c5a4701ac


Revert "[SPARK-21417][SQL] Infer join conditions using propagated constraints"

This reverts commit 6ac57fd0d1c82b834eb4bf0dd57596b92a99d6de.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c5a4701a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c5a4701a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c5a4701a

Branch: refs/heads/master
Commit: c5a4701acc6972ed7ccb11c506fe718d5503f140
Parents: 8eb5609
Author: gatorsmile 
Authored: Wed Dec 13 11:50:04 2017 -0800
Committer: gatorsmile 
Committed: Wed Dec 13 11:50:04 2017 -0800

--
 .../expressions/EquivalentExpressionMap.scala   |  66 -
 .../catalyst/expressions/ExpressionSet.scala|   2 -
 .../sql/catalyst/optimizer/Optimizer.scala  |   1 -
 .../spark/sql/catalyst/optimizer/joins.scala|  60 -
 .../EquivalentExpressionMapSuite.scala  |  56 -
 .../optimizer/EliminateCrossJoinSuite.scala | 238 ---
 6 files changed, 423 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c5a4701a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala
deleted file mode 100644
index cf1614a..000
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala
+++ /dev/null
@@ -1,66 +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.expressions
-
-import scala.collection.mutable
-
-import 
org.apache.spark.sql.catalyst.expressions.EquivalentExpressionMap.SemanticallyEqualExpr
-
-/**
- * A class that allows you to map an expression into a set of equivalent 
expressions. The keys are
- * handled based on their semantic meaning and ignoring cosmetic differences. 
The values are
- * represented as [[ExpressionSet]]s.
- *
- * The underlying representation of keys depends on the 
[[Expression.semanticHash]] and
- * [[Expression.semanticEquals]] methods.
- *
- * {{{
- *   val map = new EquivalentExpressionMap()
- *
- *   map.put(1 + 2, a)
- *   map.put(rand(), b)
- *
- *   map.get(2 + 1) => Set(a) // 1 + 2 and 2 + 1 are semantically equivalent
- *   map.get(1 + 2) => Set(a) // 1 + 2 and 2 + 1 are semantically equivalent
- *   map.get(rand()) => Set() // non-deterministic expressions are not 
equivalent
- * }}}
- */
-class EquivalentExpressionMap {
-
-  private val equivalenceMap = mutable.HashMap.empty[SemanticallyEqualExpr, 
ExpressionSet]
-
-  def put(expression: Expression, equivalentExpression: Expression): Unit = {
-val equivalentExpressions = equivalenceMap.getOrElseUpdate(expression, 
ExpressionSet.empty)
-equivalenceMap(expression) = equivalentExpressions + equivalentExpression
-  }
-
-  def get(expression: Expression): Set[Expression] =
-equivalenceMap.getOrElse(expression, ExpressionSet.empty)
-}
-
-object EquivalentExpressionMap {
-
-  private implicit class SemanticallyEqualExpr(val expr: Expression) {
-override def equals(obj: Any): Boolean = obj match {
-  case other: SemanticallyEqualExpr => expr.semanticEquals(other.expr)
-  case _ => false
-}
-
-override def hashCode: Int = expr.semanticHash()
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/c5a4701a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala
index e989083..7e8e7b8 100644
--- 

spark git commit: [SPARK-22042][SQL] ReorderJoinPredicates can break when child's partitioning is not decided

2017-12-12 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 874350905 -> 682eb4f2e


[SPARK-22042][SQL] ReorderJoinPredicates can break when child's partitioning is 
not decided

## What changes were proposed in this pull request?

See jira description for the bug : 
https://issues.apache.org/jira/browse/SPARK-22042

Fix done in this PR is:  In `EnsureRequirements`, apply `ReorderJoinPredicates` 
over the input tree before doing its core logic. Since the tree is transformed 
bottom-up, we can assure that the children are resolved before doing 
`ReorderJoinPredicates`.

Theoretically this will guarantee to cover all such cases while keeping the 
code simple. My small grudge is for cosmetic reasons. This PR will look weird 
given that we don't call rules from other rules (not to my knowledge). I could 
have moved all the logic for `ReorderJoinPredicates` into `EnsureRequirements` 
but that will make it a but crowded. I am happy to discuss if there are better 
options.

## How was this patch tested?

Added a new test case

Author: Tejas Patil 

Closes #19257 from tejasapatil/SPARK-22042_ReorderJoinPredicates.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/682eb4f2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/682eb4f2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/682eb4f2

Branch: refs/heads/master
Commit: 682eb4f2ea152ce1043fbe689ea95318926b91b0
Parents: 8743509
Author: Tejas Patil 
Authored: Tue Dec 12 23:30:06 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 12 23:30:06 2017 -0800

--
 .../spark/sql/execution/QueryExecution.scala|  2 -
 .../execution/exchange/EnsureRequirements.scala | 76 +++-
 .../execution/joins/ReorderJoinPredicates.scala | 94 
 .../spark/sql/sources/BucketedReadSuite.scala   | 31 +++
 4 files changed, 106 insertions(+), 97 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/682eb4f2/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
index f404621..946475a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
@@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.catalyst.util.DateTimeUtils
 import org.apache.spark.sql.execution.command.{DescribeTableCommand, 
ExecutedCommandExec, ShowTablesCommand}
 import org.apache.spark.sql.execution.exchange.{EnsureRequirements, 
ReuseExchange}
-import org.apache.spark.sql.execution.joins.ReorderJoinPredicates
 import org.apache.spark.sql.types.{BinaryType, DateType, DecimalType, 
TimestampType, _}
 import org.apache.spark.util.Utils
 
@@ -104,7 +103,6 @@ class QueryExecution(val sparkSession: SparkSession, val 
logical: LogicalPlan) {
   protected def preparations: Seq[Rule[SparkPlan]] = Seq(
 python.ExtractPythonUDFs,
 PlanSubqueries(sparkSession),
-new ReorderJoinPredicates,
 EnsureRequirements(sparkSession.sessionState.conf),
 CollapseCodegenStages(sparkSession.sessionState.conf),
 ReuseExchange(sparkSession.sessionState.conf),

http://git-wip-us.apache.org/repos/asf/spark/blob/682eb4f2/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
index 4e2ca37..82f0b9f 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
@@ -17,10 +17,14 @@
 
 package org.apache.spark.sql.execution.exchange
 
+import scala.collection.mutable.ArrayBuffer
+
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, 
ShuffledHashJoinExec,
+  SortMergeJoinExec}
 import org.apache.spark.sql.internal.SQLConf
 
 /**
@@ -248,6 +252,75 @@ case class EnsureRequirements(conf: SQLConf) extends 
Rule[SparkPlan] {
 operator.withNewChildren(children)
   }
 
+  /**
+   * When the physical operators are created for JOIN, the ordering of join 
keys is based on order
+ 

spark git commit: [SPARK-22759][SQL] Filters can be combined iff both are deterministic

2017-12-12 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 6b80ce4fb -> 13e489b67


[SPARK-22759][SQL] Filters can be combined iff both are deterministic

## What changes were proposed in this pull request?
The query execution/optimization does not guarantee the expressions are 
evaluated in order. We only can combine them if and only if both are 
deterministic. We need to update the optimizer rule: CombineFilters.

## How was this patch tested?
Updated the existing tests.

Author: gatorsmile 

Closes #19947 from gatorsmile/combineFilters.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/13e489b6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/13e489b6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/13e489b6

Branch: refs/heads/master
Commit: 13e489b6754f4d3569dad99bf5be2d5b0914dd68
Parents: 6b80ce4
Author: gatorsmile 
Authored: Tue Dec 12 22:48:31 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 12 22:48:31 2017 -0800

--
 .../apache/spark/sql/catalyst/optimizer/Optimizer.scala |  4 +++-
 .../sql/catalyst/optimizer/FilterPushdownSuite.scala| 12 
 2 files changed, 7 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/13e489b6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 484cd8c..5776935 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -695,7 +695,9 @@ object CombineUnions extends Rule[LogicalPlan] {
  */
 object CombineFilters extends Rule[LogicalPlan] with PredicateHelper {
   def apply(plan: LogicalPlan): LogicalPlan = plan transform {
-case Filter(fc, nf @ Filter(nc, grandChild)) =>
+// The query execution/optimization does not guarantee the expressions are 
evaluated in order.
+// We only can combine them if and only if both are deterministic.
+case Filter(fc, nf @ Filter(nc, grandChild)) if fc.deterministic && 
nc.deterministic =>
   (ExpressionSet(splitConjunctivePredicates(fc)) --
 ExpressionSet(splitConjunctivePredicates(nc))).reduceOption(And) match 
{
 case Some(ac) =>

http://git-wip-us.apache.org/repos/asf/spark/blob/13e489b6/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
index de0e7c7..641824e 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
@@ -94,19 +94,15 @@ class FilterPushdownSuite extends PlanTest {
 comparePlans(optimized, correctAnswer)
   }
 
-  test("combine redundant deterministic filters") {
+  test("do not combine non-deterministic filters even if they are identical") {
 val originalQuery =
   testRelation
 .where(Rand(0) > 0.1 && 'a === 1)
-.where(Rand(0) > 0.1 && 'a === 1)
+.where(Rand(0) > 0.1 && 'a === 1).analyze
 
-val optimized = Optimize.execute(originalQuery.analyze)
-val correctAnswer =
-  testRelation
-.where(Rand(0) > 0.1 && 'a === 1 && Rand(0) > 0.1)
-.analyze
+val optimized = Optimize.execute(originalQuery)
 
-comparePlans(optimized, correctAnswer)
+comparePlans(optimized, originalQuery)
   }
 
   test("SPARK-16164: Filter pushdown should keep the ordering in the logical 
plan") {


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



spark git commit: [SPARK-19809][SQL][TEST][FOLLOWUP] Move the test case to HiveOrcQuerySuite

2017-12-12 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 0e36ba621 -> 6b80ce4fb


[SPARK-19809][SQL][TEST][FOLLOWUP] Move the test case to HiveOrcQuerySuite

## What changes were proposed in this pull request?

As a follow-up of #19948 , this PR moves the test case and adds comments.

## How was this patch tested?

Pass the Jenkins.

Author: Dongjoon Hyun 

Closes #19960 from dongjoon-hyun/SPARK-19809-2.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6b80ce4f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6b80ce4f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6b80ce4f

Branch: refs/heads/master
Commit: 6b80ce4fb20da57c9513b94ab02b53a5fd7571d0
Parents: 0e36ba6
Author: Dongjoon Hyun 
Authored: Tue Dec 12 22:41:38 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 12 22:41:38 2017 -0800

--
 .../sql/hive/execution/SQLQuerySuite.scala  | 36 ---
 .../spark/sql/hive/orc/HiveOrcQuerySuite.scala  | 48 +++-
 2 files changed, 47 insertions(+), 37 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6b80ce4f/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index 93c91d3..c11e37a 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -2153,40 +2153,4 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
   }
 }
   }
-
-  test("SPARK-22267 Spark SQL incorrectly reads ORC files when column order is 
different") {
-Seq("native", "hive").foreach { orcImpl =>
-  withSQLConf(SQLConf.ORC_IMPLEMENTATION.key -> orcImpl) {
-withTempPath { f =>
-  val path = f.getCanonicalPath
-  Seq(1 -> 2).toDF("c1", "c2").write.orc(path)
-  checkAnswer(spark.read.orc(path), Row(1, 2))
-
-  withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "true") { // 
default since 2.3.0
-withTable("t") {
-  sql(s"CREATE EXTERNAL TABLE t(c2 INT, c1 INT) STORED AS ORC 
LOCATION '$path'")
-  checkAnswer(spark.table("t"), Row(2, 1))
-}
-  }
-}
-  }
-}
-  }
-
-  test("SPARK-19809 NullPointerException on zero-size ORC file") {
-Seq("native", "hive").foreach { orcImpl =>
-  withSQLConf(SQLConf.ORC_IMPLEMENTATION.key -> orcImpl) {
-withTempPath { dir =>
-  withTable("spark_19809") {
-sql(s"CREATE TABLE spark_19809(a int) STORED AS ORC LOCATION 
'$dir'")
-Files.touch(new File(s"${dir.getCanonicalPath}", "zero.orc"))
-
-withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "true") { // 
default since 2.3.0
-  checkAnswer(sql("SELECT * FROM spark_19809"), Seq.empty)
-}
-  }
-}
-  }
-}
-  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/6b80ce4f/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala
index 7244c36..92b2f06 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala
@@ -17,7 +17,11 @@
 
 package org.apache.spark.sql.hive.orc
 
-import org.apache.spark.sql.AnalysisException
+import java.io.File
+
+import com.google.common.io.Files
+
+import org.apache.spark.sql.{AnalysisException, Row}
 import org.apache.spark.sql.catalyst.catalog.HiveTableRelation
 import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, 
LogicalRelation}
 import org.apache.spark.sql.execution.datasources.orc.OrcQueryTest
@@ -162,4 +166,46 @@ class HiveOrcQuerySuite extends OrcQueryTest with 
TestHiveSingleton {
 }
 }
   }
+
+  // Since Hive 1.2.1 library code path still has this problem, users may hit 
this
+  // when spark.sql.hive.convertMetastoreOrc=false. However, after SPARK-22279,
+  // Apache Spark with the default configuration doesn't hit this bug.
+  test("SPARK-22267 Spark SQL incorrectly reads ORC files when column order is 
different") {
+Seq("native", "hive").foreach { orcImpl =>
+  withSQLConf(SQLConf.ORC_IMPLEMENTATION.key -> orcImpl) {
+withTempPath { 

spark git commit: Revert "[SPARK-22574][MESOS][SUBMIT] Check submission request parameters"

2017-12-12 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.2 728a45e5a -> 0230515a2


Revert "[SPARK-22574][MESOS][SUBMIT] Check submission request parameters"

This reverts commit 728a45e5a68a20bdd17227edc70e6a38d178af1c.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0230515a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0230515a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0230515a

Branch: refs/heads/branch-2.2
Commit: 0230515a2df7996cde1329a2f574b9a349b93452
Parents: 728a45e
Author: gatorsmile 
Authored: Tue Dec 12 13:42:21 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 12 13:42:21 2017 -0800

--
 .../spark/deploy/rest/SubmitRestProtocolRequest.scala   |  2 --
 .../spark/deploy/rest/SubmitRestProtocolSuite.scala |  2 --
 .../spark/deploy/rest/mesos/MesosRestServer.scala   | 12 
 3 files changed, 4 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0230515a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
 
b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
index 86ddf95..0d50a76 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
@@ -46,8 +46,6 @@ private[rest] class CreateSubmissionRequest extends 
SubmitRestProtocolRequest {
 super.doValidate()
 assert(sparkProperties != null, "No Spark properties set!")
 assertFieldIsSet(appResource, "appResource")
-assertFieldIsSet(appArgs, "appArgs")
-assertFieldIsSet(environmentVariables, "environmentVariables")
 assertPropertyIsSet("spark.app.name")
 assertPropertyIsBoolean("spark.driver.supervise")
 assertPropertyIsNumeric("spark.driver.cores")

http://git-wip-us.apache.org/repos/asf/spark/blob/0230515a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
 
b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
index 75c50af..725b884 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
@@ -86,8 +86,6 @@ class SubmitRestProtocolSuite extends SparkFunSuite {
 message.clientSparkVersion = "1.2.3"
 message.appResource = "honey-walnut-cherry.jar"
 message.mainClass = "org.apache.spark.examples.SparkPie"
-message.appArgs = Array("two slices")
-message.environmentVariables = Map("PATH" -> "/dev/null")
 val conf = new SparkConf(false)
 conf.set("spark.app.name", "SparkPie")
 message.sparkProperties = conf.getAll.toMap

http://git-wip-us.apache.org/repos/asf/spark/blob/0230515a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
--
diff --git 
a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
 
b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
index bb8dfee..ff60b88 100644
--- 
a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
+++ 
b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
@@ -77,16 +77,10 @@ private[mesos] class MesosSubmitRequestServlet(
   private def buildDriverDescription(request: CreateSubmissionRequest): 
MesosDriverDescription = {
 // Required fields, including the main class because python is not yet 
supported
 val appResource = Option(request.appResource).getOrElse {
-  throw new SubmitRestMissingFieldException("Application jar 'appResource' 
is missing.")
+  throw new SubmitRestMissingFieldException("Application jar is missing.")
 }
 val mainClass = Option(request.mainClass).getOrElse {
-  throw new SubmitRestMissingFieldException("Main class 'mainClass' is 
missing.")
-}
-val appArgs = Option(request.appArgs).getOrElse {
-  throw new SubmitRestMissingFieldException("Application arguments 
'appArgs' are missing.")
-}
-val environmentVariables = Option(request.environmentVariables).getOrElse {
-  throw new SubmitRestMissingFieldException("Environment variables 
'environmentVariables' are missing.")
+  throw new 

spark git commit: Revert "[SPARK-22574][MESOS][SUBMIT] Check submission request parameters"

2017-12-12 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 7a51e7135 -> 704af4bd6


Revert "[SPARK-22574][MESOS][SUBMIT] Check submission request parameters"

This reverts commit 7a51e71355485bb176a1387d99ec430c5986cbec.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/704af4bd
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/704af4bd
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/704af4bd

Branch: refs/heads/master
Commit: 704af4bd67d981a2efb5eb00110e19c5f7e7c924
Parents: 7a51e71
Author: gatorsmile 
Authored: Tue Dec 12 13:40:01 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 12 13:40:01 2017 -0800

--
 .../spark/deploy/rest/SubmitRestProtocolRequest.scala   |  2 --
 .../spark/deploy/rest/SubmitRestProtocolSuite.scala |  2 --
 .../spark/deploy/rest/mesos/MesosRestServer.scala   | 12 
 3 files changed, 4 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/704af4bd/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
 
b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
index 86ddf95..0d50a76 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala
@@ -46,8 +46,6 @@ private[rest] class CreateSubmissionRequest extends 
SubmitRestProtocolRequest {
 super.doValidate()
 assert(sparkProperties != null, "No Spark properties set!")
 assertFieldIsSet(appResource, "appResource")
-assertFieldIsSet(appArgs, "appArgs")
-assertFieldIsSet(environmentVariables, "environmentVariables")
 assertPropertyIsSet("spark.app.name")
 assertPropertyIsBoolean("spark.driver.supervise")
 assertPropertyIsNumeric("spark.driver.cores")

http://git-wip-us.apache.org/repos/asf/spark/blob/704af4bd/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
 
b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
index 75c50af..725b884 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala
@@ -86,8 +86,6 @@ class SubmitRestProtocolSuite extends SparkFunSuite {
 message.clientSparkVersion = "1.2.3"
 message.appResource = "honey-walnut-cherry.jar"
 message.mainClass = "org.apache.spark.examples.SparkPie"
-message.appArgs = Array("two slices")
-message.environmentVariables = Map("PATH" -> "/dev/null")
 val conf = new SparkConf(false)
 conf.set("spark.app.name", "SparkPie")
 message.sparkProperties = conf.getAll.toMap

http://git-wip-us.apache.org/repos/asf/spark/blob/704af4bd/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
--
diff --git 
a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
 
b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
index bb8dfee..ff60b88 100644
--- 
a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
+++ 
b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
@@ -77,16 +77,10 @@ private[mesos] class MesosSubmitRequestServlet(
   private def buildDriverDescription(request: CreateSubmissionRequest): 
MesosDriverDescription = {
 // Required fields, including the main class because python is not yet 
supported
 val appResource = Option(request.appResource).getOrElse {
-  throw new SubmitRestMissingFieldException("Application jar 'appResource' 
is missing.")
+  throw new SubmitRestMissingFieldException("Application jar is missing.")
 }
 val mainClass = Option(request.mainClass).getOrElse {
-  throw new SubmitRestMissingFieldException("Main class 'mainClass' is 
missing.")
-}
-val appArgs = Option(request.appArgs).getOrElse {
-  throw new SubmitRestMissingFieldException("Application arguments 
'appArgs' are missing.")
-}
-val environmentVariables = Option(request.environmentVariables).getOrElse {
-  throw new SubmitRestMissingFieldException("Environment variables 
'environmentVariables' are missing.")
+  throw new 

spark git commit: [SPARK-22729][SQL] Add getTruncateQuery to JdbcDialect

2017-12-12 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master d5007734b -> e6dc5f280


[SPARK-22729][SQL] Add getTruncateQuery to JdbcDialect

In order to enable truncate for PostgreSQL databases in Spark JDBC, a change is 
needed to the query used for truncating a PostgreSQL table. By default, 
PostgreSQL will automatically truncate any descendant tables if a TRUNCATE 
query is executed. As this may result in (unwanted) side-effects, the query 
used for the truncate should be specified separately for PostgreSQL, specifying 
only to TRUNCATE a single table.

## What changes were proposed in this pull request?

Add `getTruncateQuery` function to `JdbcDialect.scala`, with default query. 
Overridden this function for PostgreSQL to only truncate a single table. Also 
sets `isCascadingTruncateTable` to false, as this will allow truncates for 
PostgreSQL.

## How was this patch tested?

Existing tests all pass. Added test for `getTruncateQuery`

Author: Daniel van der Ende 

Closes #19911 from danielvdende/SPARK-22717.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e6dc5f28
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e6dc5f28
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e6dc5f28

Branch: refs/heads/master
Commit: e6dc5f280786bdb068abb7348b787324f76c0e4a
Parents: d500773
Author: Daniel van der Ende 
Authored: Tue Dec 12 10:41:37 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 12 10:41:37 2017 -0800

--
 .../datasources/jdbc/JdbcRelationProvider.scala |  2 +-
 .../sql/execution/datasources/jdbc/JdbcUtils.scala  |  7 ---
 .../apache/spark/sql/jdbc/AggregatedDialect.scala   |  6 +-
 .../org/apache/spark/sql/jdbc/JdbcDialects.scala| 12 
 .../org/apache/spark/sql/jdbc/PostgresDialect.scala | 14 --
 .../scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 16 
 6 files changed, 50 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e6dc5f28/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala
index 37e7bb0..cc506e5 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala
@@ -68,7 +68,7 @@ class JdbcRelationProvider extends CreatableRelationProvider
   case SaveMode.Overwrite =>
 if (options.isTruncate && isCascadingTruncateTable(options.url) == 
Some(false)) {
   // In this case, we should truncate table and then load.
-  truncateTable(conn, options.table)
+  truncateTable(conn, options)
   val tableSchema = JdbcUtils.getSchemaOption(conn, options)
   saveTable(df, tableSchema, isCaseSensitive, options)
 } else {

http://git-wip-us.apache.org/repos/asf/spark/blob/e6dc5f28/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
index bbc95df..e6dc2fd 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
@@ -96,12 +96,13 @@ object JdbcUtils extends Logging {
   }
 
   /**
-   * Truncates a table from the JDBC database.
+   * Truncates a table from the JDBC database without side effects.
*/
-  def truncateTable(conn: Connection, table: String): Unit = {
+  def truncateTable(conn: Connection, options: JDBCOptions): Unit = {
+val dialect = JdbcDialects.get(options.url)
 val statement = conn.createStatement
 try {
-  statement.executeUpdate(s"TRUNCATE TABLE $table")
+  statement.executeUpdate(dialect.getTruncateQuery(options.table))
 } finally {
   statement.close()
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/e6dc5f28/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala 

spark git commit: [SPARK-20557][SQL] Only support TIMESTAMP WITH TIME ZONE for Oracle Dialect

2017-12-11 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 3d82f6eb7 -> a4002651a


[SPARK-20557][SQL] Only support TIMESTAMP WITH TIME ZONE for Oracle Dialect

## What changes were proposed in this pull request?
In the previous PRs, https://github.com/apache/spark/pull/17832 and 
https://github.com/apache/spark/pull/17835 , we convert `TIMESTAMP WITH TIME 
ZONE` and `TIME WITH TIME ZONE` to `TIMESTAMP` for all the JDBC sources. 
However, this conversion could be risky since it does not respect our SQL 
configuration `spark.sql.session.timeZone`.

In addition, each vendor might have different semantics for these two types. 
For example, Postgres simply returns `TIMESTAMP` types for `TIMESTAMP WITH TIME 
ZONE`. For such supports, we should do it case by case. This PR reverts the 
general support of `TIMESTAMP WITH TIME ZONE` and `TIME WITH TIME ZONE` for 
JDBC sources, except ORACLE Dialect.

When supporting the ORACLE's `TIMESTAMP WITH TIME ZONE`, we only support it 
when the JVM default timezone is the same as the user-specified configuration 
`spark.sql.session.timeZone` (whose default is the JVM default timezone). Now, 
we still treat `TIMESTAMP WITH TIME ZONE` as `TIMESTAMP` when fetching the 
values via the Oracle JDBC connector, whose client converts the timestamp 
values with time zone to the timestamp values using the local JVM default 
timezone (a test case is added to `OracleIntegrationSuite.scala` in this PR for 
showing the behavior). Thus, to avoid any future behavior change, we will not 
support it if JVM default timezone is different from 
`spark.sql.session.timeZone`

No regression because the previous two PRs were just merged to be unreleased 
master branch.

## How was this patch tested?
Added the test cases

Author: gatorsmile 

Closes #19939 from gatorsmile/timezoneUpdate.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a4002651
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a4002651
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a4002651

Branch: refs/heads/master
Commit: a4002651a3ea673cf3eff7927531c1659663d194
Parents: 3d82f6e
Author: gatorsmile 
Authored: Mon Dec 11 16:33:06 2017 -0800
Committer: gatorsmile 
Committed: Mon Dec 11 16:33:06 2017 -0800

--
 .../spark/sql/jdbc/OracleIntegrationSuite.scala | 67 +++-
 .../sql/jdbc/PostgresIntegrationSuite.scala |  2 +
 .../execution/datasources/jdbc/JdbcUtils.scala  |  4 +-
 .../apache/spark/sql/jdbc/OracleDialect.scala   | 13 +++-
 .../org/apache/spark/sql/jdbc/JDBCSuite.scala   |  4 +-
 5 files changed, 82 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a4002651/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
--
diff --git 
a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
 
b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
index 9034318..8512496 100644
--- 
a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
+++ 
b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
@@ -18,11 +18,12 @@
 package org.apache.spark.sql.jdbc
 
 import java.sql.{Connection, Date, Timestamp}
-import java.util.Properties
+import java.util.{Properties, TimeZone}
 import java.math.BigDecimal
 
-import org.apache.spark.sql.{DataFrame, Row, SaveMode}
-import org.apache.spark.sql.execution.{WholeStageCodegenExec, 
RowDataSourceScanExec}
+import org.apache.spark.sql.{DataFrame, QueryTest, Row, SaveMode}
+import org.apache.spark.sql.execution.{RowDataSourceScanExec, 
WholeStageCodegenExec}
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SharedSQLContext
 import org.apache.spark.sql.types._
 import org.apache.spark.tags.DockerTest
@@ -77,6 +78,9 @@ class OracleIntegrationSuite extends 
DockerJDBCIntegrationSuite with SharedSQLCo
 conn.prepareStatement(
   "INSERT INTO ts_with_timezone VALUES " +
 "(1, to_timestamp_tz('1999-12-01 11:00:00 UTC','-MM-DD HH:MI:SS 
TZR'))").executeUpdate()
+conn.prepareStatement(
+  "INSERT INTO ts_with_timezone VALUES " +
+"(2, to_timestamp_tz('1999-12-01 12:00:00 PST','-MM-DD HH:MI:SS 
TZR'))").executeUpdate()
 conn.commit()
 
 conn.prepareStatement(
@@ -235,6 +239,63 @@ class OracleIntegrationSuite extends 
DockerJDBCIntegrationSuite with SharedSQLCo
 assert(types(1).equals("class java.sql.Timestamp"))
   }
 
+  test("Column type TIMESTAMP with SESSION_LOCAL_TIMEZONE is 

[2/2] spark git commit: [SPARK-22726][TEST] Basic tests for Binary Comparison and ImplicitTypeCasts

2017-12-11 Thread lixiao
[SPARK-22726][TEST] Basic tests for Binary Comparison and ImplicitTypeCasts

## What changes were proposed in this pull request?
Before we deliver the Hive compatibility mode, we plan to write a set of test 
cases that can be easily run in both Spark and Hive sides. We can easily 
compare whether they are the same or not. When new typeCoercion rules are 
added, we also can easily track the changes. These test cases can also be 
backported to the previous Spark versions for determining the changes we made.

This PR is the first attempt for improving the test coverage for type coercion 
compatibility. We generate these test cases for our binary comparison and 
ImplicitTypeCasts based on the Apache Derby test cases in 
https://github.com/apache/derby/blob/10.14/java/testing/org/apache/derbyTesting/functionTests/tests/lang/implicitConversions.sql

## How was this patch tested?
N/A

Author: gatorsmile 

Closes #19918 from gatorsmile/typeCoercionTests.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3d82f6eb
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3d82f6eb
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3d82f6eb

Branch: refs/heads/master
Commit: 3d82f6eb782315b05453b3a0334d3bc05ab4298a
Parents: 3f4060c
Author: gatorsmile 
Authored: Mon Dec 11 15:55:23 2017 -0800
Committer: gatorsmile 
Committed: Mon Dec 11 15:55:23 2017 -0800

--
 .../typeCoercion/native/binaryComparison.sql|  287 +++
 .../typeCoercion/native/implicitTypeCasts.sql   |   72 +
 .../sql-tests/results/datetime.sql.out  |2 +
 .../results/predicate-functions.sql.out |  152 +-
 .../native/binaryComparison.sql.out | 2146 ++
 .../native/implicitTypeCasts.sql.out|  354 +++
 .../apache/spark/sql/SQLQueryTestSuite.scala|2 +-
 .../org/apache/spark/sql/TPCDSQuerySuite.scala  |2 +-
 .../hive/execution/HiveCompatibilitySuite.scala |2 +-
 9 files changed, 2940 insertions(+), 79 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3d82f6eb/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/binaryComparison.sql
--
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/binaryComparison.sql
 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/binaryComparison.sql
new file mode 100644
index 000..522322a
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/binaryComparison.sql
@@ -0,0 +1,287 @@
+--
+--   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.
+--
+
+-- Binary Comparison
+
+CREATE TEMPORARY VIEW t AS SELECT 1;
+
+SELECT cast(1 as binary) = '1' FROM t;
+SELECT cast(1 as binary) > '2' FROM t;
+SELECT cast(1 as binary) >= '2' FROM t;
+SELECT cast(1 as binary) < '2' FROM t;
+SELECT cast(1 as binary) <= '2' FROM t;
+SELECT cast(1 as binary) <> '2' FROM t;
+SELECT cast(1 as binary) = cast(null as string) FROM t;
+SELECT cast(1 as binary) > cast(null as string) FROM t;
+SELECT cast(1 as binary) >= cast(null as string) FROM t;
+SELECT cast(1 as binary) < cast(null as string) FROM t;
+SELECT cast(1 as binary) <= cast(null as string) FROM t;
+SELECT cast(1 as binary) <> cast(null as string) FROM t;
+SELECT '1' = cast(1 as binary) FROM t;
+SELECT '2' > cast(1 as binary) FROM t;
+SELECT '2' >= cast(1 as binary) FROM t;
+SELECT '2' < cast(1 as binary) FROM t;
+SELECT '2' <= cast(1 as binary) FROM t;
+SELECT '2' <> cast(1 as binary) FROM t;
+SELECT cast(null as string) = cast(1 as binary) FROM t;
+SELECT cast(null as string) > cast(1 as binary) FROM t;
+SELECT cast(null as string) >= cast(1 as binary) FROM t;
+SELECT cast(null as string) < cast(1 as binary) FROM t;
+SELECT cast(null as string) <= cast(1 as binary) FROM t;
+SELECT cast(null as string) <> cast(1 as binary) FROM t;
+SELECT cast(1 as tinyint) = '1' FROM t;
+SELECT cast(1 as tinyint) > '2' FROM t;
+SELECT 

[1/2] spark git commit: [SPARK-22726][TEST] Basic tests for Binary Comparison and ImplicitTypeCasts

2017-12-11 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 3f4060c34 -> 3d82f6eb7


http://git-wip-us.apache.org/repos/asf/spark/blob/3d82f6eb/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out
--
diff --git 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out
new file mode 100644
index 000..44fa48e
--- /dev/null
+++ 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out
@@ -0,0 +1,354 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 44
+
+
+-- !query 0
+CREATE TEMPORARY VIEW t AS SELECT 1
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+SELECT 1 + '2' FROM t
+-- !query 1 schema
+struct<(CAST(1 AS DOUBLE) + CAST(2 AS DOUBLE)):double>
+-- !query 1 output
+3.0
+
+
+-- !query 2
+SELECT 1 - '2' FROM t
+-- !query 2 schema
+struct<(CAST(1 AS DOUBLE) - CAST(2 AS DOUBLE)):double>
+-- !query 2 output
+-1.0
+
+
+-- !query 3
+SELECT 1 * '2' FROM t
+-- !query 3 schema
+struct<(CAST(1 AS DOUBLE) * CAST(2 AS DOUBLE)):double>
+-- !query 3 output
+2.0
+
+
+-- !query 4
+SELECT 4 / '2' FROM t
+-- !query 4 schema
+struct<(CAST(4 AS DOUBLE) / CAST(CAST(2 AS DOUBLE) AS DOUBLE)):double>
+-- !query 4 output
+2.0
+
+
+-- !query 5
+SELECT 1.1 + '2' FROM t
+-- !query 5 schema
+struct<(CAST(1.1 AS DOUBLE) + CAST(2 AS DOUBLE)):double>
+-- !query 5 output
+3.1
+
+
+-- !query 6
+SELECT 1.1 - '2' FROM t
+-- !query 6 schema
+struct<(CAST(1.1 AS DOUBLE) - CAST(2 AS DOUBLE)):double>
+-- !query 6 output
+-0.8999
+
+
+-- !query 7
+SELECT 1.1 * '2' FROM t
+-- !query 7 schema
+struct<(CAST(1.1 AS DOUBLE) * CAST(2 AS DOUBLE)):double>
+-- !query 7 output
+2.2
+
+
+-- !query 8
+SELECT 4.4 / '2' FROM t
+-- !query 8 schema
+struct<(CAST(4.4 AS DOUBLE) / CAST(2 AS DOUBLE)):double>
+-- !query 8 output
+2.2
+
+
+-- !query 9
+SELECT 1.1 + '2.2' FROM t
+-- !query 9 schema
+struct<(CAST(1.1 AS DOUBLE) + CAST(2.2 AS DOUBLE)):double>
+-- !query 9 output
+3.3003
+
+
+-- !query 10
+SELECT 1.1 - '2.2' FROM t
+-- !query 10 schema
+struct<(CAST(1.1 AS DOUBLE) - CAST(2.2 AS DOUBLE)):double>
+-- !query 10 output
+-1.1
+
+
+-- !query 11
+SELECT 1.1 * '2.2' FROM t
+-- !query 11 schema
+struct<(CAST(1.1 AS DOUBLE) * CAST(2.2 AS DOUBLE)):double>
+-- !query 11 output
+2.4204
+
+
+-- !query 12
+SELECT 4.4 / '2.2' FROM t
+-- !query 12 schema
+struct<(CAST(4.4 AS DOUBLE) / CAST(2.2 AS DOUBLE)):double>
+-- !query 12 output
+2.0
+
+
+-- !query 13
+SELECT '$' || cast(1 as smallint) || '$' FROM t
+-- !query 13 schema
+struct
+-- !query 13 output
+$1$
+
+
+-- !query 14
+SELECT '$' || 1 || '$' FROM t
+-- !query 14 schema
+struct
+-- !query 14 output
+$1$
+
+
+-- !query 15
+SELECT '$' || cast(1 as bigint) || '$' FROM t
+-- !query 15 schema
+struct
+-- !query 15 output
+$1$
+
+
+-- !query 16
+SELECT '$' || cast(1.1 as float) || '$' FROM t
+-- !query 16 schema
+struct
+-- !query 16 output
+$1.1$
+
+
+-- !query 17
+SELECT '$' || cast(1.1 as double) || '$' FROM t
+-- !query 17 schema
+struct
+-- !query 17 output
+$1.1$
+
+
+-- !query 18
+SELECT '$' || 1.1 || '$' FROM t
+-- !query 18 schema
+struct
+-- !query 18 output
+$1.1$
+
+
+-- !query 19
+SELECT '$' || cast(1.1 as decimal(8,3)) || '$' FROM t
+-- !query 19 schema
+struct
+-- !query 19 output
+$1.100$
+
+
+-- !query 20
+SELECT '$' || 'abcd' || '$' FROM t
+-- !query 20 schema
+struct
+-- !query 20 output
+$abcd$
+
+
+-- !query 21
+SELECT '$' || date('1996-09-09') || '$' FROM t
+-- !query 21 schema
+struct
+-- !query 21 output
+$1996-09-09$
+
+
+-- !query 22
+SELECT '$' || timestamp('1996-09-09 10:11:12.4' )|| '$' FROM t
+-- !query 22 schema
+struct
+-- !query 22 output
+$1996-09-09 10:11:12.4$
+
+
+-- !query 23
+SELECT length(cast(1 as smallint)) FROM t
+-- !query 23 schema
+struct
+-- !query 23 output
+1
+
+
+-- !query 24
+SELECT length(cast(1 as int)) FROM t
+-- !query 24 schema
+struct
+-- !query 24 output
+1
+
+
+-- !query 25
+SELECT length(cast(1 as bigint)) FROM t
+-- !query 25 schema
+struct
+-- !query 25 

spark git commit: [SPARK-22746][SQL] Avoid the generation of useless mutable states by SortMergeJoin

2017-12-11 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master a04f2bea6 -> c235b5f97


[SPARK-22746][SQL] Avoid the generation of useless mutable states by 
SortMergeJoin

## What changes were proposed in this pull request?

This PR reduce the number of global mutable variables in generated code of 
`SortMergeJoin`.

Before this PR, global mutable variables are used to extend lifetime of 
variables in the nested loop. This can be achieved by declaring variable at the 
outer most loop level where the variables are used.
In the following example, `smj_value8`, `smj_value8`, and `smj_value9` are 
declared as local variable at lines 145-147 in `With this PR`.

This PR fixes potential assertion error by #19865. Without this PR, a global 
mutable variable is potentially passed to arguments in generated code of split 
function.

Without this PR
```
/* 010 */   int smj_value8;
/* 011 */   boolean smj_value8;
/* 012 */   int smj_value9;
..
/* 143 */   protected void processNext() throws java.io.IOException {
/* 144 */ while (findNextInnerJoinRows(smj_leftInput, smj_rightInput)) {
/* 145 */   boolean smj_loaded = false;
/* 146 */   smj_isNull6 = smj_leftRow.isNullAt(1);
/* 147 */   smj_value9 = smj_isNull6 ? -1 : (smj_leftRow.getInt(1));
/* 148 */   scala.collection.Iterator smj_iterator = 
smj_matches.generateIterator();
/* 149 */   while (smj_iterator.hasNext()) {
/* 150 */ InternalRow smj_rightRow1 = (InternalRow) smj_iterator.next();
/* 151 */ boolean smj_isNull8 = smj_rightRow1.isNullAt(1);
/* 152 */ int smj_value11 = smj_isNull8 ? -1 : 
(smj_rightRow1.getInt(1));
/* 153 */
/* 154 */ boolean smj_value12 = (smj_isNull6 && smj_isNull8) ||
/* 155 */ (!smj_isNull6 && !smj_isNull8 && smj_value9 == smj_value11);
/* 156 */ if (false || !smj_value12) continue;
/* 157 */ if (!smj_loaded) {
/* 158 */   smj_loaded = true;
/* 159 */   smj_value8 = smj_leftRow.getInt(0);
/* 160 */ }
/* 161 */ int smj_value10 = smj_rightRow1.getInt(0);
/* 162 */ smj_numOutputRows.add(1);
/* 163 */
/* 164 */ smj_rowWriter.zeroOutNullBytes();
/* 165 */
/* 166 */ smj_rowWriter.write(0, smj_value8);
/* 167 */
/* 168 */ if (smj_isNull6) {
/* 169 */   smj_rowWriter.setNullAt(1);
/* 170 */ } else {
/* 171 */   smj_rowWriter.write(1, smj_value9);
/* 172 */ }
/* 173 */
/* 174 */ smj_rowWriter.write(2, smj_value10);
/* 175 */
/* 176 */ if (smj_isNull8) {
/* 177 */   smj_rowWriter.setNullAt(3);
/* 178 */ } else {
/* 179 */   smj_rowWriter.write(3, smj_value11);
/* 180 */ }
/* 181 */ append(smj_result.copy());
/* 182 */
/* 183 */   }
/* 184 */   if (shouldStop()) return;
/* 185 */ }
/* 186 */   }
```

With this PR
```
/* 143 */   protected void processNext() throws java.io.IOException {
/* 144 */ while (findNextInnerJoinRows(smj_leftInput, smj_rightInput)) {
/* 145 */   int smj_value8 = -1;
/* 146 */   boolean smj_isNull6 = false;
/* 147 */   int smj_value9 = -1;
/* 148 */   boolean smj_loaded = false;
/* 149 */   smj_isNull6 = smj_leftRow.isNullAt(1);
/* 150 */   smj_value9 = smj_isNull6 ? -1 : (smj_leftRow.getInt(1));
/* 151 */   scala.collection.Iterator smj_iterator = 
smj_matches.generateIterator();
/* 152 */   while (smj_iterator.hasNext()) {
/* 153 */ InternalRow smj_rightRow1 = (InternalRow) smj_iterator.next();
/* 154 */ boolean smj_isNull8 = smj_rightRow1.isNullAt(1);
/* 155 */ int smj_value11 = smj_isNull8 ? -1 : 
(smj_rightRow1.getInt(1));
/* 156 */
/* 157 */ boolean smj_value12 = (smj_isNull6 && smj_isNull8) ||
/* 158 */ (!smj_isNull6 && !smj_isNull8 && smj_value9 == smj_value11);
/* 159 */ if (false || !smj_value12) continue;
/* 160 */ if (!smj_loaded) {
/* 161 */   smj_loaded = true;
/* 162 */   smj_value8 = smj_leftRow.getInt(0);
/* 163 */ }
/* 164 */ int smj_value10 = smj_rightRow1.getInt(0);
/* 165 */ smj_numOutputRows.add(1);
/* 166 */
/* 167 */ smj_rowWriter.zeroOutNullBytes();
/* 168 */
/* 169 */ smj_rowWriter.write(0, smj_value8);
/* 170 */
/* 171 */ if (smj_isNull6) {
/* 172 */   smj_rowWriter.setNullAt(1);
/* 173 */ } else {
/* 174 */   smj_rowWriter.write(1, smj_value9);
/* 175 */ }
/* 176 */
/* 177 */ smj_rowWriter.write(2, smj_value10);
/* 178 */
/* 179 */ if (smj_isNull8) {
/* 180 */   smj_rowWriter.setNullAt(3);
/* 181 */ } else {
/* 182 */   smj_rowWriter.write(3, smj_value11);
/* 183 */ }
/* 184 */ append(smj_result.copy());
/* 185 */
/* 186 */   }
/* 187 */   if (shouldStop()) return;
/* 188 */ }
/* 189 */   }
```

## How was this patch tested?

Existing test cases

Author: Kazuaki Ishizaki 

Closes #19937 from 

spark git commit: Revert "[SPARK-22496][SQL] thrift server adds operation logs"

2017-12-11 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master bf20abb2d -> a04f2bea6


Revert "[SPARK-22496][SQL] thrift server adds operation logs"

This reverts commit 4289ac9d8dbbc45fc2ee6d0250a2113107bf08d0.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a04f2bea
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a04f2bea
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a04f2bea

Branch: refs/heads/master
Commit: a04f2bea67c9abf4149d33ac6c319cd4f85344d5
Parents: bf20abb
Author: gatorsmile 
Authored: Mon Dec 11 13:08:42 2017 -0800
Committer: gatorsmile 
Committed: Mon Dec 11 13:08:42 2017 -0800

--
 .../cli/operation/ExecuteStatementOperation.java   | 13 -
 .../hive/service/cli/operation/SQLOperation.java   | 12 
 .../thriftserver/SparkExecuteStatementOperation.scala  |  1 -
 3 files changed, 12 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a04f2bea/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
--
diff --git 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
index dc7de3c..3f2de10 100644
--- 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
+++ 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
@@ -23,7 +23,6 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.ql.processors.CommandProcessor;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
-import org.apache.hadoop.hive.ql.session.OperationLog;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationType;
 import org.apache.hive.service.cli.session.HiveSession;
@@ -68,16 +67,4 @@ public abstract class ExecuteStatementOperation extends 
Operation {
   this.confOverlay = confOverlay;
 }
   }
-
-  protected void registerCurrentOperationLog() {
-if (isOperationLogEnabled) {
-  if (operationLog == null) {
-LOG.warn("Failed to get current OperationLog object of Operation: " +
-getHandle().getHandleIdentifier());
-isOperationLogEnabled = false;
-return;
-  }
-  OperationLog.setCurrentOperationLog(operationLog);
-}
-  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/a04f2bea/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
--
diff --git 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
index fd9108e..5014ced 100644
--- 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -274,6 +274,18 @@ public class SQLOperation extends 
ExecuteStatementOperation {
 }
   }
 
+  private void registerCurrentOperationLog() {
+if (isOperationLogEnabled) {
+  if (operationLog == null) {
+LOG.warn("Failed to get current OperationLog object of Operation: " +
+getHandle().getHandleIdentifier());
+isOperationLogEnabled = false;
+return;
+  }
+  OperationLog.setCurrentOperationLog(operationLog);
+}
+  }
+
   private void cleanup(OperationState state) throws HiveSQLException {
 setState(state);
 if (shouldRunAsync()) {

http://git-wip-us.apache.org/repos/asf/spark/blob/a04f2bea/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
index 664bc20..f5191fa 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
@@ -170,7 +170,6 @@ private[hive] class SparkExecuteStatementOperation(
 override def run(): Unit = {
   val doAsAction = new PrivilegedExceptionAction[Unit]() {
 override def 

spark git commit: [SPARK-22496][SQL] thrift server adds operation logs

2017-12-10 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master ab1b6ee73 -> 4289ac9d8


[SPARK-22496][SQL] thrift server adds operation logs

## What changes were proposed in this pull request?
since hive 2.0+  upgrades log4j to log4j2,a lot of 
[changes](https://issues.apache.org/jira/browse/HIVE-11304) are made working on 
it.
as spark is not to ready to update its inner hive version(1.2.1) , so I manage 
to make little changes.
the function registerCurrentOperationLog  is moved from SQLOperstion to its 
parent class ExecuteStatementOperation so spark can use it.

## How was this patch tested?
manual test

Author: zouchenjun 

Closes #19721 from ChenjunZou/operation-log.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4289ac9d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4289ac9d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4289ac9d

Branch: refs/heads/master
Commit: 4289ac9d8dbbc45fc2ee6d0250a2113107bf08d0
Parents: ab1b6ee
Author: zouchenjun 
Authored: Sun Dec 10 20:36:14 2017 -0800
Committer: gatorsmile 
Committed: Sun Dec 10 20:36:14 2017 -0800

--
 .../cli/operation/ExecuteStatementOperation.java   | 13 +
 .../hive/service/cli/operation/SQLOperation.java   | 12 
 .../thriftserver/SparkExecuteStatementOperation.scala  |  1 +
 3 files changed, 14 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4289ac9d/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
--
diff --git 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
index 3f2de10..dc7de3c 100644
--- 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
+++ 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
@@ -23,6 +23,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.ql.processors.CommandProcessor;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
+import org.apache.hadoop.hive.ql.session.OperationLog;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationType;
 import org.apache.hive.service.cli.session.HiveSession;
@@ -67,4 +68,16 @@ public abstract class ExecuteStatementOperation extends 
Operation {
   this.confOverlay = confOverlay;
 }
   }
+
+  protected void registerCurrentOperationLog() {
+if (isOperationLogEnabled) {
+  if (operationLog == null) {
+LOG.warn("Failed to get current OperationLog object of Operation: " +
+getHandle().getHandleIdentifier());
+isOperationLogEnabled = false;
+return;
+  }
+  OperationLog.setCurrentOperationLog(operationLog);
+}
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/4289ac9d/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
--
diff --git 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
index 5014ced..fd9108e 100644
--- 
a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ 
b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -274,18 +274,6 @@ public class SQLOperation extends 
ExecuteStatementOperation {
 }
   }
 
-  private void registerCurrentOperationLog() {
-if (isOperationLogEnabled) {
-  if (operationLog == null) {
-LOG.warn("Failed to get current OperationLog object of Operation: " +
-getHandle().getHandleIdentifier());
-isOperationLogEnabled = false;
-return;
-  }
-  OperationLog.setCurrentOperationLog(operationLog);
-}
-  }
-
   private void cleanup(OperationState state) throws HiveSQLException {
 setState(state);
 if (shouldRunAsync()) {

http://git-wip-us.apache.org/repos/asf/spark/blob/4289ac9d/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
 

spark git commit: [SPARK-22279][SQL] Turn on spark.sql.hive.convertMetastoreOrc by default

2017-12-07 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 18b75d465 -> aa1764ba1


[SPARK-22279][SQL] Turn on spark.sql.hive.convertMetastoreOrc by default

## What changes were proposed in this pull request?

Like Parquet, this PR aims to turn on `spark.sql.hive.convertMetastoreOrc` by 
default.

## How was this patch tested?

Pass all the existing test cases.

Author: Dongjoon Hyun 

Closes #19499 from dongjoon-hyun/SPARK-22279.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/aa1764ba
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/aa1764ba
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/aa1764ba

Branch: refs/heads/master
Commit: aa1764ba1addbe7ad79344d5640bf6426267a38c
Parents: 18b75d4
Author: Dongjoon Hyun 
Authored: Thu Dec 7 15:45:23 2017 -0800
Committer: gatorsmile 
Committed: Thu Dec 7 15:45:23 2017 -0800

--
 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/aa1764ba/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
--
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
index f5e6720..c489690 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
@@ -109,7 +109,7 @@ private[spark] object HiveUtils extends Logging {
 .doc("When set to true, the built-in ORC reader and writer are used to 
process " +
   "ORC tables created by using the HiveQL syntax, instead of Hive serde.")
 .booleanConf
-.createWithDefault(false)
+.createWithDefault(true)
 
   val HIVE_METASTORE_SHARED_PREFIXES = 
buildConf("spark.sql.hive.metastore.sharedPrefixes")
 .doc("A comma separated list of class prefixes that should be loaded using 
the classloader " +


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



spark git commit: [SPARK-22719][SQL] Refactor ConstantPropagation

2017-12-07 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master f41c0a93f -> 18b75d465


[SPARK-22719][SQL] Refactor ConstantPropagation

## What changes were proposed in this pull request?

The current time complexity of ConstantPropagation is O(n^2), which can be slow 
when the query is complex.
Refactor the implementation with O( n ) time complexity, and some pruning to 
avoid traversing the whole `Condition`

## How was this patch tested?

Unit test.

Also simple benchmark test in ConstantPropagationSuite
```
  val condition = (1 to 500).map{_ => Rand(0) === Rand(0)}.reduce(And)
  val query = testRelation
.select(columnA)
.where(condition)
  val start = System.currentTimeMillis()
  (1 to 40).foreach { _ =>
Optimize.execute(query.analyze)
  }
  val end = System.currentTimeMillis()
  println(end - start)
```
Run time before changes: 18989ms (474ms per loop)
Run time after changes: 1275 ms (32ms per loop)

Author: Wang Gengliang 

Closes #19912 from gengliangwang/ConstantPropagation.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/18b75d46
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/18b75d46
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/18b75d46

Branch: refs/heads/master
Commit: 18b75d465b7563de926c5690094086a72a75c09f
Parents: f41c0a9
Author: Wang Gengliang 
Authored: Thu Dec 7 10:24:49 2017 -0800
Committer: gatorsmile 
Committed: Thu Dec 7 10:24:49 2017 -0800

--
 .../sql/catalyst/optimizer/expressions.scala| 106 +--
 1 file changed, 73 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/18b75d46/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
index 785e815..6305b6c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
@@ -64,49 +64,89 @@ object ConstantFolding extends Rule[LogicalPlan] {
  * }}}
  *
  * Approach used:
- * - Start from AND operator as the root
- * - Get all the children conjunctive predicates which are EqualTo / 
EqualNullSafe such that they
- *   don't have a `NOT` or `OR` operator in them
  * - Populate a mapping of attribute => constant value by looking at all the 
equals predicates
  * - Using this mapping, replace occurrence of the attributes with the 
corresponding constant values
  *   in the AND node.
  */
 object ConstantPropagation extends Rule[LogicalPlan] with PredicateHelper {
-  private def containsNonConjunctionPredicates(expression: Expression): 
Boolean = expression.find {
-case _: Not | _: Or => true
-case _ => false
-  }.isDefined
-
   def apply(plan: LogicalPlan): LogicalPlan = plan transform {
-case f: Filter => f transformExpressionsUp {
-  case and: And =>
-val conjunctivePredicates =
-  splitConjunctivePredicates(and)
-.filter(expr => expr.isInstanceOf[EqualTo] || 
expr.isInstanceOf[EqualNullSafe])
-.filterNot(expr => containsNonConjunctionPredicates(expr))
-
-val equalityPredicates = conjunctivePredicates.collect {
-  case e @ EqualTo(left: AttributeReference, right: Literal) => 
((left, right), e)
-  case e @ EqualTo(left: Literal, right: AttributeReference) => 
((right, left), e)
-  case e @ EqualNullSafe(left: AttributeReference, right: Literal) => 
((left, right), e)
-  case e @ EqualNullSafe(left: Literal, right: AttributeReference) => 
((right, left), e)
-}
+case f: Filter =>
+  val (newCondition, _) = traverse(f.condition, replaceChildren = true)
+  if (newCondition.isDefined) {
+f.copy(condition = newCondition.get)
+  } else {
+f
+  }
+  }
 
-val constantsMap = AttributeMap(equalityPredicates.map(_._1))
-val predicates = equalityPredicates.map(_._2).toSet
+  type EqualityPredicates = Seq[((AttributeReference, Literal), 
BinaryComparison)]
 
-def replaceConstants(expression: Expression) = expression transform {
-  case a: AttributeReference =>
-constantsMap.get(a) match {
-  case Some(literal) => literal
-  case None => a
-}
+  /**
+   * Traverse a condition as a tree and replace attributes with constant 
values.
+   * - On matching [[And]], recursively traverse each children and get 
propagated mappings.
+   *   If the current node is not child of another [[And]], 

spark git commit: [SPARK-22688][SQL] Upgrade Janino version to 3.0.8

2017-12-06 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master f110a7f88 -> 8ae004b46


[SPARK-22688][SQL] Upgrade Janino version to 3.0.8

## What changes were proposed in this pull request?

This PR upgrade Janino version to 3.0.8. [Janino 
3.0.8](https://janino-compiler.github.io/janino/changelog.html) includes an 
important fix to reduce the number of constant pool entries by using 'sipush' 
java bytecode.

* SIPUSH bytecode is not used for short integer constant 
[#33](https://github.com/janino-compiler/janino/issues/33).

Please see detail in [this discussion 
thread](https://github.com/apache/spark/pull/19518#issuecomment-346674976).

## How was this patch tested?

Existing tests

Author: Kazuaki Ishizaki 

Closes #19890 from kiszk/SPARK-22688.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8ae004b4
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8ae004b4
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8ae004b4

Branch: refs/heads/master
Commit: 8ae004b4602266d1f210e4c1564246d590412c06
Parents: f110a7f
Author: Kazuaki Ishizaki 
Authored: Wed Dec 6 16:15:25 2017 -0800
Committer: gatorsmile 
Committed: Wed Dec 6 16:15:25 2017 -0800

--
 dev/deps/spark-deps-hadoop-2.6 | 4 ++--
 dev/deps/spark-deps-hadoop-2.7 | 4 ++--
 pom.xml| 2 +-
 .../spark/sql/catalyst/expressions/codegen/CodeGenerator.scala | 6 +++---
 .../main/scala/org/apache/spark/sql/execution/SparkPlan.scala  | 4 ++--
 5 files changed, 10 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8ae004b4/dev/deps/spark-deps-hadoop-2.6
--
diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6
index 3b5a694..1831f33 100644
--- a/dev/deps/spark-deps-hadoop-2.6
+++ b/dev/deps/spark-deps-hadoop-2.6
@@ -35,7 +35,7 @@ commons-beanutils-core-1.8.0.jar
 commons-cli-1.2.jar
 commons-codec-1.10.jar
 commons-collections-3.2.2.jar
-commons-compiler-3.0.7.jar
+commons-compiler-3.0.8.jar
 commons-compress-1.4.1.jar
 commons-configuration-1.6.jar
 commons-crypto-1.0.0.jar
@@ -96,7 +96,7 @@ jackson-mapper-asl-1.9.13.jar
 jackson-module-paranamer-2.7.9.jar
 jackson-module-scala_2.11-2.6.7.1.jar
 jackson-xc-1.9.13.jar
-janino-3.0.7.jar
+janino-3.0.8.jar
 java-xmlbuilder-1.1.jar
 javassist-3.18.1-GA.jar
 javax.annotation-api-1.2.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/8ae004b4/dev/deps/spark-deps-hadoop-2.7
--
diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7
index 64136ba..fe14c05 100644
--- a/dev/deps/spark-deps-hadoop-2.7
+++ b/dev/deps/spark-deps-hadoop-2.7
@@ -35,7 +35,7 @@ commons-beanutils-core-1.8.0.jar
 commons-cli-1.2.jar
 commons-codec-1.10.jar
 commons-collections-3.2.2.jar
-commons-compiler-3.0.7.jar
+commons-compiler-3.0.8.jar
 commons-compress-1.4.1.jar
 commons-configuration-1.6.jar
 commons-crypto-1.0.0.jar
@@ -96,7 +96,7 @@ jackson-mapper-asl-1.9.13.jar
 jackson-module-paranamer-2.7.9.jar
 jackson-module-scala_2.11-2.6.7.1.jar
 jackson-xc-1.9.13.jar
-janino-3.0.7.jar
+janino-3.0.8.jar
 java-xmlbuilder-1.1.jar
 javassist-3.18.1-GA.jar
 javax.annotation-api-1.2.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/8ae004b4/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 07bca9d..52db79e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -170,7 +170,7 @@
 
 3.5
 3.2.10
-3.0.7
+3.0.8
 2.22.2
 2.9.3
 3.5.2

http://git-wip-us.apache.org/repos/asf/spark/blob/8ae004b4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index 670c82e..5c9e604 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -29,7 +29,7 @@ import scala.util.control.NonFatal
 import com.google.common.cache.{CacheBuilder, CacheLoader}
 import com.google.common.util.concurrent.{ExecutionError, 
UncheckedExecutionException}
 import org.codehaus.commons.compiler.CompileException
-import org.codehaus.janino.{ByteArrayClassLoader, ClassBodyEvaluator, 
JaninoRuntimeException, SimpleCompiler}

spark git commit: [SPARK-22693][SQL] CreateNamedStruct and InSet should not use global variables

2017-12-06 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 9948b860a -> f110a7f88


[SPARK-22693][SQL] CreateNamedStruct and InSet should not use global variables

## What changes were proposed in this pull request?

CreateNamedStruct and InSet are using a global variable which is not needed. 
This can generate some unneeded entries in the constant pool.

The PR removes the unnecessary mutable states and makes them local variables.

## How was this patch tested?

added UT

Author: Marco Gaido 
Author: Marco Gaido 

Closes #19896 from mgaido91/SPARK-22693.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f110a7f8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f110a7f8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f110a7f8

Branch: refs/heads/master
Commit: f110a7f884cb09f01a20462038328ddc5662b46f
Parents: 9948b86
Author: Marco Gaido 
Authored: Wed Dec 6 14:12:16 2017 -0800
Committer: gatorsmile 
Committed: Wed Dec 6 14:12:16 2017 -0800

--
 .../expressions/complexTypeCreator.scala| 27 +++-
 .../sql/catalyst/expressions/predicates.scala   | 22 
 .../catalyst/expressions/ComplexTypeSuite.scala |  7 +
 .../catalyst/expressions/PredicateSuite.scala   |  7 +
 4 files changed, 40 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f110a7f8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
index 087b210..3dc2ee0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
@@ -356,22 +356,25 @@ case class CreateNamedStruct(children: Seq[Expression]) 
extends CreateNamedStruc
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
 val rowClass = classOf[GenericInternalRow].getName
 val values = ctx.freshName("values")
-ctx.addMutableState("Object[]", values, s"$values = null;")
+val valCodes = valExprs.zipWithIndex.map { case (e, i) =>
+  val eval = e.genCode(ctx)
+  s"""
+ |${eval.code}
+ |if (${eval.isNull}) {
+ |  $values[$i] = null;
+ |} else {
+ |  $values[$i] = ${eval.value};
+ |}
+   """.stripMargin
+}
 val valuesCode = ctx.splitExpressionsWithCurrentInputs(
-  valExprs.zipWithIndex.map { case (e, i) =>
-val eval = e.genCode(ctx)
-s"""
-  ${eval.code}
-  if (${eval.isNull}) {
-$values[$i] = null;
-  } else {
-$values[$i] = ${eval.value};
-  }"""
-  })
+  expressions = valCodes,
+  funcName = "createNamedStruct",
+  extraArguments = "Object[]" -> values :: Nil)
 
 ev.copy(code =
   s"""
- |$values = new Object[${valExprs.size}];
+ |Object[] $values = new Object[${valExprs.size}];
  |$valuesCode
  |final InternalRow ${ev.value} = new $rowClass($values);
  |$values = null;

http://git-wip-us.apache.org/repos/asf/spark/blob/f110a7f8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
index 04e6694..a42dd7e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -344,17 +344,17 @@ case class InSet(child: Expression, hset: Set[Any]) 
extends UnaryExpression with
 } else {
   ""
 }
-ctx.addMutableState(setName, setTerm,
-  s"$setTerm = (($InSetName)references[${ctx.references.size - 
1}]).getSet();")
-ev.copy(code = s"""
-  ${childGen.code}
-  boolean ${ev.isNull} = ${childGen.isNull};
-  boolean ${ev.value} = false;
-  if (!${ev.isNull}) {
-${ev.value} = $setTerm.contains(${childGen.value});
-$setNull
-  }
- """)
+ev.copy(code =
+  s"""
+ |${childGen.code}
+ |${ctx.JAVA_BOOLEAN} ${ev.isNull} = ${childGen.isNull};
+ |${ctx.JAVA_BOOLEAN} ${ev.value} = false;
+ |if (!${ev.isNull}) {
+

spark git commit: [SPARK-22720][SS] Make EventTimeWatermark Extend UnaryNode

2017-12-06 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 51066b437 -> effca9868


[SPARK-22720][SS] Make EventTimeWatermark Extend UnaryNode

## What changes were proposed in this pull request?
Our Analyzer and Optimizer have multiple rules for `UnaryNode`. After making 
`EventTimeWatermark` extend `UnaryNode`, we do not need a special handling for 
`EventTimeWatermark`.

## How was this patch tested?
The existing tests

Author: gatorsmile 

Closes #19913 from gatorsmile/eventtimewatermark.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/effca986
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/effca986
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/effca986

Branch: refs/heads/master
Commit: effca9868e3feae16c5722c36878b23e616d01a2
Parents: 51066b4
Author: gatorsmile 
Authored: Wed Dec 6 13:11:38 2017 -0800
Committer: gatorsmile 
Committed: Wed Dec 6 13:11:38 2017 -0800

--
 .../spark/sql/catalyst/plans/logical/EventTimeWatermark.scala| 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/effca986/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala
index 06196b5..7a927e1 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala
@@ -38,7 +38,7 @@ object EventTimeWatermark {
 case class EventTimeWatermark(
 eventTime: Attribute,
 delay: CalendarInterval,
-child: LogicalPlan) extends LogicalPlan {
+child: LogicalPlan) extends UnaryNode {
 
   // Update the metadata on the eventTime column to include the desired delay.
   override val output: Seq[Attribute] = child.output.map { a =>
@@ -60,6 +60,4 @@ case class EventTimeWatermark(
   a
 }
   }
-
-  override val children: Seq[LogicalPlan] = child :: Nil
 }


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



spark git commit: [SPARK-22710] ConfigBuilder.fallbackConf should trigger onCreate function

2017-12-06 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master e98f9647f -> 4286cba7d


[SPARK-22710] ConfigBuilder.fallbackConf should trigger onCreate function

## What changes were proposed in this pull request?
I was looking at the config code today and found that configs defined using 
ConfigBuilder.fallbackConf didn't trigger onCreate function. This patch fixes 
it.

This doesn't require backporting since we currently have no configs that use it.

## How was this patch tested?
Added a test case for all the config final creator functions in 
ConfigEntrySuite.

Author: Reynold Xin 

Closes #19905 from rxin/SPARK-22710.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4286cba7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4286cba7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4286cba7

Branch: refs/heads/master
Commit: 4286cba7dacf4b457fff91da3743ac2518699945
Parents: e98f964
Author: Reynold Xin 
Authored: Wed Dec 6 10:11:25 2017 -0800
Committer: gatorsmile 
Committed: Wed Dec 6 10:11:25 2017 -0800

--
 .../spark/internal/config/ConfigBuilder.scala   |  4 +++-
 .../internal/config/ConfigEntrySuite.scala  | 20 
 2 files changed, 23 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4286cba7/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala 
b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
index 8f4c1b6..b0cd711 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
@@ -235,7 +235,9 @@ private[spark] case class ConfigBuilder(key: String) {
   }
 
   def fallbackConf[T](fallback: ConfigEntry[T]): ConfigEntry[T] = {
-new FallbackConfigEntry(key, _alternatives, _doc, _public, fallback)
+val entry = new FallbackConfigEntry(key, _alternatives, _doc, _public, 
fallback)
+_onCreate.foreach(_(entry))
+entry
   }
 
   def regexConf: TypedConfigBuilder[Regex] = {

http://git-wip-us.apache.org/repos/asf/spark/blob/4286cba7/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala 
b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala
index bf08276..02514dc 100644
--- 
a/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala
@@ -288,4 +288,24 @@ class ConfigEntrySuite extends SparkFunSuite {
 conf.remove(testKey("b"))
 assert(conf.get(iConf) === 3)
   }
+
+  test("onCreate") {
+var onCreateCalled = false
+ConfigBuilder(testKey("oc1")).onCreate(_ => onCreateCalled = 
true).intConf.createWithDefault(1)
+assert(onCreateCalled)
+
+onCreateCalled = false
+ConfigBuilder(testKey("oc2")).onCreate(_ => onCreateCalled = 
true).intConf.createOptional
+assert(onCreateCalled)
+
+onCreateCalled = false
+ConfigBuilder(testKey("oc3")).onCreate(_ => onCreateCalled = true).intConf
+  .createWithDefaultString("1.0")
+assert(onCreateCalled)
+
+val fallback = ConfigBuilder(testKey("oc4")).intConf.createWithDefault(1)
+onCreateCalled = false
+ConfigBuilder(testKey("oc5")).onCreate(_ => onCreateCalled = 
true).fallbackConf(fallback)
+assert(onCreateCalled)
+  }
 }


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



spark git commit: [SPARK-20392][SQL] Set barrier to prevent re-entering a tree

2017-12-05 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 82183f7b5 -> 00d176d2f


[SPARK-20392][SQL] Set barrier to prevent re-entering a tree

## What changes were proposed in this pull request?

The SQL `Analyzer` goes through a whole query plan even most part of it is 
analyzed. This increases the time spent on query analysis for long pipelines in 
ML, especially.

This patch adds a logical node called `AnalysisBarrier` that wraps an analyzed 
logical plan to prevent it from analysis again. The barrier is applied to the 
analyzed logical plan in `Dataset`. It won't change the output of wrapped 
logical plan and just acts as a wrapper to hide it from analyzer. New 
operations on the dataset will be put on the barrier, so only the new nodes 
created will be analyzed.

This analysis barrier will be removed at the end of analysis stage.

## How was this patch tested?

Added tests.

Author: Liang-Chi Hsieh 

Closes #19873 from viirya/SPARK-20392-reopen.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/00d176d2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/00d176d2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/00d176d2

Branch: refs/heads/master
Commit: 00d176d2fe7bbdf55cb3146a9cb04ca99b1858b7
Parents: 82183f7
Author: Liang-Chi Hsieh 
Authored: Tue Dec 5 21:43:41 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 5 21:43:41 2017 -0800

--
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 77 ++---
 .../sql/catalyst/analysis/CheckAnalysis.scala   |  4 -
 .../catalyst/analysis/DecimalPrecision.scala|  2 +-
 .../analysis/ResolveTableValuedFunctions.scala  |  2 +-
 .../analysis/SubstituteUnresolvedOrdinals.scala |  2 +-
 .../sql/catalyst/analysis/TypeCoercion.scala| 32 ---
 .../catalyst/analysis/timeZoneAnalysis.scala|  2 +-
 .../spark/sql/catalyst/analysis/view.scala  |  2 +-
 .../spark/sql/catalyst/optimizer/subquery.scala |  2 +-
 .../catalyst/plans/logical/LogicalPlan.scala| 49 ---
 .../plans/logical/basicLogicalOperators.scala   | 19 
 .../sql/catalyst/analysis/AnalysisSuite.scala   | 14 +++
 .../sql/catalyst/plans/LogicalPlanSuite.scala   | 42 ++---
 .../scala/org/apache/spark/sql/Dataset.scala| 91 ++--
 .../spark/sql/execution/datasources/rules.scala |  2 +-
 .../spark/sql/execution/PlannerSuite.scala  |  2 +-
 .../apache/spark/sql/hive/HiveStrategies.scala  |  6 +-
 17 files changed, 185 insertions(+), 165 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/00d176d2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index e5c93b5..0d5e866 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -165,14 +165,15 @@ class Analyzer(
 Batch("Subquery", Once,
   UpdateOuterReferences),
 Batch("Cleanup", fixedPoint,
-  CleanupAliases)
+  CleanupAliases,
+  EliminateBarriers)
   )
 
   /**
* Analyze cte definitions and substitute child plan with analyzed cte 
definitions.
*/
   object CTESubstitution extends Rule[LogicalPlan] {
-def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators  {
+def apply(plan: LogicalPlan): LogicalPlan = plan.transformUp  {
   case With(child, relations) =>
 substituteCTE(child, relations.foldLeft(Seq.empty[(String, 
LogicalPlan)]) {
   case (resolved, (name, relation)) =>
@@ -200,7 +201,7 @@ class Analyzer(
* Substitute child plan with WindowSpecDefinitions.
*/
   object WindowsSubstitution extends Rule[LogicalPlan] {
-def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
+def apply(plan: LogicalPlan): LogicalPlan = plan.transformUp {
   // Lookup WindowSpecDefinitions. This rule works with unresolved 
children.
   case WithWindowDefinition(windowDefinitions, child) =>
 child.transform {
@@ -242,7 +243,7 @@ class Analyzer(
 private def hasUnresolvedAlias(exprs: Seq[NamedExpression]) =
   exprs.exists(_.find(_.isInstanceOf[UnresolvedAlias]).isDefined)
 
-def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
+def apply(plan: LogicalPlan): LogicalPlan = plan.transformUp {
   case Aggregate(groups, aggs, child) if child.resolved && 
hasUnresolvedAlias(aggs) =>
 Aggregate(groups, assignAliases(aggs), child)
 
@@ -611,7 +612,7 @@ class Analyzer(
   case 

spark git commit: [SPARK-22662][SQL] Failed to prune columns after rewriting predicate subquery

2017-12-05 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 132a3f470 -> 1e17ab83d


[SPARK-22662][SQL] Failed to prune columns after rewriting predicate subquery

## What changes were proposed in this pull request?

As a simple example:
```
spark-sql> create table base (a int, b int) using parquet;
Time taken: 0.066 seconds
spark-sql> create table relInSubq ( x int, y int, z int) using parquet;
Time taken: 0.042 seconds
spark-sql> explain select a from base where a in (select x from relInSubq);
== Physical Plan ==
*Project [a#83]
+- *BroadcastHashJoin [a#83], [x#85], LeftSemi, BuildRight
   :- *FileScan parquet default.base[a#83,b#84] Batched: true, Format: Parquet, 
Location: InMemoryFileIndex[hdfs://100.0.0.4:9000/wzh/base], PartitionFilters: 
[], PushedFilters: [], ReadSchema: struct
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, 
true] as bigint)))
  +- *Project [x#85]
 +- *FileScan parquet default.relinsubq[x#85] Batched: true, Format: 
Parquet, Location: InMemoryFileIndex[hdfs://100.0.0.4:9000/wzh/relinsubq], 
PartitionFilters: [], PushedFilters: [], ReadSchema: struct
```
We only need column `a` in table `base`, but all columns (`a`, `b`) are fetched.

The reason is that, in "Operator Optimizations" batch, `ColumnPruning` first 
produces a `Project` on table `base`, but then it's removed by 
`removeProjectBeforeFilter`. Because at that time, the predicate subquery is in 
filter form. Then, in "Rewrite Subquery" batch, `RewritePredicateSubquery` 
converts the subquery into a LeftSemi join, but this batch doesn't have the 
`ColumnPruning` rule. This results in reading all columns for the `base` table.

## How was this patch tested?
Added a new test case.

Author: Zhenhua Wang 

Closes #19855 from wzhfy/column_pruning_subquery.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1e17ab83
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1e17ab83
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1e17ab83

Branch: refs/heads/master
Commit: 1e17ab83de29bca1823a537d7c57ffc4de8a26ee
Parents: 132a3f4
Author: Zhenhua Wang 
Authored: Tue Dec 5 15:15:32 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 5 15:15:32 2017 -0800

--
 .../sql/catalyst/optimizer/Optimizer.scala  |  4 +-
 .../optimizer/RewriteSubquerySuite.scala| 55 
 2 files changed, 58 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1e17ab83/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 8a5c486..484cd8c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -141,7 +141,9 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
   CheckCartesianProducts) ::
 Batch("RewriteSubquery", Once,
   RewritePredicateSubquery,
-  CollapseProject) :: Nil
+  ColumnPruning,
+  CollapseProject,
+  RemoveRedundantProject) :: Nil
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/1e17ab83/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
new file mode 100644
index 000..6b3739c
--- /dev/null
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala
@@ -0,0 +1,55 @@
+/*
+ * 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

spark git commit: [SPARK-22500][SQL][FOLLOWUP] cast for struct can split code even with whole stage codegen

2017-12-05 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master ced6ccf0d -> 132a3f470


[SPARK-22500][SQL][FOLLOWUP] cast for struct can split code even with whole 
stage codegen

## What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/19730, we can split the code 
for casting struct even with whole stage codegen.

This PR also has some renaming to make the code easier to read.

## How was this patch tested?

existing test

Author: Wenchen Fan 

Closes #19891 from cloud-fan/cast.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/132a3f47
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/132a3f47
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/132a3f47

Branch: refs/heads/master
Commit: 132a3f470811bb98f265d0c9ad2c161698e0237b
Parents: ced6ccf
Author: Wenchen Fan 
Authored: Tue Dec 5 11:40:13 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 5 11:40:13 2017 -0800

--
 .../spark/sql/catalyst/expressions/Cast.scala   | 52 +---
 1 file changed, 24 insertions(+), 28 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/132a3f47/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index f4ecbdb..b8d3661 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -548,8 +548,8 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
   castCode(ctx, eval.value, eval.isNull, ev.value, ev.isNull, dataType, 
nullSafeCast))
   }
 
-  // three function arguments are: child.primitive, result.primitive and 
result.isNull
-  // it returns the code snippets to be put in null safe evaluation region
+  // The function arguments are: `input`, `result` and `resultIsNull`. We 
don't need `inputIsNull`
+  // in parameter list, because the returned code will be put in null safe 
evaluation region.
   private[this] type CastFunction = (String, String, String) => String
 
   private[this] def nullSafeCastFunction(
@@ -584,15 +584,15 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
   throw new SparkException(s"Cannot cast $from to $to.")
   }
 
-  // Since we need to cast child expressions recursively inside ComplexTypes, 
such as Map's
+  // Since we need to cast input expressions recursively inside ComplexTypes, 
such as Map's
   // Key and Value, Struct's field, we need to name out all the variable names 
involved in a cast.
-  private[this] def castCode(ctx: CodegenContext, childPrim: String, 
childNull: String,
-resultPrim: String, resultNull: String, resultType: DataType, cast: 
CastFunction): String = {
+  private[this] def castCode(ctx: CodegenContext, input: String, inputIsNull: 
String,
+result: String, resultIsNull: String, resultType: DataType, cast: 
CastFunction): String = {
 s"""
-  boolean $resultNull = $childNull;
-  ${ctx.javaType(resultType)} $resultPrim = 
${ctx.defaultValue(resultType)};
-  if (!$childNull) {
-${cast(childPrim, resultPrim, resultNull)}
+  boolean $resultIsNull = $inputIsNull;
+  ${ctx.javaType(resultType)} $result = ${ctx.defaultValue(resultType)};
+  if (!$inputIsNull) {
+${cast(input, result, resultIsNull)}
   }
 """
   }
@@ -1014,8 +1014,8 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
   case (fromField, toField) => nullSafeCastFunction(fromField.dataType, 
toField.dataType, ctx)
 }
 val rowClass = classOf[GenericInternalRow].getName
-val result = ctx.freshName("result")
-val tmpRow = ctx.freshName("tmpRow")
+val tmpResult = ctx.freshName("tmpResult")
+val tmpInput = ctx.freshName("tmpInput")
 
 val fieldsEvalCode = fieldsCasts.zipWithIndex.map { case (cast, i) =>
   val fromFieldPrim = ctx.freshName("ffp")
@@ -1024,37 +1024,33 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
   val toFieldNull = ctx.freshName("tfn")
   val fromType = ctx.javaType(from.fields(i).dataType)
   s"""
-boolean $fromFieldNull = $tmpRow.isNullAt($i);
+boolean $fromFieldNull = $tmpInput.isNullAt($i);
 if ($fromFieldNull) {
-  $result.setNullAt($i);
+  $tmpResult.setNullAt($i);
 } else {
   $fromType $fromFieldPrim =
-${ctx.getValue(tmpRow, 

spark git commit: [SPARK-22701][SQL] add ctx.splitExpressionsWithCurrentInputs

2017-12-05 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 03fdc92e4 -> ced6ccf0d


[SPARK-22701][SQL] add ctx.splitExpressionsWithCurrentInputs

## What changes were proposed in this pull request?

This pattern appears many times in the codebase:
```
if (ctx.INPUT_ROW == null || ctx.currentVars != null) {
  exprs.mkString("\n")
} else {
  ctx.splitExpressions(...)
}
```

This PR adds a `ctx.splitExpressionsWithCurrentInputs` for this pattern

## How was this patch tested?

existing tests

Author: Wenchen Fan 

Closes #19895 from cloud-fan/splitExpression.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ced6ccf0
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ced6ccf0
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ced6ccf0

Branch: refs/heads/master
Commit: ced6ccf0d6f362e299f270ed2a474f2e14f845da
Parents: 03fdc92
Author: Wenchen Fan 
Authored: Tue Dec 5 10:15:15 2017 -0800
Committer: gatorsmile 
Committed: Tue Dec 5 10:15:15 2017 -0800

--
 .../sql/catalyst/expressions/arithmetic.scala   |  4 +-
 .../expressions/codegen/CodeGenerator.scala | 44 -
 .../codegen/GenerateMutableProjection.scala |  4 +-
 .../codegen/GenerateSafeProjection.scala|  2 +-
 .../expressions/complexTypeCreator.scala|  6 +-
 .../expressions/conditionalExpressions.scala| 84 -
 .../sql/catalyst/expressions/generators.scala   |  2 +-
 .../spark/sql/catalyst/expressions/hash.scala   | 55 +---
 .../catalyst/expressions/nullExpressions.scala  | 94 +---
 .../catalyst/expressions/objects/objects.scala  |  6 +-
 .../sql/catalyst/expressions/predicates.scala   | 47 +-
 .../expressions/stringExpressions.scala | 37 
 12 files changed, 179 insertions(+), 206 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ced6ccf0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index d98f7b3..739bd13 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -614,7 +614,7 @@ case class Least(children: Seq[Expression]) extends 
Expression {
 }
   """
 }
-val codes = ctx.splitExpressions(evalChildren.map(updateEval))
+val codes = 
ctx.splitExpressionsWithCurrentInputs(evalChildren.map(updateEval))
 ev.copy(code = s"""
   ${ev.isNull} = true;
   ${ev.value} = ${ctx.defaultValue(dataType)};
@@ -680,7 +680,7 @@ case class Greatest(children: Seq[Expression]) extends 
Expression {
 }
   """
 }
-val codes = ctx.splitExpressions(evalChildren.map(updateEval))
+val codes = 
ctx.splitExpressionsWithCurrentInputs(evalChildren.map(updateEval))
 ev.copy(code = s"""
   ${ev.isNull} = true;
   ${ev.value} = ${ctx.defaultValue(dataType)};

http://git-wip-us.apache.org/repos/asf/spark/blob/ced6ccf0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index 1645db1..670c82e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -781,29 +781,26 @@ class CodegenContext {
* beyond 1000kb, we declare a private, inner sub-class, and the function is 
inlined to it
* instead, because classes have a constant pool limit of 65,536 named 
values.
*
-   * Note that we will extract the current inputs of this context and pass 
them to the generated
-   * functions. The input is `INPUT_ROW` for normal codegen path, and 
`currentVars` for whole
-   * stage codegen path. Whole stage codegen path is not supported yet.
-   *
-   * @param expressions the codes to evaluate expressions.
-   */
-  def splitExpressions(expressions: Seq[String]): String = {
-splitExpressions(expressions, funcName = "apply", extraArguments = Nil)
-  }
-
-  /**
-   * Similar to [[splitExpressions(expressions: Seq[String])]], but has 
customized function name
-   * and extra arguments.
+   * Note that different from 

spark git commit: [SPARK-22665][SQL] Avoid repartitioning with empty list of expressions

2017-12-04 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 1d5597b40 -> 3887b7eef


[SPARK-22665][SQL] Avoid repartitioning with empty list of expressions

## What changes were proposed in this pull request?

Repartitioning by empty set of expressions is currently possible, even though 
it is a case which is not handled properly. Indeed, in `HashExpression` there 
is a check to avoid to run it on an empty set, but this check is not performed 
while repartitioning.
Thus, the PR adds a check to avoid this wrong situation.

## How was this patch tested?

added UT

Author: Marco Gaido 

Closes #19870 from mgaido91/SPARK-22665.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3887b7ee
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3887b7ee
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3887b7ee

Branch: refs/heads/master
Commit: 3887b7eef7b89d3aeecadebc0fdafa47586a232b
Parents: 1d5597b
Author: Marco Gaido 
Authored: Mon Dec 4 17:08:56 2017 -0800
Committer: gatorsmile 
Committed: Mon Dec 4 17:08:56 2017 -0800

--
 .../catalyst/plans/logical/basicLogicalOperators.scala  | 12 +++-
 .../spark/sql/catalyst/analysis/AnalysisSuite.scala |  5 -
 2 files changed, 11 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3887b7ee/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
index 93de7c1..ba5f97d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
@@ -22,8 +22,8 @@ import 
org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.logical.statsEstimation._
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, 
Partitioning, RangePartitioning}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, 
Partitioning,
+  RangePartitioning, RoundRobinPartitioning}
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
 import org.apache.spark.util.random.RandomSampler
@@ -847,14 +847,16 @@ case class RepartitionByExpression(
 "`SortOrder`, which means `RangePartitioning`, or none of them are 
`SortOrder`, which " +
 "means `HashPartitioning`. In this case we have:" +
   s"""
- |SortOrder: ${sortOrder}
- |NonSortOrder: ${nonSortOrder}
+ |SortOrder: $sortOrder
+ |NonSortOrder: $nonSortOrder
""".stripMargin)
 
 if (sortOrder.nonEmpty) {
   RangePartitioning(sortOrder.map(_.asInstanceOf[SortOrder]), 
numPartitions)
-} else {
+} else if (nonSortOrder.nonEmpty) {
   HashPartitioning(nonSortOrder, numPartitions)
+} else {
+  RoundRobinPartitioning(numPartitions)
 }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/3887b7ee/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index 0e2e706..109fb32 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -27,7 +27,8 @@ import org.apache.spark.sql.catalyst.dsl.plans._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.{Cross, Inner}
 import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, 
Partitioning, RangePartitioning}
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, 
Partitioning,
+  RangePartitioning, RoundRobinPartitioning}
 import org.apache.spark.sql.types._
 
 
@@ -530,6 +531,8 @@ class AnalysisSuite extends AnalysisTest with Matchers {
 checkPartitioning[RangePartitioning](numPartitions = 10,
   exprs = SortOrder('a.attr, Ascending), SortOrder('b.attr, Descending))
 
+

spark git commit: [SPARK-22626][SQL][FOLLOWUP] improve documentation and simplify test case

2017-12-04 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master e1dd03e42 -> 1d5597b40


[SPARK-22626][SQL][FOLLOWUP] improve documentation and simplify test case

## What changes were proposed in this pull request?

This PR improves documentation for not using zero `numRows` statistics and 
simplifies the test case.

The reason why some Hive tables have zero `numRows` is that, in Hive, when 
stats gathering is disabled, `numRows` is always zero after INSERT command:
```
hive> create table src (key int, value string) stored as orc;
hive> desc formatted src;
Table Parameters:
COLUMN_STATS_ACCURATE   {\"BASIC_STATS\":\"true\"}
numFiles0
numRows 0
rawDataSize 0
totalSize   0
transient_lastDdlTime   1512399590

hive> set hive.stats.autogather=false;
hive> insert into src select 1, 'a';
hive> desc formatted src;
Table Parameters:
numFiles1
numRows 0
rawDataSize 0
totalSize   275
transient_lastDdlTime   1512399647

hive> insert into src select 1, 'b';
hive> desc formatted src;
Table Parameters:
numFiles2
numRows 0
rawDataSize 0
totalSize   550
transient_lastDdlTime   1512399687
```

## How was this patch tested?

Modified existing test.

Author: Zhenhua Wang 

Closes #19880 from wzhfy/doc_zero_rowCount.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1d5597b4
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1d5597b4
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1d5597b4

Branch: refs/heads/master
Commit: 1d5597b408485e41812f3645a670864ad88570a0
Parents: e1dd03e
Author: Zhenhua Wang 
Authored: Mon Dec 4 15:08:07 2017 -0800
Committer: gatorsmile 
Committed: Mon Dec 4 15:08:07 2017 -0800

--
 .../apache/spark/sql/hive/client/HiveClientImpl.scala|  8 +---
 .../org/apache/spark/sql/hive/StatisticsSuite.scala  | 11 +--
 2 files changed, 10 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1d5597b4/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
index 77e8360..08eb5c7 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
@@ -426,9 +426,11 @@ private[hive] class HiveClientImpl(
   // TODO: stats should include all the other two fields (`numFiles` and 
`numPartitions`).
   // (see StatsSetupConst in Hive)
   val stats =
-// When table is external, `totalSize` is always zero, which will 
influence join strategy
-// so when `totalSize` is zero, use `rawDataSize` instead. When 
`rawDataSize` is also zero,
-// return None. Later, we will use the other ways to estimate the 
statistics.
+// When table is external, `totalSize` is always zero, which will 
influence join strategy.
+// So when `totalSize` is zero, use `rawDataSize` instead. When 
`rawDataSize` is also zero,
+// return None.
+// In Hive, when statistics gathering is disabled, `rawDataSize` and 
`numRows` is always
+// zero after INSERT command. So they are used here only if they are 
larger than zero.
 if (totalSize.isDefined && totalSize.get > 0L) {
   Some(CatalogStatistics(sizeInBytes = totalSize.get, rowCount = 
rowCount.filter(_ > 0)))
 } else if (rawDataSize.isDefined && rawDataSize.get > 0) {

http://git-wip-us.apache.org/repos/asf/spark/blob/1d5597b4/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
index ee027e5..13f06a2 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
@@ -1366,17 +1366,16 @@ class StatisticsSuite extends 
StatisticsCollectionTestBase with TestHiveSingleto
   sql("CREATE TABLE maybe_big (c1 bigint)" +
 "TBLPROPERTIES ('numRows'='0', 'rawDataSize'='600', 
'totalSize'='8')")
 
-  val relation = 

spark git commit: [SPARK-22489][DOC][FOLLOWUP] Update broadcast behavior changes in migration section

2017-12-03 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master dff440f1e -> 4131ad03f


[SPARK-22489][DOC][FOLLOWUP] Update broadcast behavior changes in migration 
section

## What changes were proposed in this pull request?

Update broadcast behavior changes in migration section.

## How was this patch tested?

N/A

Author: Yuming Wang 

Closes #19858 from wangyum/SPARK-22489-migration.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4131ad03
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4131ad03
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4131ad03

Branch: refs/heads/master
Commit: 4131ad03f4d2dfcfb1e166e5dfdf0752479f7340
Parents: dff440f
Author: Yuming Wang 
Authored: Sun Dec 3 23:52:37 2017 -0800
Committer: gatorsmile 
Committed: Sun Dec 3 23:52:37 2017 -0800

--
 docs/sql-programming-guide.md | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4131ad03/docs/sql-programming-guide.md
--
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index a1b9c3b..b76be91 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1776,6 +1776,8 @@ options.
 Note that, for DecimalType(38,0)*, the table above intentionally 
does not cover all other combinations of scales and precisions because 
currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 
1.1 is inferred as double type.
   - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas 
related functionalities, such as `toPandas`, `createDataFrame` from Pandas 
DataFrame, etc.
   - In PySpark, the behavior of timestamp values for Pandas related 
functionalities was changed to respect session timezone. If you want to use the 
old behavior, you need to set a configuration 
`spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See 
[SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details.
+ 
+ - Since Spark 2.3, when either broadcast hash join or broadcast nested loop 
join is applicable, we prefer to broadcasting the table that is explicitly 
specified in a broadcast hint. For details, see the section [Broadcast 
Hint](#broadcast-hint-for-sql-queries) and 
[SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489).
 
 ## Upgrading From Spark SQL 2.1 to 2.2
 


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



spark git commit: [SPARK-22601][SQL] Data load is getting displayed successful on providing non existing nonlocal file path

2017-11-30 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.2 af8a692d6 -> ba00bd961


[SPARK-22601][SQL] Data load is getting displayed successful on providing non 
existing nonlocal file path

## What changes were proposed in this pull request?
When user tries to load data with a non existing hdfs file path system is not 
validating it and the load command operation is getting successful.
This is misleading to the user. already there is a validation in the scenario 
of none existing local file path. This PR has added validation in the scenario 
of nonexisting hdfs file path
## How was this patch tested?
UT has been added for verifying the issue, also snapshots has been added after 
the verification in a spark yarn cluster

Author: sujith71955 

Closes #19823 from sujith71955/master_LoadComand_Issue.

(cherry picked from commit 16adaf634bcca3074b448d95e72177eefdf50069)
Signed-off-by: gatorsmile 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ba00bd96
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ba00bd96
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ba00bd96

Branch: refs/heads/branch-2.2
Commit: ba00bd9615cc37a903f4333dad57e0eeafbdfd0c
Parents: af8a692
Author: sujith71955 
Authored: Thu Nov 30 20:45:30 2017 -0800
Committer: gatorsmile 
Committed: Thu Nov 30 20:46:46 2017 -0800

--
 .../org/apache/spark/sql/execution/command/tables.scala | 9 -
 .../org/apache/spark/sql/hive/execution/HiveDDLSuite.scala  | 9 +
 2 files changed, 17 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ba00bd96/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index 8b61240..126c1cb 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -333,7 +333,7 @@ case class LoadDataCommand(
 uri
   } else {
 val uri = new URI(path)
-if (uri.getScheme() != null && uri.getAuthority() != null) {
+val hdfsUri = if (uri.getScheme() != null && uri.getAuthority() != 
null) {
   uri
 } else {
   // Follow Hive's behavior:
@@ -373,6 +373,13 @@ case class LoadDataCommand(
   }
   new URI(scheme, authority, absolutePath, uri.getQuery(), 
uri.getFragment())
 }
+val hadoopConf = sparkSession.sessionState.newHadoopConf()
+val srcPath = new Path(hdfsUri)
+val fs = srcPath.getFileSystem(hadoopConf)
+if (!fs.exists(srcPath)) {
+  throw new AnalysisException(s"LOAD DATA input path does not exist: 
$path")
+}
+hdfsUri
   }
 
 if (partition.nonEmpty) {

http://git-wip-us.apache.org/repos/asf/spark/blob/ba00bd96/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index c1c8281..f4c2625 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -1983,4 +1983,13 @@ class HiveDDLSuite
   }
 }
   }
+
+  test("load command for non local invalid path validation") {
+withTable("tbl") {
+  sql("CREATE TABLE tbl(i INT, j STRING)")
+  val e = intercept[AnalysisException](
+sql("load data inpath '/doesnotexist.csv' into table tbl"))
+  assert(e.message.contains("LOAD DATA input path does not exist"))
+}
+  }
 }


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



spark git commit: [SPARK-22601][SQL] Data load is getting displayed successful on providing non existing nonlocal file path

2017-11-30 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master dc365422b -> 16adaf634


[SPARK-22601][SQL] Data load is getting displayed successful on providing non 
existing nonlocal file path

## What changes were proposed in this pull request?
When user tries to load data with a non existing hdfs file path system is not 
validating it and the load command operation is getting successful.
This is misleading to the user. already there is a validation in the scenario 
of none existing local file path. This PR has added validation in the scenario 
of nonexisting hdfs file path
## How was this patch tested?
UT has been added for verifying the issue, also snapshots has been added after 
the verification in a spark yarn cluster

Author: sujith71955 

Closes #19823 from sujith71955/master_LoadComand_Issue.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/16adaf63
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/16adaf63
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/16adaf63

Branch: refs/heads/master
Commit: 16adaf634bcca3074b448d95e72177eefdf50069
Parents: dc36542
Author: sujith71955 
Authored: Thu Nov 30 20:45:30 2017 -0800
Committer: gatorsmile 
Committed: Thu Nov 30 20:45:30 2017 -0800

--
 .../org/apache/spark/sql/execution/command/tables.scala | 9 -
 .../org/apache/spark/sql/hive/execution/HiveDDLSuite.scala  | 9 +
 2 files changed, 17 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/16adaf63/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index c9f6e57..c42e6c3 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -340,7 +340,7 @@ case class LoadDataCommand(
 uri
   } else {
 val uri = new URI(path)
-if (uri.getScheme() != null && uri.getAuthority() != null) {
+val hdfsUri = if (uri.getScheme() != null && uri.getAuthority() != 
null) {
   uri
 } else {
   // Follow Hive's behavior:
@@ -380,6 +380,13 @@ case class LoadDataCommand(
   }
   new URI(scheme, authority, absolutePath, uri.getQuery(), 
uri.getFragment())
 }
+val hadoopConf = sparkSession.sessionState.newHadoopConf()
+val srcPath = new Path(hdfsUri)
+val fs = srcPath.getFileSystem(hadoopConf)
+if (!fs.exists(srcPath)) {
+  throw new AnalysisException(s"LOAD DATA input path does not exist: 
$path")
+}
+hdfsUri
   }
 
 if (partition.nonEmpty) {

http://git-wip-us.apache.org/repos/asf/spark/blob/16adaf63/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index 9063ef0..6c11905 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -2141,4 +2141,13 @@ class HiveDDLSuite
   }
 }
   }
+
+  test("load command for non local invalid path validation") {
+withTable("tbl") {
+  sql("CREATE TABLE tbl(i INT, j STRING)")
+  val e = intercept[AnalysisException](
+sql("load data inpath '/doesnotexist.csv' into table tbl"))
+  assert(e.message.contains("LOAD DATA input path does not exist"))
+}
+  }
 }


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



spark git commit: [SPARK-22614] Dataset API: repartitionByRange(...)

2017-11-30 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master bcceab649 -> f5f8e84d9


[SPARK-22614] Dataset API: repartitionByRange(...)

## What changes were proposed in this pull request?

This PR introduces a way to explicitly range-partition a Dataset. So far, only 
round-robin and hash partitioning were possible via `df.repartition(...)`, but 
sometimes range partitioning might be desirable: e.g. when writing to disk, for 
better compression without the cost of global sort.

The current implementation piggybacks on the existing `RepartitionByExpression` 
`LogicalPlan` and simply adds the following logic: If its expressions are of 
type `SortOrder`, then it will do `RangePartitioning`; otherwise 
`HashPartitioning`. This was by far the least intrusive solution I could come 
up with.

## How was this patch tested?
Unit test for `RepartitionByExpression` changes, a test to ensure we're not 
changing the behavior of existing `.repartition()` and a few end-to-end tests 
in `DataFrameSuite`.

Author: Adrian Ionescu 

Closes #19828 from adrian-ionescu/repartitionByRange.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f5f8e84d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f5f8e84d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f5f8e84d

Branch: refs/heads/master
Commit: f5f8e84d9d35751dad51490b6ae22931aa88db7b
Parents: bcceab6
Author: Adrian Ionescu 
Authored: Thu Nov 30 15:41:34 2017 -0800
Committer: gatorsmile 
Committed: Thu Nov 30 15:41:34 2017 -0800

--
 .../plans/logical/basicLogicalOperators.scala   | 20 +++
 .../sql/catalyst/analysis/AnalysisSuite.scala   | 26 +
 .../scala/org/apache/spark/sql/Dataset.scala| 57 ++--
 .../spark/sql/execution/SparkStrategies.scala   |  5 +-
 .../org/apache/spark/sql/DataFrameSuite.scala   | 57 
 5 files changed, 157 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f5f8e84d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
index c2750c3..93de7c1 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
@@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical.statsEstimation._
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, 
Partitioning, RangePartitioning}
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
 import org.apache.spark.util.random.RandomSampler
@@ -838,6 +839,25 @@ case class RepartitionByExpression(
 
   require(numPartitions > 0, s"Number of partitions ($numPartitions) must be 
positive.")
 
+  val partitioning: Partitioning = {
+val (sortOrder, nonSortOrder) = 
partitionExpressions.partition(_.isInstanceOf[SortOrder])
+
+require(sortOrder.isEmpty || nonSortOrder.isEmpty,
+  s"${getClass.getSimpleName} expects that either all its 
`partitionExpressions` are of type " +
+"`SortOrder`, which means `RangePartitioning`, or none of them are 
`SortOrder`, which " +
+"means `HashPartitioning`. In this case we have:" +
+  s"""
+ |SortOrder: ${sortOrder}
+ |NonSortOrder: ${nonSortOrder}
+   """.stripMargin)
+
+if (sortOrder.nonEmpty) {
+  RangePartitioning(sortOrder.map(_.asInstanceOf[SortOrder]), 
numPartitions)
+} else {
+  HashPartitioning(nonSortOrder, numPartitions)
+}
+  }
+
   override def maxRows: Option[Long] = child.maxRows
   override def shuffle: Boolean = true
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f5f8e84d/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index e56a5d6..0e2e706 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ 

spark git commit: [SPARK-22489][SQL] Shouldn't change broadcast join buildSide if user clearly specified

2017-11-30 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 6ac57fd0d -> bcceab649


[SPARK-22489][SQL] Shouldn't change broadcast join buildSide if user clearly 
specified

## What changes were proposed in this pull request?

How to reproduce:
```scala
import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec

spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", 
"value").createTempView("table1")
spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", 
"value").createTempView("table2")

val bl = sql("SELECT /*+ MAPJOIN(t1) */ * FROM table1 t1 JOIN table2 t2 ON 
t1.key = t2.key").queryExecution.executedPlan

println(bl.children.head.asInstanceOf[BroadcastHashJoinExec].buildSide)
```
The result is `BuildRight`, but should be `BuildLeft`. This PR fix this issue.
## How was this patch tested?

unit tests

Author: Yuming Wang 

Closes #19714 from wangyum/SPARK-22489.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bcceab64
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bcceab64
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bcceab64

Branch: refs/heads/master
Commit: bcceab649510a45f4c4b8e44b157c9987adff6f4
Parents: 6ac57fd
Author: Yuming Wang 
Authored: Thu Nov 30 15:36:26 2017 -0800
Committer: gatorsmile 
Committed: Thu Nov 30 15:36:26 2017 -0800

--
 docs/sql-programming-guide.md   | 58 
 .../spark/sql/execution/SparkStrategies.scala   | 67 ++-
 .../execution/joins/BroadcastJoinSuite.scala| 69 +++-
 3 files changed, 177 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/bcceab64/docs/sql-programming-guide.md
--
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 983770d..a1b9c3b 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1492,6 +1492,64 @@ that these options will be deprecated in future release 
as more optimizations ar
   
 
 
+## Broadcast Hint for SQL Queries
+
+The `BROADCAST` hint guides Spark to broadcast each specified table when 
joining them with another table or view.
+When Spark deciding the join methods, the broadcast hash join (i.e., BHJ) is 
preferred, 
+even if the statistics is above the configuration 
`spark.sql.autoBroadcastJoinThreshold`.
+When both sides of a join are specified, Spark broadcasts the one having the 
lower statistics.
+Note Spark does not guarantee BHJ is always chosen, since not all cases (e.g. 
full outer join) 
+support BHJ. When the broadcast nested loop join is selected, we still respect 
the hint.
+
+
+
+
+
+{% highlight scala %}
+import org.apache.spark.sql.functions.broadcast
+broadcast(spark.table("src")).join(spark.table("records"), "key").show()
+{% endhighlight %}
+
+
+
+
+
+{% highlight java %}
+import static org.apache.spark.sql.functions.broadcast;
+broadcast(spark.table("src")).join(spark.table("records"), "key").show();
+{% endhighlight %}
+
+
+
+
+
+{% highlight python %}
+from pyspark.sql.functions import broadcast
+broadcast(spark.table("src")).join(spark.table("records"), "key").show()
+{% endhighlight %}
+
+
+
+
+
+{% highlight r %}
+src <- sql("SELECT * FROM src")
+records <- sql("SELECT * FROM records")
+head(join(broadcast(src), records, src$key == records$key))
+{% endhighlight %}
+
+
+
+
+
+{% highlight sql %}
+-- We accept BROADCAST, BROADCASTJOIN and MAPJOIN for broadcast hint
+SELECT /*+ BROADCAST(r) */ * FROM records r JOIN src s ON r.key = s.key
+{% endhighlight %}
+
+
+
+
 # Distributed SQL Engine
 
 Spark SQL can also act as a distributed query engine using its JDBC/ODBC or 
command-line interface.

http://git-wip-us.apache.org/repos/asf/spark/blob/bcceab64/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 19b858f..1fe3cb1 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.execution.columnar.{InMemoryRelation, 
InMemoryTableScanExec}
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
-import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight}
+import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide}
 import 

spark git commit: [SPARK-21417][SQL] Infer join conditions using propagated constraints

2017-11-30 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 999ec137a -> 6ac57fd0d


[SPARK-21417][SQL] Infer join conditions using propagated constraints

## What changes were proposed in this pull request?

This PR adds an optimization rule that infers join conditions using propagated 
constraints.

For instance, if there is a join, where the left relation has 'a = 1' and the 
right relation has 'b = 1', then the rule infers 'a = b' as a join predicate. 
Only semantically new predicates are appended to the existing join condition.

Refer to the corresponding ticket and tests for more details.

## How was this patch tested?

This patch comes with a new test suite to cover the implemented logic.

Author: aokolnychyi 

Closes #18692 from aokolnychyi/spark-21417.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6ac57fd0
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6ac57fd0
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6ac57fd0

Branch: refs/heads/master
Commit: 6ac57fd0d1c82b834eb4bf0dd57596b92a99d6de
Parents: 999ec13
Author: aokolnychyi 
Authored: Thu Nov 30 14:25:10 2017 -0800
Committer: gatorsmile 
Committed: Thu Nov 30 14:25:10 2017 -0800

--
 .../expressions/EquivalentExpressionMap.scala   |  66 +
 .../catalyst/expressions/ExpressionSet.scala|   2 +
 .../sql/catalyst/optimizer/Optimizer.scala  |   1 +
 .../spark/sql/catalyst/optimizer/joins.scala|  60 +
 .../EquivalentExpressionMapSuite.scala  |  56 +
 .../optimizer/EliminateCrossJoinSuite.scala | 238 +++
 6 files changed, 423 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6ac57fd0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala
new file mode 100644
index 000..cf1614a
--- /dev/null
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala
@@ -0,0 +1,66 @@
+/*
+ * 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.expressions
+
+import scala.collection.mutable
+
+import 
org.apache.spark.sql.catalyst.expressions.EquivalentExpressionMap.SemanticallyEqualExpr
+
+/**
+ * A class that allows you to map an expression into a set of equivalent 
expressions. The keys are
+ * handled based on their semantic meaning and ignoring cosmetic differences. 
The values are
+ * represented as [[ExpressionSet]]s.
+ *
+ * The underlying representation of keys depends on the 
[[Expression.semanticHash]] and
+ * [[Expression.semanticEquals]] methods.
+ *
+ * {{{
+ *   val map = new EquivalentExpressionMap()
+ *
+ *   map.put(1 + 2, a)
+ *   map.put(rand(), b)
+ *
+ *   map.get(2 + 1) => Set(a) // 1 + 2 and 2 + 1 are semantically equivalent
+ *   map.get(1 + 2) => Set(a) // 1 + 2 and 2 + 1 are semantically equivalent
+ *   map.get(rand()) => Set() // non-deterministic expressions are not 
equivalent
+ * }}}
+ */
+class EquivalentExpressionMap {
+
+  private val equivalenceMap = mutable.HashMap.empty[SemanticallyEqualExpr, 
ExpressionSet]
+
+  def put(expression: Expression, equivalentExpression: Expression): Unit = {
+val equivalentExpressions = equivalenceMap.getOrElseUpdate(expression, 
ExpressionSet.empty)
+equivalenceMap(expression) = equivalentExpressions + equivalentExpression
+  }
+
+  def get(expression: Expression): Set[Expression] =
+equivalenceMap.getOrElse(expression, ExpressionSet.empty)
+}
+
+object EquivalentExpressionMap {
+
+  private implicit class SemanticallyEqualExpr(val expr: Expression) {
+override def equals(obj: Any): Boolean = obj match {
+  case other: SemanticallyEqualExpr => 

spark git commit: [SPARK-22615][SQL] Handle more cases in PropagateEmptyRelation

2017-11-29 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 20b239845 -> 57687280d


[SPARK-22615][SQL] Handle more cases in PropagateEmptyRelation

## What changes were proposed in this pull request?

Currently, in the optimize rule `PropagateEmptyRelation`, the following cases 
is not handled:
1.  empty relation as right child in left outer join
2. empty relation as left child in right outer join
3. empty relation as right child  in left semi join
4. empty relation as right child  in left anti join
5. only one empty relation in full outer join

case 1 / 2 / 5 can be treated as **Cartesian product** and cause exception. See 
the new test cases.

## How was this patch tested?
Unit test

Author: Wang Gengliang 

Closes #19825 from gengliangwang/SPARK-22615.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/57687280
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/57687280
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/57687280

Branch: refs/heads/master
Commit: 57687280d4171db98d4d9404c7bd3374f51deac0
Parents: 20b2398
Author: Wang Gengliang 
Authored: Wed Nov 29 09:17:39 2017 -0800
Committer: gatorsmile 
Committed: Wed Nov 29 09:17:39 2017 -0800

--
 .../optimizer/PropagateEmptyRelation.scala  |  36 +++-
 .../optimizer/PropagateEmptyRelationSuite.scala |  16 +-
 .../sql-tests/inputs/join-empty-relation.sql|  28 +++
 .../results/join-empty-relation.sql.out | 194 +++
 4 files changed, 257 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/57687280/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
index 52fbb4d..a6e5aa6 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
@@ -41,6 +41,10 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] with 
PredicateHelper {
   private def empty(plan: LogicalPlan) =
 LocalRelation(plan.output, data = Seq.empty, isStreaming = 
plan.isStreaming)
 
+  // Construct a project list from plan's output, while the value is always 
NULL.
+  private def nullValueProjectList(plan: LogicalPlan): Seq[NamedExpression] =
+plan.output.map{ a => Alias(Literal(null), a.name)(a.exprId) }
+
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
 case p: Union if p.children.forall(isEmptyLocalRelation) =>
   empty(p)
@@ -49,16 +53,28 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] 
with PredicateHelper {
 // as stateful streaming joins need to perform other state management 
operations other than
 // just processing the input data.
 case p @ Join(_, _, joinType, _)
-if !p.children.exists(_.isStreaming) && 
p.children.exists(isEmptyLocalRelation) =>
-  joinType match {
-case _: InnerLike => empty(p)
-// Intersect is handled as LeftSemi by `ReplaceIntersectWithSemiJoin` 
rule.
-// Except is handled as LeftAnti by `ReplaceExceptWithAntiJoin` rule.
-case LeftOuter | LeftSemi | LeftAnti if isEmptyLocalRelation(p.left) 
=> empty(p)
-case RightOuter if isEmptyLocalRelation(p.right) => empty(p)
-case FullOuter if p.children.forall(isEmptyLocalRelation) => empty(p)
-case _ => p
-}
+if !p.children.exists(_.isStreaming) =>
+  val isLeftEmpty = isEmptyLocalRelation(p.left)
+  val isRightEmpty = isEmptyLocalRelation(p.right)
+  if (isLeftEmpty || isRightEmpty) {
+joinType match {
+  case _: InnerLike => empty(p)
+  // Intersect is handled as LeftSemi by 
`ReplaceIntersectWithSemiJoin` rule.
+  // Except is handled as LeftAnti by `ReplaceExceptWithAntiJoin` rule.
+  case LeftOuter | LeftSemi | LeftAnti if isLeftEmpty => empty(p)
+  case LeftSemi if isRightEmpty => empty(p)
+  case LeftAnti if isRightEmpty => p.left
+  case FullOuter if isLeftEmpty && isRightEmpty => empty(p)
+  case LeftOuter | FullOuter if isRightEmpty =>
+Project(p.left.output ++ nullValueProjectList(p.right), p.left)
+  case RightOuter if isRightEmpty => empty(p)
+  case RightOuter | FullOuter if isLeftEmpty =>
+Project(nullValueProjectList(p.left) ++ p.right.output, p.right)
+  case _ => p
+}
+  } else {
+p
+  }
 
 case p: 

spark git commit: [SPARK-22637][SQL] Only refresh a logical plan once.

2017-11-28 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.2 eef72d3f0 -> 38a0532cf


[SPARK-22637][SQL] Only refresh a logical plan once.

## What changes were proposed in this pull request?
`CatalogImpl.refreshTable` uses `foreach(..)` to refresh all tables in a view. 
This traverses all nodes in the subtree and calls `LogicalPlan.refresh()` on 
these nodes. However `LogicalPlan.refresh()` is also refreshing its children, 
as a result refreshing a large view can be quite expensive.

This PR just calls `LogicalPlan.refresh()` on the top node.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell 

Closes #19837 from hvanhovell/SPARK-22637.

(cherry picked from commit 475a29f11ef488e7cb19bf7e0696d9d099d77c92)
Signed-off-by: gatorsmile 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/38a0532c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/38a0532c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/38a0532c

Branch: refs/heads/branch-2.2
Commit: 38a0532cfc59adbd182ad925da99d32c27a12f2b
Parents: eef72d3
Author: Herman van Hovell 
Authored: Tue Nov 28 16:03:47 2017 -0800
Committer: gatorsmile 
Committed: Tue Nov 28 16:04:02 2017 -0800

--
 .../src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/38a0532c/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
index e3ed419..f5338ac 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
@@ -468,7 +468,7 @@ class CatalogImpl(sparkSession: SparkSession) extends 
Catalog {
 if (tableMetadata.tableType == CatalogTableType.VIEW) {
   // Temp or persistent views: refresh (or invalidate) any metadata/data 
cached
   // in the plan recursively.
-  table.queryExecution.analyzed.foreach(_.refresh())
+  table.queryExecution.analyzed.refresh()
 } else {
   // Non-temp tables: refresh the metadata cache.
   sessionCatalog.refreshTable(tableIdent)


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



spark git commit: [SPARK-22637][SQL] Only refresh a logical plan once.

2017-11-28 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master a10b328db -> 475a29f11


[SPARK-22637][SQL] Only refresh a logical plan once.

## What changes were proposed in this pull request?
`CatalogImpl.refreshTable` uses `foreach(..)` to refresh all tables in a view. 
This traverses all nodes in the subtree and calls `LogicalPlan.refresh()` on 
these nodes. However `LogicalPlan.refresh()` is also refreshing its children, 
as a result refreshing a large view can be quite expensive.

This PR just calls `LogicalPlan.refresh()` on the top node.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell 

Closes #19837 from hvanhovell/SPARK-22637.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/475a29f1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/475a29f1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/475a29f1

Branch: refs/heads/master
Commit: 475a29f11ef488e7cb19bf7e0696d9d099d77c92
Parents: a10b328
Author: Herman van Hovell 
Authored: Tue Nov 28 16:03:47 2017 -0800
Committer: gatorsmile 
Committed: Tue Nov 28 16:03:47 2017 -0800

--
 .../src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/475a29f1/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
index fdd2533..6ae307b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
@@ -480,7 +480,7 @@ class CatalogImpl(sparkSession: SparkSession) extends 
Catalog {
 if (tableMetadata.tableType == CatalogTableType.VIEW) {
   // Temp or persistent views: refresh (or invalidate) any metadata/data 
cached
   // in the plan recursively.
-  table.queryExecution.analyzed.foreach(_.refresh())
+  table.queryExecution.analyzed.refresh()
 } else {
   // Non-temp tables: refresh the metadata cache.
   sessionCatalog.refreshTable(tableIdent)


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



spark git commit: [SPARK-22515][SQL] Estimation relation size based on numRows * rowSize

2017-11-28 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master b70e483cb -> da3557429


[SPARK-22515][SQL] Estimation relation size based on numRows * rowSize

## What changes were proposed in this pull request?

Currently, relation size is computed as the sum of file size, which is 
error-prone because storage format like parquet may have a much smaller file 
size compared to in-memory size. When we choose broadcast join based on file 
size, there's a risk of OOM. But if the number of rows is available in 
statistics, we can get a better estimation by `numRows * rowSize`, which helps 
to alleviate this problem.

## How was this patch tested?

Added a new test case for data source table and hive table.

Author: Zhenhua Wang 
Author: Zhenhua Wang 

Closes #19743 from wzhfy/better_leaf_size.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/da355742
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/da355742
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/da355742

Branch: refs/heads/master
Commit: da35574297d96eb750bdfee755a48defc36e284a
Parents: b70e483
Author: Zhenhua Wang 
Authored: Tue Nov 28 11:43:21 2017 -0800
Committer: gatorsmile 
Committed: Tue Nov 28 11:43:21 2017 -0800

--
 .../spark/sql/catalyst/catalog/interface.scala  | 14 +
 .../SizeInBytesOnlyStatsPlanVisitor.scala   |  1 -
 .../sql/StatisticsCollectionTestBase.scala  |  2 +-
 .../apache/spark/sql/hive/StatisticsSuite.scala | 30 +++-
 4 files changed, 38 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/da355742/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
index b10ce05..95b6fbb 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
@@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, 
InternalRow, TableIden
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, 
AttributeReference, Cast, ExprId, Literal}
 import org.apache.spark.sql.catalyst.plans.logical._
+import 
org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils
 import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
 import org.apache.spark.sql.catalyst.util.quoteIdentifier
 import org.apache.spark.sql.types.StructType
@@ -367,13 +368,14 @@ case class CatalogStatistics(
* on column names.
*/
   def toPlanStats(planOutput: Seq[Attribute], cboEnabled: Boolean): Statistics 
= {
-if (cboEnabled) {
-  val attrStats = planOutput.flatMap(a => colStats.get(a.name).map(a -> _))
-  Statistics(sizeInBytes = sizeInBytes, rowCount = rowCount,
-attributeStats = AttributeMap(attrStats))
+if (cboEnabled && rowCount.isDefined) {
+  val attrStats = AttributeMap(planOutput.flatMap(a => 
colStats.get(a.name).map(a -> _)))
+  // Estimate size as number of rows * row size.
+  val size = EstimationUtils.getOutputSize(planOutput, rowCount.get, 
attrStats)
+  Statistics(sizeInBytes = size, rowCount = rowCount, attributeStats = 
attrStats)
 } else {
-  // When CBO is disabled, we apply the size-only estimation strategy, so 
there's no need to
-  // propagate other statistics from catalog to the plan.
+  // When CBO is disabled or the table doesn't have other statistics, we 
apply the size-only
+  // estimation strategy and only propagate sizeInBytes in statistics.
   Statistics(sizeInBytes = sizeInBytes)
 }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/da355742/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
index d701a95..5e1c4e0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
+++ 

spark git commit: [SPARK-22602][SQL] remove ColumnVector#loadBytes

2017-11-26 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master d49d9e403 -> 5a02e3a2a


[SPARK-22602][SQL] remove ColumnVector#loadBytes

## What changes were proposed in this pull request?

`ColumnVector#loadBytes` is only used as an optimization for reading UTF8String 
in `WritableColumnVector`, this PR moves this optimization to 
`WritableColumnVector` and simplified it.

## How was this patch tested?

existing test

Author: Wenchen Fan 

Closes #19815 from cloud-fan/load-bytes.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5a02e3a2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5a02e3a2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5a02e3a2

Branch: refs/heads/master
Commit: 5a02e3a2ac8a25d92d98d3b3b0d1173dddb9cc91
Parents: d49d9e4
Author: Wenchen Fan 
Authored: Sun Nov 26 21:49:09 2017 -0800
Committer: gatorsmile 
Committed: Sun Nov 26 21:49:09 2017 -0800

--
 .../execution/vectorized/ArrowColumnVector.java |  5 
 .../sql/execution/vectorized/ColumnVector.java  |  8 ++
 .../sql/execution/vectorized/ColumnarArray.java |  8 --
 .../vectorized/OffHeapColumnVector.java | 23 ++--
 .../vectorized/OnHeapColumnVector.java  | 12 
 .../vectorized/WritableColumnVector.java| 29 +---
 6 files changed, 24 insertions(+), 61 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5a02e3a2/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
index 3a10e98..5c502c9 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
@@ -240,11 +240,6 @@ public final class ArrowColumnVector extends ColumnVector {
 return accessor.getArrayOffset(rowId);
   }
 
-  @Override
-  public void loadBytes(ColumnarArray array) {
-throw new UnsupportedOperationException();
-  }
-
   //
   // APIs dealing with Decimals
   //

http://git-wip-us.apache.org/repos/asf/spark/blob/5a02e3a2/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java
index 360ed83e..940457f 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java
@@ -181,11 +181,6 @@ public abstract class ColumnVector implements 
AutoCloseable {
   }
 
   /**
-   * Loads the data into array.byteArray.
-   */
-  public abstract void loadBytes(ColumnarArray array);
-
-  /**
* Returns the value for rowId.
*/
   public MapData getMap(int ordinal) {
@@ -198,7 +193,8 @@ public abstract class ColumnVector implements AutoCloseable 
{
   public abstract Decimal getDecimal(int rowId, int precision, int scale);
 
   /**
-   * Returns the UTF8String for rowId.
+   * Returns the UTF8String for rowId. Note that the returned UTF8String may 
point to the data of
+   * this column vector, please copy it if you want to keep it after this 
column vector is freed.
*/
   public abstract UTF8String getUTF8String(int rowId);
 

http://git-wip-us.apache.org/repos/asf/spark/blob/5a02e3a2/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarArray.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarArray.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarArray.java
index 34bde3e..b9da641 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarArray.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarArray.java
@@ -33,14 +33,6 @@ public final class ColumnarArray extends ArrayData {
   public int length;
   public int offset;
 
-  // Populate if binary data is required for the Array. This is stored here as 
an optimization
-  // for string data.
-  public byte[] byteArray;
-  public int byteArrayOffset;
-
-  // Reused staging buffer, used for loading from offheap.
-  protected byte[] tmpByteArray = new byte[1];
-
   ColumnarArray(ColumnVector data) {
 this.data = 

spark git commit: [SPARK-22604][SQL] remove the get address methods from ColumnVector

2017-11-24 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 70221903f -> e3fd93f14


[SPARK-22604][SQL] remove the get address methods from ColumnVector

## What changes were proposed in this pull request?

`nullsNativeAddress` and `valuesNativeAddress` are only used in tests and 
benchmark, no need to be top class API.

## How was this patch tested?

existing tests

Author: Wenchen Fan 

Closes #19818 from cloud-fan/minor.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e3fd93f1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e3fd93f1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e3fd93f1

Branch: refs/heads/master
Commit: e3fd93f149ff0ff1caff28a5191215e2a29749a9
Parents: 7022190
Author: Wenchen Fan 
Authored: Fri Nov 24 22:43:47 2017 -0800
Committer: gatorsmile 
Committed: Fri Nov 24 22:43:47 2017 -0800

--
 .../execution/vectorized/ArrowColumnVector.java | 10 ---
 .../sql/execution/vectorized/ColumnVector.java  |  7 --
 .../vectorized/OffHeapColumnVector.java |  6 +-
 .../vectorized/OnHeapColumnVector.java  |  9 ---
 .../vectorized/ColumnarBatchBenchmark.scala | 32 
 .../vectorized/ColumnarBatchSuite.scala | 82 +++-
 6 files changed, 47 insertions(+), 99 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e3fd93f1/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
index 949035b..3a10e98 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java
@@ -60,16 +60,6 @@ public final class ArrowColumnVector extends ColumnVector {
   }
 
   @Override
-  public long nullsNativeAddress() {
-throw new RuntimeException("Cannot get native address for arrow column");
-  }
-
-  @Override
-  public long valuesNativeAddress() {
-throw new RuntimeException("Cannot get native address for arrow column");
-  }
-
-  @Override
   public void close() {
 if (childColumns != null) {
   for (int i = 0; i < childColumns.length; i++) {

http://git-wip-us.apache.org/repos/asf/spark/blob/e3fd93f1/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java
index 666fd63..360ed83e 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java
@@ -63,13 +63,6 @@ public abstract class ColumnVector implements AutoCloseable {
   public abstract boolean anyNullsSet();
 
   /**
-   * Returns the off heap ptr for the arrays backing the NULLs and values 
buffer. Only valid
-   * to call for off heap columns.
-   */
-  public abstract long nullsNativeAddress();
-  public abstract long valuesNativeAddress();
-
-  /**
* Returns whether the value at rowId is NULL.
*/
   public abstract boolean isNullAt(int rowId);

http://git-wip-us.apache.org/repos/asf/spark/blob/e3fd93f1/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java
index 2bf523b..6b5c783 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java
@@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.vectorized;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.spark.sql.types.*;
 import org.apache.spark.unsafe.Platform;
 
@@ -73,12 +75,12 @@ public final class OffHeapColumnVector extends 
WritableColumnVector {
 reset();
   }
 
-  @Override
+  @VisibleForTesting
   public long valuesNativeAddress() {
 return data;
   }
 
-  @Override
+  @VisibleForTesting
   public long nullsNativeAddress() {
 return nulls;
   }


spark git commit: [SPARK-22596][SQL] set ctx.currentVars in CodegenSupport.consume

2017-11-24 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master a1877f45c -> 70221903f


[SPARK-22596][SQL] set ctx.currentVars in CodegenSupport.consume

## What changes were proposed in this pull request?

`ctx.currentVars` means the input variables for the current operator, which is 
already decided in `CodegenSupport`, we can set it there instead of `doConsume`.

also add more comments to help people understand the codegen framework.

After this PR, we now have a principle about setting `ctx.currentVars` and 
`ctx.INPUT_ROW`:
1. for non-whole-stage-codegen path, never set them. (permit some special cases 
like generating ordering)
2. for whole-stage-codegen `produce` path, mostly we don't need to set them, 
but blocking operators may need to set them for expressions that produce data 
from data source, sort buffer, aggregate buffer, etc.
3. for whole-stage-codegen `consume` path, mostly we don't need to set them 
because `currentVars` is automatically set to child input variables and 
`INPUT_ROW` is mostly not used. A few plans need to tweak them as they may have 
different inputs, or they use the input row.

## How was this patch tested?

existing tests.

Author: Wenchen Fan 

Closes #19803 from cloud-fan/codegen.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/70221903
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/70221903
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/70221903

Branch: refs/heads/master
Commit: 70221903f54eaa0514d5d189dfb6f175a62228a8
Parents: a1877f4
Author: Wenchen Fan 
Authored: Fri Nov 24 21:50:30 2017 -0800
Committer: gatorsmile 
Committed: Fri Nov 24 21:50:30 2017 -0800

--
 .../catalyst/expressions/BoundAttribute.scala   | 23 +
 .../expressions/codegen/CodeGenerator.scala | 14 +++---
 .../sql/execution/DataSourceScanExec.scala  | 14 +-
 .../apache/spark/sql/execution/ExpandExec.scala |  3 ---
 .../spark/sql/execution/GenerateExec.scala  |  2 --
 .../sql/execution/WholeStageCodegenExec.scala   | 27 +++-
 .../sql/execution/basicPhysicalOperators.scala  |  6 +
 .../apache/spark/sql/execution/objects.scala| 20 +--
 8 files changed, 59 insertions(+), 50 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/70221903/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
index 7d16118..6a17a39 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
@@ -59,21 +59,24 @@ case class BoundReference(ordinal: Int, dataType: DataType, 
nullable: Boolean)
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-val javaType = ctx.javaType(dataType)
-val value = ctx.getValue(ctx.INPUT_ROW, dataType, ordinal.toString)
 if (ctx.currentVars != null && ctx.currentVars(ordinal) != null) {
   val oev = ctx.currentVars(ordinal)
   ev.isNull = oev.isNull
   ev.value = oev.value
-  val code = oev.code
-  oev.code = ""
-  ev.copy(code = code)
-} else if (nullable) {
-  ev.copy(code = s"""
-boolean ${ev.isNull} = ${ctx.INPUT_ROW}.isNullAt($ordinal);
-$javaType ${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(dataType)} : 
($value);""")
+  ev.copy(code = oev.code)
 } else {
-  ev.copy(code = s"""$javaType ${ev.value} = $value;""", isNull = "false")
+  assert(ctx.INPUT_ROW != null, "INPUT_ROW and currentVars cannot both be 
null.")
+  val javaType = ctx.javaType(dataType)
+  val value = ctx.getValue(ctx.INPUT_ROW, dataType, ordinal.toString)
+  if (nullable) {
+ev.copy(code =
+  s"""
+ |boolean ${ev.isNull} = ${ctx.INPUT_ROW}.isNullAt($ordinal);
+ |$javaType ${ev.value} = ${ev.isNull} ? 
${ctx.defaultValue(dataType)} : ($value);
+   """.stripMargin)
+  } else {
+ev.copy(code = s"$javaType ${ev.value} = $value;", isNull = "false")
+  }
 }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/70221903/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 

spark git commit: [SPARK-22592][SQL] cleanup filter converting for hive

2017-11-23 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 42f83d7c4 -> c1217565e


[SPARK-22592][SQL] cleanup filter converting for hive

## What changes were proposed in this pull request?

We have 2 different methods to convert filters for hive, regarding a config. 
This introduces duplicated and inconsistent code(e.g. one use helper objects 
for pattern match and one doesn't).

## How was this patch tested?

existing tests

Author: Wenchen Fan 

Closes #19801 from cloud-fan/cleanup.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c1217565
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c1217565
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c1217565

Branch: refs/heads/master
Commit: c1217565e20bd3297f3b1bc8f18f5dea933211c0
Parents: 42f83d7
Author: Wenchen Fan 
Authored: Thu Nov 23 15:33:26 2017 -0800
Committer: gatorsmile 
Committed: Thu Nov 23 15:33:26 2017 -0800

--
 .../apache/spark/sql/hive/client/HiveShim.scala | 144 +--
 1 file changed, 69 insertions(+), 75 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c1217565/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
index bd1b300..1eac70d 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
@@ -585,53 +585,17 @@ private[client] class Shim_v0_13 extends Shim_v0_12 {
* Unsupported predicates are skipped.
*/
   def convertFilters(table: Table, filters: Seq[Expression]): String = {
-if (SQLConf.get.advancedPartitionPredicatePushdownEnabled) {
-  convertComplexFilters(table, filters)
-} else {
-  convertBasicFilters(table, filters)
-}
-  }
-
-
-  /**
-   * An extractor that matches all binary comparison operators except 
null-safe equality.
-   *
-   * Null-safe equality is not supported by Hive metastore partition predicate 
pushdown
-   */
-  object SpecialBinaryComparison {
-def unapply(e: BinaryComparison): Option[(Expression, Expression)] = e 
match {
-  case _: EqualNullSafe => None
-  case _ => Some((e.left, e.right))
+/**
+ * An extractor that matches all binary comparison operators except 
null-safe equality.
+ *
+ * Null-safe equality is not supported by Hive metastore partition 
predicate pushdown
+ */
+object SpecialBinaryComparison {
+  def unapply(e: BinaryComparison): Option[(Expression, Expression)] = e 
match {
+case _: EqualNullSafe => None
+case _ => Some((e.left, e.right))
+  }
 }
-  }
-
-  private def convertBasicFilters(table: Table, filters: Seq[Expression]): 
String = {
-// hive varchar is treated as catalyst string, but hive varchar can't be 
pushed down.
-lazy val varcharKeys = table.getPartitionKeys.asScala
-  .filter(col => col.getType.startsWith(serdeConstants.VARCHAR_TYPE_NAME) 
||
-col.getType.startsWith(serdeConstants.CHAR_TYPE_NAME))
-  .map(col => col.getName).toSet
-
-filters.collect {
-  case op @ SpecialBinaryComparison(a: Attribute, Literal(v, _: 
IntegralType)) =>
-s"${a.name} ${op.symbol} $v"
-  case op @ SpecialBinaryComparison(Literal(v, _: IntegralType), a: 
Attribute) =>
-s"$v ${op.symbol} ${a.name}"
-  case op @ SpecialBinaryComparison(a: Attribute, Literal(v, _: 
StringType))
-if !varcharKeys.contains(a.name) =>
-s"""${a.name} ${op.symbol} ${quoteStringLiteral(v.toString)}"""
-  case op @ SpecialBinaryComparison(Literal(v, _: StringType), a: 
Attribute)
-if !varcharKeys.contains(a.name) =>
-s"""${quoteStringLiteral(v.toString)} ${op.symbol} ${a.name}"""
-}.mkString(" and ")
-  }
-
-  private def convertComplexFilters(table: Table, filters: Seq[Expression]): 
String = {
-// hive varchar is treated as catalyst string, but hive varchar can't be 
pushed down.
-lazy val varcharKeys = table.getPartitionKeys.asScala
-  .filter(col => col.getType.startsWith(serdeConstants.VARCHAR_TYPE_NAME) 
||
-col.getType.startsWith(serdeConstants.CHAR_TYPE_NAME))
-  .map(col => col.getName).toSet
 
 object ExtractableLiteral {
   def unapply(expr: Expression): Option[String] = expr match {
@@ -643,9 +607,11 @@ private[client] class Shim_v0_13 extends Shim_v0_12 {
 
 object ExtractableLiterals {
   def unapply(exprs: Seq[Expression]): Option[Seq[String]] = {
-

spark git commit: [SPARK-22543][SQL] fix java 64kb compile error for deeply nested expressions

2017-11-22 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/master 327d25fe1 -> 0605ad761


[SPARK-22543][SQL] fix java 64kb compile error for deeply nested expressions

## What changes were proposed in this pull request?

A frequently reported issue of Spark is the Java 64kb compile error. This is 
because Spark generates a very big method and it's usually caused by 3 reasons:

1. a deep expression tree, e.g. a very complex filter condition
2. many individual expressions, e.g. expressions can have many children, 
operators can have many expressions.
3. a deep query plan tree (with whole stage codegen)

This PR focuses on 1. There are already several patches(#15620  #18972 #18641) 
trying to fix this issue and some of them are already merged. However this is 
an endless job as every non-leaf expression has this issue.

This PR proposes to fix this issue in `Expression.genCode`, to make sure the 
code for a single expression won't grow too big.

According to maropu 's benchmark, no regression is found with TPCDS (thanks 
maropu !): 
https://docs.google.com/spreadsheets/d/1K3_7lX05-ZgxDXi9X_GleNnDjcnJIfoSlSCDZcL4gdg/edit?usp=sharing

## How was this patch tested?

existing test

Author: Wenchen Fan 
Author: Wenchen Fan 

Closes #19767 from cloud-fan/codegen.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0605ad76
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0605ad76
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0605ad76

Branch: refs/heads/master
Commit: 0605ad761438b202ab077a6af342f48cab2825d8
Parents: 327d25f
Author: Wenchen Fan 
Authored: Wed Nov 22 10:05:46 2017 -0800
Committer: gatorsmile 
Committed: Wed Nov 22 10:05:46 2017 -0800

--
 .../sql/catalyst/expressions/Expression.scala   | 40 +-
 .../expressions/codegen/CodeGenerator.scala | 33 +---
 .../expressions/conditionalExpressions.scala| 60 --
 .../catalyst/expressions/namedExpressions.scala |  4 +-
 .../sql/catalyst/expressions/predicates.scala   | 82 +---
 .../expressions/CodeGenerationSuite.scala   |  4 +-
 .../execution/aggregate/HashAggregateExec.scala |  2 +
 7 files changed, 62 insertions(+), 163 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0605ad76/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
--
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index a3b722a..743782a 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -104,16 +104,48 @@ abstract class Expression extends TreeNode[Expression] {
 }.getOrElse {
   val isNull = ctx.freshName("isNull")
   val value = ctx.freshName("value")
-  val ve = doGenCode(ctx, ExprCode("", isNull, value))
-  if (ve.code.nonEmpty) {
+  val eval = doGenCode(ctx, ExprCode("", isNull, value))
+  reduceCodeSize(ctx, eval)
+  if (eval.code.nonEmpty) {
 // Add `this` in the comment.
-ve.copy(code = s"${ctx.registerComment(this.toString)}\n" + 
ve.code.trim)
+eval.copy(code = s"${ctx.registerComment(this.toString)}\n" + 
eval.code.trim)
   } else {
-ve
+eval
   }
 }
   }
 
+  private def reduceCodeSize(ctx: CodegenContext, eval: ExprCode): Unit = {
+// TODO: support whole stage codegen too
+if (eval.code.trim.length > 1024 && ctx.INPUT_ROW != null && 
ctx.currentVars == null) {
+  val setIsNull = if (eval.isNull != "false" && eval.isNull != "true") {
+val globalIsNull = ctx.freshName("globalIsNull")
+ctx.addMutableState(ctx.JAVA_BOOLEAN, globalIsNull)
+val localIsNull = eval.isNull
+eval.isNull = globalIsNull
+s"$globalIsNull = $localIsNull;"
+  } else {
+""
+  }
+
+  val javaType = ctx.javaType(dataType)
+  val newValue = ctx.freshName("value")
+
+  val funcName = ctx.freshName(nodeName)
+  val funcFullName = ctx.addNewFunction(funcName,
+s"""
+   |private $javaType $funcName(InternalRow ${ctx.INPUT_ROW}) {
+   |  ${eval.code.trim}
+   |  $setIsNull
+   |  return ${eval.value};
+   |}
+   """.stripMargin)
+
+  eval.value = newValue
+  eval.code = s"$javaType $newValue = $funcFullName(${ctx.INPUT_ROW});"
+}
+  }
+
   /**
* Returns Java source code that can be compiled to evaluate this expression.
* The 

spark git commit: [SPARK-17920][SPARK-19580][SPARK-19878][SQL] Backport PR 19779 to branch-2.2 - Support writing to Hive table which uses Avro schema url 'avro.schema.url'

2017-11-22 Thread lixiao
Repository: spark
Updated Branches:
  refs/heads/branch-2.2 df9228b49 -> b17f4063c


[SPARK-17920][SPARK-19580][SPARK-19878][SQL] Backport PR 19779 to branch-2.2 - 
Support writing to Hive table which uses Avro schema url 'avro.schema.url'

## What changes were proposed in this pull request?

> Backport https://github.com/apache/spark/pull/19779 to branch-2.2

SPARK-19580 Support for avro.schema.url while writing to hive table
SPARK-19878 Add hive configuration when initialize hive serde in 
InsertIntoHiveTable.scala
SPARK-17920 HiveWriterContainer passes null configuration to serde.initialize, 
causing NullPointerException in AvroSerde when using avro.schema.url

Support writing to Hive table which uses Avro schema url 'avro.schema.url'
For ex:
create external table avro_in (a string) stored as avro location '/avro-in/' 
tblproperties ('avro.schema.url'='/avro-schema/avro.avsc');

create external table avro_out (a string) stored as avro location '/avro-out/' 
tblproperties ('avro.schema.url'='/avro-schema/avro.avsc');

insert overwrite table avro_out select * from avro_in; // fails with 
java.lang.NullPointerException

WARN AvroSerDe: Encountered exception determining schema. Returning signal 
schema to indicate problem
java.lang.NullPointerException
at org.apache.hadoop.fs.FileSystem.getDefaultUri(FileSystem.java:182)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:174)
## Changes proposed in this fix
Currently 'null' value is passed to serializer, which causes NPE during insert 
operation, instead pass Hadoop configuration object
## How was this patch tested?
Added new test case in VersionsSuite

Author: vinodkc 

Closes #19795 from vinodkc/br_Fix_SPARK-17920_branch-2.2.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b17f4063
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b17f4063
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b17f4063

Branch: refs/heads/branch-2.2
Commit: b17f4063cdf52c101ae562ac2a885918acd172ac
Parents: df9228b
Author: vinodkc 
Authored: Wed Nov 22 09:21:26 2017 -0800
Committer: gatorsmile 
Committed: Wed Nov 22 09:21:26 2017 -0800

--
 .../sql/hive/execution/HiveFileFormat.scala |  4 +-
 .../spark/sql/hive/client/VersionsSuite.scala   | 53 +++-
 2 files changed, 54 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b17f4063/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala
--
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala
index ac735e8..4a7cd69 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala
@@ -116,7 +116,7 @@ class HiveOutputWriter(
 
   private val serializer = {
 val serializer = 
tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer]
-serializer.initialize(null, tableDesc.getProperties)
+serializer.initialize(jobConf, tableDesc.getProperties)
 serializer
   }
 
@@ -130,7 +130,7 @@ class HiveOutputWriter(
 
   private val standardOI = ObjectInspectorUtils
 .getStandardObjectInspector(
-  tableDesc.getDeserializer.getObjectInspector,
+  tableDesc.getDeserializer(jobConf).getObjectInspector,
   ObjectInspectorCopyOption.JAVA)
 .asInstanceOf[StructObjectInspector]
 

http://git-wip-us.apache.org/repos/asf/spark/blob/b17f4063/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
--
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
index 7aff49c..d48a23f 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.hive.client
 
-import java.io.{ByteArrayOutputStream, File, PrintStream}
+import java.io.{ByteArrayOutputStream, File, PrintStream, PrintWriter}
 import java.net.URI
 
 import org.apache.hadoop.conf.Configuration
@@ -697,6 +697,57 @@ class VersionsSuite extends SparkFunSuite with Logging {
 assert(versionSpark.table("t1").collect() === Array(Row(2)))
   }
 }
+
+test(s"$version: SPARK-17920: Insert into/overwrite avro table") {
+  withTempDir { dir =>
+val path = dir.getAbsolutePath
+val schemaPath = 

<    5   6   7   8   9   10   11   12   13   14   >