[GitHub] spark issue #20611: [SPARK-23425][SQL]Support wildcard in HDFS path for load...

2018-04-15 Thread sujith71955
Github user sujith71955 commented on the issue:

https://github.com/apache/spark/pull/20611
  
@wzhfy @HyukjinKwon @jiangxb1987 @gatorsmile @vinodkc 
All  test-cases has been passed, please let me know for any other 
suggestions. 
With current solution following improvement has been done
a) We are able to validate the correctness of load path for both HDFS and 
local file systems
b) Wildcards can be used in both folder and file level in local file  and 
HDFS file paths


---

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



[GitHub] spark issue #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark pull request #21033: [SPARK-19320][MESOS]allow specifying a hard limit...

2018-04-15 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/21033#discussion_r181627739
  
--- Diff: 
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
 ---
@@ -495,9 +500,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
   launchTasks = true
   val taskId = newMesosTaskId()
   val offerCPUs = getResource(resources, "cpus").toInt
-  val taskGPUs = Math.min(
-Math.max(0, maxGpus - totalGpusAcquired), 
getResource(resources, "gpus").toInt)
-
+  val offerGPUs = getResource(resources, "gpus").toInt
+  var taskGPUs = executorGpus
--- End diff --

so looks like we are changing the behavior for the value set in 
`spark.mesos.gpus.max` (since 2.1)? we are ok with that/that might break 
existing deployment? is there a migration guide for something like this?

in addition, is there other changes by default - specifically now taskGPUs 
defaults to 0?

also, should we warn if `spark.mesos.executor.gpus` is > 
`spark.mesos.gpus.max`?


---

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



