I have reported this issue to the Parquet community:
https://github.com/apache/parquet-java/issues/3193

On Tue, Apr 15, 2025 at 9:47 AM Wenchen Fan <cloud0...@gmail.com> wrote:

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

Reply via email to