This release uses Parquet 1.15.1. It seems Parquet 1.15.1 may cause deadlock.
Found one Java-level deadlock: ============================= "Executor 566 task launch worker for task 202024534, task 19644.1 in stage 13967543.0 of app application_1736396393732_100191": waiting to lock monitor 0x00007f9435525aa0 (object 0x00007f9575000a70, a java.lang.Object), which is held by "Task reaper-9" "Task reaper-9": waiting to lock monitor 0x00007fa06b315500 (object 0x00007f963d0af788, a org.apache.hadoop.hdfs.DFSOutputStream), which is held by "Executor 566 task launch worker for task 202024534, task 19644.1 in stage 13967543.0 of app application_1736396393732_100191" Java stack information for the threads listed above: =================================================== "Executor 566 task launch worker for task 202024534, task 19644.1 in stage 13967543.0 of app application_1736396393732_100191": at org.apache.spark.util.UninterruptibleThread.interrupt(UninterruptibleThread.scala:96) - waiting to lock <0x00007f9575000a70> (a java.lang.Object) at org.apache.hadoop.hdfs.DataStreamer.waitAndQueuePacket(DataStreamer.java:989) - locked <0x00007f963d0af760> (a java.util.LinkedList) at org.apache.hadoop.hdfs.DFSOutputStream.enqueueCurrentPacket(DFSOutputStream.java:496) at org.apache.hadoop.hdfs.DFSOutputStream.enqueueCurrentPacketFull(DFSOutputStream.java:505) - locked <0x00007f963d0af788> (a org.apache.hadoop.hdfs.DFSOutputStream) at org.apache.hadoop.hdfs.DFSOutputStream.writeChunk(DFSOutputStream.java:445) - locked <0x00007f963d0af788> (a org.apache.hadoop.hdfs.DFSOutputStream) at org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunks(FSOutputSummer.java:218) at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:165) - eliminated <0x00007f963d0af788> (a org.apache.hadoop.hdfs.DFSOutputStream) at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:146) - locked <0x00007f963d0af788> (a org.apache.hadoop.hdfs.DFSOutputStream) at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:137) at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:112) - locked <0x00007f963d0af788> (a org.apache.hadoop.hdfs.DFSOutputStream) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:62) at java.io.DataOutputStream.write(java.base@17.0.6 /DataOutputStream.java:112) - locked <0x00007f963d0b0a70> (a org.apache.hadoop.hdfs.client.HdfsDataOutputStream) at org.apache.parquet.hadoop.util.HadoopPositionOutputStream.write(HadoopPositionOutputStream.java:50) at java.nio.channels.Channels$WritableByteChannelImpl.write(java.base@17.0.6 /Channels.java:463) - locked <0x00007f965dfad498> (a java.lang.Object) at org.apache.parquet.bytes.ConcatenatingByteBufferCollector.writeAllTo(ConcatenatingByteBufferCollector.java:77) at org.apache.parquet.hadoop.ParquetFileWriter.writeColumnChunk(ParquetFileWriter.java:1341) at org.apache.parquet.hadoop.ParquetFileWriter.writeColumnChunk(ParquetFileWriter.java:1262) at org.apache.parquet.hadoop.ColumnChunkPageWriteStore$ColumnChunkPageWriter.writeToFileWriter(ColumnChunkPageWriteStore.java:408) at org.apache.parquet.hadoop.ColumnChunkPageWriteStore.flushToFileWriter(ColumnChunkPageWriteStore.java:675) at org.apache.parquet.hadoop.InternalParquetRecordWriter.flushRowGroupToStore(InternalParquetRecordWriter.java:210) at org.apache.parquet.hadoop.InternalParquetRecordWriter.checkBlockSizeReached(InternalParquetRecordWriter.java:178) at org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:154) at org.apache.parquet.hadoop.ParquetRecordWriter.write(ParquetRecordWriter.java:240) at org.apache.parquet.hadoop.ParquetRecordWriter.write(ParquetRecordWriter.java:41) at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.write(ParquetOutputWriter.scala:39) at org.apache.spark.sql.execution.datasources.BaseDynamicPartitionDataWriter.writeRecord(FileFormatDataWriter.scala:357) at org.apache.spark.sql.execution.datasources.DynamicPartitionDataSingleWriter.write(FileFormatDataWriter.scala:403) at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.writeWithMetrics(FileFormatDataWriter.scala:86) at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.writeWithIterator(FileFormatDataWriter.scala:93) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$executeTask$2(FileFormatWriter.scala:501) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$$Lambda$5526/0x00000008033c9870.apply(Unknown Source) at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1412) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:508) at org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$3(WriteFiles.scala:126) at org.apache.spark.sql.execution.datasources.WriteFilesExec$$Lambda$5678/0x00000008034b9b10.apply(Unknown Source) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:931) at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:931) at org.apache.spark.rdd.RDD$$Lambda$1530/0x00000008016c6000.apply(Unknown Source) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:405) at org.apache.spark.rdd.RDD.iterator(RDD.scala:329) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93) at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161) at org.apache.spark.scheduler.Task.run(Task.scala:154) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:622) at org.apache.spark.executor.Executor$TaskRunner$$Lambda$965/0x0000000801376d80.apply(Unknown Source) at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64) at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:94) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:625) at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@17.0.6 /ThreadPoolExecutor.java:1136) at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@17.0.6 /ThreadPoolExecutor.java:635) at java.lang.Thread.run(java.base@17.0.6/Thread.java:833) "Task reaper-9": at org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:636) - waiting to lock <0x00007f963d0af788> (a org.apache.hadoop.hdfs.DFSOutputStream) at org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:585) at org.apache.hadoop.fs.FSDataOutputStream.hflush(FSDataOutputStream.java:136) at org.apache.parquet.hadoop.util.HadoopPositionOutputStream.close(HadoopPositionOutputStream.java:65) at java.nio.channels.Channels$WritableByteChannelImpl.implCloseChannel(java.base@17.0.6 /Channels.java:475) at java.nio.channels.spi.AbstractInterruptibleChannel$1.interrupt(java.base@17.0.6 /AbstractInterruptibleChannel.java:162) - locked <0x00007f965dfb1340> (a java.lang.Object) at java.lang.Thread.interrupt(java.base@17.0.6/Thread.java:997) - locked <0x00007f9575003ab0> (a java.lang.Object) at org.apache.spark.util.UninterruptibleThread.interrupt(UninterruptibleThread.scala:99) - locked <0x00007f9575000a70> (a java.lang.Object) at org.apache.spark.scheduler.Task.kill(Task.scala:263) at org.apache.spark.executor.Executor$TaskRunner.kill(Executor.scala:495) - locked <0x00007f963d1364e0> (a org.apache.spark.executor.Executor$TaskRunner) at org.apache.spark.executor.Executor$TaskReaper.run(Executor.scala:1001) at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@17.0.6 /ThreadPoolExecutor.java:1136) at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@17.0.6 /ThreadPoolExecutor.java:635) at java.lang.Thread.run(java.base@17.0.6/Thread.java:833) Found 1 deadlock. On Mon, Apr 14, 2025 at 11:13 AM Hyukjin Kwon <gurwls...@apache.org> wrote: > Made a fix at https://github.com/apache/spark/pull/50575 👍 > > On Mon, 14 Apr 2025 at 11:42, Wenchen Fan <cloud0...@gmail.com> wrote: > >> I'm testing the new spark-connect distribution and here is the result: >> >> 4 packages are tested: pip install pyspark, pip install pyspark_connect (I >> installed them with the RC4 pyspark tarballs), the classic tarball >> (spark-4.0.0-bin-hadoop3.tgz), the connect tarball >> (spark-4.0.0-bin-hadoop3-spark-connect.tgz). >> >> *case 1*: run command pyspark and spark-shell, with and without --master >> local, it should use the default mode (classic or connect depending on >> the distribution package) >> Everything works as expected. >> >> *case 2*: run command pyspark and spark-shell with --remote local, it >> should use the connect mode >> Everything works as expected. >> >> *case 3*: run command pyspark and spark-shell with --master local --conf >> spark.api.mode=classic, >> it should use the classic mode >> The connect packages fail with TypeError: 'JavaPackage' object is not >> callable when running the pyspark command. >> >> @Hyukjin Kwon <gurwls...@gmail.com> is looking into it now and will >> share the findings later. >> >> Please let me know if you find any other issues with RC4, either >> functionality issues with Spark itself, or integration issues with >> downstream libraries. >> >> Thanks! >> Wenchen >> >> On Thu, Apr 10, 2025 at 11:21 PM Wenchen Fan <cloud0...@gmail.com> wrote: >> >>> Please vote on releasing the following candidate as Apache Spark version >>> 4.0.0. >>> >>> The vote is open until April 15 (PST) and passes if a majority +1 PMC >>> votes are cast, with a minimum of 3 +1 votes. >>> >>> [ ] +1 Release this package as Apache Spark 4.0.0 >>> [ ] -1 Do not release this package because ... >>> >>> To learn more about Apache Spark, please see https://spark.apache.org/ >>> >>> The tag to be voted on is v4.0.0-rc4 (commit >>> e0801d9d8e33cd8835f3e3beed99a3588c16b776) >>> https://github.com/apache/spark/tree/v4.0.0-rc4 >>> >>> The release files, including signatures, digests, etc. can be found at: >>> https://dist.apache.org/repos/dist/dev/spark/v4.0.0-rc4-bin/ >>> >>> Signatures used for Spark RCs can be found in this file: >>> https://dist.apache.org/repos/dist/dev/spark/KEYS >>> >>> The staging repository for this release can be found at: >>> https://repository.apache.org/content/repositories/orgapachespark-1480/ >>> >>> The documentation corresponding to this release can be found at: >>> https://dist.apache.org/repos/dist/dev/spark/v4.0.0-rc4-docs/ >>> >>> The list of bug fixes going into 4.0.0 can be found at the following URL: >>> https://issues.apache.org/jira/projects/SPARK/versions/12353359 >>> >>> This release is using the release script of the tag v4.0.0-rc4. >>> >>> FAQ >>> >>> ========================= >>> How can I help test this release? >>> ========================= >>> >>> If you are a Spark user, you can help us test this release by taking >>> an existing Spark workload and running on this release candidate, then >>> reporting any regressions. >>> >>> If you're working in PySpark you can set up a virtual env and install >>> the current RC and see if anything important breaks, in the Java/Scala >>> you can add the staging repository to your projects resolvers and test >>> with the RC (make sure to clean up the artifact cache before/after so >>> you don't end up building with a out of date RC going forward). >>> >>