Hi Yuming, 1.51.1 is the latest release of Apache Parquet for the 1.x line. Is it a known issue the Parquet community is working on, or are you still investigating it? If the issue is confirmed by the Parquet community, we can probably roll back to the previous Parquet version for Spark 4.0.
Thanks, Wenchen On Tue, Apr 15, 2025 at 7:24 AM Yuming Wang <yumw...@apache.org> wrote: > 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). >>>> >>>