Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Trino worker parallelism becomes 0 #15055

Open
vinay-kl opened this issue Nov 16, 2022 · 24 comments · Fixed by #15060
Open

Trino worker parallelism becomes 0 #15055

vinay-kl opened this issue Nov 16, 2022 · 24 comments · Fixed by #15060
Labels
bug Something isn't working

Comments

@vinay-kl
Copy link
Contributor

Hello Team,
We are running into WORKER PARALLELISM becoming 0 and it stays for 30-40 minutes even though there are enough queries in the queue and in running state.

Attaching the thread dump and images of the same
Screenshot 2022-11-16 at 2 54 51 PM
Screenshot 2022-11-16 at 3 11 47 PM
Screenshot 2022-11-16 at 3 12 33 PM
threadDump_16Nov2.out.zip

When we looked at the thread dump, we found out that one of the thread which has entered the synchronized block which is interacting with hadoop API in our case azure storage is stuck under retries due to some underlying reason unknown. And all the other threads are waiting upon this Thread to release the lock to enter the block

The stack trace for initial thread which is holding the lock

dispatcher-query-22sleeping , holding [ 0x00007fa04fc79108 0x00007fa0e37d52f0 ]
at java.lang.Thread.sleep([email protected]/Native Method)
at io.trino.hadoop.$internal.com.microsoft.azure.storage.core.ExecutionEngine.executeWithRetry(ExecutionEngine.java:265)
at io.trino.hadoop.$internal.com.microsoft.azure.storage.blob.CloudBlobContainer.exists(CloudBlobContainer.java:769)
at io.trino.hadoop.$internal.com.microsoft.azure.storage.blob.CloudBlobContainer.exists(CloudBlobContainer.java:756)
at org.apache.hadoop.fs.azure.StorageInterfaceImpl$CloudBlobContainerWrapperImpl.exists(StorageInterfaceImpl.java:233)
at io.trino.plugin.hive.fs.TrinoFileSystemCache.createFileSystem(TrinoFileSystemCache.java:164)
at io.trino.plugin.hive.fs.TrinoFileSystemCache.getInternal(TrinoFileSystemCache.java:120)
at io.trino.plugin.hive.fs.TrinoFileSystemCache.get(TrinoFileSystemCache.java:88)
at org.apache.hadoop.fs.ForwardingFileSystemCache.get(ForwardingFileSystemCache.java:38)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:479)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
at io.trino.plugin.hive.HdfsEnvironment.lambda$getFileSystem$0(HdfsEnvironment.java:78)
at io.trino.plugin.hive.HdfsEnvironment$$Lambda$13335/0x00007f94df1bcd08.run(Unknown Source)
at io.trino.plugin.hive.authentication.NoHdfsAuthentication.doAs(NoHdfsAuthentication.java:25)
at io.trino.plugin.hive.HdfsEnvironment.getFileSystem(HdfsEnvironment.java:77)
at io.trino.plugin.hive.HdfsEnvironment.getFileSystem(HdfsEnvironment.java:71)
at io.trino.plugin.hive.util.HiveWriteUtils.isS3FileSystem(HiveWriteUtils.java:482)
at io.trino.plugin.hive.HiveMetadata.checkExternalPath(HiveMetadata.java:1121)
at io.trino.plugin.hive.HiveMetadata.createTable(HiveMetadata.java:908)
at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata.createTable(ClassLoaderSafeConnectorMetadata.java:382)
at io.trino.metadata.MetadataManager.createTable(MetadataManager.java:640)
at io.trino.execution.CreateTableTask.internalExecute(CreateTableTask.java:270)
at io.trino.execution.CreateTableTask.execute(CreateTableTask.java:119)
at io.trino.execution.CreateTableTask.execute(CreateTableTask.java:85)
at io.trino.execution.DataDefinitionExecution.start(DataDefinitionExecution.java:145)
at io.trino.execution.SqlQueryManager.createQuery(SqlQueryManager.java:243)
at io.trino.dispatcher.LocalDispatchQueryFactory$$Lambda$9271/0x00007f96eb0fa8b8.accept(Unknown Source)
at io.trino.dispatcher.LocalDispatchQuery.lambda$startExecution$7(LocalDispatchQuery.java:143)
at io.trino.dispatcher.LocalDispatchQuery$$Lambda$9871/0x00007f96e868b8b8.run(Unknown Source)
at io.trino.$gen.Trino_387____20221116_085646_2.run(Unknown Source)
at java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1128)
at java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:628)
at java.lang.Thread.run([email protected]/Thread.java:829)

