You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by dongjoon-hyun <gi...@git.apache.org> on 2018/02/15 16:59:13 UTC

[GitHub] spark pull request #20619: [SPARK-23390][SQL] Register task completion liste...

GitHub user dongjoon-hyun opened a pull request:

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

    [SPARK-23390][SQL] Register task completion listerners first in ParquetFileFormat

    ## What changes were proposed in this pull request?
    
    ParquetFileFormat leaks opened files in some cases. This PR prevents that by register task completion listers first before initialization.
    
    - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-branch-2.3-test-sbt-hadoop-2.7/205/testReport/org.apache.spark.sql/FileBasedDataSourceSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/
    
    ```
    Caused by: sbt.ForkMain$ForkError: java.lang.Throwable: null
    	at org.apache.spark.DebugFilesystem$.addOpenStream(DebugFilesystem.scala:36)
    	at org.apache.spark.DebugFilesystem.open(DebugFilesystem.scala:70)
    	at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:769)
    	at org.apache.parquet.hadoop.ParquetFileReader.<init>(ParquetFileReader.java:538)
    	at org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase.initialize(SpecificParquetRecordReaderBase.java:149)
    	at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initialize(VectorizedParquetRecordReader.java:133)
    	at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anonfun$buildReaderWithPartitionValues$1.apply(ParquetFileFormat.scala:400)
    	at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anonfun$buildReaderWithPartitionValues$1.apply(ParquetFileFormat.scala:356)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:125)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:179)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:106)
    ```
    
    ## How was this patch tested?
    
    N/A

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

    $ git pull https://github.com/dongjoon-hyun/spark SPARK-23390

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

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

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

    This closes #20619
    
----
commit 43f809fd2ff619c901e05bc062ab70aa65371a46
Author: Dongjoon Hyun <do...@...>
Date:   2018-02-15T16:55:43Z

    [SPARK-23390][SQL] Register task completion listerners first in ParquetFileFormat

----


---

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


