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
[Problem] Inconsistent blocks when reading shuffle data #198
Comments
It may be a bug of our rss. I ever try to fix the similar problems by #40 and Tencent/Firestorm#92. Could you reproduce this problem? Are the missing blocks disk storage or memory storage? |
It occurs in our users' jobs and maybe I have to write test spark code to reproduce
I dont know which storage's block missing. But I check the request number in shuffle server and the request log entry number in tasks, they are equal. |
It looks this problem is not the same as the problem you said. This problem is caused by lacking partial blocks. |
We seem to have the same problem, but the reason for the inconsistent blocks number is that an exception occurred when the index file was read. |
It seems that we read incomplete index file. |
We should fail fast when encountering BufferUnderflowException instead of ignoring. incubator-uniffle/common/src/main/java/org/apache/uniffle/common/util/RssUtils.java Line 224 in 07f70ed
|
For MEMORY_HDFS or MEMORY_LOCALFILE, it's normal to read incomplete index data, so we choose to ignore instead of failing fast. |
What I am curious about is whether this is caused by the fact that the data of the stream is not flushed normally or the stream is closed abnormally. |
Maybe shuffle index is writing.. |
Oh, make sense.
Firstly, maybe we should reproduce this exception that make reader/writer operate this file at the same time. |
As I know, the exception of BufferUnderflowException is caused by the incomplete incubator-uniffle/common/src/main/java/org/apache/uniffle/common/util/RssUtils.java Line 224 in 07f70ed
The operation of spilling is a common action for uniffle and it has done the operation of filtering blocks. So this inconsistent blocks means it exist bug when spilling |
Our production environment also had such an exception once, but so far it has not been able to reproduce. |
I think maybe the shuffle server is writing this file and the output stream has not been closed. |
Can you help me understand #issue_63? @jerqi |
In our production environment, certain tasks will have this problem for a certain period of time. I am trying to reproduce this situation and keep the index file and data file. |
If you use MEMORY_HDFS or MEMORY_LOCALFILE_HDFS, you can read data from memory and then you will read data from HDFS, but the data of memory may be flushed, so you can read the incomplete data of memory flushed to HDFS. |
After collecting all the failed tasks whose exception is related to I think I will raise a PR to fix it. |
Follow up this problem. I found the Grpc client sometimes will throw DEADLINE exception like as follows
But I found this response has been sent by shuffle server, but the client side still throw exception. What will cause this? Network? GC? Did you meet similar problems? @jerqi |
Response may not be sent by shuffle server timely. |
Do we have some ways to avoid this problem? |
This is the response timeout, which should be caused by the high load of ShuffleServer and slow response. |
We can increase the rpc timeout, and find the reason for the slow response of the shuffle server. |
But I dont find any exception through the metrics of grpc. How did u locate this problem caused by the high-pressure shuffle server? |
It's not a fundamental way of increasing the rpc timeout. And I found the rpc has been accepted by shuffle server(handled log has been shown), maybe it's stucked on sending. |
We had a similar rpc timeout exception before, and the task of running 10T data will appear. The investigation found that it was because the inflush_memory and used_memory were too high, which caused the client to frequently retry to send data and apply for buffer. |
You're right, not only the service response, but also the network io, etc. |
Yes, the failure of requiring memory will also cause this exception, but I didn't find any exception of server about require-memory failed. Besides, the problem you mentioned I have submit a new PR to fix #218 |
we also met this error, but in fact it's rpc timeout, when we in compose client mode, the one time read exception will be catch, then try to read other storage , and if the result is empty, then the exception will be gone. at last client will check the blocks map, found the Inconsistent blocks issue. so am thinking there is exception confuse, some case is not inconsistent blocks issue, just read timeout maybe cased by high load shuffle server. suggest to change the code to raise another exception for this case to void the confuse |
+1. |
+1, you can raise a pr if you have time. |
What you want seems to be implemented in this pr #276? Is it right? |
#276 just fix the block inconsistency in multiple replica. I think we just fast fail when encountering rpc timeout in this issue. |
hi, I had meet same error, my server conf settting rss.storage.type MEMORY_LOCALFILE_HDFS , there is 10 millions records, spark sql is : insert into ssb_new.lineorder_partition_sml_parquet_partition9 Driver logs have Warn log : There is No FULL GC and server.out error happened : 警告: Stream Error @zuston @leixm Hi, I find ShuffleServerNettyHandler.RPC_TIMEOUT is hard coded by 60000 MILLISECONDS Do you have ways to find why or slove ? |
The key point is the client request timeout. I think you should check the shuffle-server metric to find out whether the grpc thread pool is enough and grpc blocking queue size is high? |
@zuston Hi, Thanks for fast reply !!! Do you mean check the shuffle-server metric : grpc_server_executor_active_threads and grpc_server_executor_blocking_queue_size ? need to adjust which key in shuffle-server conf ? |
Let's see the metric firstly. |
ok , I will rerun the sql and check the metrics , thanks !!! |
Hi, I rerun the sql, don`t reproduce the error , the metrics dashboard as follow: server.conf : rss.server.flush.thread.alive 5 |
From the dashboard metrics, it looks good. Please attach pictures when this error happens and then ping me. Thanks. |
@zuston |
@zuston hi, when huge table join huge table, shuffle server have blocked threads , Is it right? the executor have no daemon thread holding and hava no error log |
Could you create a new issue? |
ok , thanks !!! |
I found some tasks of spark jobs will throw the exceptions that the inconsistent blocks number. The stacktrace is as follows
I didn't find any error/warn log in shuffle server which stored the corresponding partition data.
We dont set any replica config and directly use the MEMORY_LOCALFILE storageType. Does this exception caused by the disk error?
The text was updated successfully, but these errors were encountered: