[GitHub] [spark] kiszk commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

2020-07-18 Thread GitBox


kiszk commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456862990



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
##
@@ -42,12 +49,13 @@ class InMemoryColumnarQuerySuite extends QueryTest with 
SharedSparkSession {
 data.createOrReplaceTempView(s"testData$dataType")
 val storageLevel = MEMORY_ONLY
 val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan
-val inMemoryRelation = InMemoryRelation(useCompression = true, 5, 
storageLevel, plan, None,
+val inMemoryRelation = InMemoryRelation(new 
TestCachedBatchSerializer(useCompression = true, 5),
+  storageLevel, plan, None,

Review comment:
   nit: `data.logicalPlan)` can be moved to the end of this line.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] kiszk commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

2020-07-18 Thread GitBox


kiszk commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456862806



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##
@@ -130,34 +97,32 @@ case class InMemoryTableScanExec(
 val numOutputRows = longMetric("numOutputRows")
 // Using these variables here to avoid serialization of entire objects (if 
referenced
 // directly) within the map Partitions closure.
-val relOutput: AttributeSeq = relation.output
+val relOutput = relation.output
+val serializer = relation.cacheBuilder.serializer
 
-filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator =>
-  // Find the ordinals and data types of the requested columns.
-  val (requestedColumnIndices, requestedColumnDataTypes) =
-attributes.map { a =>
-  relOutput.indexOf(a.exprId) -> a.dataType
-}.unzip
-
-  // update SQL metrics
-  val withMetrics = cachedBatchIterator.map { batch =>
-if (enableAccumulatorsForTest) {
-  readBatches.add(1)
-}
-numOutputRows += batch.numRows
+// update SQL metrics
+val withMetrics = if (enableAccumulatorsForTest) {
+  filteredCachedBatches().map{ batch =>
+readBatches.add(1)
 batch
   }
-
-  val columnTypes = requestedColumnDataTypes.map {
-case udt: UserDefinedType[_] => udt.sqlType
-case other => other
-  }.toArray
-  val columnarIterator = GenerateColumnAccessor.generate(columnTypes)
-  columnarIterator.initialize(withMetrics, columnTypes, 
requestedColumnIndices.toArray)
-  if (enableAccumulatorsForTest && columnarIterator.hasNext) {
-readPartitions.add(1)
+} else {
+  filteredCachedBatches()
+}.map{ batch =>
+  numOutputRows += batch.numRows
+  batch
+}
+val ret = serializer.decompressToRows(withMetrics, relOutput, attributes, 
conf)

Review comment:
   nit: Can we use better name instead of `ret`?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29079: [SPARK-32286][SQL] Coalesce bucketed table for shuffled hash join if applicable

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29079:
URL: https://github.com/apache/spark/pull/29079#issuecomment-660590396







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29079: [SPARK-32286][SQL] Coalesce bucketed table for shuffled hash join if applicable

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29079:
URL: https://github.com/apache/spark/pull/29079#issuecomment-660590396







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29079: [SPARK-32286][SQL] Coalesce bucketed table for shuffled hash join if applicable

2020-07-18 Thread GitBox


SparkQA commented on pull request #29079:
URL: https://github.com/apache/spark/pull/29079#issuecomment-660590273


   **[Test build #126121 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126121/testReport)**
 for PR 29079 at commit 
[`4c65c7f`](https://github.com/apache/spark/commit/4c65c7f6293f1a4459123d0d9cb57c00f4ebf364).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] kiszk commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

2020-07-18 Thread GitBox


kiszk commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456862232



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##
@@ -85,69 +208,55 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-val output = cachedPlan.output
-val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-  new Iterator[CachedBatch] {
-def next(): CachedBatch = {
-  val columnBuilders = output.map { attribute =>
-ColumnBuilder(attribute.dataType, batchSize, attribute.name, 
useCompression)
-  }.toArray
-
-  var rowCount = 0
-  var totalSize = 0L
-  while (rowIterator.hasNext && rowCount < batchSize
-&& totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-val row = rowIterator.next()
-
-// Added for SPARK-6082. This assertion can be useful for 
scenarios when something
-// like Hive TRANSFORM is used. The external data generation 
script used in TRANSFORM
-// may result malformed rows, causing 
ArrayIndexOutOfBoundsException, which is somewhat
-// hard to decipher.
-assert(
-  row.numFields == columnBuilders.length,
-  s"Row column number mismatch, expected ${output.size} columns, " 
+
-s"but got ${row.numFields}." +
-s"\nRow content: $row")
-
-var i = 0
-totalSize = 0
-while (i < row.numFields) {
-  columnBuilders(i).appendFrom(row, i)
-  totalSize += columnBuilders(i).columnStats.sizeInBytes
-  i += 1
-}
-rowCount += 1
-  }
-
-  sizeInBytesStats.add(totalSize)
-  rowCountStats.add(rowCount)
-
-  val stats = InternalRow.fromSeq(
-columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-  CachedBatch(rowCount, columnBuilders.map { builder =>
-JavaUtils.bufferToArray(builder.build())
-  }, stats)
-}
-
-def hasNext: Boolean = rowIterator.hasNext
-  }
-}.persist(storageLevel)
-
+val cached = serializer.convertForCache(cachedPlan)
+.map { batch =>
+  sizeInBytesStats.add(batch.sizeInBytes)
+  rowCountStats.add(batch.numRows)
+  batch
+}.persist(storageLevel)
 cached.setName(cachedName)
 cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None

Review comment:
   Can we implement this without double-checked locking? It is not expected 
in general. Can we do it in initializer?
   
   If we need to use double-checked locking, we need `@volatile` for `ser`. See 
https://www.cs.umd.edu/~pugh/java/memoryModel/DoubleCheckedLocking.html
   





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] c21 commented on pull request #29079: [SPARK-32286][SQL] Coalesce bucketed table for shuffled hash join if applicable

2020-07-18 Thread GitBox


c21 commented on pull request #29079:
URL: https://github.com/apache/spark/pull/29079#issuecomment-660590187


   Addressed all comments and rebased to latest master. Thanks. cc @maropu, 
@cloud-fan and @viirya.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] c21 commented on a change in pull request #29079: [SPARK-32286][SQL] Coalesce bucketed table for shuffled hash join if applicable

2020-07-18 Thread GitBox


c21 commented on a change in pull request #29079:
URL: https://github.com/apache/spark/pull/29079#discussion_r456862050



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##
@@ -2651,12 +2651,13 @@ object SQLConf {
   .booleanConf
   .createWithDefault(true)
 
-  val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED =
-buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled")
+  val COALESCE_BUCKETS_IN_JOIN_ENABLED =
+buildConf("spark.sql.bucketing.coalesceBucketsInJoin.enabled")
   .doc("When true, if two bucketed tables with the different number of 
buckets are joined, " +
 "the side with a bigger number of buckets will be coalesced to have 
the same number " +
-"of buckets as the other side. Bucket coalescing is applied only to 
sort-merge joins " +
-"and only when the bigger number of buckets is divisible by the 
smaller number of buckets.")
+"of buckets as the other side. Bigger number of buckets is divisible 
by the smaller " +
+"number of buckets. Bucket coalescing is applied to sort-merge joins 
and " +
+"shuffled hash join.")

Review comment:
   @viirya - sure. added.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] c21 commented on a change in pull request #29079: [SPARK-32286][SQL] Coalesce bucketed table for shuffled hash join if applicable

2020-07-18 Thread GitBox


c21 commented on a change in pull request #29079:
URL: https://github.com/apache/spark/pull/29079#discussion_r456862018



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##
@@ -2659,12 +2660,24 @@ object SQLConf {
 
buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.maxBucketRatio")
   .doc("The ratio of the number of two buckets being coalesced should be 
less than or " +
 "equal to this value for bucket coalescing to be applied. This 
configuration only " +
-s"has an effect when 
'${COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key}' is set to true.")
+s"has an effect when '${COALESCE_BUCKETS_IN_JOIN_ENABLED.key}' is set 
to true.")
   .version("3.1.0")
   .intConf
   .checkValue(_ > 0, "The difference must be positive.")
   .createWithDefault(4)
 
+  val COALESCE_BUCKETS_IN_SHUFFLED_HASH_JOIN_MAX_BUCKET_RATIO =
+
buildConf("spark.sql.bucketing.coalesceBucketsInShuffledHashJoin.maxBucketRatio")
+  .doc("The ratio of the number of two buckets being coalesced should be 
less than or " +
+"equal to this value for bucket coalescing to be applied. This 
configuration only " +
+s"has an effect when '${COALESCE_BUCKETS_IN_JOIN_ENABLED.key}' is set 
to true. " +
+"Note as coalescing reduces parallelism, there might be a higher risk 
for " +
+"out of memory error at shuffled hash join build side.")
+  .version("3.1.0")
+  .intConf
+  .checkValue(_ > 0, "The difference must be positive.")
+  .createWithDefault(2)

Review comment:
   Sounds good. Updated to single ratio config for SMJ and SHJ. Thanks. cc 
@maropu and @viirya.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29154: [MINOR][DOCS] add link for Debugging your Application in running-on-yarn.html#launching-spark-on-yarn

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29154:
URL: https://github.com/apache/spark/pull/29154#issuecomment-660588814


   Can one of the admins verify this patch?



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29154: [MINOR][DOCS] add link for Debugging your Application in running-on-yarn.html#launching-spark-on-yarn

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29154:
URL: https://github.com/apache/spark/pull/29154#issuecomment-660588714


   Can one of the admins verify this patch?



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29154: [MINOR][DOCS] add link for Debugging your Application in running-on-yarn.html#launching-spark-on-yarn

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29154:
URL: https://github.com/apache/spark/pull/29154#issuecomment-660588714


   Can one of the admins verify this patch?



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29085: [SPARK-32106][SQL]Implement SparkScriptTransformationExec in sql/core

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29085:
URL: https://github.com/apache/spark/pull/29085#issuecomment-660588232







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29085: [SPARK-32106][SQL]Implement SparkScriptTransformationExec in sql/core

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29085:
URL: https://github.com/apache/spark/pull/29085#issuecomment-660588232







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] brandonJY opened a new pull request #29154: [MINOR][DOCS] add link for Debugging your Application in running-on-yarn.html#launching-spark-on-yarn

2020-07-18 Thread GitBox


brandonJY opened a new pull request #29154:
URL: https://github.com/apache/spark/pull/29154


   ### What changes were proposed in this pull request?
add link for Debugging your Application in 
`running-on-yarn.html#launching-spark-on-yar`
   
   
   
   ### Why are the changes needed?
   Currrently on running-on-yarn.html page launching-spark-on-yarn section, it 
mentions to refer for Debugging your Application. It is better to add a direct 
link for it to save reader time to find the section
 
![image](https://user-images.githubusercontent.com/20021316/87867542-80cc5500-c9c0-11ea-8560-5ddcb5a308bc.png)
   
   
   
   
   
   ### Does this PR introduce _any_ user-facing change?
   Yes. 
   Docs changes.  add link for Debugging your Application in 
`running-on-yarn.html#launching-spark-on-yarn` section
   Updated behavior:
   
![image](https://user-images.githubusercontent.com/20021316/87867534-6eeab200-c9c0-11ea-94ee-d3fa58157156.png)
   
   
   
   ### How was this patch tested?
   manual test has been performed to test the updated
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29085: [SPARK-32106][SQL]Implement SparkScriptTransformationExec in sql/core

2020-07-18 Thread GitBox


SparkQA commented on pull request #29085:
URL: https://github.com/apache/spark/pull/29085#issuecomment-660588126


   **[Test build #126120 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126120/testReport)**
 for PR 29085 at commit 
[`4ef4d76`](https://github.com/apache/spark/commit/4ef4d76bfd0e044dc4d5a0a9d674770a35ede408).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29085: [SPARK-32106][SQL]Implement SparkScriptTransformationExec in sql/core

2020-07-18 Thread GitBox


AngersZh commented on a change in pull request #29085:
URL: https://github.com/apache/spark/pull/29085#discussion_r456860293



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
##
@@ -87,17 +170,59 @@ trait BaseScriptTransformationExec extends UnaryExecNode {
   }
 }
   }
+
+  private lazy val fieldWriters: Seq[String => Any] = output.map { attr =>
+val converter = 
CatalystTypeConverters.createToCatalystConverter(attr.dataType)
+attr.dataType match {
+  case StringType => wrapperConvertException(data => data, converter)
+  case BooleanType => wrapperConvertException(data => data.toBoolean, 
converter)
+  case ByteType => wrapperConvertException(data => data.toByte, converter)
+  case BinaryType => wrapperConvertException(data => data.getBytes, 
converter)
+  case IntegerType => wrapperConvertException(data => data.toInt, 
converter)

Review comment:
   @marope Add type convert for BooleanType and BinaryType, and I have add 
it in UT `test("SPARK-32106: TRANSFORM should support more data types (no 
serde)")`





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29085: [SPARK-32106][SQL]Implement SparkScriptTransformationExec in sql/core

2020-07-18 Thread GitBox


AngersZh commented on a change in pull request #29085:
URL: https://github.com/apache/spark/pull/29085#discussion_r456860128



##
File path: sql/core/src/test/resources/sql-tests/inputs/transform.sql
##
@@ -0,0 +1,47 @@
+-- Test data.
+CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES
+('a'), ('b'), ('v')
+as t1(a);
+
+CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES
+(1, '1', 1.0, Decimal(1.0), timestamp(1)),
+(2, '2', 2.0, Decimal(2.0), timestamp(2)),
+(2, '2', 3.0, Decimal(3.0), timestamp(3))

Review comment:
   > Could you add all the typed data for tests where possible, e.g., 
`date` and `float`.
   
   Updated.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] holdenk commented on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

2020-07-18 Thread GitBox


holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660582133


   The python packaging tests are failing on Jenkins post upgrade and this 
passes all of the GH actions so unless there is any more discussion I intend to 
merge this tomorrow.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on pull request #29085: [SPARK-32106][SQL]Implement SparkScriptTransformationExec in sql/core

2020-07-18 Thread GitBox


AngersZh commented on pull request #29085:
URL: https://github.com/apache/spark/pull/29085#issuecomment-660580865


   > > Some some can't run without hive
   > 
   > Which kind of queries?
   
   DataType such as CalenderIntervalType, 
https://github.com/apache/spark/pull/29085#discussion_r456770157
   
   Schemaless when input column more than 2. 
https://github.com/apache/spark/pull/29085#discussion_r456751726
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29138: [SPARK-32338] [SQL] Overload slice to accept Column for start and length

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29138:
URL: https://github.com/apache/spark/pull/29138#issuecomment-660579657


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126116/
   Test FAILed.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29138: [SPARK-32338] [SQL] Overload slice to accept Column for start and length

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29138:
URL: https://github.com/apache/spark/pull/29138#issuecomment-660579655


   Merged build finished. Test FAILed.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA removed a comment on pull request #29138: [SPARK-32338] [SQL] Overload slice to accept Column for start and length

2020-07-18 Thread GitBox


SparkQA removed a comment on pull request #29138:
URL: https://github.com/apache/spark/pull/29138#issuecomment-660553870


   **[Test build #126116 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126116/testReport)**
 for PR 29138 at commit 
[`43d4f18`](https://github.com/apache/spark/commit/43d4f18e398e17eda7181e0c82d3f22495bbfcfe).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29138: [SPARK-32338] [SQL] Overload slice to accept Column for start and length

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29138:
URL: https://github.com/apache/spark/pull/29138#issuecomment-660579655







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29138: [SPARK-32338] [SQL] Overload slice to accept Column for start and length

2020-07-18 Thread GitBox


SparkQA commented on pull request #29138:
URL: https://github.com/apache/spark/pull/29138#issuecomment-660579492


   **[Test build #126116 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126116/testReport)**
 for PR 29138 at commit 
[`43d4f18`](https://github.com/apache/spark/commit/43d4f18e398e17eda7181e0c82d3f22495bbfcfe).
* This patch **fails PySpark pip packaging tests**.
* This patch merges cleanly.
* This patch adds no public classes.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] huaxingao commented on pull request #29056: [SPARK-31753][SQL][DOCS] Add missing keywords in the SQL docs

2020-07-18 Thread GitBox


huaxingao commented on pull request #29056:
URL: https://github.com/apache/spark/pull/29056#issuecomment-660577750


   @GuoPhilipse Sorry I have more nitpicks on grammar. I know this is very 
tedious. I went through all these when I did the doc PRs. Thanks a lot for 
doing this!
   I have no more comments :)



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon closed pull request #29133: [SPARK-32253][INFRA] Show errors only for the sbt tests of github actions

2020-07-18 Thread GitBox


HyukjinKwon closed pull request #29133:
URL: https://github.com/apache/spark/pull/29133


   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon commented on pull request #29133: [SPARK-32253][INFRA] Show errors only for the sbt tests of github actions

2020-07-18 Thread GitBox


HyukjinKwon commented on pull request #29133:
URL: https://github.com/apache/spark/pull/29133#issuecomment-660577416


   I filed new JIRA for test reporter (SPARK-32357)..  let's see how it goes.
   
   Merged to master.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] baohe-zhang commented on a change in pull request #29149: [SPARK-32350][CORE] Add batch-write on LevelDB to improve performance of HybridStore

2020-07-18 Thread GitBox


baohe-zhang commented on a change in pull request #29149:
URL: https://github.com/apache/spark/pull/29149#discussion_r456851608



##
File path: 
common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java
##
@@ -171,6 +172,58 @@ public void write(Object value) throws Exception {
 }
   }
 
+  public void writeAll(List values) throws Exception {
+Preconditions.checkArgument(values != null && !values.isEmpty(),
+  "Non-empty values required.");
+
+// Group by class, in case there are values from different classes in the 
values
+// Typical usecase is for this to be a single class.
+// A NullPointerException will be thrown if values contain null object.
+for (Map.Entry, ? extends List> entry :
+
values.stream().collect(Collectors.groupingBy(Object::getClass)).entrySet()) {
+
+  final Iterator valueIter = entry.getValue().iterator();
+  final Iterator serializedValueIter;
+
+  // Deserialize outside synchronized block
+  List list = new ArrayList<>(entry.getValue().size());
+  for (Object value : values) {
+list.add(serializer.serialize(value));
+  }
+  serializedValueIter = list.iterator();
+
+  final Class klass = entry.getKey();
+  final LevelDBTypeInfo ti = getTypeInfo(klass);
+
+  synchronized (ti) {
+final LevelDBTypeInfo.Index naturalIndex = ti.naturalIndex();
+final Collection indices = ti.indices();
+
+try (WriteBatch batch = db().createWriteBatch()) {
+  while (valueIter.hasNext()) {
+final Object value = valueIter.next();

Review comment:
   Done





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] huaxingao commented on a change in pull request #29056: [SPARK-31753][SQL][DOCS] Add missing keywords in the SQL docs

2020-07-18 Thread GitBox


huaxingao commented on a change in pull request #29056:
URL: https://github.com/apache/spark/pull/29056#discussion_r456849627



##
File path: docs/sql-ref-syntax-qry-select-lateral-view.md
##
@@ -0,0 +1,130 @@
+---
+layout: global
+title: LATERAL VIEW Clause
+displayTitle: LATERAL VIEW Clause
+license: |
+  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.
+---
+
+### Description
+
+`LATERAL VIEW` clause is used in conjunction with generator functions such as 
explode(), which will generate a virtual table containing one or more rows. 
`LATERAL VIEW` will apply the rows to each original output row.
+
+### Syntax
+
+```sql
+LATERAL VIEW [ OUTER ] generator_function ( expression [ , ... ] ) [ 
table_alias ] AS column_alias [ , ... ]
+```
+
+### Parameters
+
+* **OUTER**
+
+If `LATERAL VIEW` is used without `OUTER`, and `generator_function` 
returns empty, then no results will be output in `SELECT` clause.
+If `LATERAL VIEW` is used with `OUTER`, and `generator_function` returns 
empty, then results will be output normally with `NULL` as `generator_function` 
output.
+
+* **generator_function**
+
+This expression will output a virtual table with single input row.
+
+* **expression**
+
+Paramters for  generating_function.

Review comment:
   typo: Parameters

##
File path: docs/sql-ref-syntax-qry-select-groupby.md
##
@@ -91,6 +93,18 @@ aggregate_name ( [ DISTINCT ] expression [ , ... ] ) [ 
FILTER ( WHERE boolean_ex
 Filters the input rows for which the `boolean_expression` in the `WHERE` 
clause evaluates
 to true are passed to the aggregate function; other rows are discarded.
 
+* **FIRST**
+
+`FIRST` selects a first expression value from the data set. We can specify 
an optional `IGNORE NULL` clause to ignore NULL values.

Review comment:
   I thought data set is one word. I searched docs, both data set and 
dataset are used, so I guess either way is OK. 

##
File path: docs/sql-ref-syntax-qry-select-lateral-view.md
##
@@ -0,0 +1,130 @@
+---
+layout: global
+title: LATERAL VIEW Clause
+displayTitle: LATERAL VIEW Clause
+license: |
+  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.
+---
+
+### Description
+
+`LATERAL VIEW` clause is used in conjunction with generator functions such as 
explode(), which will generate a virtual table containing one or more rows. 
`LATERAL VIEW` will apply the rows to each original output row.
+
+### Syntax
+
+```sql
+LATERAL VIEW [ OUTER ] generator_function ( expression [ , ... ] ) [ 
table_alias ] AS column_alias [ , ... ]
+```
+
+### Parameters
+
+* **OUTER**
+
+If `LATERAL VIEW` is used without `OUTER`, and `generator_function` 
returns empty, then no results will be output in `SELECT` clause.
+If `LATERAL VIEW` is used with `OUTER`, and `generator_function` returns 
empty, then results will be output normally with `NULL` as `generator_function` 
output.
+
+* **generator_function**
+
+This expression will output a virtual table with single input row.
+
+* **expression**
+
+Paramters for  generating_function.
+
+* **table_alias**
+
+It is the alias for `generator_function`, which is optional.
+ 
+* **column_alias**
+
+It lists the column aliases of `generator_function`, which may be used in 
output rows, we may have multiple alias if `generator_function` have multiple 
output columns.

Review comment:
   Remove it?
   , we -> . We?
   multiple alias -> multiple aliases?
   ```
   Lists the 

[GitHub] [spark] HyukjinKwon commented on pull request #29143: [SPARK-32344][SQL] Unevaluable expr is set to FIRST/LAST ignoreNullsExpr in distinct aggregates

2020-07-18 Thread GitBox


HyukjinKwon commented on pull request #29143:
URL: https://github.com/apache/spark/pull/29143#issuecomment-660573442


   Merged to master and branch-3.0.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon closed pull request #29143: [SPARK-32344][SQL] Unevaluable expr is set to FIRST/LAST ignoreNullsExpr in distinct aggregates

2020-07-18 Thread GitBox


HyukjinKwon closed pull request #29143:
URL: https://github.com/apache/spark/pull/29143


   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon commented on a change in pull request #29143: [SPARK-32344][SQL] Unevaluable expr is set to FIRST/LAST ignoreNullsExpr in distinct aggregates

2020-07-18 Thread GitBox


HyukjinKwon commented on a change in pull request #29143:
URL: https://github.com/apache/spark/pull/29143#discussion_r456848709



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala
##
@@ -120,3 +120,11 @@ case class First(child: Expression, ignoreNullsExpr: 
Expression)
 
   override def toString: String = s"$prettyName($child)${if (ignoreNulls) " 
ignore nulls"}"
 }
+
+object FirstLast {

Review comment:
   I think this deduplication is a little bit too much but I guess it's 
fine.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660573211







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA removed a comment on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


SparkQA removed a comment on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660569876


   **[Test build #126118 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126118/testReport)**
 for PR 29117 at commit 
[`d7974a4`](https://github.com/apache/spark/commit/d7974a4d58bd51f99d6c010ac536e63a5094fbf3).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660573211







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


SparkQA commented on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660573178


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



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon commented on pull request #29151: [SPARK-29802][BUILD] Use python3 in build scripts

2020-07-18 Thread GitBox


HyukjinKwon commented on pull request #29151:
URL: https://github.com/apache/spark/pull/29151#issuecomment-660572682


   With the test results from Jenkins and GitHub Actions, seems all tests pass.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon closed pull request #29151: [SPARK-29802][BUILD] Use python3 in build scripts

2020-07-18 Thread GitBox


HyukjinKwon closed pull request #29151:
URL: https://github.com/apache/spark/pull/29151


   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon commented on pull request #29151: [SPARK-29802][BUILD] Use python3 in build scripts

2020-07-18 Thread GitBox


HyukjinKwon commented on pull request #29151:
URL: https://github.com/apache/spark/pull/29151#issuecomment-660572614


   Merged to master.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660572338


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126119/
   Test FAILed.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660572337


   Merged build finished. Test FAILed.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA removed a comment on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


SparkQA removed a comment on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660570649


   **[Test build #126119 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126119/testReport)**
 for PR 29117 at commit 
[`d7974a4`](https://github.com/apache/spark/commit/d7974a4d58bd51f99d6c010ac536e63a5094fbf3).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


SparkQA commented on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660572332


   **[Test build #126119 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126119/testReport)**
 for PR 29117 at commit 
[`d7974a4`](https://github.com/apache/spark/commit/d7974a4d58bd51f99d6c010ac536e63a5094fbf3).
* This patch **fails PySpark pip packaging tests**.
* This patch merges cleanly.
* This patch adds no public classes.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660572337







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon commented on pull request #29151: [SPARK-29909][BUILD] Use python3 in build scripts

2020-07-18 Thread GitBox


HyukjinKwon commented on pull request #29151:
URL: https://github.com/apache/spark/pull/29151#issuecomment-660572238


   @srowen, I am investigating it in #29117. I will be able to fix soon I guess.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

2020-07-18 Thread GitBox


HyukjinKwon commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456847406



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##
@@ -0,0 +1,279 @@
+/*
+ * 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.columnar
+
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.expressions.{And, Attribute, 
AttributeReference, BindReferences, EqualNullSafe, EqualTo, Expression, 
GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, Length, LessThan, 
LessThanOrEqual, Literal, Or, Predicate, StartsWith}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, 
PartitionStatistics}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AtomicType, BinaryType}
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+/**
+ * Basic interface that all cached batches of data must support. This is 
primarily to allow
+ * for metrics to be handled outside of the encoding and decoding steps in a 
standard way.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
+
+/**
+ * Provides APIs for compressing, filtering, and decompressing SQL data that 
will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Run the given plan and convert its output to a implementation of 
[[CachedBatch]].
+   * @param cachedPlan the plan to run.
+   * @return the RDD containing the batches of data to cache.
+   */
+  def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch]
+
+  /**
+   * Builds a function that can be used to filter which batches are loaded.
+   * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will 
provide the filter logic
+   * necessary.  You will need to provide metrics for this to work. 
[[SimpleMetricsCachedBatch]]
+   * provides the APIs to hold those metrics and explains the metrics used, 
really just min and max.
+   * Note that this is intended to skip batches that are not needed, and the 
actual filtering of
+   * individual rows is handled later.
+   * @param predicates the set of expressions to use for filtering.
+   * @param cachedAttributes the schema/attributes of the data that is cached. 
This can be helpful
+   * if you don't store it with the data.
+   * @return a function that takes the partition id and the iterator of 
batches in the partition.
+   * It returns an iterator of batches that should be loaded.
+   */
+  def buildFilter(predicates: Seq[Expression],
+  cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => 
Iterator[CachedBatch]
+
+  /**
+   * Decompress the cached data into a ColumnarBatch. This currently is only 
used for basic types
+   * BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | 
DoubleType
+   * That may change in the future.
+   * @param input the cached batches that should be decompressed.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the field that should be loaded from the data, 
and the order they
+   *   should appear in the output batch.
+   * @param conf the configuration for the job.
+   * @return an RDD of the input cached batches transformed into the 
ColumnarBatch format.
+   */
+  def decompressColumnar(
+  input: RDD[CachedBatch],
+  cacheAttributes: Seq[Attribute],
+  selectedAttributes: Seq[Attribute],
+  conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Decompress the cached batch into [[InternalRow]]. If you want this to be 
performant, code

Review comment:
I think the error seems from here:
   
   ```
   [warn] 
/home/runner/work/spark/spark/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala:90:
 Could not find 

[GitHub] [spark] HyukjinKwon commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

2020-07-18 Thread GitBox


HyukjinKwon commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456847406



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##
@@ -0,0 +1,279 @@
+/*
+ * 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.columnar
+
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.expressions.{And, Attribute, 
AttributeReference, BindReferences, EqualNullSafe, EqualTo, Expression, 
GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, Length, LessThan, 
LessThanOrEqual, Literal, Or, Predicate, StartsWith}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, 
PartitionStatistics}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AtomicType, BinaryType}
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+/**
+ * Basic interface that all cached batches of data must support. This is 
primarily to allow
+ * for metrics to be handled outside of the encoding and decoding steps in a 
standard way.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
+
+/**
+ * Provides APIs for compressing, filtering, and decompressing SQL data that 
will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Run the given plan and convert its output to a implementation of 
[[CachedBatch]].
+   * @param cachedPlan the plan to run.
+   * @return the RDD containing the batches of data to cache.
+   */
+  def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch]
+
+  /**
+   * Builds a function that can be used to filter which batches are loaded.
+   * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will 
provide the filter logic
+   * necessary.  You will need to provide metrics for this to work. 
[[SimpleMetricsCachedBatch]]
+   * provides the APIs to hold those metrics and explains the metrics used, 
really just min and max.
+   * Note that this is intended to skip batches that are not needed, and the 
actual filtering of
+   * individual rows is handled later.
+   * @param predicates the set of expressions to use for filtering.
+   * @param cachedAttributes the schema/attributes of the data that is cached. 
This can be helpful
+   * if you don't store it with the data.
+   * @return a function that takes the partition id and the iterator of 
batches in the partition.
+   * It returns an iterator of batches that should be loaded.
+   */
+  def buildFilter(predicates: Seq[Expression],
+  cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => 
Iterator[CachedBatch]
+
+  /**
+   * Decompress the cached data into a ColumnarBatch. This currently is only 
used for basic types
+   * BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | 
DoubleType
+   * That may change in the future.
+   * @param input the cached batches that should be decompressed.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the field that should be loaded from the data, 
and the order they
+   *   should appear in the output batch.
+   * @param conf the configuration for the job.
+   * @return an RDD of the input cached batches transformed into the 
ColumnarBatch format.
+   */
+  def decompressColumnar(
+  input: RDD[CachedBatch],
+  cacheAttributes: Seq[Attribute],
+  selectedAttributes: Seq[Attribute],
+  conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Decompress the cached batch into [[InternalRow]]. If you want this to be 
performant, code

Review comment:
I think the error seems from here:
   
   ```
   [warn] 
/home/runner/work/spark/spark/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala:90:
 Could not find 

[GitHub] [spark] maropu commented on a change in pull request #29085: [SPARK-32106][SQL]Implement SparkScriptTransformationExec in sql/core

2020-07-18 Thread GitBox


maropu commented on a change in pull request #29085:
URL: https://github.com/apache/spark/pull/29085#discussion_r456847553



##
File path: sql/core/src/test/resources/sql-tests/inputs/transform.sql
##
@@ -0,0 +1,47 @@
+-- Test data.
+CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES
+('a'), ('b'), ('v')
+as t1(a);
+
+CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES
+(1, '1', 1.0, Decimal(1.0), timestamp(1)),
+(2, '2', 2.0, Decimal(2.0), timestamp(2)),
+(2, '2', 3.0, Decimal(3.0), timestamp(3))

Review comment:
   Could you add all the typed data for tests where possible, e.g., `date` 
and `float`.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] maropu commented on a change in pull request #29085: [SPARK-32106][SQL]Implement SparkScriptTransformationExec in sql/core

2020-07-18 Thread GitBox


maropu commented on a change in pull request #29085:
URL: https://github.com/apache/spark/pull/29085#discussion_r456847499



##
File path: sql/core/src/test/resources/sql-tests/inputs/transform.sql
##
@@ -0,0 +1,47 @@
+-- Test data.
+CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES
+('a'), ('b'), ('v')
+as t1(a);
+
+CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES
+(1, '1', 1.0, Decimal(1.0), timestamp(1)),
+(2, '2', 2.0, Decimal(2.0), timestamp(2)),
+(2, '2', 3.0, Decimal(3.0), timestamp(3))
+as t2(a,b,c,d,e);
+
+SELECT TRANSFORM(a)
+USING 'cat' AS (a)
+FROM t1;
+
+
+-- with non-exist command
+SELECT TRANSFORM(a)
+USING 'some_non_existent_command' AS (a)
+FROM t1;
+
+-- with non-exist file
+SELECT TRANSFORM(a)
+USING 'python some_non_existent_file' AS (a)
+FROM t1;
+
+
+-- support different data type
+SELECT TRANSFORM(a, b, c, d, e)
+USING 'CAT' AS (a, b, c, d, e)
+FROM t2;
+
+
+-- handle schema less
+SELECT TRANSFORM(a, b)
+USING 'CAT'
+FROM t2;
+
+-- return null when return string incompatible(no serde)
+SELECT TRANSFORM(a, b, c)
+USING 'cat' as (a int, b int , c int)
+FROM (
+SELECT
+1 AS a,

Review comment:
   nit: add indents for readability.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

2020-07-18 Thread GitBox


HyukjinKwon commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456847406



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##
@@ -0,0 +1,279 @@
+/*
+ * 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.columnar
+
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.expressions.{And, Attribute, 
AttributeReference, BindReferences, EqualNullSafe, EqualTo, Expression, 
GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, Length, LessThan, 
LessThanOrEqual, Literal, Or, Predicate, StartsWith}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, 
PartitionStatistics}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AtomicType, BinaryType}
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+/**
+ * Basic interface that all cached batches of data must support. This is 
primarily to allow
+ * for metrics to be handled outside of the encoding and decoding steps in a 
standard way.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
+
+/**
+ * Provides APIs for compressing, filtering, and decompressing SQL data that 
will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Run the given plan and convert its output to a implementation of 
[[CachedBatch]].
+   * @param cachedPlan the plan to run.
+   * @return the RDD containing the batches of data to cache.
+   */
+  def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch]
+
+  /**
+   * Builds a function that can be used to filter which batches are loaded.
+   * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will 
provide the filter logic
+   * necessary.  You will need to provide metrics for this to work. 
[[SimpleMetricsCachedBatch]]
+   * provides the APIs to hold those metrics and explains the metrics used, 
really just min and max.
+   * Note that this is intended to skip batches that are not needed, and the 
actual filtering of
+   * individual rows is handled later.
+   * @param predicates the set of expressions to use for filtering.
+   * @param cachedAttributes the schema/attributes of the data that is cached. 
This can be helpful
+   * if you don't store it with the data.
+   * @return a function that takes the partition id and the iterator of 
batches in the partition.
+   * It returns an iterator of batches that should be loaded.
+   */
+  def buildFilter(predicates: Seq[Expression],
+  cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => 
Iterator[CachedBatch]
+
+  /**
+   * Decompress the cached data into a ColumnarBatch. This currently is only 
used for basic types
+   * BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | 
DoubleType
+   * That may change in the future.
+   * @param input the cached batches that should be decompressed.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the field that should be loaded from the data, 
and the order they
+   *   should appear in the output batch.
+   * @param conf the configuration for the job.
+   * @return an RDD of the input cached batches transformed into the 
ColumnarBatch format.
+   */
+  def decompressColumnar(
+  input: RDD[CachedBatch],
+  cacheAttributes: Seq[Attribute],
+  selectedAttributes: Seq[Attribute],
+  conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Decompress the cached batch into [[InternalRow]]. If you want this to be 
performant, code

Review comment:
I think the error seems from here:
   
   ```
   [warn] 
/home/runner/work/spark/spark/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala:90:
 Could not find 

[GitHub] [spark] HyukjinKwon commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

2020-07-18 Thread GitBox


HyukjinKwon commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456847406



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##
@@ -0,0 +1,279 @@
+/*
+ * 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.columnar
+
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.expressions.{And, Attribute, 
AttributeReference, BindReferences, EqualNullSafe, EqualTo, Expression, 
GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, Length, LessThan, 
LessThanOrEqual, Literal, Or, Predicate, StartsWith}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, 
PartitionStatistics}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AtomicType, BinaryType}
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+/**
+ * Basic interface that all cached batches of data must support. This is 
primarily to allow
+ * for metrics to be handled outside of the encoding and decoding steps in a 
standard way.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
+
+/**
+ * Provides APIs for compressing, filtering, and decompressing SQL data that 
will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Run the given plan and convert its output to a implementation of 
[[CachedBatch]].
+   * @param cachedPlan the plan to run.
+   * @return the RDD containing the batches of data to cache.
+   */
+  def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch]
+
+  /**
+   * Builds a function that can be used to filter which batches are loaded.
+   * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will 
provide the filter logic
+   * necessary.  You will need to provide metrics for this to work. 
[[SimpleMetricsCachedBatch]]
+   * provides the APIs to hold those metrics and explains the metrics used, 
really just min and max.
+   * Note that this is intended to skip batches that are not needed, and the 
actual filtering of
+   * individual rows is handled later.
+   * @param predicates the set of expressions to use for filtering.
+   * @param cachedAttributes the schema/attributes of the data that is cached. 
This can be helpful
+   * if you don't store it with the data.
+   * @return a function that takes the partition id and the iterator of 
batches in the partition.
+   * It returns an iterator of batches that should be loaded.
+   */
+  def buildFilter(predicates: Seq[Expression],
+  cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => 
Iterator[CachedBatch]
+
+  /**
+   * Decompress the cached data into a ColumnarBatch. This currently is only 
used for basic types
+   * BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | 
DoubleType
+   * That may change in the future.
+   * @param input the cached batches that should be decompressed.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the field that should be loaded from the data, 
and the order they
+   *   should appear in the output batch.
+   * @param conf the configuration for the job.
+   * @return an RDD of the input cached batches transformed into the 
ColumnarBatch format.
+   */
+  def decompressColumnar(
+  input: RDD[CachedBatch],
+  cacheAttributes: Seq[Attribute],
+  selectedAttributes: Seq[Attribute],
+  conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Decompress the cached batch into [[InternalRow]]. If you want this to be 
performant, code

Review comment:
I think the error seems from here:
   
   ```
   [warn] 
/home/runner/work/spark/spark/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala:90:
 Could not find 

[GitHub] [spark] maropu commented on pull request #29085: [SPARK-32106][SQL]Implement SparkScriptTransformationExec in sql/core

2020-07-18 Thread GitBox


maropu commented on pull request #29085:
URL: https://github.com/apache/spark/pull/29085#issuecomment-660571530


   > Some some can't run without hive
   
   Which kind of queries?
   
   > For doc, I prefer to update after all these done. Ok?
   
   Yea, it looks okay to me.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660570773







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660570773







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] viirya commented on pull request #29133: [SPARK-32253][INFRA] Show errors only for the sbt tests of github actions

2020-07-18 Thread GitBox


viirya commented on pull request #29133:
URL: https://github.com/apache/spark/pull/29133#issuecomment-660570738


   Looks okay to me. We can keep improving this.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


SparkQA commented on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660570649


   **[Test build #126119 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126119/testReport)**
 for PR 29117 at commit 
[`d7974a4`](https://github.com/apache/spark/commit/d7974a4d58bd51f99d6c010ac536e63a5094fbf3).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon commented on pull request #29133: [SPARK-32253][INFRA] Show errors only for the sbt tests of github actions

2020-07-18 Thread GitBox


HyukjinKwon commented on pull request #29133:
URL: https://github.com/apache/spark/pull/29133#issuecomment-660570181


   I am okay to try this out for now until we have a way to report the test 
results but I'll leave it to @dongjoon-hyun and @viirya.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660569987







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660569987







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon commented on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


HyukjinKwon commented on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660569991


   retest this please



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


SparkQA commented on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660569876


   **[Test build #126118 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126118/testReport)**
 for PR 29117 at commit 
[`d7974a4`](https://github.com/apache/spark/commit/d7974a4d58bd51f99d6c010ac536e63a5094fbf3).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon commented on pull request #29133: [SPARK-32253][INFRA] Show errors only for the sbt tests of github actions

2020-07-18 Thread GitBox


HyukjinKwon commented on pull request #29133:
URL: https://github.com/apache/spark/pull/29133#issuecomment-660569838


   @dongjoon-hyun, I am debugging the flakiness at 
https://github.com/apache/spark/pull/29117 to make a complete fix. I think it 
was caused by my recent fixes and Jenkins environment differences vs Github 
Actions. I will be able to fix at least in the next week.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] HyukjinKwon commented on pull request #29117: [WIP] Debug flaky pip installation test failure

2020-07-18 Thread GitBox


HyukjinKwon commented on pull request #29117:
URL: https://github.com/apache/spark/pull/29117#issuecomment-660569710


   retest this please



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29145: [WIP][SPARK-32346][SQL] Support filters pushdown in Avro datasource

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29145:
URL: https://github.com/apache/spark/pull/29145#issuecomment-660566442







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29145: [WIP][SPARK-32346][SQL] Support filters pushdown in Avro datasource

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29145:
URL: https://github.com/apache/spark/pull/29145#issuecomment-660566442







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29145: [WIP][SPARK-32346][SQL] Support filters pushdown in Avro datasource

2020-07-18 Thread GitBox


SparkQA commented on pull request #29145:
URL: https://github.com/apache/spark/pull/29145#issuecomment-660566210


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



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA removed a comment on pull request #29145: [WIP][SPARK-32346][SQL] Support filters pushdown in Avro datasource

2020-07-18 Thread GitBox


SparkQA removed a comment on pull request #29145:
URL: https://github.com/apache/spark/pull/29145#issuecomment-660536420


   **[Test build #126115 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126115/testReport)**
 for PR 29145 at commit 
[`072eab0`](https://github.com/apache/spark/commit/072eab0a7c35f5827b1c59d884fe3eea18251bc9).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29153: [SPARK-32310][ML][PySpark][WIP] ML params default value parity in feature and tuning

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29153:
URL: https://github.com/apache/spark/pull/29153#issuecomment-660564857


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126117/
   Test FAILed.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29153: [SPARK-32310][ML][PySpark][WIP] ML params default value parity in feature and tuning

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29153:
URL: https://github.com/apache/spark/pull/29153#issuecomment-660564855


   Merged build finished. Test FAILed.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29153: [SPARK-32310][ML][PySpark][WIP] ML params default value parity in feature and tuning

2020-07-18 Thread GitBox


SparkQA commented on pull request #29153:
URL: https://github.com/apache/spark/pull/29153#issuecomment-660564831


   **[Test build #126117 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126117/testReport)**
 for PR 29153 at commit 
[`86e0579`](https://github.com/apache/spark/commit/86e0579cd53d52f77871a0256d41f622d2bad474).
* This patch **fails Spark unit tests**.
* This patch merges cleanly.
* This patch adds no public classes.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29153: [SPARK-32310][ML][PySpark][WIP] ML params default value parity in feature and tuning

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29153:
URL: https://github.com/apache/spark/pull/29153#issuecomment-660564855







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA removed a comment on pull request #29153: [SPARK-32310][ML][PySpark][WIP] ML params default value parity in feature and tuning

2020-07-18 Thread GitBox


SparkQA removed a comment on pull request #29153:
URL: https://github.com/apache/spark/pull/29153#issuecomment-660561046


   **[Test build #126117 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126117/testReport)**
 for PR 29153 at commit 
[`86e0579`](https://github.com/apache/spark/commit/86e0579cd53d52f77871a0256d41f622d2bad474).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] frankyin-factual commented on pull request #28898: [SPARK-32059][SQL] Allow nested schema pruning thru window/sort/filter plans

2020-07-18 Thread GitBox


frankyin-factual commented on pull request #28898:
URL: https://github.com/apache/spark/pull/28898#issuecomment-660563330


   @maropu @viirya @dongjoon-hyun friendly bump 



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29153: [SPARK-32310][ML][PySpark][WIP] ML params default value parity in feature and tuning

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29153:
URL: https://github.com/apache/spark/pull/29153#issuecomment-660561180







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29153: [SPARK-32310][ML][PySpark][WIP] ML params default value parity in feature and tuning

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29153:
URL: https://github.com/apache/spark/pull/29153#issuecomment-660561180







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29153: [SPARK-32310][ML][PySpark][WIP] ML params default value parity in feature and tuning

2020-07-18 Thread GitBox


SparkQA commented on pull request #29153:
URL: https://github.com/apache/spark/pull/29153#issuecomment-660561046


   **[Test build #126117 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126117/testReport)**
 for PR 29153 at commit 
[`86e0579`](https://github.com/apache/spark/commit/86e0579cd53d52f77871a0256d41f622d2bad474).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] huaxingao opened a new pull request #29153: [SPARK-32310][ML][PySpark][WIP] ML params default value parity in feature and tuning

2020-07-18 Thread GitBox


huaxingao opened a new pull request #29153:
URL: https://github.com/apache/spark/pull/29153


   
   
   ### What changes were proposed in this pull request?
   set params default values in trait Params for feature and tuning in both 
Scala and Python. 
   
   
   ### Why are the changes needed?
   Make ML has the same default param values between estimator and its 
corresponding transformer, and also between Scala and Python.
   
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   
   ### How was this patch tested?
   Existing and modified tests
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] viirya commented on a change in pull request #29079: [SPARK-32286][SQL] Coalesce bucketed table for shuffled hash join if applicable

2020-07-18 Thread GitBox


viirya commented on a change in pull request #29079:
URL: https://github.com/apache/spark/pull/29079#discussion_r456839808



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##
@@ -2651,12 +2651,13 @@ object SQLConf {
   .booleanConf
   .createWithDefault(true)
 
-  val COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED =
-buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.enabled")
+  val COALESCE_BUCKETS_IN_JOIN_ENABLED =
+buildConf("spark.sql.bucketing.coalesceBucketsInJoin.enabled")
   .doc("When true, if two bucketed tables with the different number of 
buckets are joined, " +
 "the side with a bigger number of buckets will be coalesced to have 
the same number " +
-"of buckets as the other side. Bucket coalescing is applied only to 
sort-merge joins " +
-"and only when the bigger number of buckets is divisible by the 
smaller number of buckets.")
+"of buckets as the other side. Bigger number of buckets is divisible 
by the smaller " +
+"number of buckets. Bucket coalescing is applied to sort-merge joins 
and " +
+"shuffled hash join.")

Review comment:
   Can we add more doc like "Coalescing bucketed table can avoid 
unnecessary shuffling during joining but it also reduces parallelism and could 
possibly cause OOM  for shuffled hash join"?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] viirya commented on a change in pull request #29079: [SPARK-32286][SQL] Coalesce bucketed table for shuffled hash join if applicable

2020-07-18 Thread GitBox


viirya commented on a change in pull request #29079:
URL: https://github.com/apache/spark/pull/29079#discussion_r456839329



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##
@@ -2659,12 +2660,24 @@ object SQLConf {
 
buildConf("spark.sql.bucketing.coalesceBucketsInSortMergeJoin.maxBucketRatio")
   .doc("The ratio of the number of two buckets being coalesced should be 
less than or " +
 "equal to this value for bucket coalescing to be applied. This 
configuration only " +
-s"has an effect when 
'${COALESCE_BUCKETS_IN_SORT_MERGE_JOIN_ENABLED.key}' is set to true.")
+s"has an effect when '${COALESCE_BUCKETS_IN_JOIN_ENABLED.key}' is set 
to true.")
   .version("3.1.0")
   .intConf
   .checkValue(_ > 0, "The difference must be positive.")
   .createWithDefault(4)
 
+  val COALESCE_BUCKETS_IN_SHUFFLED_HASH_JOIN_MAX_BUCKET_RATIO =
+
buildConf("spark.sql.bucketing.coalesceBucketsInShuffledHashJoin.maxBucketRatio")
+  .doc("The ratio of the number of two buckets being coalesced should be 
less than or " +
+"equal to this value for bucket coalescing to be applied. This 
configuration only " +
+s"has an effect when '${COALESCE_BUCKETS_IN_JOIN_ENABLED.key}' is set 
to true. " +
+"Note as coalescing reduces parallelism, there might be a higher risk 
for " +
+"out of memory error at shuffled hash join build side.")
+  .version("3.1.0")
+  .intConf
+  .checkValue(_ > 0, "The difference must be positive.")
+  .createWithDefault(2)

Review comment:
   It looks a bit premature optimization to me for now. I think we can 
leave as one ratio config first and add it when it is required to have?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29138: [SPARK-32338] [SQL] Overload slice to accept Column for start and length

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29138:
URL: https://github.com/apache/spark/pull/29138#issuecomment-660553983







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29138: [SPARK-32338] [SQL] Overload slice to accept Column for start and length

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29138:
URL: https://github.com/apache/spark/pull/29138#issuecomment-660553983







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29138: [SPARK-32338] [SQL] Overload slice to accept Column for start and length

2020-07-18 Thread GitBox


SparkQA commented on pull request #29138:
URL: https://github.com/apache/spark/pull/29138#issuecomment-660553870


   **[Test build #126116 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126116/testReport)**
 for PR 29138 at commit 
[`43d4f18`](https://github.com/apache/spark/commit/43d4f18e398e17eda7181e0c82d3f22495bbfcfe).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] nvander1 commented on a change in pull request #29138: [SPARK-32338] [SQL] Overload slice to accept Column for start and length

2020-07-18 Thread GitBox


nvander1 commented on a change in pull request #29138:
URL: https://github.com/apache/spark/pull/29138#discussion_r456835780



##
File path: sql/core/src/main/scala/org/apache/spark/sql/functions.scala
##
@@ -3406,6 +3406,21 @@ object functions {
 Slice(x.expr, Literal(start), Literal(length))

Review comment:
   Updated @ueshin 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] nvander1 commented on a change in pull request #29138: [SPARK-32338] [SQL] Overload slice to accept Column for start and length

2020-07-18 Thread GitBox


nvander1 commented on a change in pull request #29138:
URL: https://github.com/apache/spark/pull/29138#discussion_r456834819



##
File path: sql/core/src/main/scala/org/apache/spark/sql/functions.scala
##
@@ -3406,6 +3406,21 @@ object functions {
 Slice(x.expr, Literal(start), Literal(length))

Review comment:
   Yes, I'm making the change now.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #28731: [SPARK-31909][SQL] Add SPARK_SUBMIT_OPTS to Beeline Script

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #28731:
URL: https://github.com/apache/spark/pull/28731#issuecomment-660550204


   Can one of the admins verify this patch?



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] dongjoon-hyun commented on pull request #29151: [SPARK-29909][BUILD] Use python3 in build scripts

2020-07-18 Thread GitBox


dongjoon-hyun commented on pull request #29151:
URL: https://github.com/apache/spark/pull/29151#issuecomment-660550190


   Give the following log, it seems to be related to `virtualenv`-side bug.
   ```
   Constructing virtual env for testing
   Using conda virtual environments
   Testing pip installation with python 3.6
   Using /tmp/tmp.2RCoqQqNP8 for virtualenv
   Fetching package metadata ...
   Solving package specifications: .
   
   Package plan for installation in environment /tmp/tmp.2RCoqQqNP8/3.6:
   ...
   ```



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #28731: [SPARK-31909][SQL] Add SPARK_SUBMIT_OPTS to Beeline Script

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #28731:
URL: https://github.com/apache/spark/pull/28731#issuecomment-639158594


   Can one of the admins verify this patch?



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] dongjoon-hyun closed pull request #29147: [SPARK-29292][YARN][K8S][MESOS] Fix Scala 2.13 compilation for remaining modules

2020-07-18 Thread GitBox


dongjoon-hyun closed pull request #29147:
URL: https://github.com/apache/spark/pull/29147


   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] dongjoon-hyun commented on pull request #29147: [SPARK-29292][YARN][K8S][MESOS] Fix Scala 2.13 compilation for remaining modules

2020-07-18 Thread GitBox


dongjoon-hyun commented on pull request #29147:
URL: https://github.com/apache/spark/pull/29147#issuecomment-660548998


   I didn't notice that. It seems that we update the Job to use R 3.5+.
   ```
   ++ /usr/bin/R CMD INSTALL 
--library=/home/jenkins/workspace/SparkPullRequestBuilder-K8s/R/lib 
/home/jenkins/workspace/SparkPullRequestBuilder-K8s/R/pkg/
   ERROR: this R is version 3.4.4, package 'SparkR' requires R >= 3.5
   ```



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] agrawaldevesh commented on pull request #29014: [SPARK-32199][SPARK-32198] Reduce job failures during decommissioning

2020-07-18 Thread GitBox


agrawaldevesh commented on pull request #29014:
URL: https://github.com/apache/spark/pull/29014#issuecomment-660540565


   The test failure in 
`org.apache.spark.sql.execution.adaptive.AdaptiveQueryExecSuite` looks 
unrelated and I cannot reproduce it locally.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29014: [SPARK-32199][SPARK-32198] Reduce job failures during decommissioning

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29014:
URL: https://github.com/apache/spark/pull/29014#issuecomment-660539329


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126113/
   Test FAILed.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29014: [SPARK-32199][SPARK-32198] Reduce job failures during decommissioning

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29014:
URL: https://github.com/apache/spark/pull/29014#issuecomment-660539324


   Merged build finished. Test FAILed.



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29014: [SPARK-32199][SPARK-32198] Reduce job failures during decommissioning

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29014:
URL: https://github.com/apache/spark/pull/29014#issuecomment-660539324







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA removed a comment on pull request #29014: [SPARK-32199][SPARK-32198] Reduce job failures during decommissioning

2020-07-18 Thread GitBox


SparkQA removed a comment on pull request #29014:
URL: https://github.com/apache/spark/pull/29014#issuecomment-660520796


   **[Test build #126113 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126113/testReport)**
 for PR 29014 at commit 
[`2417f03`](https://github.com/apache/spark/commit/2417f030ad0397839ab04940cdc46aedbfd66835).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29014: [SPARK-32199][SPARK-32198] Reduce job failures during decommissioning

2020-07-18 Thread GitBox


SparkQA commented on pull request #29014:
URL: https://github.com/apache/spark/pull/29014#issuecomment-660539135


   **[Test build #126113 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126113/testReport)**
 for PR 29014 at commit 
[`2417f03`](https://github.com/apache/spark/commit/2417f030ad0397839ab04940cdc46aedbfd66835).
* This patch **fails Spark unit tests**.
* This patch merges cleanly.
* This patch adds the following public classes _(experimental)_:
 * `case class ExecutorProcessLost(`



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29145: [WIP][SPARK-32346][SQL] Support filters pushdown in Avro datasource

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29145:
URL: https://github.com/apache/spark/pull/29145#issuecomment-660536547







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins commented on pull request #29145: [WIP][SPARK-32346][SQL] Support filters pushdown in Avro datasource

2020-07-18 Thread GitBox


AmplabJenkins commented on pull request #29145:
URL: https://github.com/apache/spark/pull/29145#issuecomment-660536547







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #29107: [SPARK-32308][SQL] Move by-name resolution logic of unionByName from API code to analysis phase

2020-07-18 Thread GitBox


AmplabJenkins removed a comment on pull request #29107:
URL: https://github.com/apache/spark/pull/29107#issuecomment-660536317







This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] SparkQA commented on pull request #29145: [WIP][SPARK-32346][SQL] Support filters pushdown in Avro datasource

2020-07-18 Thread GitBox


SparkQA commented on pull request #29145:
URL: https://github.com/apache/spark/pull/29145#issuecomment-660536420


   **[Test build #126115 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126115/testReport)**
 for PR 29145 at commit 
[`072eab0`](https://github.com/apache/spark/commit/072eab0a7c35f5827b1c59d884fe3eea18251bc9).



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



  1   2   3   4   >