[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/21320#discussion_r204288547
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala
 ---
@@ -0,0 +1,156 @@
+/*
+ * 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.execution.datasources.parquet
+
+import java.io.File
+
+import org.apache.spark.sql.{QueryTest, Row}
+import org.apache.spark.sql.execution.FileSchemaPruningTest
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSQLContext
+
+class ParquetSchemaPruningSuite
+extends QueryTest
+with ParquetTest
+with FileSchemaPruningTest
+with SharedSQLContext {
+  case class FullName(first: String, middle: String, last: String)
+  case class Contact(name: FullName, address: String, pets: Int, friends: 
Array[FullName] = Array(),
+relatives: Map[String, FullName] = Map())
+
+  val contacts =
+Contact(FullName("Jane", "X.", "Doe"), "123 Main Street", 1) ::
+Contact(FullName("John", "Y.", "Doe"), "321 Wall Street", 3) :: Nil
+
+  case class Name(first: String, last: String)
+  case class BriefContact(name: Name, address: String)
+
+  val briefContacts =
+BriefContact(Name("Janet", "Jones"), "567 Maple Drive") ::
+BriefContact(Name("Jim", "Jones"), "6242 Ash Street") :: Nil
+
+  case class ContactWithDataPartitionColumn(name: FullName, address: 
String, pets: Int,
+friends: Array[FullName] = Array(), relatives: Map[String, FullName] = 
Map(), p: Int)
+
+  case class BriefContactWithDataPartitionColumn(name: Name, address: 
String, p: Int)
+
+  val contactsWithDataPartitionColumn =
+contacts.map { case Contact(name, address, pets, friends, relatives) =>
+  ContactWithDataPartitionColumn(name, address, pets, friends, 
relatives, 1) }
+  val briefContactsWithDataPartitionColumn =
+briefContacts.map { case BriefContact(name: Name, address: String) =>
+  BriefContactWithDataPartitionColumn(name, address, 2) }
+
+  testSchemaPruning("select a single complex field") {
+val query = sql("select name.middle from contacts")
+checkScanSchemata(query, "struct>")
+checkAnswer(query, Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: 
Nil)
+  }
+
+  testSchemaPruning("select a single complex field and the partition 
column") {
+val query = sql("select name.middle, p from contacts")
+checkScanSchemata(query, "struct>")
+checkAnswer(query, Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: 
Row(null, 2) :: Nil)
+  }
+
+  ignore("partial schema intersection - select missing subfield") {
+val query = sql("select name.middle, address from contacts where p=2")
+checkScanSchemata(query, 
"struct,address:string>")
+checkAnswer(query,
+  Row(null, "567 Maple Drive") ::
+  Row(null, "6242 Ash Street") :: Nil)
+  }
+
+  testSchemaPruning("no unnecessary schema pruning") {
+val query =
+  sql("select name.last, name.middle, name.first, relatives[''].last, 
" +
+"relatives[''].middle, relatives[''].first, friends[0].last, 
friends[0].middle, " +
+"friends[0].first, pets, address from contacts where p=2")
+// We've selected every field in the schema. Therefore, no schema 
pruning should be performed.
+// We check this by asserting that the scanned schema of the query is 
identical to the schema
+// of the contacts relation, even though the fields are selected in 
different orders.
+checkScanSchemata(query,
+  
"struct,address:string,pets:int,"
 +
+  "friends:array>," +
+  
"relatives:map>>")
+checkAnswer(query,
+  Row("Jones", null, "Janet", null, null, null, null, null, null, 
null, "567 Maple Drive") ::
+  Row("Jones", null, "Jim", null, null, null, null, null, null, null, 

[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/21320#discussion_r204288381
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala
 ---
@@ -0,0 +1,388 @@
+/*
+ * 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.planning
+
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.exceptions.TestFailedException
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.types._
+
+// scalastyle:off line.size.limit
--- End diff --

tiny nit: I think we don't really have to disable the length limit.


---

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



[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/21320#discussion_r204288233
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala
 ---
@@ -0,0 +1,153 @@
+/*
+ * 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.execution.datasources.parquet
+
+import org.apache.spark.sql.catalyst.expressions.{And, Attribute, 
Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, 
ProjectionOverSchema, SelectedField}
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, 
Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, 
LogicalRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{ArrayType, DataType, MapType, 
StructField, StructType}
+
+/**
+ * Prunes unnecessary Parquet columns given a [[PhysicalOperation]] over a
+ * [[ParquetRelation]]. By "Parquet column", we mean a column as defined 
in the
+ * Parquet format. In Spark SQL, a root-level Parquet column corresponds 
to a
+ * SQL column, and a nested Parquet column corresponds to a 
[[StructField]].
+ */
+private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan =
+if (SQLConf.get.nestedSchemaPruningEnabled) {
+  apply0(plan)
+} else {
+  plan
+}
+
+  private def apply0(plan: LogicalPlan): LogicalPlan =
+plan transformDown {
+  case op @ PhysicalOperation(projects, filters,
+  l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, _,
+dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) 
=>
+val projectionRootFields = projects.flatMap(getRootFields)
+val filterRootFields = filters.flatMap(getRootFields)
+val requestedRootFields = (projectionRootFields ++ 
filterRootFields).distinct
+
+// If [[requestedRootFields]] includes a nested field, continue. 
Otherwise,
+// return [[op]]
+if (requestedRootFields.exists { case RootField(_, derivedFromAtt) 
=> !derivedFromAtt }) {
+  val prunedSchema = requestedRootFields
+.map { case RootField(field, _) => StructType(Array(field)) }
+.reduceLeft(_ merge _)
+  val dataSchemaFieldNames = dataSchema.fieldNames.toSet
+  val prunedDataSchema =
+StructType(prunedSchema.filter(f => 
dataSchemaFieldNames.contains(f.name)))
+
+  // If the data schema is different from the pruned data schema, 
continue. Otherwise,
+  // return [[op]]. We effect this comparison by counting the 
number of "leaf" fields in
+  // each schemata, assuming the fields in [[prunedDataSchema]] 
are a subset of the fields
+  // in [[dataSchema]].
+  if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) {
+val prunedParquetRelation =
+  hadoopFsRelation.copy(dataSchema = 
prunedDataSchema)(hadoopFsRelation.sparkSession)
+
+// We need to replace the expression ids of the pruned 
relation output attributes
+// with the expression ids of the original relation output 
attributes so that
+// references to the original relation's output are not broken
+val outputIdMap = l.output.map(att => (att.name, 
att.exprId)).toMap
+val prunedRelationOutput =
+  prunedParquetRelation
+.schema
+.toAttributes
+.map {
+  case att if outputIdMap.contains(att.name) =>
+att.withExprId(outputIdMap(att.name))
+  case att => att
+}
+val prunedRelation =
+  l.copy(relation = 

[GitHub] spark issue #21775: [SPARK-24812][SQL] Last Access Time in the table descrip...

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21775
  
**[Test build #93431 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93431/testReport)**
 for PR 21775 at commit 
[`76a34c6`](https://github.com/apache/spark/commit/76a34c6d3c05c3f729be5893210b199ebb6c093c).


---

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



[GitHub] spark pull request #21775: [SPARK-24812][SQL] Last Access Time in the table ...

2018-07-22 Thread sujith71955
Github user sujith71955 commented on a diff in the pull request:

https://github.com/apache/spark/pull/21775#discussion_r204287713
  
--- Diff: docs/sql-programming-guide.md ---
@@ -1843,6 +1843,7 @@ working with timestamps in `pandas_udf`s to get the 
best performance, see
 
 ## Upgrading From Spark SQL 2.3 to 2.4
 
+  - Since Spark 2.4, Spark will display hive table description column  
`Last Access` value  as `UNKNOWN` following the Hive system.
--- End diff --

done.


---

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



[GitHub] spark issue #21839: [SPARK-24339][SQL] Prunes the unused columns from child ...

2018-07-22 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/21839
  
@xuanyuanking Could you add an end-to-end test into 
ScriptTransformationSuite to verify the results?


---

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



[GitHub] spark pull request #21839: [SPARK-24339][SQL] Prunes the unused columns from...

2018-07-22 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/21839#discussion_r204286954
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -450,13 +450,16 @@ object ColumnPruning extends Rule[LogicalPlan] {
 case d @ DeserializeToObject(_, _, child) if (child.outputSet -- 
d.references).nonEmpty =>
   d.copy(child = prunedChild(child, d.references))
 
-// Prunes the unused columns from child of Aggregate/Expand/Generate
+// Prunes the unused columns from child of 
Aggregate/Expand/Generate/ScriptTransformation
 case a @ Aggregate(_, _, child) if (child.outputSet -- 
a.references).nonEmpty =>
   a.copy(child = prunedChild(child, a.references))
 case f @ FlatMapGroupsInPandas(_, _, _, child) if (child.outputSet -- 
f.references).nonEmpty =>
   f.copy(child = prunedChild(child, f.references))
 case e @ Expand(_, _, child) if (child.outputSet -- 
e.references).nonEmpty =>
   e.copy(child = prunedChild(child, e.references))
+case s @ ScriptTransformation(_, _, _, child, _)
+  if (child.outputSet -- s.references).nonEmpty =>
--- End diff --

Nit: two more spaces before `if`. 


---

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



[GitHub] spark issue #21845: [SPARK-24886][INFRA] Fix the testing script to increase ...

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21845
  
**[Test build #93430 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93430/testReport)**
 for PR 21845 at commit 
[`7afc5c5`](https://github.com/apache/spark/commit/7afc5c52fa31595b1eb458100d37fe92f62e31aa).


---

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



[GitHub] spark issue #21845: [SPARK-24886][INFRA] Fix the testing script to increase ...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21845
  
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-unified/1230/
Test PASSed.


---

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



[GitHub] spark issue #21845: [SPARK-24886][INFRA] Fix the testing script to increase ...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21845
  
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 #21123: [SPARK-24045][SQL]Create base class for file data source...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21123: [SPARK-24045][SQL]Create base class for file data source...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21123
  
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 #21123: [SPARK-24045][SQL]Create base class for file data source...

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21123
  
**[Test build #93422 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93422/testReport)**
 for PR 21123 at commit 
[`5ed7854`](https://github.com/apache/spark/commit/5ed7854c98a226e5f31bd212b80546d60fda65c8).
 * 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 #21775: [SPARK-24812][SQL] Last Access Time in the table ...

2018-07-22 Thread sujith71955
Github user sujith71955 commented on a diff in the pull request:

https://github.com/apache/spark/pull/21775#discussion_r204286227
  
--- Diff: docs/sql-programming-guide.md ---
@@ -1843,6 +1843,7 @@ working with timestamps in `pandas_udf`s to get the 
best performance, see
 
 ## Upgrading From Spark SQL 2.3 to 2.4
 
+  - Since Spark 2.4, Spark will display hive table description column  
`Last Access` value  as `UNKNOWN` following the Hive system.
--- End diff --

Right, its applicable for both type, i will update the message as per your 
comment. Thanks


---

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



[GitHub] spark issue #21845: [SPARK-24886][INFRA] Fix the testing script to increase ...

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #21845: [SPARK-24886][INFRA] Fix the testing script to increase ...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21845
  
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-unified/1229/
Test PASSed.


---

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



[GitHub] spark issue #21845: [SPARK-24886][INFRA] Fix the testing script to increase ...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21845
  
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 #21775: [SPARK-24812][SQL] Last Access Time in the table ...

2018-07-22 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/21775#discussion_r204285948
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -114,7 +114,10 @@ case class CatalogTablePartition(
   map.put("Partition Parameters", s"{${parameters.map(p => p._1 + "=" 
+ p._2).mkString(", ")}}")
 }
 map.put("Created Time", new Date(createTime).toString)
-map.put("Last Access", new Date(lastAccessTime).toString)
+val lastAccess = {
+  if (-1 == lastAccessTime) "UNKNOWN" else new 
Date(lastAccessTime).toString
+}
+map.put("Last Access", lastAccess)
--- End diff --

the current way is also fine


---

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



[GitHub] spark pull request #21845: [SPARK-24886][INFRA] Fix the testing script to in...

2018-07-22 Thread HyukjinKwon
GitHub user HyukjinKwon opened a pull request:

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

[SPARK-24886][INFRA] Fix the testing script to increase timeout for Jenkins 
build (from 300m to 350m)

## What changes were proposed in this pull request?

Currently, looks we hit the time limit time to time. Looks better 
increasing the time a bit.

For instance, please see https://github.com/apache/spark/pull/21822

For clarification, current Jenkins timeout is already 361m:

![screen shot 2018-07-23 at 1 28 57 
pm](https://user-images.githubusercontent.com/6477701/43058956-c0f94880-8e7c-11e8-86ba-20cdd92a1f14.png)

This PR just proposes to fix the test script to increase it correspondingly.

## How was this patch tested?

Jenkins tests.

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

$ git pull https://github.com/HyukjinKwon/spark SPARK-24886

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

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


commit c57b74546b457d44e0c9d28d92a672fe5324d3ff
Author: hyukjinkwon 
Date:   2018-07-23T05:30:16Z

Fix the testing script to increase timeout for Jenkins build (from 300m to 
350m)




---

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



[GitHub] spark pull request #21775: [SPARK-24812][SQL] Last Access Time in the table ...

2018-07-22 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/21775#discussion_r204285853
  
--- Diff: docs/sql-programming-guide.md ---
@@ -1843,6 +1843,7 @@ working with timestamps in `pandas_udf`s to get the 
best performance, see
 
 ## Upgrading From Spark SQL 2.3 to 2.4
 
+  - Since Spark 2.4, Spark will display hive table description column  
`Last Access` value  as `UNKNOWN` following the Hive system.
--- End diff --

This is applicable to both native and hive tables. How about changing it to 

> Spark will display table description column  `Last Access` value  as 
`UNKNOWN` when the value was `Jan 01 1970`. 


---

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



[GitHub] spark issue #21845: [SPARK-24886][INFRA] Fix the testing script to increase ...

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21845
  
cc @rxin 


---

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



[GitHub] spark issue #21775: [SPARK-24812][SQL] Last Access Time in the table descrip...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21775
  
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 #21775: [SPARK-24812][SQL] Last Access Time in the table descrip...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21775: [SPARK-24812][SQL] Last Access Time in the table descrip...

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21775
  
**[Test build #93421 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93421/testReport)**
 for PR 21775 at commit 
[`140c4ce`](https://github.com/apache/spark/commit/140c4ce68f87ada4720fb1f142f49ba3e7f284b6).
 * 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 #21764: [SPARK-24802][SQL] Add a new config for Optimization Rul...

2018-07-22 Thread maropu
Github user maropu commented on the issue:

https://github.com/apache/spark/pull/21764
  
LGTM, too


---

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



[GitHub] spark issue #21764: [SPARK-24802][SQL] Add a new config for Optimization Rul...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21764
  
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 #21764: [SPARK-24802][SQL] Add a new config for Optimization Rul...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21764: [SPARK-24802][SQL] Add a new config for Optimization Rul...

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21764
  
**[Test build #93419 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93419/testReport)**
 for PR 21764 at commit 
[`87afe4f`](https://github.com/apache/spark/commit/87afe4fbcaf71d303b07612f9ceb9ad25dd3dcda).
 * 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 #21764: [SPARK-24802][SQL] Add a new config for Optimization Rul...

2018-07-22 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/21764
  
LGTM pending Jenkins 


---

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



[GitHub] spark issue #21764: [SPARK-24802][SQL] Add a new config for Optimization Rul...

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21764
  
**[Test build #93428 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93428/testReport)**
 for PR 21764 at commit 
[`39b6ce9`](https://github.com/apache/spark/commit/39b6ce9548c99363e81cb246b4cbe5534d710f3e).


---

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



[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

2018-07-22 Thread hthuynh2
Github user hthuynh2 commented on the issue:

https://github.com/apache/spark/pull/21653
  
@tgravescs Can you please run the test again, thank you. 


---

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



[GitHub] spark issue #21764: [SPARK-24802][SQL] Add a new config for Optimization Rul...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21764
  
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-unified/1228/
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 #21764: [SPARK-24802][SQL] Add a new config for Optimizat...

2018-07-22 Thread maryannxue
Github user maryannxue commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r204279843
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -175,6 +191,41 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
* Override to provide additional rules for the operator optimization 
batch.
*/
   def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
+
+  override def batches: Seq[Batch] = {
+val excludedRulesConf =
+  
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(_.split(",").map(_.trim).filter(_.nonEmpty))
--- End diff --

No reason. It's just I didn't know about it. Thank you for point this out!


---

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



[GitHub] spark issue #21764: [SPARK-24802][SQL] Add a new config for Optimization Rul...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21764
  
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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21653
  
**[Test build #93418 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93418/testReport)**
 for PR 21653 at commit 
[`b6585da`](https://github.com/apache/spark/commit/b6585da0f137d3d3675925368c4668c884de900c).
 * 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 #21764: [SPARK-24802][SQL] Add a new config for Optimizat...

2018-07-22 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r204278968
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -175,6 +191,41 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
* Override to provide additional rules for the operator optimization 
batch.
*/
   def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
+
+  override def batches: Seq[Batch] = {
+val excludedRulesConf =
+  
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(_.split(",").map(_.trim).filter(_.nonEmpty))
--- End diff --

+1



---

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



[GitHub] spark issue #21826: [SPARK-24872] Remove the symbol “||” of the “OR”...

2018-07-22 Thread httfighter
Github user httfighter commented on the issue:

https://github.com/apache/spark/pull/21826
  
I have a suggestion that I don't know if it is reasonable.
In our spark, since we already support “||” as a string concatenation 
function, I don't know if we can make such an improvement. In the SQL project 
part, we will use "||" as a string concatenation function. In the SQL filter 
part, we will use  "||" as the “Or” operation.
@HyukjinKwon @viirya @gatorsmile @rxin @hvanhovell


---

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



[GitHub] spark pull request #21830: [SPARK-24878][SQL] Fix reverse function for array...

2018-07-22 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21830#discussion_r204277225
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -1244,46 +1244,50 @@ case class Reverse(child: Expression) extends 
UnaryExpression with ImplicitCastI
   }
 
   private def arrayCodeGen(ctx: CodegenContext, ev: ExprCode, childName: 
String): String = {
-val length = ctx.freshName("length")
-val javaElementType = CodeGenerator.javaType(elementType)
+
 val isPrimitiveType = CodeGenerator.isPrimitiveType(elementType)
 
+val numElements = ctx.freshName("numElements")
+val arrayData = ctx.freshName("arrayData")
+
 val initialization = if (isPrimitiveType) {
-  s"$childName.copy()"
+  ctx.createUnsafeArray(arrayData, numElements, elementType, s" 
$prettyName failed.")
--- End diff --

I talked with @ueshin and no problem about this. 


---

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



[GitHub] spark issue #21772: [SPARK-24809] [SQL] Serializing LongHashedRelation in ex...

2018-07-22 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/21772
  
@liutang123 Thanks for this work. I'm curious that if this is an actual 
problem you hit in real application, or you just think it is problematic?


---

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



[GitHub] spark pull request #21772: [SPARK-24809] [SQL] Serializing LongHashedRelatio...

2018-07-22 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/21772#discussion_r204276973
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
 ---
@@ -278,6 +278,39 @@ class HashedRelationSuite extends SparkFunSuite with 
SharedSQLContext {
 map.free()
   }
 
+  test("SPARK-24809: Serializing LongHashedRelation in executor may result 
in data error") {
--- End diff --

Is it possible to have an end-to-end test for this?


---

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



[GitHub] spark issue #21826: [SPARK-24872] Remove the symbol “||” of the “OR”...

2018-07-22 Thread httfighter
Github user httfighter commented on the issue:

https://github.com/apache/spark/pull/21826
  
I did the following tests in mysql. 
 mysql> select "abc" || "def";
  ++
  | "abc" || "def" |
 ++
 |  0  |
 ++
 mysql> select "abc"  "def";
 ++
  | abc|
++
 | abcdef |
++
mysql> select * from aa where id=1 || id=2;
   +--+--+
   | id   | name |
  +--+--+
  |1 | sdf  |
  |2 | ader |
 +--+--+
   mysql> select * from aa where id=1 or id=2;
  +--+--+
  | id   | name |
  +--+--+
  |1 | sdf  |
  |2 | ader |
  +--+--+
It seems that it does not act as a string connector, but as an "or" 
operation. I don't know if my tests are correct. 


---

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



[GitHub] spark pull request #21772: [SPARK-24809] [SQL] Serializing LongHashedRelatio...

2018-07-22 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/21772#discussion_r204276886
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
 ---
@@ -770,8 +771,10 @@ private[execution] final class LongToUnsafeRowMap(val 
mm: TaskMemoryManager, cap
 val length = readLong().toInt
 mask = length - 2
 array = readLongArray(readBuffer, length)
-val pageLength = readLong().toInt
-page = readLongArray(readBuffer, pageLength)
+val usedWordsNumber = readLong().toInt
+// Set cursor because cursor is used in write function.
+cursor = usedWordsNumber * 8 + Platform.LONG_ARRAY_OFFSET
+page = readLongArray(readBuffer, usedWordsNumber)
--- End diff --

ditto. Can you just update `cursor` and revert other unrelated change?


---

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



[GitHub] spark pull request #21772: [SPARK-24809] [SQL] Serializing LongHashedRelatio...

2018-07-22 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/21772#discussion_r204276788
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
 ---
@@ -726,9 +726,10 @@ private[execution] final class LongToUnsafeRowMap(val 
mm: TaskMemoryManager, cap
 
 writeLong(array.length)
 writeLongArray(writeBuffer, array, array.length)
-val used = ((cursor - Platform.LONG_ARRAY_OFFSET) / 8).toInt
-writeLong(used)
-writeLongArray(writeBuffer, page, used)
+
+val usedWordsNumber = ((cursor - Platform.LONG_ARRAY_OFFSET) / 8).toInt
+writeLong(usedWordsNumber)
+writeLongArray(writeBuffer, page, usedWordsNumber)
--- End diff --

If no good reason, shall we revert this change? Looks like you only rename 
it?


---

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



[GitHub] spark issue #21836: [SQL][HIVE] Correct an assert message in function makeRD...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21836
  
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 #21802: [SPARK-23928][SQL] Add shuffle collection functio...

2018-07-22 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/21802#discussion_r204276502
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
 ---
@@ -1419,4 +1421,71 @@ class CollectionExpressionsSuite extends 
SparkFunSuite with ExpressionEvalHelper
 assert(ArrayUnion(a20, 
a21).dataType.asInstanceOf[ArrayType].containsNull === false)
 assert(ArrayUnion(a20, 
a22).dataType.asInstanceOf[ArrayType].containsNull === true)
   }
+
+  test("Shuffle") {
+// Primitive-type elements
+val ai0 = Literal.create(Seq(1, 2, 3, 4, 5), ArrayType(IntegerType, 
containsNull = false))
+val ai1 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType, 
containsNull = false))
+val ai2 = Literal.create(Seq(null, 1, null, 3), ArrayType(IntegerType, 
containsNull = true))
+val ai3 = Literal.create(Seq(2, null, 4, null), ArrayType(IntegerType, 
containsNull = true))
+val ai4 = Literal.create(Seq(null, null, null), ArrayType(IntegerType, 
containsNull = true))
+val ai5 = Literal.create(Seq(1), ArrayType(IntegerType, containsNull = 
false))
+val ai6 = Literal.create(Seq.empty, ArrayType(IntegerType, 
containsNull = false))
+val ai7 = Literal.create(null, ArrayType(IntegerType, containsNull = 
true))
+
+checkEvaluation(Shuffle(ai0, Some(0)), Seq(4, 1, 2, 3, 5))
+checkEvaluation(Shuffle(ai1, Some(0)), Seq(3, 1, 2))
+checkEvaluation(Shuffle(ai2, Some(0)), Seq(3, null, 1, null))
+checkEvaluation(Shuffle(ai3, Some(0)), Seq(null, 2, null, 4))
+checkEvaluation(Shuffle(ai4, Some(0)), Seq(null, null, null))
+checkEvaluation(Shuffle(ai5, Some(0)), Seq(1))
+checkEvaluation(Shuffle(ai6, Some(0)), Seq.empty)
+checkEvaluation(Shuffle(ai7, Some(0)), null)
+
+// Non-primitive-type elements
+val as0 = Literal.create(Seq("a", "b", "c", "d"), 
ArrayType(StringType, containsNull = false))
+val as1 = Literal.create(Seq("a", "b", "c"), ArrayType(StringType, 
containsNull = false))
+val as2 = Literal.create(Seq(null, "a", null, "c"), 
ArrayType(StringType, containsNull = true))
+val as3 = Literal.create(Seq("b", null, "d", null), 
ArrayType(StringType, containsNull = true))
+val as4 = Literal.create(Seq(null, null, null), ArrayType(StringType, 
containsNull = true))
+val as5 = Literal.create(Seq("a"), ArrayType(StringType, containsNull 
= false))
+val as6 = Literal.create(Seq.empty, ArrayType(StringType, containsNull 
= false))
+val as7 = Literal.create(null, ArrayType(StringType, containsNull = 
true))
+val aa = Literal.create(
+  Seq(Seq("a", "b"), Seq("c", "d"), Seq("e")),
+  ArrayType(ArrayType(StringType)))
+
+checkEvaluation(Shuffle(as0, Some(0)), Seq("d", "a", "b", "c"))
+checkEvaluation(Shuffle(as1, Some(0)), Seq("c", "a", "b"))
+checkEvaluation(Shuffle(as2, Some(0)), Seq("c", null, "a", null))
+checkEvaluation(Shuffle(as3, Some(0)), Seq(null, "b", null, "d"))
+checkEvaluation(Shuffle(as4, Some(0)), Seq(null, null, null))
+checkEvaluation(Shuffle(as5, Some(0)), Seq("a"))
+checkEvaluation(Shuffle(as6, Some(0)), Seq.empty)
+checkEvaluation(Shuffle(as7, Some(0)), null)
+checkEvaluation(Shuffle(aa, Some(0)), Seq(Seq("e"), Seq("a", "b"), 
Seq("c", "d")))
+
+val r = new Random()
+val seed1 = Some(r.nextLong())
+assert(evaluateWithoutCodegen(Shuffle(ai0, seed1)) ===
+  evaluateWithoutCodegen(Shuffle(ai0, seed1)))
+assert(evaluateWithGeneratedMutableProjection(Shuffle(ai0, seed1)) ===
+  evaluateWithGeneratedMutableProjection(Shuffle(ai0, seed1)))
+assert(evaluateWithUnsafeProjection(Shuffle(ai0, seed1)) ===
+  evaluateWithUnsafeProjection(Shuffle(ai0, seed1)))
+
+val seed2 = Some(r.nextLong())
--- End diff --

I think this is what we expect. The result is decided by the random seed. 
So if using different random seeds, I think the results should be different.


---

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



[GitHub] spark issue #21836: [SQL][HIVE] Correct an assert message in function makeRD...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


---

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



[GitHub] spark issue #21836: [SQL][HIVE] Correct an assert message in function makeRD...

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21836
  
**[Test build #93427 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93427/testReport)**
 for PR 21836 at commit 
[`6a0f729`](https://github.com/apache/spark/commit/6a0f7290a7ba2764ab9dd437a07afdf419ce7bf0).
 * This patch **fails Scala style 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 #21836: [SQL][HIVE] Correct an assert message in function makeRD...

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #21836: [SQL][HIVE] Correct an assert message in function makeRD...

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21836
  
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 #21836: [SQL][HIVE] Correct an assert message in function makeRD...

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21836
  
test this please


---

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



[GitHub] spark issue #21440: [SPARK-24307][CORE] Support reading remote cached partit...

2018-07-22 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/21440
  
cc @squito @mridulm @markhamstra @jerryshao @vanzin @JoshRosen @rxin 
@zsxwing 


---

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



[GitHub] spark issue #21440: [SPARK-24307][CORE] Support reading remote cached partit...

2018-07-22 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/21440
  
Although the code quality is pretty good, I am still afraid it could 
introduce some unexpected issues. Is that possible we can introduce a conf to 
fall back to the previous implementation? We can remove the conf in the next 
release. 


---

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



[GitHub] spark pull request #21830: [SPARK-24878][SQL] Fix reverse function for array...

2018-07-22 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21830#discussion_r204275800
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -1244,46 +1244,50 @@ case class Reverse(child: Expression) extends 
UnaryExpression with ImplicitCastI
   }
 
   private def arrayCodeGen(ctx: CodegenContext, ev: ExprCode, childName: 
String): String = {
-val length = ctx.freshName("length")
-val javaElementType = CodeGenerator.javaType(elementType)
+
 val isPrimitiveType = CodeGenerator.isPrimitiveType(elementType)
 
+val numElements = ctx.freshName("numElements")
+val arrayData = ctx.freshName("arrayData")
+
 val initialization = if (isPrimitiveType) {
-  s"$childName.copy()"
+  ctx.createUnsafeArray(arrayData, numElements, elementType, s" 
$prettyName failed.")
--- End diff --

You can check this? 
https://gist.github.com/maropu/e9e8afd64ce30cdf824bb4e18d0c9b4f


---

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



[GitHub] spark pull request #21802: [SPARK-23928][SQL] Add shuffle collection functio...

2018-07-22 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21802#discussion_r204275498
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
 ---
@@ -1419,4 +1421,71 @@ class CollectionExpressionsSuite extends 
SparkFunSuite with ExpressionEvalHelper
 assert(ArrayUnion(a20, 
a21).dataType.asInstanceOf[ArrayType].containsNull === false)
 assert(ArrayUnion(a20, 
a22).dataType.asInstanceOf[ArrayType].containsNull === true)
   }
+
+  test("Shuffle") {
+// Primitive-type elements
+val ai0 = Literal.create(Seq(1, 2, 3, 4, 5), ArrayType(IntegerType, 
containsNull = false))
+val ai1 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType, 
containsNull = false))
+val ai2 = Literal.create(Seq(null, 1, null, 3), ArrayType(IntegerType, 
containsNull = true))
+val ai3 = Literal.create(Seq(2, null, 4, null), ArrayType(IntegerType, 
containsNull = true))
+val ai4 = Literal.create(Seq(null, null, null), ArrayType(IntegerType, 
containsNull = true))
+val ai5 = Literal.create(Seq(1), ArrayType(IntegerType, containsNull = 
false))
+val ai6 = Literal.create(Seq.empty, ArrayType(IntegerType, 
containsNull = false))
+val ai7 = Literal.create(null, ArrayType(IntegerType, containsNull = 
true))
+
+checkEvaluation(Shuffle(ai0, Some(0)), Seq(4, 1, 2, 3, 5))
+checkEvaluation(Shuffle(ai1, Some(0)), Seq(3, 1, 2))
+checkEvaluation(Shuffle(ai2, Some(0)), Seq(3, null, 1, null))
+checkEvaluation(Shuffle(ai3, Some(0)), Seq(null, 2, null, 4))
+checkEvaluation(Shuffle(ai4, Some(0)), Seq(null, null, null))
+checkEvaluation(Shuffle(ai5, Some(0)), Seq(1))
+checkEvaluation(Shuffle(ai6, Some(0)), Seq.empty)
+checkEvaluation(Shuffle(ai7, Some(0)), null)
+
+// Non-primitive-type elements
+val as0 = Literal.create(Seq("a", "b", "c", "d"), 
ArrayType(StringType, containsNull = false))
+val as1 = Literal.create(Seq("a", "b", "c"), ArrayType(StringType, 
containsNull = false))
+val as2 = Literal.create(Seq(null, "a", null, "c"), 
ArrayType(StringType, containsNull = true))
+val as3 = Literal.create(Seq("b", null, "d", null), 
ArrayType(StringType, containsNull = true))
+val as4 = Literal.create(Seq(null, null, null), ArrayType(StringType, 
containsNull = true))
+val as5 = Literal.create(Seq("a"), ArrayType(StringType, containsNull 
= false))
+val as6 = Literal.create(Seq.empty, ArrayType(StringType, containsNull 
= false))
+val as7 = Literal.create(null, ArrayType(StringType, containsNull = 
true))
+val aa = Literal.create(
+  Seq(Seq("a", "b"), Seq("c", "d"), Seq("e")),
+  ArrayType(ArrayType(StringType)))
+
+checkEvaluation(Shuffle(as0, Some(0)), Seq("d", "a", "b", "c"))
+checkEvaluation(Shuffle(as1, Some(0)), Seq("c", "a", "b"))
+checkEvaluation(Shuffle(as2, Some(0)), Seq("c", null, "a", null))
+checkEvaluation(Shuffle(as3, Some(0)), Seq(null, "b", null, "d"))
+checkEvaluation(Shuffle(as4, Some(0)), Seq(null, null, null))
+checkEvaluation(Shuffle(as5, Some(0)), Seq("a"))
+checkEvaluation(Shuffle(as6, Some(0)), Seq.empty)
+checkEvaluation(Shuffle(as7, Some(0)), null)
+checkEvaluation(Shuffle(aa, Some(0)), Seq(Seq("e"), Seq("a", "b"), 
Seq("c", "d")))
+
+val r = new Random()
+val seed1 = Some(r.nextLong())
+assert(evaluateWithoutCodegen(Shuffle(ai0, seed1)) ===
+  evaluateWithoutCodegen(Shuffle(ai0, seed1)))
+assert(evaluateWithGeneratedMutableProjection(Shuffle(ai0, seed1)) ===
+  evaluateWithGeneratedMutableProjection(Shuffle(ai0, seed1)))
+assert(evaluateWithUnsafeProjection(Shuffle(ai0, seed1)) ===
+  evaluateWithUnsafeProjection(Shuffle(ai0, seed1)))
+
+val seed2 = Some(r.nextLong())
--- End diff --

I also followed a test for `Uuid` 
[MiscExpressionsSuite.scala#L46-L68](https://github.com/ueshin/apache-spark/blob/2ca12302e08d60ab9534d7d65fad9854fe1d6f28/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscExpressionsSuite.scala#L46-L68)
 here.
@viirya WDYT about this?


---

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



[GitHub] spark issue #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-22 Thread maropu
Github user maropu commented on the issue:

https://github.com/apache/spark/pull/21764
  
Also, can you update the title? You need to at least add `[SQL]` in the 
title: https://github.com/apache/spark/pull/21764#issuecomment-405832763


---

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



[GitHub] spark pull request #21764: [SPARK-24802] Optimization Rule Exclusion

2018-07-22 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21764#discussion_r204275071
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -175,6 +191,41 @@ abstract class Optimizer(sessionCatalog: 
SessionCatalog)
* Override to provide additional rules for the operator optimization 
batch.
*/
   def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil
+
+  override def batches: Seq[Batch] = {
+val excludedRulesConf =
+  
SQLConf.get.optimizerExcludedRules.toSeq.flatMap(_.split(",").map(_.trim).filter(_.nonEmpty))
--- End diff --

Any reason not to use `Utils.stringToSeq`?
https://github.com/apache/spark/pull/21764#discussion_r204202735


---

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



[GitHub] spark issue #21784: [SPARK-24873][YARN] Turn off spark-shell noisy log outpu...

2018-07-22 Thread srowen
Github user srowen commented on the issue:

https://github.com/apache/spark/pull/21784
  
Logging is already configurable; a switch is overkill. This seems fine.


---

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



[GitHub] spark pull request #21439: [SPARK-24391][SQL] Support arrays of any types by...

2018-07-22 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/21439#discussion_r204274790
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
 ---
@@ -101,6 +102,17 @@ class JacksonParser(
 }
   }
 
+  private def makeArrayRootConverter(at: ArrayType): JsonParser => 
Seq[InternalRow] = {
+val elemConverter = makeConverter(at.elementType)
+(parser: JsonParser) => parseJsonToken[Seq[InternalRow]](parser, at) {
+  case START_ARRAY => Seq(InternalRow(convertArray(parser, 
elemConverter)))
+  case START_OBJECT if at.elementType.isInstanceOf[StructType] =>
--- End diff --

I think it is a super weird case...can we put this special handling code 
for back-compatibility in `JsonToStructs`?
Ether way, I think we should leave code comments here to make others 
understood easily?


---

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



[GitHub] spark issue #21772: [SPARK-24809] [SQL] Serializing LongHashedRelation in ex...

2018-07-22 Thread liutang123
Github user liutang123 commented on the issue:

https://github.com/apache/spark/pull/21772
  
@viirya Hi, Could you have more time to review this PR?


---

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



[GitHub] spark pull request #21826: [SPARK-24872] Remove the symbol “||” of the �...

2018-07-22 Thread httfighter
Github user httfighter commented on a diff in the pull request:

https://github.com/apache/spark/pull/21826#discussion_r204274497
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
 ---
@@ -442,8 +442,6 @@ case class Or(left: Expression, right: Expression) 
extends BinaryOperator with P
 
   override def inputType: AbstractDataType = BooleanType
 
-  override def symbol: String = "||"
--- End diff --

I am sorry for this error, I will try to avoid it in the future.


---

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



[GitHub] spark pull request #21826: [SPARK-24872] Remove the symbol “||” of the �...

2018-07-22 Thread httfighter
Github user httfighter commented on a diff in the pull request:

https://github.com/apache/spark/pull/21826#discussion_r204274481
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
 ---
@@ -442,8 +442,6 @@ case class Or(left: Expression, right: Expression) 
extends BinaryOperator with P
 
   override def inputType: AbstractDataType = BooleanType
 
-  override def symbol: String = "||"
--- End diff --

I am sorry for this error, I will try to avoid it in the future.


---

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



[GitHub] spark issue #21784: [SPARK-24873][YARN] Turn off spark-shell noisy log outpu...

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21784
  
Adding a configuration to control some logs sounds an overkill. I wouldn't 
go for this way.


---

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



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

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



[GitHub] spark issue #21784: [SPARK-24873][YARN] Turn off spark-shell noisy log outpu...

2018-07-22 Thread guoxiaolongzte
Github user guoxiaolongzte commented on the issue:

https://github.com/apache/spark/pull/21784
  
We need to listen to @vanzin opinion.
Because the relevant code is what he wrote.


---

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



[GitHub] spark issue #21828: Update regression.py

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21828
  
@woodthom2, if you have some plans to update this PR quite soon, please see 
https://spark.apache.org/contributing.html and proceed. Otherwise, I would 
suggest to leave this closed so that active PRs are left for reviewing.


---

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



[GitHub] spark issue #21844: Spark 24873

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21844
  
I don't think we should as discussed in 
https://github.com/apache/spark/pull/21784. I think this PR would better be 
left closed and go discuss there.


---

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



[GitHub] spark issue #21844: Spark 24873

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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

2018-07-22 Thread hejiefang
GitHub user hejiefang opened a pull request:

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

Spark 24873

[https://issues.apache.org/jira/browse/SPARK-24873](url)
I think it is better to add a switch to increase a switch to shielding 
frequent interaction reports with yarn.


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

$ git pull https://github.com/hejiefang/spark spark-24873

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

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


commit 7f9ef388679e8b9a282befc3c5a031a2199d0eb0
Author: hejiefang 
Date:   2018-07-20T09:39:11Z

[SPARK-24873]Increase switch to shielding frequent interaction reports with 
yarn

commit 912878e88d684df966831c903a021b73d0ced778
Author: hejiefang 
Date:   2018-07-23T02:51:22Z

[SPARK-24873]Increase switch to shielding frequent interaction reports with 
yarn




---

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



[GitHub] spark issue #21784: [SPARK-24873][YARN] Turn off spark-shell noisy log outpu...

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21784
  
It has never been printed before, right? I think we can consider to turn it 
on specifically for spark-submit applications for some spark-submit 
applications although I am not fully sure if it's something worth doing so yet.


---

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



[GitHub] spark issue #21784: [SPARK-24873][YARN] Turn off spark-shell noisy log outpu...

2018-07-22 Thread guoxiaolongzte
Github user guoxiaolongzte commented on the issue:

https://github.com/apache/spark/pull/21784
  
But for some spark-submit applications, I want these Application report for 
information. 
What should I do?


---

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



[GitHub] spark pull request #21843: for test

2018-07-22 Thread cuijunyao
Github user cuijunyao closed the pull request at:

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


---

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



[GitHub] spark issue #21843: for test

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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

2018-07-22 Thread cuijunyao
GitHub user cuijunyao opened a pull request:

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

for test

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

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


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

$ git pull https://github.com/cuijunyao/spark-1 master

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

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


commit 1c0ba0bdc23cd28b56a32a35114fbf987ade5513
Author: xiaocui 
Date:   2018-07-23T03:04:00Z

for test




---

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



[GitHub] spark pull request #20861: [SPARK-23599][SQL] Use RandomUUIDGenerator in Uui...

2018-07-22 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/20861#discussion_r204272567
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -1994,6 +1996,20 @@ class Analyzer(
 }
   }
 
+  /**
+   * Set the seed for random number generation in Uuid expressions.
+   */
+  object ResolvedUuidExpressions extends Rule[LogicalPlan] {
+private lazy val random = new Random()
+
+override def apply(plan: LogicalPlan): LogicalPlan = plan.transformUp {
+  case p if p.resolved => p
+  case p => p transformExpressionsUp {
+case Uuid(None) => Uuid(Some(random.nextLong()))
--- End diff --

One thing I think is, is any use case that we need to re-initialize random 
seed for `Rand`? Maybe streaming? For streaming query, I think `Rand` should 
use different random seed in each execution. For now, the random seed is 
initialized when constructing, even we re-analyze the query, it still uses same 
seed.




---

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



[GitHub] spark issue #21830: [SPARK-24878][SQL] Fix reverse function for array type o...

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21830
  
**[Test build #93425 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93425/testReport)**
 for PR 21830 at commit 
[`30d08ca`](https://github.com/apache/spark/commit/30d08ca32402a6006387b135b8455681bab15e10).


---

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



[GitHub] spark issue #21118: SPARK-23325: Use InternalRow when reading with DataSourc...

2018-07-22 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/21118
  
so where are we on this? looks like we have 2 LGTM?


---

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



[GitHub] spark issue #21830: [SPARK-24878][SQL] Fix reverse function for array type o...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21830
  
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-unified/1227/
Test PASSed.


---

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



[GitHub] spark issue #21830: [SPARK-24878][SQL] Fix reverse function for array type o...

2018-07-22 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/21830
  
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 #21839: [SPARK-24339][SQL] Prunes the unused columns from child ...

2018-07-22 Thread maropu
Github user maropu commented on the issue:

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


---

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



[GitHub] spark issue #21516: [SPARK-24501][MESOS] Add Dispatcher and Driver metrics

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/21516
  
**[Test build #93424 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93424/testReport)**
 for PR 21516 at commit 
[`50c1c1e`](https://github.com/apache/spark/commit/50c1c1e810fa27480ae7e72640cc8f67b44a60f1).


---

-
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-07-22 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/21075
  
@tnachen @susanxhuynh @mgummelt @skonto 


---

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



[GitHub] spark issue #21516: [SPARK-24501][MESOS] Add Dispatcher and Driver metrics

2018-07-22 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/21516
  
@tnachen @susanxhuynh @mgummelt @skonto 


---

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



[GitHub] spark issue #21516: [SPARK-24501][MESOS] Add Dispatcher and Driver metrics

2018-07-22 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/21516
  
Jenkins, ok to test


---

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



[GitHub] spark issue #21754: [SPARK-24705][SQL] Cannot reuse an exchange operator wit...

2018-07-22 Thread maropu
Github user maropu commented on the issue:

https://github.com/apache/spark/pull/21754
  
@markhamstra @cloud-fan How about the current fix in this pr? Or, another 
approach idea to fix is: 
https://github.com/apache/spark/compare/master...maropu:SPARK-24705-3
In the latter idea, I added the `CoordinatorSupport` trait to handle the 
reused case.


---

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



[GitHub] spark issue #21784: [SPARK-24873][YARN] Turn off spark-shell noisy log outpu...

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21784
  
Maybe we could consider avoiding this logs in shell specifically. Adding a 
switch for disable/enable logs sounds an overkill.


---

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



[GitHub] spark issue #21784: [SPARK-24873][YARN] Turn off spark-shell noisy log outpu...

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21784
  
? I think we don't need a switch.


---

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



[GitHub] spark issue #21827: [SPARK-24873]Increase switch to shielding frequent inter...

2018-07-22 Thread guoxiaolongzte
Github user guoxiaolongzte commented on the issue:

https://github.com/apache/spark/pull/21827
  
Please add a switch. And represented by a constant. This configuration is 
added to the running-on-yarn.md document. @hejiefang


---

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



[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

2018-07-22 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/21584
  
I see. you are referring to 
https://github.com/apache/spark/pull/21584#issuecomment-406091994
got it.


---

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



[GitHub] spark issue #21517: Testing k8s change - please ignore (13)

2018-07-22 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/21517
  
I think we can close this


---

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



[GitHub] spark pull request #21802: [SPARK-23928][SQL] Add shuffle collection functio...

2018-07-22 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21802#discussion_r204270437
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 ---
@@ -1184,6 +1186,137 @@ case class ArraySort(child: Expression) extends 
UnaryExpression with ArraySortLi
   override def prettyName: String = "array_sort"
 }
 
+/**
+ * Returns a random permutation of the given array.
+ *
+ * This implementation uses the modern version of Fisher-Yates algorithm.
+ * Reference: 
https://en.wikipedia.org/wiki/Fisher%E2%80%93Yates_shuffle#Modern_method
--- End diff --

In the latest commit, "inside-out" looks simpler without swapping (using 
just an assignment).


---

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



[GitHub] spark issue #21784: [SPARK-24873][YARN] Turn off spark-shell noisy log outpu...

2018-07-22 Thread guoxiaolongzte
Github user guoxiaolongzte commented on the issue:

https://github.com/apache/spark/pull/21784
  
what? I think we need to add a switch.
https://github.com/apache/spark/pull/21827


---

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



[GitHub] spark pull request #20838: [SPARK-23698] Resolve undefined names in Python 3

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/20838#discussion_r204269682
  
--- Diff: python/pyspark/sql/conf.py ---
@@ -59,7 +62,7 @@ def unset(self, key):
 
 def _checkType(self, obj, identifier):
 """Assert that an object is of type str."""
-if not isinstance(obj, str) and not isinstance(obj, unicode):
+if not isinstance(obj, basestring):
--- End diff --

Nope


---

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



[GitHub] spark issue #21841: [SPARK-24768][SQL]Avro: remove implicit class AvroDataFr...

2018-07-22 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/21841
  
@gengliangwang, looks JIRA link is wrong. you mean SPARK-24883?


---

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



[GitHub] spark issue #21474: [SPARK-24297][CORE] Fetch-to-disk by default for > 2gb

2018-07-22 Thread SparkQA
Github user SparkQA commented on the issue:

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


---

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



  1   2   3   >