Other threads which are blocked

"dispatcher-query-103" #97965 daemon prio=5 os_prio=0 cpu=70.98ms elapsed=52.19s tid=0x00007f9a28011420 nid=0x8d40 waiting for monitor entry  [0x00007f95035f3000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at io.trino.plugin.hive.fs.TrinoFileSystemCache.getInternal(TrinoFileSystemCache.java:108)
waiting to lock <0x00007fa04fc79108> (a io.trino.plugin.hive.fs.TrinoFileSystemCache)
        at io.trino.plugin.hive.fs.TrinoFileSystemCache.get(TrinoFileSystemCache.java:88)
        at org.apache.hadoop.fs.ForwardingFileSystemCache.get(ForwardingFileSystemCache.java:38)
        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:479)
        at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
        at io.trino.plugin.hive.HdfsEnvironment.lambda$getFileSystem$0(HdfsEnvironment.java:78)
        at io.trino.plugin.hive.HdfsEnvironment$$Lambda$13335/0x00007f94df1bcd08.run(Unknown Source)
        at io.trino.plugin.hive.authentication.NoHdfsAuthentication.doAs(NoHdfsAuthentication.java:25)
        at io.trino.plugin.hive.HdfsEnvironment.getFileSystem(HdfsEnvironment.java:77)
        at io.trino.plugin.hive.HdfsEnvironment.getFileSystem(HdfsEnvironment.java:71)
        at io.trino.plugin.hive.util.HiveWriteUtils.isS3FileSystem(HiveWriteUtils.java:482)
        at io.trino.plugin.hive.HiveMetadata.checkExternalPath(HiveMetadata.java:1121)
        at io.trino.plugin.hive.HiveMetadata.createTable(HiveMetadata.java:908)
        at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata.createTable(ClassLoaderSafeConnectorMetadata.java:382)
        at io.trino.metadata.MetadataManager.createTable(MetadataManager.java:640)
        at io.trino.execution.CreateTableTask.internalExecute(CreateTableTask.java:270)
        at io.trino.execution.CreateTableTask.execute(CreateTableTask.java:119)
        at io.trino.execution.CreateTableTask.execute(CreateTableTask.java:85)
        at io.trino.execution.DataDefinitionExecution.start(DataDefinitionExecution.java:145)
        at io.trino.execution.SqlQueryManager.createQuery(SqlQueryManager.java:243)
        at io.trino.dispatcher.LocalDispatchQueryFactory$$Lambda$9271/0x00007f96eb0fa8b8.accept(Unknown Source)
        at io.trino.dispatcher.LocalDispatchQuery.lambda$startExecution$7(LocalDispatchQuery.java:143)
        at io.trino.dispatcher.LocalDispatchQuery$$Lambda$9871/0x00007f96e868b8b8.run(Unknown Source)
        at io.trino.$gen.Trino_387____20221116_085646_2.run(Unknown Source)
        at java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1128)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:628)
        at java.lang.Thread.run([email protected]/Thread.java:829)

TrinoFileSystemCache.java is the class and getInternal is the method.

This PR seems to have solved the issue https://github.com/trinodb/trino-hadoop-apache/pull/39, this was already merged but i couldn't see the change even in the latest branch.

@hashhar
Copy link
Member

hashhar commented Nov 16, 2022

cc: @findepi @losipiuk

No newer version of trino-hadoop-apache is released with that change included yet.

@hashhar hashhar added the bug Something isn't working label Nov 16, 2022
@sopel39
Copy link
Member

sopel39 commented Nov 16, 2022

cc @arhimondr

@findepi
Copy link
Member

findepi commented Nov 16, 2022

i wasn't involved in trinodb/trino-hadoop-apache#39.
cc @phd3 @electrum

@arhimondr
Copy link
Contributor

#15060

@phd3
Copy link
Member

