[ 
https://issues.apache.org/jira/browse/FLINK-31104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17694431#comment-17694431
 ] 

dalongliu commented on FLINK-31104:
-----------------------------------

I also occur the query timeout when running tpcds in the yarn cluster, the 
thread stack as follows:
{code:java}
"HashJoin[3174] [Source: store_sales[3210], Source: 
household_demographics[3185], Source: store_sales[3144]] -> Calc[3175] -> 
HashAggregate[3176] -> Calc[3177] (940/1500)#0" Id=7959 WAITING on 
java.util.concurrent.CompletableFuture$Signaller@4a270956
    at sun.misc.Unsafe.park(Native Method)
    -  waiting on java.util.concurrent.CompletableFuture$Signaller@4a270956
    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
    at 
java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1707)
    at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
    at 
java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1742)
    at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
    at 
org.apache.flink.runtime.io.network.buffer.LocalBufferPool.requestMemorySegmentBlocking(LocalBufferPool.java:384)
    at 
org.apache.flink.runtime.io.network.buffer.LocalBufferPool.requestMemorySegmentBlocking(LocalBufferPool.java:350)
    at 
org.apache.flink.runtime.io.network.partition.SortMergeResultPartition.requestNetworkBuffers(SortMergeResultPartition.java:339)
    at 
org.apache.flink.runtime.io.network.partition.SortMergeResultPartition.createNewDataBuffer(SortMergeResultPartition.java:307)
    at 
org.apache.flink.runtime.io.network.partition.SortMergeResultPartition.getBroadcastDataBuffer(SortMergeResultPartition.java:302)
    at 
org.apache.flink.runtime.io.network.partition.SortMergeResultPartition.emit(SortMergeResultPartition.java:256)
    at 
org.apache.flink.runtime.io.network.partition.SortMergeResultPartition.broadcast(SortMergeResultPartition.java:248)
    at 
org.apache.flink.runtime.io.network.partition.SortMergeResultPartition.broadcastRecord(SortMergeResultPartition.java:223)
    at 
org.apache.flink.runtime.io.network.api.writer.BroadcastRecordWriter.broadcastEmit(BroadcastRecordWriter.java:48)
    at 
org.apache.flink.streaming.runtime.io.RecordWriterOutput.emitWatermark(RecordWriterOutput.java:121)
    at 
org.apache.flink.streaming.api.operators.CountingOutput.emitWatermark(CountingOutput.java:43)
    at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.processWatermark(AbstractStreamOperator.java:604)
    at 
org.apache.flink.table.runtime.operators.TableStreamOperator.processWatermark(TableStreamOperator.java:57)
    at 
org.apache.flink.streaming.runtime.tasks.ChainingOutput.emitWatermark(ChainingOutput.java:107)
    at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.processWatermark(AbstractStreamOperator.java:604)
    at 
org.apache.flink.table.runtime.operators.TableStreamOperator.processWatermark(TableStreamOperator.java:57)
    at 
org.apache.flink.streaming.runtime.tasks.ChainingOutput.emitWatermark(ChainingOutput.java:107)
    at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.processWatermark(AbstractStreamOperator.java:604)
    at 
org.apache.flink.table.runtime.operators.TableStreamOperator.processWatermark(TableStreamOperator.java:57)
    at 
org.apache.flink.streaming.runtime.tasks.ChainingOutput.emitWatermark(ChainingOutput.java:107)
    at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.processWatermark(AbstractStreamOperator.java:604)
    at 
org.apache.flink.table.runtime.operators.TableStreamOperator.processWatermark(TableStreamOperator.java:57)
    at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.processWatermark(AbstractStreamOperator.java:609)
    at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.processWatermark2(AbstractStreamOperator.java:618)
    at 
org.apache.flink.streaming.runtime.io.StreamTwoInputProcessorFactory$StreamTaskNetworkOutput.emitWatermark(StreamTwoInputProcessorFactory.java:268)
    at 
org.apache.flink.streaming.runtime.watermarkstatus.StatusWatermarkValve.findAndOutputNewMinWatermarkAcrossAlignedChannels(StatusWatermarkValve.java:199)
    at 
org.apache.flink.streaming.runtime.watermarkstatus.StatusWatermarkValve.inputWatermark(StatusWatermarkValve.java:114)
    at 
org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.processElement(AbstractStreamTaskNetworkInput.java:148)
    at 
org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.emitNext(AbstractStreamTaskNetworkInput.java:110)
    at 
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
    at 
org.apache.flink.streaming.runtime.io.StreamMultipleInputProcessor.processInput(StreamMultipleInputProcessor.java:85)
    at 
org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:547)
    at 
org.apache.flink.streaming.runtime.tasks.StreamTask$$Lambda$653/803691740.runDefaultAction(Unknown
 Source)
    at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231)
    at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:834)
    at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:783)
    at 
org.apache.flink.runtime.taskmanager.Task$$Lambda$1273/1500345048.run(Unknown 
Source)
    at 
org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:952)
    at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:931)
    at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:745)
    at org.apache.flink.runtime.taskmanager.Task.run(Task.java:562)
    at java.lang.Thread.run(Thread.java:750)"System Time Trigger for 
HashJoin[3174] [Source: store_sales[3210], Source: 
household_demographics[3185], Source: store_sales[3144]] -> Calc[3175] -> 
HashAggregate[3176] -> Calc[3177] (707/1500)#0" Id=7955 WAITING on 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@5358647
    at sun.misc.Unsafe.park(Native Method)
    -  waiting on 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@5358647
    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
    at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
    at 
java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1081)
    at 
java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
    at 
java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
    at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
    at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    at java.lang.Thread.run(Thread.java:750) {code}

> TPC-DS test timed out in query 36
> ---------------------------------
>
>                 Key: FLINK-31104
>                 URL: https://issues.apache.org/jira/browse/FLINK-31104
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Runtime, Tests
>    Affects Versions: 1.17.0
>            Reporter: Matthias Pohl
>            Priority: Blocker
>              Labels: test-stability
>
> There has a timeout happened in 
> [apache-flink:flink-end-to-end-tests/flink-tpcds-test/tpcds-tool/query/query36.sql|https://github.com/apache/flink/blob/20c983c26262057c4d59bd591aed89969a8ff525/flink-end-to-end-tests/flink-tpcds-test/tpcds-tool/query/query36.sql]
>  of the TPC-DS test suite:
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=46202&view=logs&j=6e8542d7-de38-5a33-4aca-458d6c87066d&t=5846934b-7a4f-545b-e5b0-eb4d8bda32e1&l=880
> {code}
> [...]
> Feb 16 04:58:23 [INFO]Run TPC-DS query 36 ...
> Feb 16 04:58:23 Job has been submitted with JobID 
> 4d0c1e6cbde9f0b6ae8b9f9afd159c06
> {code}
> Unfortunately, no further logs are provided.



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

Reply via email to