[GitHub] spark pull request #20619: [SPARK-23457][SQL] Register task completion liste...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark pull request #20619: [SPARK-23390][SQL] Register task completion liste...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20619#discussion_r168556797
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -414,16 +417,16 @@ class ParquetFileFormat
             } else {
               new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz))
             }
    +        val recordReaderIterator = new RecordReaderIterator(reader)
    +        // Register a task completion lister before `initalization`.
    +        taskContext.foreach(_.addTaskCompletionListener(_ => recordReaderIterator.close()))
             reader.initialize(split, hadoopAttemptContext)
    -        reader
    +        recordReaderIterator
           }
     
    -      val iter = new RecordReaderIterator(parquetReader)
    -      taskContext.foreach(_.addTaskCompletionListener(_ => iter.close()))
     
           // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy.
    -      if (parquetReader.isInstanceOf[VectorizedParquetRecordReader] &&
    -          enableVectorizedReader) {
    +      if (enableVectorizedReader) {
    --- End diff --
    
    Yep. It looks possible. I'll update together after getting more reviews. Thanks, @kiszk .


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Thank you for last-minute review before vacation. :)


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark pull request #20619: [SPARK-23390][SQL] Register task completion liste...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20619#discussion_r168552292
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -414,16 +417,16 @@ class ParquetFileFormat
             } else {
               new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz))
             }
    +        val recordReaderIterator = new RecordReaderIterator(reader)
    +        // Register a task completion lister before `initalization`.
    +        taskContext.foreach(_.addTaskCompletionListener(_ => recordReaderIterator.close()))
             reader.initialize(split, hadoopAttemptContext)
    -        reader
    +        recordReaderIterator
           }
     
    -      val iter = new RecordReaderIterator(parquetReader)
    -      taskContext.foreach(_.addTaskCompletionListener(_ => iter.close()))
     
           // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy.
    -      if (parquetReader.isInstanceOf[VectorizedParquetRecordReader] &&
    -          enableVectorizedReader) {
    +      if (enableVectorizedReader) {
    --- End diff --
    
    Would it be possible to merge this if-statement into the above if-statement?


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    LGTM


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Retest this please.


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Thank you, @kiszk . I added SPARK-23390 in the PR description.
    > Would it be worth to add this JIRA number in a comment as we did for ORC?


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Thank you all!


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/946/
    Test PASSed.


---

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


[GitHub] spark pull request #20619: [SPARK-23457][SQL] Register task completion liste...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20619#discussion_r168929352
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -395,16 +395,21 @@ class ParquetFileFormat
             ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
           }
           val taskContext = Option(TaskContext.get())
    -      val parquetReader = if (enableVectorizedReader) {
    +      if (enableVectorizedReader) {
             val vectorizedReader = new VectorizedParquetRecordReader(
               convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity)
    +        val iter = new RecordReaderIterator(vectorizedReader)
    +        // SPARK-23457 Register a task completion lister before `initialization`.
    --- End diff --
    
    Now, `SPARK-23457` is added.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listerners f...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Hi, @cloud-fan and @gatorsmile .
    This is the same kind of PR about opened file leakage for ParquetFileFormat. Could you review this?


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Umm, we still see the following exception in [the log](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87523/testReport/) ...
    
    ```
    Caused by: sbt.ForkMain$ForkError: java.lang.Throwable: null
    	at org.apache.spark.DebugFilesystem$.addOpenStream(DebugFilesystem.scala:36)
    	at org.apache.spark.DebugFilesystem.open(DebugFilesystem.scala:70)
    	at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:766)
    	at org.apache.orc.impl.RecordReaderUtils$DefaultDataReader.open(RecordReaderUtils.java:173)
    	at org.apache.orc.impl.RecordReaderImpl.<init>(RecordReaderImpl.java:254)
    	at org.apache.orc.impl.ReaderImpl.rows(ReaderImpl.java:633)
    	at org.apache.spark.sql.execution.datasources.orc.OrcColumnarBatchReader.initialize(OrcColumnarBatchReader.java:140)
    	at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anonfun$buildReaderWithPartitionValues$2.apply(OrcFileFormat.scala:197)
    	at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anonfun$buildReaderWithPartitionValues$2.apply(OrcFileFormat.scala:161)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:125)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:179)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:106)
    	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.scan_nextBatch$(Unknown Source)
    	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
    	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
    	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$10$$anon$1.hasNext(WholeStageCodegenExec.scala:614)
    	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
    	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
    	at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1834)
    	at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1162)
    	at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1162)
    	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2063)
    	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2063)
    	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
    	at org.apache.spark.scheduler.Task.run(Task.scala:109)
    	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)
    	... 3 more
    ```


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listerners f...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/954/
    Test PASSed.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    **[Test build #87537 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87537/testReport)** for PR 20619 at commit [`8bd02d8`](https://github.com/apache/spark/commit/8bd02d8692708ab58e31e19a3682af3a94550369).


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Hi, @cloud-fan .
    Since 2.3 is announced, can we have this in `branch-2.3` for Apache Spark 2.3.1?


---

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


[GitHub] spark pull request #20619: [SPARK-23390][SQL] Register task completion liste...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20619#discussion_r168711619
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -395,16 +395,19 @@ class ParquetFileFormat
             ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
           }
           val taskContext = Option(TaskContext.get())
    -      val parquetReader = if (enableVectorizedReader) {
    +      val iter = if (enableVectorizedReader) {
             val vectorizedReader = new VectorizedParquetRecordReader(
               convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity)
    +        val recordReaderIterator = new RecordReaderIterator(vectorizedReader)
    +        // Register a task completion lister before `initalization`.
    --- End diff --
    
    Those constructors didn't look heavy to me.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listerners f...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listerners f...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    **[Test build #87533 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87533/testReport)** for PR 20619 at commit [`8bd02d8`](https://github.com/apache/spark/commit/8bd02d8692708ab58e31e19a3682af3a94550369).


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    thanks, merging to master!


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/955/
    Test PASSed.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listerners f...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/952/
    Test PASSed.


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    **[Test build #87534 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87534/testReport)** for PR 20619 at commit [`8bd02d8`](https://github.com/apache/spark/commit/8bd02d8692708ab58e31e19a3682af3a94550369).


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Thank you for review, @mgaido91 .


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/953/
    Test PASSed.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    For the following, I'll.
    > Would it be worth to add this JIRA number in a comment as we did for ORC?


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    The final failure is irrelevant to this.
    ```
    org.apache.spark.sql.sources.CreateTableAsSelectSuite.(It is not a test it is a sbt.testing.SuiteSelector)
    ```


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listerners f...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Yep. I'll try for this, too. @cloud-fan .


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/949/
    Test PASSed.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/943/
    Test PASSed.


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Thank you, @cloud-fan !


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/950/
    Test PASSed.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Retest this please.


---

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


[GitHub] spark pull request #20619: [SPARK-23390][SQL] Register task completion liste...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20619#discussion_r168714722
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -395,16 +395,19 @@ class ParquetFileFormat
             ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
           }
           val taskContext = Option(TaskContext.get())
    -      val parquetReader = if (enableVectorizedReader) {
    +      val iter = if (enableVectorizedReader) {
             val vectorizedReader = new VectorizedParquetRecordReader(
               convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity)
    +        val recordReaderIterator = new RecordReaderIterator(vectorizedReader)
    +        // Register a task completion lister before `initalization`.
    --- End diff --
    
    ok


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Oh..


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Yea, please go ahead.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/942/
    Test PASSed.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    **[Test build #87535 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87535/testReport)** for PR 20619 at commit [`8bd02d8`](https://github.com/apache/spark/commit/8bd02d8692708ab58e31e19a3682af3a94550369).


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Yep. @kiszk . @mgaido91 also reports that, so I'm investigating that more.
    
    However, that doesn't mean this approach is not proper. You can see the manual test case example in previous ORC-related PR and this PR. This approach definitely reduces the number of point of failures. 
    
    For the remaining issue, I think we may need a different approach in a different code path.


---

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


[GitHub] spark pull request #20619: [SPARK-23390][SQL] Register task completion liste...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20619#discussion_r168557092
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -414,16 +417,16 @@ class ParquetFileFormat
             } else {
               new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz))
             }
    +        val recordReaderIterator = new RecordReaderIterator(reader)
    +        // Register a task completion lister before `initalization`.
    +        taskContext.foreach(_.addTaskCompletionListener(_ => recordReaderIterator.close()))
             reader.initialize(split, hadoopAttemptContext)
    -        reader
    +        recordReaderIterator
           }
     
    -      val iter = new RecordReaderIterator(parquetReader)
    -      taskContext.foreach(_.addTaskCompletionListener(_ => iter.close()))
    --- End diff --
    
    According to the reported leakage, this is too late.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/944/
    Test PASSed.


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    **[Test build #87528 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87528/testReport)** for PR 20619 at commit [`8bd02d8`](https://github.com/apache/spark/commit/8bd02d8692708ab58e31e19a3682af3a94550369).


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listerners f...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    It looks good to me that we move the registrations to the new (earlier) places.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listerners f...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Would it be worth to add this JIRA number in a comment as we did for ORC?


---

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


[GitHub] spark pull request #20619: [SPARK-23390][SQL] Register task completion liste...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20619#discussion_r168709918
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -395,16 +395,19 @@ class ParquetFileFormat
             ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
           }
           val taskContext = Option(TaskContext.get())
    -      val parquetReader = if (enableVectorizedReader) {
    +      val iter = if (enableVectorizedReader) {
             val vectorizedReader = new VectorizedParquetRecordReader(
               convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity)
    +        val recordReaderIterator = new RecordReaderIterator(vectorizedReader)
    +        // Register a task completion lister before `initalization`.
    --- End diff --
    
    could `new VectorizedParquetRecordReader` or `new RecordReaderIterator` fail?


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Thank you for retriggering, @gatorsmile .


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Retest this please.


---

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


[GitHub] spark pull request #20619: [SPARK-23390][SQL] Register task completion liste...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20619#discussion_r168910218
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -414,16 +417,16 @@ class ParquetFileFormat
             } else {
               new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz))
             }
    +        val recordReaderIterator = new RecordReaderIterator(reader)
    +        // Register a task completion lister before `initalization`.
    +        taskContext.foreach(_.addTaskCompletionListener(_ => recordReaderIterator.close()))
             reader.initialize(split, hadoopAttemptContext)
    -        reader
    +        recordReaderIterator
           }
     
    -      val iter = new RecordReaderIterator(parquetReader)
    -      taskContext.foreach(_.addTaskCompletionListener(_ => iter.close()))
     
           // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy.
    -      if (parquetReader.isInstanceOf[VectorizedParquetRecordReader] &&
    -          enableVectorizedReader) {
    +      if (enableVectorizedReader) {
    --- End diff --
    
    yea it seems more reasonable to merge this if-else now.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listerners f...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    can we provide a manual test like the OOM one in your ORC PR?


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by mgaido91 <gi...@git.apache.org>.
Github user mgaido91 commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    LGTM


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Oh, @kiszk .  The following meat really `comment` in the code. Sorry, I misunderstood.
    > Would it be worth to add this JIRA number in a comment as we did for ORC?


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    He is already on vacation. : ) 


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    The reproducible test case is added into PR description and the code is updated according to @kiszk and @cloud-fan 's comments.


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    LGTM with one minor comment


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

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


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listeners fi...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    The failure is irrelevant to this PR.
    ```
    org.apache.spark.sql.hive.client.HiveClientSuites.(It is not a test it is a sbt.testing.NestedSuiteSelector)
    ```


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listerners f...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    **[Test build #87482 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87482/testReport)** for PR 20619 at commit [`43f809f`](https://github.com/apache/spark/commit/43f809fd2ff619c901e05bc062ab70aa65371a46).


---

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


[GitHub] spark issue #20619: [SPARK-23390][SQL] Register task completion listerners f...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/919/
    Test PASSed.


---

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


[GitHub] spark issue #20619: [SPARK-23457][SQL] Register task completion listeners fi...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20619
  
    **[Test build #87534 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/87534/testReport)** for PR 20619 at commit [`8bd02d8`](https://github.com/apache/spark/commit/8bd02d8692708ab58e31e19a3682af3a94550369).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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