phd3 commented Nov 16, 2022

Thanks @arhimondr

@vinay-kl have we verified if the change works? not sure yet if that PR will necessarily solve this particular issue - given the locks here are on getInternal from the stacktraces. And looks like thread holding the lock is hanging - but not deadlock. (Although the first screenshot does show one being held while waiting for others, sorry I haven't looked at the thread-dump.)

@phd3
Copy link
Member

phd3 commented Nov 16, 2022

cc @jitheshtr

@vinay-kl
Copy link
Contributor Author

vinay-kl commented Nov 17, 2022

@phd3 as long as the current thread which is inside the getInternal hangs, the remaining ones would be blocked right?
And this could be happen for numerous reason, throttling at storage side in our case.

getInternal of TrinoFileSysyemCache is a private method which is still synchronized, and the thread is stuck within createFileSystem function of the same class.

 dispatcher-query-22sleeping , holding [ 0x00007fa04fc79108 0x00007fa0e37d52f0 ]
at java.lang.Thread.sleep([email protected]/Native Method)
at io.trino.hadoop.$internal.com.microsoft.azure.storage.core.ExecutionEngine.executeWithRetry(ExecutionEngine.java:265)
at io.trino.hadoop.$internal.com.microsoft.azure.storage.blob.CloudBlobContainer.exists(CloudBlobContainer.java:769)
at io.trino.hadoop.$internal.com.microsoft.azure.storage.blob.CloudBlobContainer.exists(CloudBlobContainer.java:756)
at org.apache.hadoop.fs.azure.StorageInterfaceImpl$CloudBlobContainerWrapperImpl.exists(StorageInterfaceImpl.java:233)
at io.trino.plugin.hive.fs.TrinoFileSystemCache.createFileSystem(TrinoFileSystemCache.java:164)
at io.trino.plugin.hive.fs.TrinoFileSystemCache.getInternal(TrinoFileSystemCache.java:120)
at io.trino.plugin.hive.fs.TrinoFileSystemCache.get(TrinoFileSystemCache.java:88)
at org.apache.hadoop.fs.ForwardingFileSystemCache.get(ForwardingFileSystemCache.java:38)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:479)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)

@arhimondr
Copy link
Contributor

Actually yeah, it looks like there could be more to it. Let me reopen it for now

@arhimondr arhimondr reopened this Nov 17, 2022
@phd3
Copy link
Member

phd3 commented Nov 17, 2022

yes, for the other change the deadlock is happening because of two different locks. If only the method holding one lock is hanging - not sure if that fix helps.

@jitheshtr
Copy link
Contributor

jitheshtr commented Nov 17, 2022

For the lock contention due to slow createFileSystem() invocation, https://github.com/trinodb/trino-hadoop-apache/pull/39 wont fix it. There is this draft PR - #13243 - that can fix this as the createFileSystem() is invoked outside of cache's lock in there. I have just pushed the latest local changes in there, but still need a rebase with master and fix product test failure (due to interaction between various tests and the new test added in there, both accessing global TrinoFileSystemCache within the same jvm)

@jitheshtr
Copy link
Contributor

@vinay-kl - Would you be able to try out the TrinoFileSystemCache change from above in your use case to confirm if it fixes your issue?

@vinay-kl
Copy link
Contributor Author

@jitheshtr as i'm on leave for few weeks, i would not be able to test the same. sorry about this.

phd3 added a commit to phd3/trino that referenced this issue Nov 28, 2022
@phd3
Copy link
Member

phd3 commented Nov 28, 2022

sample reproduction by mocking a hanging createFileSystem call here: phd3@a24c2b2#diff-00fbdadb3c559a4c490b567fb1e23166699640a44ea4e1e5e530a579b8c3392bR82

@phd3
Copy link
Member

phd3 commented Nov 28, 2022

@vinay-kl From your heapdump - there're ~400 instances of TrinoFileSystemCache.getInternal waiting on the lock that's being held by a thread taking too long to create FS. So to me, looks like that's the cause for 0 parellism.

@phd3
Copy link
Member

phd3 commented Nov 28, 2022

From cluster stability perspective - we need to remove this contention. Either by somehow timing the call out soon enough to get out of the lock - OR avoid having to lock in the creation process (which seems unnecessary but happens currently).

