[ 
https://issues.apache.org/jira/browse/SPARK-44843?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kent Yao reassigned SPARK-44843:
--------------------------------

    Assignee: Yang Jie

> flaky test: RocksDBStateStoreStreamingAggregationSuite
> ------------------------------------------------------
>
>                 Key: SPARK-44843
>                 URL: https://issues.apache.org/jira/browse/SPARK-44843
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL, Tests
>    Affects Versions: 4.0.0
>            Reporter: Yang Jie
>            Assignee: Yang Jie
>            Priority: Major
>              Labels: pull-request-available
>
> I've seen this more than once, let's record it for now.
> [https://github.com/apache/spark/actions/runs/5875252243/job/15931264374]
> {code:java}
> 2023-08-16T06:49:14.0550627Z [info] - 
> SPARK-35896: metrics in StateOperatorProgress are output correctly 
> (RocksDBStateStore) *** FAILED *** (1 minute, 1 second)
> 2023-08-16T06:49:14.0560354Z [info]   Timed out 
> waiting for stream: The code passed to failAfter did not complete within 60 
> seconds.
> 2023-08-16T06:49:14.0568703Z [info]   
> java.lang.Thread.getStackTrace(Thread.java:1564)
> 2023-08-16T06:49:14.0578526Z [info]      
> org.scalatest.concurrent.TimeLimits$.failAfterImpl(TimeLimits.scala:277)
> 2023-08-16T06:49:14.0600495Z [info]      
> org.scalatest.concurrent.TimeLimits.failAfter(TimeLimits.scala:231)
> 2023-08-16T06:49:14.0609443Z [info]      
> org.scalatest.concurrent.TimeLimits.failAfter$(TimeLimits.scala:230)
> 2023-08-16T06:49:14.0630028Z [info]      
> org.apache.spark.SparkFunSuite.failAfter(SparkFunSuite.scala:69)
> 2023-08-16T06:49:14.0638142Z [info]      
> org.apache.spark.sql.streaming.StreamTest.$anonfun$testStream$7(StreamTest.scala:481)
> 2023-08-16T06:49:14.0704798Z [info]      
> org.apache.spark.sql.streaming.StreamTest.$anonfun$testStream$7$adapted(StreamTest.scala:480)
> 2023-08-16T06:49:14.0732716Z [info]      
> scala.collection.mutable.HashMap.$anonfun$foreach$1(HashMap.scala:149)
> 2023-08-16T06:49:14.0743783Z [info]      
> scala.collection.mutable.HashTable.foreachEntry(HashTable.scala:237)
> 2023-08-16T06:49:14.0753421Z [info]      
> scala.collection.mutable.HashTable.foreachEntry$(HashTable.scala:230)
> 2023-08-16T06:49:14.0765553Z [info]   
> 2023-08-16T06:49:14.0773522Z [info]      Caused 
> by:      null
> 2023-08-16T06:49:14.0787123Z [info]      
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2014)
> 2023-08-16T06:49:14.0796604Z [info]              
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2173)
> 2023-08-16T06:49:14.0808419Z [info]              
> org.apache.spark.sql.execution.streaming.StreamExecution.awaitOffset(StreamExecution.scala:481)
> 2023-08-16T06:49:14.0817018Z [info]              
> org.apache.spark.sql.streaming.StreamTest.$anonfun$testStream$8(StreamTest.scala:482)
> 2023-08-16T06:49:14.0824218Z [info]              
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> 2023-08-16T06:49:14.0831608Z [info]              
> org.scalatest.enablers.Timed$$anon$1.timeoutAfter(Timed.scala:127)
> 2023-08-16T06:49:14.0838059Z [info]              
> org.scalatest.concurrent.TimeLimits$.failAfterImpl(TimeLimits.scala:282)
> 2023-08-16T06:49:14.0847335Z [info]              
> org.scalatest.concurrent.TimeLimits.failAfter(TimeLimits.scala:231)
> 2023-08-16T06:49:14.0854180Z [info]              
> org.scalatest.concurrent.TimeLimits.failAfter$(TimeLimits.scala:230)
> 2023-08-16T06:49:14.0861298Z [info]              
> org.apache.spark.SparkFunSuite.failAfter(SparkFunSuite.scala:69)
> 2023-08-16T06:49:14.0866845Z [info]   
> 2023-08-16T06:49:14.0872599Z [info]   
> 2023-08-16T06:49:14.0880688Z [info]   == 
> Progress ==
> 2023-08-16T06:49:14.0887109Z [info]      
> StartStream(ProcessingTimeTrigger(0),org.apache.spark.util.SystemClock@432e877b,Map(spark.sql.shuffle.partitions
>  -> 3),null)
> 2023-08-16T06:49:14.0901797Z [info]      AddData 
> to MemoryStream[value#19338]: 3,2,1,3
> 2023-08-16T06:49:14.0913348Z [info]   => 
> CheckLastBatch: [3,2],[2,1],[1,1]
> 2023-08-16T06:49:14.0919235Z [info]      
> AssertOnQuery(<condition>, Check total state rows = List(3), updated state 
> rows = List(3), rows dropped by watermark = List(0), removed state rows = 
> Some(List(0)))
> 2023-08-16T06:49:14.0925028Z [info]      AddData 
> to MemoryStream[value#19338]: 1,4
> 2023-08-16T06:49:14.0931608Z [info]      
> CheckLastBatch: [3,2],[2,1],[1,2],[4,1]
> 2023-08-16T06:49:14.0938968Z [info]      
> AssertOnQuery(<condition>, Check operator progress metrics: operatorName = 
> stateStoreSave, numShufflePartitions = 3, numStateStoreInstances = 3)
> 2023-08-16T06:49:14.0944383Z [info]   
> 2023-08-16T06:49:14.0985018Z [info]   == Stream 
> ==
> 2023-08-16T06:49:14.1010235Z [info]   Output 
> Mode: Complete
> 2023-08-16T06:49:14.1015512Z [info]   Stream 
> state: {}
> 2023-08-16T06:49:14.1020511Z [info]   Thread 
> state: alive
> 2023-08-16T06:49:14.1026531Z [info]   Thread 
> stack trace: sun.misc.Unsafe.park(Native Method)
> 2023-08-16T06:49:14.1032459Z [info]   
> java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> 2023-08-16T06:49:14.1038406Z [info]   
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
> 2023-08-16T06:49:14.1044652Z [info]   
> java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
> 2023-08-16T06:49:14.1052001Z [info]   
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
> 2023-08-16T06:49:14.1058244Z [info]   
> scala.concurrent.impl.Promise$DefaultPromise.tryAwait(Promise.scala:242)
> 2023-08-16T06:49:14.1063878Z [info]   
> scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:258)
> 2023-08-16T06:49:14.1069740Z [info]   
> scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:187)
> 2023-08-16T06:49:14.1081581Z [info]   
> org.apache.spark.util.ThreadUtils$.awaitReady(ThreadUtils.scala:342)
> 2023-08-16T06:49:14.1089123Z [info]   
> org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:974)
> 2023-08-16T06:49:14.1096254Z [info]   
> org.apache.spark.SparkContext.runJob(SparkContext.scala:2405)
> 2023-08-16T06:49:14.1104050Z [info]   
> org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2(WriteToDataSourceV2Exec.scala:385)
> 2023-08-16T06:49:14.1112551Z [info]   
> org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2$(WriteToDataSourceV2Exec.scala:359)
> 2023-08-16T06:49:14.1120764Z [info]   
> org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec.writeWithV2(WriteToDataSourceV2Exec.scala:307)
> 2023-08-16T06:49:14.1141786Z [info]   
> org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec.run(WriteToDataSourceV2Exec.scala:318)
> 2023-08-16T06:49:14.1149758Z [info]   
> org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
> 2023-08-16T06:49:14.1159611Z [info]   
> org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
> 2023-08-16T06:49:14.1166283Z [info]   
> org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
> 2023-08-16T06:49:14.1172538Z [info]   
> org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:4344)
> 2023-08-16T06:49:14.1208809Z [info]   
> org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:3585)
> 2023-08-16T06:49:14.1215346Z [info]   
> org.apache.spark.sql.Dataset$$Lambda$2415/1714895648.apply(Unknown 
> Source)
> 2023-08-16T06:49:14.1228588Z [info]   
> org.apache.spark.sql.Dataset.$anonfun$withAction$2(Dataset.scala:4334)
> 2023-08-16T06:49:14.1234684Z [info]   
> org.apache.spark.sql.Dataset$$Lambda$2442/14653745.apply(Unknown 
> Source)
> 2023-08-16T06:49:14.1246038Z [info]   
> org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:546)
> 2023-08-16T06:49:14.1246768Z [info]   
> org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:4332)
> 2023-08-16T06:49:14.1273061Z [info]   
> org.apache.spark.sql.Dataset$$Lambda$2416/619810288.apply(Unknown 
> Source)
> 2023-08-16T06:49:14.1273847Z [info]   
> org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:132)
> 2023-08-16T06:49:14.1274620Z [info]   
> org.apache.spark.sql.execution.SQLExecution$$$Lambda$2366/419259595.apply(Unknown
>  Source)
> 2023-08-16T06:49:14.1291096Z [info]   
> org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:209)
> 2023-08-16T06:49:14.1291944Z [info]   
> org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:115)
> 2023-08-16T06:49:14.1292713Z [info]   
> org.apache.spark.sql.execution.SQLExecution$$$Lambda$2353/1369125389.apply(Unknown
>  Source)
> 2023-08-16T06:49:14.1293407Z [info]   
> org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
> 2023-08-16T06:49:14.1294142Z [info]   
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:68)
> 2023-08-16T06:49:14.1305624Z [info]   
> org.apache.spark.sql.Dataset.withAction(Dataset.scala:4332)
> 2023-08-16T06:49:14.1306571Z [info]   
> org.apache.spark.sql.Dataset.collect(Dataset.scala:3585)
> 2023-08-16T06:49:14.1307380Z [info]   
> org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$runBatch$17(MicroBatchExecution.scala:741)
> 2023-08-16T06:49:14.1308412Z [info]   
> org.apache.spark.sql.execution.streaming.MicroBatchExecution$$Lambda$2352/335540119.apply(Unknown
>  Source)
> 2023-08-16T06:49:14.1318048Z [info]   
> org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:132)
> 2023-08-16T06:49:14.1324073Z [info]   
> org.apache.spark.sql.execution.SQLExecution$$$Lambda$2366/419259595.apply(Unknown
>  Source)
> 2023-08-16T06:49:14.1326362Z [info]   
> org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:209)
> 2023-08-16T06:49:14.1330068Z [info]   
> org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:115)
> 2023-08-16T06:49:14.1330922Z [info]   
> org.apache.spark.sql.execution.SQLExecution$$$Lambda$2353/1369125389.apply(Unknown
>  Source)
> 2023-08-16T06:49:14.1418063Z [info]   
> org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
> 2023-08-16T06:49:14.1418867Z [info]   
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:68)
> 2023-08-16T06:49:14.1419759Z [info]   
> org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$runBatch$16(MicroBatchExecution.scala:729)
> 2023-08-16T06:49:14.1420680Z [info]   
> org.apache.spark.sql.execution.streaming.MicroBatchExecution$$Lambda$2351/1752953868.apply(Unknown
>  Source)
> 2023-08-16T06:49:14.1421623Z [info]   
> org.apache.spark.sql.execution.streaming.ProgressReporter.reportTimeTaken(ProgressReporter.scala:405)
> 2023-08-16T06:49:14.1422639Z [info]   
> org.apache.spark.sql.execution.streaming.ProgressReporter.reportTimeTaken$(ProgressReporter.scala:403)
> 2023-08-16T06:49:14.1423746Z [info]   
> org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:67)
> 2023-08-16T06:49:14.1424717Z [info]   
> org.apache.spark.sql.execution.streaming.MicroBatchExecution.runBatch(MicroBatchExecution.scala:729)
> 2023-08-16T06:49:14.1425711Z [info]   
> org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$runActivatedStream$2(MicroBatchExecution.scala:286)
> 2023-08-16T06:49:14.1426644Z [info]   
> org.apache.spark.sql.execution.streaming.MicroBatchExecution$$Lambda$2053/1800928896.apply$mcV$sp(Unknown
>  Source)
> 2023-08-16T06:49:14.1427389Z [info]   
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> 2023-08-16T06:49:14.1428211Z [info]   
> org.apache.spark.sql.execution.streaming.ProgressReporter.reportTimeTaken(ProgressReporter.scala:405)
> 2023-08-16T06:49:14.1678185Z 06:49:14.167 WARN 
> org.apache.spark.sql.execution.streaming.ResolveWriteToStream: 
> spark.sql.adaptive.enabled is not supported in streaming DataFrames/Datasets 
> and will be disabled.
> 2023-08-16T06:49:14.1749754Z [info]   
> org.apache.spark.sql.execution.streaming.ProgressReporter.reportTimeTaken$(ProgressReporter.scala:403)
> 2023-08-16T06:49:14.1760827Z [info]   
> org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:67)
> 2023-08-16T06:49:14.1767884Z [info]   
> org.apache.spark.sql.execution.streaming.MicroBatchExecution.$anonfun$runActivatedStream$1(MicroBatchExecution.scala:249)
> 2023-08-16T06:49:14.1772443Z [info]   
> org.apache.spark.sql.execution.streaming.MicroBatchExecution$$Lambda$2050/1379075459.apply$mcZ$sp(Unknown
>  Source)
> 2023-08-16T06:49:14.1775920Z [info]   
> org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:67)
> 2023-08-16T06:49:14.1779547Z [info]   
> org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecution.scala:239)
> 2023-08-16T06:49:14.1780654Z [info]   
> org.apache.spark.sql.execution.streaming.StreamExecution.$anonfun$runStream$1(StreamExecution.scala:311)
> 2023-08-16T06:49:14.1781762Z [info]   
> org.apache.spark.sql.execution.streaming.StreamExecution$$Lambda$2042/403495914.apply$mcV$sp(Unknown
>  Source)
> 2023-08-16T06:49:14.1787505Z [info]   
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> 2023-08-16T06:49:14.1789663Z [info]   
> org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
> 2023-08-16T06:49:14.1824717Z [info]   
> org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:289)
> 2023-08-16T06:49:14.1825945Z [info]   
> org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.$anonfun$run$1(StreamExecution.scala:211)
> 2023-08-16T06:49:14.1826944Z [info]   
> org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1$$Lambda$2038/346934547.apply$mcV$sp(Unknown
>  Source)
> 2023-08-16T06:49:14.1827779Z [info]   
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> 2023-08-16T06:49:14.1828564Z [info]   
> org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94)
> 2023-08-16T06:49:14.1829439Z [info]   
> org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:211)
> 2023-08-16T06:49:14.1830060Z [info]   
> 2023-08-16T06:49:14.1830461Z [info]   
> 2023-08-16T06:49:14.1830895Z [info]   == Sink 
> ==
> 2023-08-16T06:49:14.1831298Z [info]   
> 2023-08-16T06:49:14.1831680Z [info]   
> 2023-08-16T06:49:14.1832072Z [info]   
> 2023-08-16T06:49:14.1832499Z [info]   == Plan 
> ==
> 2023-08-16T06:49:14.1833000Z [info]   == Parsed 
> Logical Plan ==
> 2023-08-16T06:49:14.1833702Z [info]   
> WriteToMicroBatchDataSource MemorySink, 0214edd5-ddb7-4c00-befd-a225a61a800e, 
> Complete, 0
> 2023-08-16T06:49:14.1835181Z [info]   +- 
> Aggregate [value#19338], [value#19338, count(1) AS count(1)#19343L]
> 2023-08-16T06:49:14.1836058Z [info]      +- 
> StreamingDataSourceV2Relation [value#19338], 
> org.apache.spark.sql.execution.streaming.MemoryStreamScanBuilder@65f877f4, 
> MemoryStream[value#19338], -1, 0
> 2023-08-16T06:49:14.1836684Z [info]   
> 2023-08-16T06:49:14.1837076Z [info]   == 
> Analyzed Logical Plan ==
> 2023-08-16T06:49:14.1837697Z [info]   
> WriteToMicroBatchDataSource MemorySink, 0214edd5-ddb7-4c00-befd-a225a61a800e, 
> Complete, 0
> 2023-08-16T06:49:14.1838328Z [info]   +- 
> Aggregate [value#19338], [value#19338, count(1) AS count(1)#19343L]
> 2023-08-16T06:49:14.1839185Z [info]      +- 
> StreamingDataSourceV2Relation [value#19338], 
> org.apache.spark.sql.execution.streaming.MemoryStreamScanBuilder@65f877f4, 
> MemoryStream[value#19338], -1, 0
> 2023-08-16T06:49:14.1839983Z [info]   
> 2023-08-16T06:49:14.1840380Z [info]   == 
> Optimized Logical Plan ==
> 2023-08-16T06:49:14.1841274Z [info]   
> WriteToDataSourceV2 MicroBatchWrite[epoch: 0, writer: 
> org.apache.spark.sql.execution.streaming.sources.MemoryStreamingWrite@26d87efa]
> 2023-08-16T06:49:14.1842056Z [info]   +- 
> Aggregate [value#19338], [value#19338, count(1) AS count(1)#19343L]
> 2023-08-16T06:49:14.1842987Z [info]      +- 
> StreamingDataSourceV2Relation [value#19338], 
> org.apache.spark.sql.execution.streaming.MemoryStreamScanBuilder@65f877f4, 
> MemoryStream[value#19338], -1, 0
> 2023-08-16T06:49:14.1843620Z [info]   
> 2023-08-16T06:49:14.1843994Z [info]   == 
> Physical Plan ==
> 2023-08-16T06:49:14.1845106Z [info]   
> WriteToDataSourceV2 MicroBatchWrite[epoch: 0, writer: 
> org.apache.spark.sql.execution.streaming.sources.MemoryStreamingWrite@26d87efa],
>  
> org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2290/663905427@7505e54c
> 2023-08-16T06:49:14.1846414Z [info]   +- *(4) 
> HashAggregate(keys=[value#19338], functions=[count(1)], output=[value#19338, 
> count(1)#19343L])
> 2023-08-16T06:49:14.1847513Z [info]      +- 
> StateStoreSave [value#19338], state info [ checkpoint = 
> file:/home/runner/work/spark/spark/target/tmp/streaming.metadata-9b48d443-954d-4338-85cf-fdf0d8ccb029/state,
>  runId = 61c94e63-d753-41bf-b5b3-cdbe2d844d26, opId = 0, ver = 0, 
> numPartitions = 3], Complete, 0, 0, 2
> 2023-08-16T06:49:14.1848443Z [info]         +- 
> *(3) HashAggregate(keys=[value#19338], functions=[merge_count(1)], 
> output=[value#19338, count#19372L])
> 2023-08-16T06:49:14.1849537Z [info]            
> +- StateStoreRestore [value#19338], state info [ checkpoint = 
> file:/home/runner/work/spark/spark/target/tmp/streaming.metadata-9b48d443-954d-4338-85cf-fdf0d8ccb029/state,
>  runId = 61c94e63-d753-41bf-b5b3-cdbe2d844d26, opId = 0, ver = 0, 
> numPartitions = 3], 2
> 2023-08-16T06:49:14.1850484Z [info]              
>  +- *(2) HashAggregate(keys=[value#19338], functions=[merge_count(1)], 
> output=[value#19338, count#19372L])
> 2023-08-16T06:49:14.1851143Z [info]              
>     +- Exchange hashpartitioning(value#19338, 3), ENSURE_REQUIREMENTS, 
> [plan_id=89400]
> 2023-08-16T06:49:14.1851838Z [info]              
>        +- *(1) HashAggregate(keys=[value#19338], 
> functions=[partial_count(1)], output=[value#19338, count#19372L])
> 2023-08-16T06:49:14.1852387Z [info]              
>           +- *(1) Project [value#19338]
> 2023-08-16T06:49:14.1852988Z [info]              
>              +- MicroBatchScan[value#19338] MemoryStreamDataSource 
> (StreamTest.scala:462)
> 2023-08-16T06:49:14.1853598Z [info]   
> org.scalatest.exceptions.TestFailedException:
> 2023-08-16T06:49:14.1854372Z [info]   at 
> org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
> 2023-08-16T06:49:14.1912224Z [info]   at 
> org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
> 2023-08-16T06:49:14.1913222Z [info]   at 
> org.scalatest.funsuite.AnyFunSuite.newAssertionFailedException(AnyFunSuite.scala:1564)
> 2023-08-16T06:49:14.1914065Z [info]   at 
> org.scalatest.Assertions.fail(Assertions.scala:933)
> 2023-08-16T06:49:14.1914653Z [info]   at 
> org.scalatest.Assertions.fail$(Assertions.scala:929)
> 2023-08-16T06:49:14.1915455Z [info]   at 
> org.scalatest.funsuite.AnyFunSuite.fail(AnyFunSuite.scala:1564)
> 2023-08-16T06:49:14.1916150Z [info]   at 
> org.apache.spark.sql.streaming.StreamTest.failTest$1(StreamTest.scala:462)
> 2023-08-16T06:49:14.1917119Z [info]   at 
> org.apache.spark.sql.streaming.StreamTest.liftedTree1$1(StreamTest.scala:800)
> 2023-08-16T06:49:14.1917876Z [info]   at 
> org.apache.spark.sql.streaming.StreamTest.testStream(StreamTest.scala:776)
> 2023-08-16T06:49:14.1918617Z [info]   at 
> org.apache.spark.sql.streaming.StreamTest.testStream$(StreamTest.scala:342)
> 2023-08-16T06:49:14.1919669Z [info]   at 
> org.apache.spark.sql.streaming.StreamingAggregationSuite.testStream(StreamingAggregationSuite.scala:54)
> 2023-08-16T06:49:14.1920735Z [info]   at 
> org.apache.spark.sql.streaming.StreamingAggregationSuite.$anonfun$new$86(StreamingAggregationSuite.scala:849)
> 2023-08-16T06:49:14.1921511Z [info]   at 
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> 2023-08-16T06:49:14.1922369Z [info]   at 
> org.apache.spark.sql.streaming.RocksDBStateStoreTest.$anonfun$test$2(RocksDBStateStoreTest.scala:39)
> 2023-08-16T06:49:14.1923184Z [info]   at 
> org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf(SQLHelper.scala:54)
> 2023-08-16T06:49:14.1923962Z [info]   at 
> org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf$(SQLHelper.scala:38)
> 2023-08-16T06:49:14.1924964Z [info]   at 
> org.apache.spark.sql.streaming.StreamingAggregationSuite.org$apache$spark$sql$test$SQLTestUtilsBase$$super$withSQLConf(StreamingAggregationSuite.scala:54)
> 2023-08-16T06:49:14.1925885Z [info]   at 
> org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf(SQLTestUtils.scala:247)
> 2023-08-16T06:49:14.1926655Z [info]   at 
> org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf$(SQLTestUtils.scala:245)
> 2023-08-16T06:49:14.1927567Z [info]   at 
> org.apache.spark.sql.streaming.StreamingAggregationSuite.withSQLConf(StreamingAggregationSuite.scala:54)
> 2023-08-16T06:49:14.1928495Z [info]   at 
> org.apache.spark.sql.streaming.RocksDBStateStoreTest.$anonfun$test$1(RocksDBStateStoreTest.scala:39)
> 2023-08-16T06:49:14.1929226Z [info]   at 
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> 2023-08-16T06:49:14.1929847Z [info]   at 
> org.scalatest.enablers.Timed$$anon$1.timeoutAfter(Timed.scala:127)
> 2023-08-16T06:49:14.1930491Z [info]   at 
> org.scalatest.concurrent.TimeLimits$.failAfterImpl(TimeLimits.scala:282)
> 2023-08-16T06:49:14.1931279Z [info]   at 
> org.scalatest.concurrent.TimeLimits.failAfter(TimeLimits.scala:231)
> 2023-08-16T06:49:14.1931975Z [info]   at 
> org.scalatest.concurrent.TimeLimits.failAfter$(TimeLimits.scala:230)
> 2023-08-16T06:49:14.1932654Z [info]   at 
> org.apache.spark.SparkFunSuite.failAfter(SparkFunSuite.scala:69)
> 2023-08-16T06:49:14.1933321Z [info]   at 
> org.apache.spark.SparkFunSuite.$anonfun$test$2(SparkFunSuite.scala:155)
> 2023-08-16T06:49:14.1934040Z [info]   at 
> org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
> 2023-08-16T06:49:14.1934683Z [info]   at 
> org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
> 2023-08-16T06:49:14.1935267Z [info]   at 
> org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
> 2023-08-16T06:49:14.1935918Z [info]   at 
> org.scalatest.Transformer.apply(Transformer.scala:22)
> 2023-08-16T06:49:14.1936614Z [info]   at 
> org.scalatest.Transformer.apply(Transformer.scala:20)
> 2023-08-16T06:49:14.1937347Z [info]   at 
> org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
> 2023-08-16T06:49:14.1938043Z [info]   at 
> org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:227)
> 2023-08-16T06:49:14.1938800Z [info]   at 
> org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
> 2023-08-16T06:49:14.1939622Z [info]   at 
> org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
> 2023-08-16T06:49:14.1940287Z [info]   at 
> org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
> 2023-08-16T06:49:14.1940965Z [info]   at 
> org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
> 2023-08-16T06:49:14.1941680Z [info]   at 
> org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
> 2023-08-16T06:49:14.1942528Z [info]   at 
> org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:69)
> 2023-08-16T06:49:14.1943259Z [info]   at 
> org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
> 2023-08-16T06:49:14.1989522Z [info]   at 
> org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
> 2023-08-16T06:49:14.1995333Z [info]   at 
> org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:69)
> 2023-08-16T06:49:14.1996628Z [info]   at 
> org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
> 2023-08-16T06:49:14.1997346Z [info]   at 
> org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
> 2023-08-16T06:49:14.1998091Z [info]   at 
> scala.collection.immutable.List.foreach(List.scala:431)
> 2023-08-16T06:49:14.1998995Z [info]   at 
> org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
> 2023-08-16T06:49:14.1999670Z [info]   at 
> org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
> 2023-08-16T06:49:14.2000302Z [info]   at 
> org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
> 2023-08-16T06:49:14.2000993Z [info]   at 
> org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:269)
> 2023-08-16T06:49:14.2004880Z [info]   at 
> org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:268)
> 2023-08-16T06:49:14.2005664Z [info]   at 
> org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1564)
> 2023-08-16T06:49:14.2006251Z [info]   at 
> org.scalatest.Suite.run(Suite.scala:1114)
> 2023-08-16T06:49:14.2007449Z [info]   at 
> org.scalatest.Suite.run$(Suite.scala:1096)
> 2023-08-16T06:49:14.2008151Z [info]   at 
> org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1564)
> 2023-08-16T06:49:14.2009643Z [info]   at 
> org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:273)
> 2023-08-16T06:49:14.2010290Z [info]   at 
> org.scalatest.SuperEngine.runImpl(Engine.scala:535)
> 2023-08-16T06:49:14.2011063Z [info]   at 
> org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:273)
> 2023-08-16T06:49:14.2013323Z [info]   at 
> org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:272)
> 2023-08-16T06:49:14.2014289Z [info]   at 
> org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:69)
> 2023-08-16T06:49:14.2017752Z [info]   at 
> org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
> 2023-08-16T06:49:14.2018563Z [info]   at 
> org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
> 2023-08-16T06:49:14.2019222Z [info]   at 
> org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
> 2023-08-16T06:49:14.2020623Z [info]   at 
> org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:69)
> 2023-08-16T06:49:14.2021440Z [info]   at 
> org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:321)
> 2023-08-16T06:49:14.2022124Z [info]   at 
> org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:517)
> 2023-08-16T06:49:14.2029114Z [info]   at 
> sbt.ForkMain$Run.lambda$runTest$1(ForkMain.java:414)
> 2023-08-16T06:49:14.2029922Z [info]   at 
> java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 2023-08-16T06:49:14.2031313Z [info]   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> 2023-08-16T06:49:14.2032083Z [info]   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> 2023-08-16T06:49:14.2036045Z [info]   at 
> java.lang.Thread.run(Thread.java:750)
> 2023-08-16T06:50:14.2328885Z 06:50:14.232 WARN 
> org.apache.spark.sql.execution.streaming.state.RocksDB 
> StateStoreId(opId=0,partId=0,name=default): Error closing RocksDB
> 2023-08-16T06:50:14.2340748Z org.apache.spark.SparkException: 
> [CANNOT_LOAD_STATE_STORE.UNRELEASED_THREAD_ERROR] An error occurred during 
> loading state. StateStoreId(opId=0,partId=0,name=default): RocksDB instance 
> could not be acquired by [ThreadId: Some(2023)] as it was not released by 
> [ThreadId: Some(2020), task: partition 0.0 in stage 541.0, TID 1401] after 
> 120010 ms.
> 2023-08-16T06:50:14.2342031Z Thread holding the lock has trace: 
> org.apache.spark.sql.execution.streaming.state.StateStore$.getStateStoreProvider(StateStore.scala:555)
> 2023-08-16T06:50:14.2342817Z 
> org.apache.spark.sql.execution.streaming.state.StateStore$.get(StateStore.scala:544)
> 2023-08-16T06:50:14.2343724Z 
> org.apache.spark.sql.execution.streaming.state.StateStoreRDD.compute(StateStoreRDD.scala:126)
> 2023-08-16T06:50:14.2344430Z 
> org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)
> 2023-08-16T06:50:14.2344964Z org.apache.spark.rdd.RDD.iterator(RDD.scala:328)
> 2023-08-16T06:50:14.2345513Z 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> 2023-08-16T06:50:14.2346107Z 
> org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)
> 2023-08-16T06:50:14.2346618Z org.apache.spark.rdd.RDD.iterator(RDD.scala:328)
> 2023-08-16T06:50:14.2347140Z 
> org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)
> 2023-08-16T06:50:14.2347736Z 
> org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)
> 2023-08-16T06:50:14.2348282Z 
> org.apache.spark.scheduler.Task.run(Task.scala:141)
> 2023-08-16T06:50:14.2348805Z 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:620)
> 2023-08-16T06:50:14.2349373Z 
> org.apache.spark.executor.Executor$TaskRunner$$Lambda$2925/1900945640.apply(Unknown
>  Source)
> 2023-08-16T06:50:14.2350017Z 
> org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
> 2023-08-16T06:50:14.2350714Z 
> org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
> 2023-08-16T06:50:14.2351555Z 
> org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:95)
> 2023-08-16T06:50:14.2352082Z 
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:623)
> 2023-08-16T06:50:14.2352688Z 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> 2023-08-16T06:50:14.2353420Z 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> 2023-08-16T06:50:14.2353927Z java.lang.Thread.run(Thread.java:750)
> 2023-08-16T06:50:14.2354527Z  at 
> org.apache.spark.sql.errors.QueryExecutionErrors$.unreleasedThreadError(QueryExecutionErrors.scala:2637)
> 2023-08-16T06:50:14.2355260Z  at 
> org.apache.spark.sql.execution.streaming.state.RocksDB.acquire(RocksDB.scala:562)
> 2023-08-16T06:50:14.2356039Z  at 
> org.apache.spark.sql.execution.streaming.state.RocksDB.close(RocksDB.scala:467)
> 2023-08-16T06:50:14.2356887Z  at 
> org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider.close(RocksDBStateStoreProvider.scala:222)
> 2023-08-16T06:50:14.2357743Z  at 
> org.apache.spark.sql.execution.streaming.state.StateStore$.$anonfun$unload$1(StateStore.scala:606)
> 2023-08-16T06:50:14.2358482Z  at 
> org.apache.spark.sql.execution.streaming.state.StateStore$.$anonfun$unload$1$adapted(StateStore.scala:606)
> 2023-08-16T06:50:14.2359162Z  at scala.Option.foreach(Option.scala:407)
> 2023-08-16T06:50:14.2359724Z  at 
> org.apache.spark.sql.execution.streaming.state.StateStore$.unload(StateStore.scala:606)
> 2023-08-16T06:50:14.2360502Z  at 
> org.apache.spark.sql.execution.streaming.state.StateStore$.$anonfun$startMaintenanceIfNeeded$4(StateStore.scala:663)
> 2023-08-16T06:50:14.2361318Z  at 
> org.apache.spark.sql.execution.streaming.state.StateStore$.$anonfun$startMaintenanceIfNeeded$4$adapted(StateStore.scala:663)
> 2023-08-16T06:50:14.2362645Z  at 
> scala.collection.mutable.HashMap$$anon$1.$anonfun$foreach$2(HashMap.scala:153)
> 2023-08-16T06:50:14.2363154Z  at 
> scala.collection.mutable.HashTable.foreachEntry(HashTable.scala:237)
> 2023-08-16T06:50:14.2363683Z  at 
> scala.collection.mutable.HashTable.foreachEntry$(HashTable.scala:230)
> 2023-08-16T06:50:14.2364196Z  at 
> scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:44)
> 2023-08-16T06:50:14.2364675Z  at 
> scala.collection.mutable.HashMap$$anon$1.foreach(HashMap.scala:153)
> 2023-08-16T06:50:14.2365279Z  at 
> org.apache.spark.sql.execution.streaming.state.StateStore$.$anonfun$startMaintenanceIfNeeded$2(StateStore.scala:663)
> 2023-08-16T06:50:14.2365989Z  at 
> org.apache.spark.sql.execution.streaming.state.StateStore$MaintenanceTask$$anon$1.run(StateStore.scala:471)
> 2023-08-16T06:50:14.2366577Z  at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 2023-08-16T06:50:14.2367068Z  at 
> java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
> 2023-08-16T06:50:14.2367706Z  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
> 2023-08-16T06:50:14.2368448Z  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
> 2023-08-16T06:50:14.2369108Z  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> 2023-08-16T06:50:14.2369683Z  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> 2023-08-16T06:50:14.2370116Z  at java.lang.Thread.run(Thread.java:750)
> 2023-08-16T06:50:14.2554839Z 06:50:14.253 WARN 
> org.apache.spark.scheduler.TaskSetManager: Lost task 2.0 in stage 541.0 (TID 
> 1403) (localhost executor driver): TaskKilled (Stage cancelled: Job 274 
> cancelled part of cancelled job group 61c94e63-d753-41bf-b5b3-cdbe2d844d26)
> 2023-08-16T06:50:14.2727333Z 06:50:14.268 WARN 
> org.apache.spark.scheduler.TaskSetManager: Lost task 0.0 in stage 541.0 (TID 
> 1401) (localhost executor driver): TaskKilled (Stage cancelled: Job 274 
> cancelled part of cancelled job group 61c94e63-d753-41bf-b5b3-cdbe2d844d26)
> 2023-08-16T06:50:14.2979761Z 06:50:14.294 ERROR 
> org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec: Data 
> source write support MicroBatchWrite[epoch: 0, writer: 
> org.apache.spark.sql.execution.streaming.sources.MemoryStreamingWrite@77a1d641]
>  is aborting.
> 2023-08-16T06:50:14.2981538Z 06:50:14.294 ERROR 
> org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec: Data 
> source write support MicroBatchWrite[epoch: 0, writer: 
> org.apache.spark.sql.execution.streaming.sources.MemoryStreamingWrite@77a1d641]
>  aborted. {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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


Reply via email to