[GitHub] spark issue #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark pull request #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/21053#discussion_r181624253
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,106 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns the value of index `right` in Array `left` or the value for key 
`right` in Map `left`.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, index) - Returns element of array at given index. If 
index < 0, accesses elements
+  from the last to the first.
+
+_FUNC_(map, key) - Returns value for given key, or NULL if the key is 
not contained in the map
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), 2);
+   2
+  > SELECT _FUNC_(map(1, 'a', 2, 'b'), 2);
+   "b"
+  """,
+  since = "2.4.0")
+case class ElementAt(left: Expression, right: Expression) extends 
GetMapValueUtil {
+
+  override def dataType: DataType = left.dataType match {
+case _: ArrayType => left.dataType.asInstanceOf[ArrayType].elementType
+case _: MapType => left.dataType.asInstanceOf[MapType].valueType
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = {
+Seq(TypeCollection(ArrayType, MapType),
+  left.dataType match {
+case _: ArrayType => IntegerType
+case _: MapType => left.dataType.asInstanceOf[MapType].keyType
+  }
+)
+  }
+
+  override def nullable: Boolean = true
--- End diff --

ah, I see. Invalid `right` can cause null result too.


---

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



[GitHub] spark pull request #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21053#discussion_r181623803
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,106 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns the value of index `right` in Array `left` or the value for key 
`right` in Map `left`.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, index) - Returns element of array at given index. If 
index < 0, accesses elements
+  from the last to the first.
+
+_FUNC_(map, key) - Returns value for given key, or NULL if the key is 
not contained in the map
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), 2);
+   2
+  > SELECT _FUNC_(map(1, 'a', 2, 'b'), 2);
+   "b"
+  """,
+  since = "2.4.0")
+case class ElementAt(left: Expression, right: Expression) extends 
GetMapValueUtil {
+
+  override def dataType: DataType = left.dataType match {
+case _: ArrayType => left.dataType.asInstanceOf[ArrayType].elementType
+case _: MapType => left.dataType.asInstanceOf[MapType].valueType
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = {
+Seq(TypeCollection(ArrayType, MapType),
+  left.dataType match {
+case _: ArrayType => IntegerType
+case _: MapType => left.dataType.asInstanceOf[MapType].keyType
+  }
+)
+  }
+
+  override def nullable: Boolean = true
--- End diff --

I'm afraid it's wrong because this returns `null` when the given index is 
"out of bounds" (`array.numElements() < math.abs(index)`) for array type or the 
given key doesn't exist for map type.


---

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



[GitHub] spark pull request #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/21053#discussion_r181623729
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,106 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns the value of index `right` in Array `left` or the value for key 
`right` in Map `left`.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, index) - Returns element of array at given index. If 
index < 0, accesses elements
+  from the last to the first.
+
+_FUNC_(map, key) - Returns value for given key, or NULL if the key is 
not contained in the map
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), 2);
+   2
+  > SELECT _FUNC_(map(1, 'a', 2, 'b'), 2);
+   "b"
+  """,
+  since = "2.4.0")
+case class ElementAt(left: Expression, right: Expression) extends 
GetMapValueUtil {
+
+  override def dataType: DataType = left.dataType match {
+case _: ArrayType => left.dataType.asInstanceOf[ArrayType].elementType
+case _: MapType => left.dataType.asInstanceOf[MapType].valueType
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = {
+Seq(TypeCollection(ArrayType, MapType),
+  left.dataType match {
+case _: ArrayType => IntegerType
+case _: MapType => left.dataType.asInstanceOf[MapType].keyType
+  }
+)
+  }
+
+  override def nullable: Boolean = true
+
+  override def nullSafeEval(value: Any, ordinal: Any): Any = {
+left.dataType match {
+  case _: ArrayType =>
+val array = value.asInstanceOf[ArrayData]
+val index = ordinal.asInstanceOf[Int]
+if (array.numElements() < math.abs(index)) {
+  null
+} else {
+  val idx = if (index == 0) {
+throw new ArrayIndexOutOfBoundsException("SQL array indices 
start at 1")
+  } else if (index > 0) {
+index - 1
+  } else {
+array.numElements() + index
+  }
+  if (array.isNullAt(idx)) {
--- End diff --

nit: 
```scala
if(left.dataType.asInstanceOf[ArrayType].containsNull && 
array.isNullAt(idx)) {
  ...
}
```


---

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



[GitHub] spark pull request #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/21053#discussion_r181623519
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,106 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns the value of index `right` in Array `left` or the value for key 
`right` in Map `left`.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, index) - Returns element of array at given index. If 
index < 0, accesses elements
+  from the last to the first.
--- End diff --

nit: `Returns NULL if the index exceeds the length of the array`.


---

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



[GitHub] spark pull request #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/21053#discussion_r181622879
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,106 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns the value of index `right` in Array `left` or the value for key 
`right` in Map `left`.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, index) - Returns element of array at given index. If 
index < 0, accesses elements
+  from the last to the first.
+
+_FUNC_(map, key) - Returns value for given key, or NULL if the key is 
not contained in the map
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), 2);
+   2
+  > SELECT _FUNC_(map(1, 'a', 2, 'b'), 2);
+   "b"
+  """,
+  since = "2.4.0")
+case class ElementAt(left: Expression, right: Expression) extends 
GetMapValueUtil {
+
+  override def dataType: DataType = left.dataType match {
+case _: ArrayType => left.dataType.asInstanceOf[ArrayType].elementType
+case _: MapType => left.dataType.asInstanceOf[MapType].valueType
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = {
+Seq(TypeCollection(ArrayType, MapType),
+  left.dataType match {
+case _: ArrayType => IntegerType
+case _: MapType => left.dataType.asInstanceOf[MapType].keyType
+  }
+)
+  }
+
+  override def nullable: Boolean = true
--- End diff --

Maybe?
```scala
override def nullable: Boolean = left.dataType match {
case a: ArrayType => a.containsNull
case m: MapType => m.valueContainsNull
} ||  left.nullable || right.nullable
```


---

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



[GitHub] spark pull request #21031: [SPARK-23923][SQL] Add cardinality function

2018-04-15 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/21031#discussion_r181622107
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 ---
@@ -407,6 +407,7 @@ object FunctionRegistry {
 expression[MapKeys]("map_keys"),
 expression[MapValues]("map_values"),
 expression[Size]("size"),
+expression[Cardinality]("cardinality"),
--- End diff --

`expression[Size]("cardinality"),` Does this work?


---

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



[GitHub] spark pull request #21031: [SPARK-23923][SQL] Add cardinality function

2018-04-15 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/21031#discussion_r181621963
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -3282,6 +3282,14 @@ object functions {
*/
   def size(e: Column): Column = withExpr { Size(e.expr) }
 
+  /**
+   * Returns length of array or map as long.
+   *
+   * @group collection_funcs
+   * @since 2.4.0
+   */
+  def cardinality(e: Column): Column = withExpr { Cardinality(e.expr) }
--- End diff --

Do not add the function APIs. Adding the SQL function is enough.


---

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



[GitHub] spark pull request #21024: [SPARK-23917][SQL] Add array_max function

2018-04-15 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark issue #21025: [SPARK-23918][SQL] Add array_min function

2018-04-15 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/21025
  
@mgaido91 Could you address the conflicts? Thanks!


---

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



[GitHub] spark issue #21024: [SPARK-23917][SQL] Add array_max function

2018-04-15 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/21024
  
LGTM. Thanks! Merged to master.


---

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



[GitHub] spark pull request #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21053#discussion_r181619426
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,106 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns the value of index `right` in Array `left` or the value for key 
`right` in Map `left`.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, index) - Returns element of array at given index. If 
index < 0, accesses elements
+  from the last to the first.
+
+_FUNC_(map, key) - Returns value for given key, or NULL if the key is 
not contained in the map
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), 2);
+   2
+  > SELECT _FUNC_(map(1, 'a', 2, 'b'), 2);
+   "b"
+  """,
+  since = "2.4.0")
+case class ElementAt(left: Expression, right: Expression) extends 
GetMapValueUtil {
+
+  override def dataType: DataType = left.dataType match {
+case _: ArrayType => left.dataType.asInstanceOf[ArrayType].elementType
+case _: MapType => left.dataType.asInstanceOf[MapType].valueType
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = {
+Seq(TypeCollection(ArrayType, MapType),
+  left.dataType match {
+case _: ArrayType => IntegerType
+case _: MapType => left.dataType.asInstanceOf[MapType].keyType
+  }
+)
+  }
+
+  override def nullable: Boolean = true
+
+  override def nullSafeEval(value: Any, ordinal: Any): Any = {
+left.dataType match {
+  case _: ArrayType =>
+val array = value.asInstanceOf[ArrayData]
+val index = ordinal.asInstanceOf[Int]
+if (array.numElements() < math.abs(index)) {
+  null
+} else {
+  val idx = if (index == 0) {
+throw new ArrayIndexOutOfBoundsException("SQL array indices 
start at 1")
+  } else if (index > 0) {
+index - 1
+  } else {
+array.numElements() + index
+  }
+  if (array.isNullAt(idx)) {
+null
+  } else {
+array.get(idx, dataType)
+  }
+}
+  case _: MapType =>
+getValueEval(value, ordinal, 
left.dataType.asInstanceOf[MapType].keyType)
+}
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+left.dataType match {
+  case _: ArrayType =>
+nullSafeCodeGen(ctx, ev, (eval1, eval2) => {
+  val index = ctx.freshName("elementAtIndex")
+  val nullCheck = if 
(left.dataType.asInstanceOf[ArrayType].containsNull) {
+s"""
+   |if ($eval1.isNullAt($index)) {
+   |  ${ev.isNull} = true;
+   |} else
+ """
+  } else {
+""
+  }
+  s"""
+ |int $index = (int) $eval2;
+ |if ($eval1.numElements() < Math.abs($index)) {
+ |  ${ev.isNull} = true;
+ |} else {
+ |  if ($index == 0) {
+ |throw new ArrayIndexOutOfBoundsException("SQL array 
indices start at 1");
+ |  } else if ($index > 0) {
+ |$index--;
+ |  } else {
+ |$index += $eval1.numElements();
+ |  }
+ |  $nullCheck
+ |  {
+ |${ev.value} = ${CodeGenerator.getValue(eval1, dataType, 
index)};
+ |  }
+ |}
+   """
--- End diff --

`stripMargin` is missing?


---

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



[GitHub] spark pull request #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21053#discussion_r181618055
  
--- Diff: python/pyspark/sql/functions.py ---
@@ -1846,6 +1846,27 @@ def array_contains(col, value):
 return Column(sc._jvm.functions.array_contains(_to_java_column(col), 
value))
 
 
+@since(2.4)
--- End diff --

We need to annotate as `@ignore_unicode_prefix`?


---

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



[GitHub] spark pull request #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21053#discussion_r181617303
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
 ---
@@ -354,3 +336,37 @@ case class GetMapValue(child: Expression, key: 
Expression)
 })
   }
 }
+
+/**
+ * Returns the value of key `key` in Map `child`.
+ *
+ * We need to do type checking here as `key` expression maybe unresolved.
+ */
+case class GetMapValue(child: Expression, key: Expression) extends 
GetMapValueUtil
+  with ExtractValue with NullIntolerant {
--- End diff --

nit: maybe `extends ...` should be this line.


---

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



[GitHub] spark pull request #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21053#discussion_r181619450
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,106 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns the value of index `right` in Array `left` or the value for key 
`right` in Map `left`.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, index) - Returns element of array at given index. If 
index < 0, accesses elements
+  from the last to the first.
+
+_FUNC_(map, key) - Returns value for given key, or NULL if the key is 
not contained in the map
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), 2);
+   2
+  > SELECT _FUNC_(map(1, 'a', 2, 'b'), 2);
+   "b"
+  """,
+  since = "2.4.0")
+case class ElementAt(left: Expression, right: Expression) extends 
GetMapValueUtil {
+
+  override def dataType: DataType = left.dataType match {
+case _: ArrayType => left.dataType.asInstanceOf[ArrayType].elementType
+case _: MapType => left.dataType.asInstanceOf[MapType].valueType
+  }
+
+  override def inputTypes: Seq[AbstractDataType] = {
+Seq(TypeCollection(ArrayType, MapType),
+  left.dataType match {
+case _: ArrayType => IntegerType
+case _: MapType => left.dataType.asInstanceOf[MapType].keyType
+  }
+)
+  }
+
+  override def nullable: Boolean = true
+
+  override def nullSafeEval(value: Any, ordinal: Any): Any = {
+left.dataType match {
+  case _: ArrayType =>
+val array = value.asInstanceOf[ArrayData]
+val index = ordinal.asInstanceOf[Int]
+if (array.numElements() < math.abs(index)) {
+  null
+} else {
+  val idx = if (index == 0) {
+throw new ArrayIndexOutOfBoundsException("SQL array indices 
start at 1")
+  } else if (index > 0) {
+index - 1
+  } else {
+array.numElements() + index
+  }
+  if (array.isNullAt(idx)) {
+null
+  } else {
+array.get(idx, dataType)
+  }
+}
+  case _: MapType =>
+getValueEval(value, ordinal, 
left.dataType.asInstanceOf[MapType].keyType)
+}
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+left.dataType match {
+  case _: ArrayType =>
+nullSafeCodeGen(ctx, ev, (eval1, eval2) => {
+  val index = ctx.freshName("elementAtIndex")
+  val nullCheck = if 
(left.dataType.asInstanceOf[ArrayType].containsNull) {
+s"""
+   |if ($eval1.isNullAt($index)) {
+   |  ${ev.isNull} = true;
+   |} else
+ """
--- End diff --

`stripMargin` is missing?


---

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



[GitHub] spark pull request #21074: [SPARK-21811][SQL] Fix the inconsistency behavior...

2018-04-15 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/21074#discussion_r181620108
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
 ---
@@ -178,7 +178,13 @@ object TypeCoercion {
   private def findWiderCommonType(types: Seq[DataType]): Option[DataType] 
= {
 types.foldLeft[Option[DataType]](Some(NullType))((r, c) => r match {
   case Some(d) => findWiderTypeForTwo(d, c)
-  case None => None
+  // Currently we find the wider common type by comparing the two 
types from left to right,
--- End diff --

This is a behavior change. We need to make it configurable. Add a conf and 
update the migration guide.


---

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



[GitHub] spark pull request #21074: [SPARK-21811][SQL] Fix the inconsistency behavior...

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

https://github.com/apache/spark/pull/21074#discussion_r181619518
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
 ---
@@ -178,7 +178,13 @@ object TypeCoercion {
   private def findWiderCommonType(types: Seq[DataType]): Option[DataType] 
= {
 types.foldLeft[Option[DataType]](Some(NullType))((r, c) => r match {
   case Some(d) => findWiderTypeForTwo(d, c)
-  case None => None
+  // Currently we find the wider common type by comparing the two 
types from left to right,
--- End diff --

The real problem is, `findWiderTypeForTwo` doesn't satisfy the associative 
law, i.e. `(a op b) op c` may not equal to `a op (b op c)`. I think 
`StringType` is the only exception here, it's more clear to do
```
val (stringType, nonStringType) = types.partition(_ == StringType)
(stringType.distinct ++ nonStringType).foldLeft...
```


---

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



[GitHub] spark pull request #21047: [SPARK-23956][YARN] Use effective RPC port in AM ...

2018-04-15 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark issue #21047: [SPARK-23956][YARN] Use effective RPC port in AM registr...

2018-04-15 Thread jerryshao
Github user jerryshao commented on the issue:

https://github.com/apache/spark/pull/21047
  
Merging to master. Thanks!


---

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



[GitHub] spark issue #21047: [SPARK-23956][YARN] Use effective RPC port in AM registr...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21047: [SPARK-23956][YARN] Use effective RPC port in AM registr...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #21047: [SPARK-23956][YARN] Use effective RPC port in AM registr...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark pull request #21037: [SPARK-23919][SQL] Add array_position function

2018-04-15 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21037#discussion_r181614224
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,60 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns the position of the first occurrence of element in the given 
array as long.
+ * Returns 0 if substr could not be found in str. Returns null if either 
of the arguments are null
+ *
+ * NOTE: that this is not zero based, but 1-based index. The first 
character in str has index 1.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, element) - Returns the (1-based) index of the first 
element of the array as long.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(3, 2, 1), 1);
+   3
+  """,
+  since = "2.4.0")
+case class ArrayPosition(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def dataType: DataType = LongType
+  override def inputTypes: Seq[AbstractDataType] =
+Seq(ArrayType, left.dataType.asInstanceOf[ArrayType].elementType)
+
+  override def nullable: Boolean = {
+left.nullable || right.nullable || 
left.dataType.asInstanceOf[ArrayType].containsNull
--- End diff --

I guess `left.dataType.asInstanceOf[ArrayType].containsNull` is not related 
to the nullability of this function?


---

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



[GitHub] spark pull request #21037: [SPARK-23919][SQL] Add array_position function

2018-04-15 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21037#discussion_r181613270
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,60 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns the position of the first occurrence of element in the given 
array as long.
+ * Returns 0 if substr could not be found in str. Returns null if either 
of the arguments are null
+ *
+ * NOTE: that this is not zero based, but 1-based index. The first 
character in str has index 1.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, element) - Returns the (1-based) index of the first 
element of the array as long.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(3, 2, 1), 1);
+   3
+  """,
+  since = "2.4.0")
+case class ArrayPosition(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def dataType: DataType = LongType
+  override def inputTypes: Seq[AbstractDataType] =
+Seq(ArrayType, left.dataType.asInstanceOf[ArrayType].elementType)
+
+  override def nullable: Boolean = {
+left.nullable || right.nullable || 
left.dataType.asInstanceOf[ArrayType].containsNull
+  }
+
+  override def nullSafeEval(arr: Any, value: Any): Any = {
+arr.asInstanceOf[ArrayData].foreach(right.dataType, (i, v) =>
+  if (v == value) {
+return (i + 1).toLong
+  }
+)
+0L
+  }
+
+  override def prettyName: String = "array_position"
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+nullSafeCodeGen(ctx, ev, (arr, value) => {
+  val pos = ctx.freshName("arrayPosition")
+  val i = ctx.freshName("i")
+  val getValue = CodeGenerator.getValue(arr, right.dataType, i)
+  s"""
+ |int ${pos} = 0;
--- End diff --

nit: `$pos`


---

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



[GitHub] spark pull request #21037: [SPARK-23919][SQL] Add array_position function

2018-04-15 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21037#discussion_r181616266
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
 ---
@@ -105,4 +106,26 @@ class CollectionExpressionsSuite extends SparkFunSuite 
with ExpressionEvalHelper
 checkEvaluation(ArrayContains(a3, Literal("")), null)
 checkEvaluation(ArrayContains(a3, Literal.create(null, StringType)), 
null)
   }
+
+
--- End diff --

nit: remove an extra line.


---

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



[GitHub] spark pull request #21037: [SPARK-23919][SQL] Add array_position function

2018-04-15 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21037#discussion_r181613044
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,60 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns the position of the first occurrence of element in the given 
array as long.
+ * Returns 0 if substr could not be found in str. Returns null if either 
of the arguments are null
+ *
+ * NOTE: that this is not zero based, but 1-based index. The first 
character in str has index 1.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, element) - Returns the (1-based) index of the first 
element of the array as long.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(3, 2, 1), 1);
+   3
+  """,
+  since = "2.4.0")
+case class ArrayPosition(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def dataType: DataType = LongType
+  override def inputTypes: Seq[AbstractDataType] =
+Seq(ArrayType, left.dataType.asInstanceOf[ArrayType].elementType)
+
+  override def nullable: Boolean = {
+left.nullable || right.nullable || 
left.dataType.asInstanceOf[ArrayType].containsNull
+  }
+
+  override def nullSafeEval(arr: Any, value: Any): Any = {
+arr.asInstanceOf[ArrayData].foreach(right.dataType, (i, v) =>
+  if (v == value) {
+return (i + 1).toLong
+  }
+)
+0L
+  }
+
+  override def prettyName: String = "array_position"
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+nullSafeCodeGen(ctx, ev, (arr, value) => {
+  val pos = ctx.freshName("arrayPosition")
+  val i = ctx.freshName("i")
+  val getValue = CodeGenerator.getValue(arr, right.dataType, i)
+  s"""
+ |int ${pos} = 0;
+ |for (int $i = 0; $i < $arr.numElements(); $i ++) {
+ |  if (${ctx.genEqual(right.dataType, value, getValue)}) {
+ |${pos} = $i + 1;
+ |break;
+ |  }
+ |}
+ |${ev.value} = (long) $pos;
+   """
--- End diff --

`stripMargin` is missing?


---

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



[GitHub] spark pull request #21037: [SPARK-23919][SQL] Add array_position function

2018-04-15 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21037#discussion_r181615751
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,60 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns the position of the first occurrence of element in the given 
array as long.
+ * Returns 0 if substr could not be found in str. Returns null if either 
of the arguments are null
+ *
+ * NOTE: that this is not zero based, but 1-based index. The first 
character in str has index 1.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, element) - Returns the (1-based) index of the first 
element of the array as long.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(3, 2, 1), 1);
+   3
+  """,
+  since = "2.4.0")
+case class ArrayPosition(left: Expression, right: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def dataType: DataType = LongType
+  override def inputTypes: Seq[AbstractDataType] =
+Seq(ArrayType, left.dataType.asInstanceOf[ArrayType].elementType)
+
+  override def nullable: Boolean = {
+left.nullable || right.nullable || 
left.dataType.asInstanceOf[ArrayType].containsNull
+  }
+
+  override def nullSafeEval(arr: Any, value: Any): Any = {
+arr.asInstanceOf[ArrayData].foreach(right.dataType, (i, v) =>
+  if (v == value) {
+return (i + 1).toLong
+  }
+)
+0L
+  }
+
+  override def prettyName: String = "array_position"
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+nullSafeCodeGen(ctx, ev, (arr, value) => {
+  val pos = ctx.freshName("arrayPosition")
+  val i = ctx.freshName("i")
+  val getValue = CodeGenerator.getValue(arr, right.dataType, i)
+  s"""
+ |int ${pos} = 0;
+ |for (int $i = 0; $i < $arr.numElements(); $i ++) {
+ |  if (${ctx.genEqual(right.dataType, value, getValue)}) {
--- End diff --

I guess we need to check null for each array element?

E.g. for the test `Array Position` in `CollectionExpressionsSuite`, if `a0` 
is as follows:

```
val a0 = Literal.create(Seq(1, null, 2, 3), ArrayType(IntegerType))
```

`checkEvaluation(ArrayPosition(a0, Literal(0)), 0L)` will fail.



---

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



[GitHub] spark pull request #21037: [SPARK-23919][SQL] Add array_position function

2018-04-15 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21037#discussion_r181614388
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +287,61 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * A function that returns the position of the first occurrence of element 
in the given array
+ * as long. Returns 0 if substr could not be found in str.
+ * Returns null if either of the arguments are null and
+ *
+ * NOTE: that this is not zero based, but 1-based index. The first 
character in str has index 1.
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array, element) - Returns the (1-based) index of the first 
element of the array as long.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(3, 2, 1), 1);
+   3
+  """,
+  since = "2.4.0")
+case class ArrayPosition(left: Expression, right: Expression)
--- End diff --

So we can't know the position of `null` in the array even if the array 
contains `null`?


---

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



[GitHub] spark issue #21047: [SPARK-23956][YARN] Use effective RPC port in AM registr...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #21047: [SPARK-23956][YARN] Use effective RPC port in AM registr...

2018-04-15 Thread jerryshao
Github user jerryshao commented on the issue:

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


---

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



[GitHub] spark issue #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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

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


---

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



[GitHub] spark issue #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21053
  
**[Test build #89383 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89383/testReport)**
 for PR 21053 at commit 
[`013a53e`](https://github.com/apache/spark/commit/013a53e065396b9b0f8f10acb2c091f05b780013).


---

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



[GitHub] spark issue #21074: [SPARK-21811][SQL] Fix the inconsistency behavior when f...

2018-04-15 Thread jiangxb1987
Github user jiangxb1987 commented on the issue:

https://github.com/apache/spark/pull/21074
  
cc @HyukjinKwon @gatorsmile @cloud-fan 


---

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



[GitHub] spark pull request #20940: [SPARK-23429][CORE] Add executor memory metrics t...

2018-04-15 Thread edwinalu
Github user edwinalu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20940#discussion_r181611071
  
--- Diff: 
core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
@@ -234,8 +244,22 @@ private[spark] class EventLoggingListener(
 }
   }
 
-  // No-op because logging every update would be overkill
-  override def onExecutorMetricsUpdate(event: 
SparkListenerExecutorMetricsUpdate): Unit = { }
+  /**
+   * Log if there is a new peak value for one of the memory metrics for 
the given executor.
+   * Metrics are cleared out when a new stage is started in 
onStageSubmitted, so this will
+   * log new peak memory metric values per executor per stage.
+   */
+  override def onExecutorMetricsUpdate(event: 
SparkListenerExecutorMetricsUpdate): Unit = {
--- End diff --

ExecutorMetrics right now has: jvmUsedHeapMemory, jvmUsedNonHeapMemory, 
onHeapExecutionMemory, offHeapExecutionMemory, onHeapStorageMemory, and 
offHeapStorageMemory. For logging at stage end, we can log the peak for each of 
these, but unified memory is more problematic. We could add new fields for on 
heap/off heap unified memory, but I'm inclined to remove unified memory (from 
all the places it is currently used), rather than add more fields. Users can 
still sum peak execution and peak storage values, which may be larger than the 
actual peak unified memory if they are not at peak values at the same time, but 
should still be a reasonable estimate for sizing.


---

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



[GitHub] spark pull request #21053: [SPARK-23924][SQL] Add element_at function

2018-04-15 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21053#discussion_r181610494
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala ---
@@ -413,6 +413,78 @@ class DataFrameFunctionsSuite extends QueryTest with 
SharedSQLContext {
 )
   }
 
+  test("element at function") {
--- End diff --

good catch, thanks


---

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



[GitHub] spark issue #18378: [SPARK-21163][SQL] DataFrame.toPandas should respect the...

2018-04-15 Thread edlee123
Github user edlee123 commented on the issue:

https://github.com/apache/spark/pull/18378
  
I see the rationale now, thank you everyone


---

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



[GitHub] spark issue #20535: [SPARK-23341][SQL] define some standard options for data...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20535: [SPARK-23341][SQL] define some standard options for data...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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

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


---

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



[GitHub] spark issue #20535: [SPARK-23341][SQL] define some standard options for data...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20535: [SPARK-23341][SQL] define some standard options for data...

2018-04-15 Thread cloud-fan
Github user cloud-fan commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark pull request #19868: [SPARK-22676] Avoid iterating all partition paths...

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

https://github.com/apache/spark/pull/19868#discussion_r181607994
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala ---
@@ -70,6 +71,45 @@ class QueryPartitionSuite extends QueryTest with 
SQLTestUtils with TestHiveSingl
 }
   }
 
+  test("Replace spark.sql.hive.verifyPartitionPath by 
spark.files.ignoreMissingFiles") {
--- End diff --

we should add some document for `spark.sql.hive.verifyPartitionPath`, say 
it's replaced by `spark.files.ignoreMissingFiles` and will be removed in future 
releases.


---

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



[GitHub] spark pull request #19868: [SPARK-22676] Avoid iterating all partition paths...

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

https://github.com/apache/spark/pull/19868#discussion_r181607863
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala ---
@@ -279,6 +293,10 @@ class HadoopRDD[K, V](
   case e: IOException if ignoreCorruptFiles =>
 logWarning(s"Skipped the rest content in the corrupted file: 
${split.inputSplit}", e)
 finished = true
+  case e: FileNotFoundException if ignoreMissingFiles =>
+logWarning(s"Skipped missing file: ${split.inputSplit}", e)
+finished = true
+null
--- End diff --

why returning null here?


---

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



[GitHub] spark pull request #19868: [SPARK-22676] Avoid iterating all partition paths...

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

https://github.com/apache/spark/pull/19868#discussion_r181607797
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala ---
@@ -260,6 +270,10 @@ class HadoopRDD[K, V](
 logWarning(s"Skipped the rest content in the corrupted file: 
${split.inputSplit}", e)
 finished = true
 null
+  case e: FileNotFoundException if ignoreMissingFiles =>
--- End diff --

`FileNotFoundException` extends `IOException`


---

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



[GitHub] spark pull request #19868: [SPARK-22676] Avoid iterating all partition paths...

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

https://github.com/apache/spark/pull/19868#discussion_r181607746
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala ---
@@ -197,17 +200,24 @@ class HadoopRDD[K, V](
 val jobConf = getJobConf()
 // add the credentials here as this can be called before SparkContext 
initialized
 SparkHadoopUtil.get.addCredentials(jobConf)
-val allInputSplits = getInputFormat(jobConf).getSplits(jobConf, 
minPartitions)
-val inputSplits = if (ignoreEmptySplits) {
-  allInputSplits.filter(_.getLength > 0)
-} else {
-  allInputSplits
-}
-val array = new Array[Partition](inputSplits.size)
-for (i <- 0 until inputSplits.size) {
-  array(i) = new HadoopPartition(id, i, inputSplits(i))
+try {
+  val allInputSplits = getInputFormat(jobConf).getSplits(jobConf, 
minPartitions)
+  val inputSplits = if (ignoreEmptySplits) {
+allInputSplits.filter(_.getLength > 0)
+  } else {
+allInputSplits
+  }
+  val array = new Array[Partition](inputSplits.size)
+  for (i <- 0 until inputSplits.size) {
+array(i) = new HadoopPartition(id, i, inputSplits(i))
+  }
+  array
+} catch {
+  case e: InvalidInputException if ignoreMissingFiles =>
--- End diff --

when will this happen? the root path doesn't exist?


---

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



[GitHub] spark issue #18378: [SPARK-21163][SQL] DataFrame.toPandas should respect the...

2018-04-15 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/18378
  
It's pretty natural to convert integer type to int32. Although Spark tries 
its best to avoid behavior changes, it's allowed to fix some wrong behaviors in 
new releases, and I believe it's well documented in the Spark 2.3 release notes.


---

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



[GitHub] spark issue #21060: [SPARK-23942][PYTHON][SQL][BRANCH-2.3] Makes collect in ...

2018-04-15 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21060
  
cc @rdblue and @steveloughran too who I guess should be interested in 
setting up a backporting policy.


---

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



[GitHub] spark issue #21060: [SPARK-23942][PYTHON][SQL][BRANCH-2.3] Makes collect in ...

2018-04-15 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21060
  
I am not saying we shouldn't be careful. I am trying to be careful when I 
backport. So, your reasons are:

- any behaviour changes shouldn't be backported and it's the basic backport 
rule

  I disagree unless it's clearly documented as a rule. Even if so, I would 
like to make this as an exception because it's less invasive, looks a bug, 
affects an actual user group and fixes the case to make it sense. That's what I 
have been used to so far.

- the query execution listener is not clearly defined

  I am seeing `collect` is included in the original commit - 
https://github.com/apache/spark/commit/15ff85b3163acbe8052d4489a00bcf1d2332fcf0.
 I don't see a reason to specifically exclude PySpark's case since Scala and R 
also work. I don't think we would exclude this on purpose.

- It's not a critical issue nor a regression.

  I don't think we should only make a backport for a critical issue or a 
regression. That's a strong reason to backport but there are still other cases 
that can be backported based on my understanding and observations. If it's a 
bug quite clearly and it affects an actual user group, I would guess it can be 
valuable for a backport. The fix is straightforward, less invasive and small.



---

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



[GitHub] spark issue #21060: [SPARK-23942][PYTHON][SQL][BRANCH-2.3] Makes collect in ...

2018-04-15 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/21060
  
Like what I said above, we need to be very careful when backporting the PR 
with the behavior changes, especially when this is **neither a critical issue 
nor a regression**. Even if this is a bug based on your understanding, we 
should still not backport such PRs. 


---

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



[GitHub] spark issue #21075: [SPARK-23988][MESOS] Improve handling of appResource in ...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21075
  
Can one of the admins verify this patch?


---

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



[GitHub] spark issue #21075: [SPARK-23988][MESOS] Improve handling of appResource in ...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21075
  
Can one of the admins verify this patch?


---

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



[GitHub] spark pull request #21075: [SPARK-23988][MESOS] Improve handling of appResou...

2018-04-15 Thread pmackles
GitHub user pmackles opened a pull request:

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

[SPARK-23988][MESOS] Improve handling of appResource in mesos dispatcher 
when using Docker

Improve/fix handling of appResource for mesos dispatcher when using docker

Tested with new unit tests as well as manually on private mesos cluster


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

$ git pull https://github.com/pmackles/spark SPARK-23988

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

https://github.com/apache/spark/pull/21075.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #21075


commit fdcd34db68a81169e71dc08c3e3dd6d573609365
Author: Paul Mackles 
Date:   2018-04-15T23:08:42Z

[SPARK-23988] Improve handling of appResource in mesos dispatcher when 
using Docker




---

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



[GitHub] spark issue #21060: [SPARK-23942][PYTHON][SQL][BRANCH-2.3] Makes collect in ...

2018-04-15 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21060
  
This is not just about just inconsistency but a bug. The previous behaivour 
doesn't make sense.

Sure, no need to rush.


---

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



[GitHub] spark issue #20611: [SPARK-23425][SQL]Support wildcard in HDFS path for load...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20611: [SPARK-23425][SQL]Support wildcard in HDFS path for load...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20611: [SPARK-23425][SQL]Support wildcard in HDFS path for load...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #21073: [SPARK-23936][SQL][WIP] Implement map_concat

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21073: [SPARK-23936][SQL][WIP] Implement map_concat

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21073: [SPARK-23936][SQL][WIP] Implement map_concat

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #21074: [SPARK-21811][SQL] Fix the inconsistency behavior when f...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21074: [SPARK-21811][SQL] Fix the inconsistency behavior when f...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21074: [SPARK-21811][SQL] Fix the inconsistency behavior when f...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark pull request #19526: [SPARK-22014][SQL] removed TypeCheckFailure: slid...

2018-04-15 Thread SimonUzL
Github user SimonUzL closed the pull request at:

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


---

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



[GitHub] spark issue #20611: [SPARK-23425][SQL]Support wildcard in HDFS path for load...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #21034: [SPARK-23926][SQL] Extending reverse function to support...

2018-04-15 Thread mn-mikke
Github user mn-mikke commented on the issue:

https://github.com/apache/spark/pull/21034
  
Any other comments?


---

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



[GitHub] spark pull request #20938: [SPARK-23821][SQL] Collection function: flatten

2018-04-15 Thread mn-mikke
Github user mn-mikke commented on a diff in the pull request:

https://github.com/apache/spark/pull/20938#discussion_r181593152
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +289,160 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Transforms an array of arrays into a single array.
+ */
+@ExpressionDescription(
+  usage = "_FUNC_(arrayOfArrays) - Transforms an array of arrays into a 
single array.",
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(array(1, 2), array(3, 4));
+   [1,2,3,4]
+  """,
+  since = "2.4.0")
+case class Flatten(child: Expression) extends UnaryExpression {
+
+  override def nullable: Boolean = child.nullable || dataType.containsNull
+
+  override def dataType: ArrayType = {
+child
+  .dataType.asInstanceOf[ArrayType]
+  .elementType.asInstanceOf[ArrayType]
+  }
+
+  override def checkInputDataTypes(): TypeCheckResult = child.dataType 
match {
+case ArrayType(_: ArrayType, _) =>
+  TypeCheckResult.TypeCheckSuccess
+case _ =>
+  TypeCheckResult.TypeCheckFailure(
+s"The argument should be an array of arrays, " +
+s"but '${child.sql}' is of ${child.dataType.simpleString} type."
+  )
+  }
+
+  override def nullSafeEval(array: Any): Any = {
+val elements = array.asInstanceOf[ArrayData].toObjectArray(dataType)
+
+if (elements.contains(null)) {
+  null
+} else {
+  val flattened = elements.flatMap(
--- End diff --

I've been searching for a well-defined constant indicating the VM limit for 
array size. It seems that the limit is platform-dependent... Any idea how to 
get the limit for a given platform?


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20937
  
**[Test build #89379 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89379/testReport)**
 for PR 20937 at commit 
[`6c0df03`](https://github.com/apache/spark/commit/6c0df038138c2d2f5bcf8579728b673a2c6a5fdc).


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark pull request #21061: [SPARK-23914][SQL] Add array_union function

2018-04-15 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21061#discussion_r181590066
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -287,3 +288,80 @@ case class ArrayContains(left: Expression, right: 
Expression)
 
   override def prettyName: String = "array_contains"
 }
+
+/**
+ * Returns an array of the elements in the union of x and y, without 
duplicates
+ */
+@ExpressionDescription(
+  usage = """
+_FUNC_(array1, array2) - Returns an array of the elements in the union 
of array1 and array2,
+  without duplicates.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5));
+   array(1, 2, 3, 5)
+  """,
+  since = "2.4.0")
+case class ArrayUnion(left: Expression, right: Expression)
+  extends BinaryExpression with ExpectsInputTypes with CodegenFallback {
--- End diff --

I see.


---

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



[GitHub] spark pull request #21070: SPARK-23972: Update Parquet to 1.10.0.

2018-04-15 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21070#discussion_r181589911
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java
 ---
@@ -63,115 +58,139 @@ public final void readBooleans(int total, 
WritableColumnVector c, int rowId) {
 }
   }
 
+  private ByteBuffer getBuffer(int length) {
+try {
+  return in.slice(length).order(ByteOrder.LITTLE_ENDIAN);
+} catch (IOException e) {
+  throw new ParquetDecodingException("Failed to read " + length + " 
bytes", e);
+}
+  }
+
   @Override
   public final void readIntegers(int total, WritableColumnVector c, int 
rowId) {
-c.putIntsLittleEndian(rowId, total, buffer, offset - 
Platform.BYTE_ARRAY_OFFSET);
-offset += 4 * total;
+int requiredBytes = total * 4;
+ByteBuffer buffer = getBuffer(requiredBytes);
+
+for (int i = 0; i < total; i += 1) {
--- End diff --

I think that better integration between `ByteBuffer` and `ColumnVector` 
would be addressed in another PR.
Since tableCache also uses `ByteBuffer`, it would be good to discuss in 
another PR.
cc: @cloud-fan 


---

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



[GitHub] spark issue #21073: [SPARK-23936][SQL][WIP] Implement map_concat

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21073
  
**[Test build #89378 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89378/testReport)**
 for PR 21073 at commit 
[`97cffbe`](https://github.com/apache/spark/commit/97cffbea429d82accfcdece5e0fdcde903678cc8).


---

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



[GitHub] spark issue #21068: [SPARK-16630][YARN] Blacklist a node if executors won't ...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21068: [SPARK-16630][YARN] Blacklist a node if executors won't ...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21068: [SPARK-16630][YARN] Blacklist a node if executors won't ...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #21060: [SPARK-23942][PYTHON][SQL][BRANCH-2.3] Makes collect in ...

2018-04-15 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/21060
  
Fixing API inconsistency should not be treated as a bug fix. 

Please give me a few days. I need to summarize the Spark 2.3 release and 
list all the PRs that were backported to the release candidate branches. Thanks!


---

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



[GitHub] spark issue #20937: [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support cus...

2018-04-15 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/20937
  
**[Test build #89377 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89377/testReport)**
 for PR 20937 at commit 
[`58fc5c6`](https://github.com/apache/spark/commit/58fc5c66f7856af4fd196d8194ab42be083324d0).


---

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



[GitHub] spark issue #21060: [SPARK-23942][PYTHON][SQL][BRANCH-2.3] Makes collect in ...

2018-04-15 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21060
  
This is not an new feature addition .. this fixes an exiting functionality 
to work as expected and consistently .. 
Sure, that'd be great. Will join in the discussion.


---

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



[GitHub] spark issue #21060: [SPARK-23942][PYTHON][SQL][BRANCH-2.3] Makes collect in ...

2018-04-15 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/21060
  
> This case specifically collect in PySpark doesn't work alone whereas all 
other actions like foreach, show and other cases in other languages works in 
all other APIs. Also, that's what a query execution listener describes. Do you 
believe you would make this exception for PySpark specifically in any case?

To improve the usability, we should change it in the master branch. My 
point is we should not backport this PR to 2.3 release. 

> Let's open a discussion in the mailing list and see if we can see the 
agreement. I think this was not the first time we talked about this and think 
it's better to open a proper discussion and make a decision.

Sure, let me lead the discussion in the dev channel and welcome you to add 
the inputs there. Next, we should also discuss the rule which PRs can be 
backported to RC branches when we do the release. In Spark 2.3 release, we 
backported many PRs that should not be merged to the release candidate 
branches. 


---

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



[GitHub] spark issue #21060: [SPARK-23942][PYTHON][SQL][BRANCH-2.3] Makes collect in ...

2018-04-15 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21060
  
> The behavior consistency among Python/Scala/R/JAVA does not mean a bug, 
right?

This case specifically `collect` in PySpark doesn't work alone whereas all 
other actions like `foreach`, `show` and other cases in other languages works 
in all other APIs. Also, that's what a query execution listener describes. Do 
you believe you would make this exception for PySpark specifically in any case?

I am seeing `foreach` and etc was fixed 
https://github.com/apache/spark/commit/154351e6dbd24c4254094477e3f7defcba979b1a 
and also see `collect` is included in the original commit - 
https://github.com/apache/spark/commit/15ff85b3163acbe8052d4489a00bcf1d2332fcf0

> I am not against this specific PR. All the committers need to be really 
careful when they make a decision to backport a behavior change. If any 
committer does it, we should jump in and stop the backport. This is what we 
should do.

Let's open a discussion in the mailing list and see if we can see the 
agreement. I think this was not the first time we talked about this and think 
it's better to open a proper discussion and make a decision. 


---

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



[GitHub] spark issue #21060: [SPARK-23942][PYTHON][SQL][BRANCH-2.3] Makes collect in ...

2018-04-15 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/21060
  
> The callback works for collect in R and Scala but Python doesn't. I think 
we should at least match the behaviour. I wonder why it's hard to say a bug 
when collect is detected in some APIs but not in some APIs.

The behavior consistency among Python/Scala/R/JAVA does not mean a bug, 
right? 

> That's because the change was big and invasive. I wouldn't backport it 
too; however, this fix is relatively small.

This is the reason why we did not backport that PR. We still can backport 
the minimal changes to the previous releases.

> I think we usually use committer's judgement when we make an exception. I 
already have been seeing many backports that actually causes behaviour changes 
and I did this because it looks being backported in general. This is the reason 
why we should formally document it if this is actually the rule.

I am not against this specific PR. All the committers need to be really 
careful when they make a decision to backport a behavior change. If any 
committer does it, we should jump in and stop the backport. This is what we 
should do.


---

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



[GitHub] spark issue #21060: [SPARK-23942][PYTHON][SQL][BRANCH-2.3] Makes collect in ...

2018-04-15 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21060
  
> We need to be very careful when backporting the PR with the behavior 
changes, especially when this is neither a critical issue nor a regression. 
Thus, I do not think we should backport this PR.

I am not saying we shouldn't be careful but affects actual user group and 
actual scenarios.


---

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



[GitHub] spark issue #21074: [SPARK-21811][SQL] Fix the inconsistency behavior when f...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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

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


---

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



[GitHub] spark issue #21060: [SPARK-23942][PYTHON][SQL][BRANCH-2.3] Makes collect in ...

2018-04-15 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21060
  
> withCallback was added in Spark 1.6 release 
https://issues.apache.org/jira/browse/SPARK-11068 Since then, my understanding 
is we never clearly define which should be part of withCallback. Thus, it is 
hard to say this is a bug fix.

The callback works for `collect` in R and Scala but Python doesn't. I think 
we should at least match the behaviour. I wonder why it's hard to say a bug 
when `collect` is detected in some APIs but not in some APIs.

> We hit the similar issue in #18064. At that time, we did not backport the 
PR to the previous releases too.

That's because the change was big and invasive. I wouldn't backport it too; 
however, this fix is relatively small.

> Thus, I do not think we should make an exception for this PR just because 
the customers of @HyukjinKwon hit this issue

It's not because my customers but I am saying it fixes an actual usecase 
and it affects actual users.

> If we make an exception, it becomes harder to decide which PRs are 
qualified for a backport.

I think we usually use committer's judgement when we make an exception. I 
already have been seeing many backports that actually causes behaviour changes 
and I did this because it looks being backported in general. This is the reason 
why we should formally document it if this is actually the rule.

What I am less sure is, why this one specifically prompted you.


---

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



[GitHub] spark issue #21074: [SPARK-21811][SQL] Fix the inconsistency behavior when f...

2018-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



  1   2   >