Add retry logic to download segment tar file in pinot server#5764
Add retry logic to download segment tar file in pinot server#5764
Conversation
e108f05 to
bce865a
Compare
| File tempTarFile = new File(tempDir, segmentName + TAR_GZ_SUFFIX); | ||
| File tempSegmentDir = new File(tempDir, segmentName); | ||
| try { | ||
| SegmentFetcherFactory.fetchSegmentToLocal(uri, tempDownloadFile); |
There was a problem hiding this comment.
fetchSegmentToLocal already adds a retry (can you document that in the API? I believe all the implementations add retries)
There was a problem hiding this comment.
Comment added.
I also added the catch block to check whether AttemptsExceededException is thrown from SegmentFetcherFactory.fetchSegmentToLocal(uri, tempDownloadFile). If so, exit the outer retry.
| LOGGER | ||
| .info("Downloaded tarred segment: {} for table: {} from: {} to: {}, file length: {}", segmentName, tableName, | ||
| uri, tempTarFile, tempTarFile.length()); | ||
| // Even if the tar file has been downloaded successfully, the file itself could be corrupted during the transmission. |
There was a problem hiding this comment.
Why would the file get corrupted during transmission?
(TCP works for communication from here to mars :)
There was a problem hiding this comment.
Agree, however, we did observe in production that the file downloaded was corrupt (likely truncated), and restarting the server downloaded the segment again and was able to untar it.
Perhaps, we can start off by first emitting a metric and alerting on it to see how often we run into the issue. If this was a one-off due to manual intervention, then perhaps retry is not warranted. But if there's a systematic problem, we should try to understand what's causing the corruption, and then decide whether retry is warranted.
There was a problem hiding this comment.
Yes, that works. And if there are cases where the segment downloaded by server is truncated, we need to fix that instead of adding retries. Obvious things to check if it is something more than manual operations:
- Are we sending the reload message before moving the segment to its final location? It appears no from a cursory look.
- Is there a possibility that multiple controllers are fielding the same segment ? (You checked the logs to see that this was not the case, but) We are not handling this case. The source file is likely to get corrupted since each controller will use
pinotFS.copyToDestination()(not exact API) to set the segment in the destination. We should enhance PinotFS to support copying to a temp destination and then moving in place, or use two apis to move/copy the file. Note that this can happen even if the sender of the segment decides to give up on a (slow) connection and retry (without closing the old connection, or closing it too late), and get a different controller to upload the segment.
There was a problem hiding this comment.
I just filed another PR to add the server metric: #5768.
We can hold off this PR a bit and see how often the exception happens on untarring segments.
7c89f3d to
22f575a
Compare
|
is this still a valid pr? |
Description
This PR adds retry logic to download segment tar file in pinot server.
Even if the tar file has been downloaded successfully, the file itself could be corrupted during the transmission. In that case, we should re-download it again.