@phd3
Copy link
Member

phd3 commented Nov 29, 2022

There're two issues here:

  • The File creation process is currently within a lock. In an environment where FileSystem objects are created independently for different users, this can result in a cluster site-up issue if one of the creation hangs because all users are impacted due to this one hang. Update TrinoFileSystemCache to represent latest hadoop implementation #13243 fixes this. The approach here is lock-striping (in somewhat oversimplified sense). The FS creation was using a common lock across the whole cache - now it only uses a per-key lock when creating FS object.
  • The Azure FileSystem creation hangs for some reason. I'm not able to reproduce the issue here on my end, but looks like it may be a bug within executeWithRetry, which should ideally fail after a point - but this still won't fix contention - i.e. other queries can remain blocked and face perf penalty for that time.

@vinay-kl
Copy link
Contributor Author

@phd3 @jitheshtr what can be done here to navigate out of this situation until the fix has been implemented?, does killing the blocked query (after identifying the one from thread dump) help here?

Once Azure storage (in our case) starts serving requests properly, then the cluster comes back to stable state. But this could take it's own time. So was wondering what could help mitigate in the meanwhile.

@jitheshtr
Copy link
Contributor

@vinay-kl - PR #13243 has been merged to version 406. Could you please let us know if this has helped fix the issue for you?

@vinay-kl
Copy link
Contributor Author

@jitheshtr The issue is still persistent in v406 and above as well, attaching few Thread dumps taken on coordinator & workers for more info. We saw many threads in awaiting (TIMED_WAITING) state.

We observed the in the newer versions the parallelism is getting recovered after 5-7 minutes, and this is better in the latest version whereas in v406 it took really long to recover.

v387 is the stable version compared to these newer ones, the occurrences of parallelism going to 0 is very ver minimal and when it does happen it recovers very quickly as opposed to others.

Also attached screenshot indicates the comparison between normal vs hampered dumps.

thread_dumps.zip
Screenshot 2023-03-07 at 10 12 17 AM

@sopel39
Copy link
Member

sopel39 commented Mar 16, 2023

@vinay-kl I don't see getInternal in dumps anymore (correct me if I'm wrong). Maybe it's different issue now? It could be an issue with split enumeration or other metadata issue. What metastore/storage you use? Do you have query JSON?

@phd3
Copy link
Member

phd3 commented Mar 16, 2023

Yeah - I didn't see any threads where splits are waiting to create/get FileSystems either - so I suspect this is an independent issue. Is there any indication that the coordinator may be CPU/network saturated? Do you see scan tasks on workers in "RUNNING" state - but just waiting on new splits? Would also be useful to get a profile for coordinator/workers to get more insight into what may be happening. https://www.baeldung.com/java-async-profiler

@vinay-kl
Copy link
Contributor Author

I don't see getInternal in dumps anymore (correct me if I'm wrong). Maybe it's different issue now? It could be an issue with split enumeration or other metadata issue. What metastore/storage you use? Do you have query JSON?

@sopel39 yes indeed the getInternal has been prevented, but the overall issue from Trino remains as is. We use Hive metastore backed by MySQL and Azure storage as data layer. We don't query JSON it's primarily ORC.

Yeah - I didn't see any threads where splits are waiting to create/get FileSystems either - so I suspect this is an independent issue. Is there any indication that the coordinator may be CPU/network saturated? Do you see scan tasks on workers in "RUNNING" state - but just waiting on new splits? Would also be useful to get a profile for coordinator/workers to get more insight into what may be happening. https://www.baeldung.com/java-async-profiler

@phd3 we don't see any saturation on Co-ordinator, the CPU is well under 15-20 % usage, and no indications at network level. There's not just enough splits for workers to process even though there are many queries in running and queue state.
we'll look into making your of the profiler mentioned to derive more insights.

@sopel39
Copy link
Member

sopel39 commented Mar 17, 2023

@vinay-kl I meant if you have JSON from query execution?

@vinay-kl
Copy link
Contributor Author

@sopel39 my bad, i'll get few JSONs and attach the same.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working
Development

Successfully merging a pull request may close this issue.

7 participants