jasperjiaguo commented on code in PR #10777: URL: https://github.com/apache/pinot/pull/10777#discussion_r1202698942
########## pinot-common/src/main/java/org/apache/pinot/common/utils/fetcher/BaseSegmentFetcher.java: ########## @@ -49,6 +50,7 @@ public abstract class BaseSegmentFetcher implements SegmentFetcher { protected int _retryDelayScaleFactor; protected AuthProvider _authProvider; + Review Comment: (nit) please remove the whitespace ########## pinot-common/src/main/java/org/apache/pinot/common/utils/fetcher/HttpSegmentFetcher.java: ########## @@ -145,6 +148,7 @@ public File fetchUntarSegmentToLocalStreamed(URI downloadURI, File dest, long ma return false; } }); + attempts.set(tries); Review Comment: ```suggestion attempts.set(tries); } catch (AttemptsExceededException e) { attempts.set(retryCount); throw e; } ``` I think we need some code like this to return the correct retry count upon failure of all attempts ########## pinot-common/src/main/java/org/apache/pinot/common/utils/fetcher/HttpSegmentFetcher.java: ########## @@ -106,7 +108,8 @@ public File fetchUntarSegmentToLocalStreamed(URI downloadURI, File dest, long ma AtomicReference<File> ret = new AtomicReference<>(); // return the untared segment directory _logger.info("Retry downloading for {} times. retryCount from pinot server config: {}, number of IP addresses for " + "download URI: {}", retryCount, _retryCount, uriProvider.numAddresses()); - RetryPolicies.exponentialBackoffRetryPolicy(retryCount, _retryWaitMs, _retryDelayScaleFactor).attempt(() -> { + int tries = RetryPolicies.exponentialBackoffRetryPolicy(retryCount, _retryWaitMs, Review Comment: ```suggestion try{ int tries = RetryPolicies.exponentialBackoffRetryPolicy(retryCount, _retryWaitMs, ``` ########## pinot-spi/src/main/java/org/apache/pinot/spi/utils/retry/RetryPolicy.java: ########## @@ -33,7 +33,8 @@ public interface RetryPolicy { * @param operation The operation to attempt, which returns true on success and false on failure. * @throws AttemptsExceededException * @throws RetriableOperationException + * @return the number of attempts used for the operation. 0 means the first try was successful. Review Comment: (nit) return the number of attempts used for the operation, upon successful attempt? In case of failure this wouldn't work right? ########## pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java: ########## @@ -628,13 +629,16 @@ private File downloadAndStreamUntarWithRateLimit(String segmentName, SegmentZKMe maxStreamRateInByte); String uri = zkMetadata.getDownloadUrl(); try { - File ret = SegmentFetcherFactory.fetchAndStreamUntarToLocal(uri, tempRootDir, maxStreamRateInByte); - LOGGER.info("Download and untarred segment: {} for table: {} from: {}", segmentName, _tableNameWithType, uri); - return ret; + AtomicInteger attempts = new AtomicInteger(0); + File ret = SegmentFetcherFactory.fetchAndStreamUntarToLocal(uri, tempRootDir, maxStreamRateInByte, attempts); + _serverMetrics.addMeteredTableValue(_tableNameWithType, ServerMeter.SEGMENT_STREAMED_DOWNLOAD_UNTAR_FAILURES, + attempts.get()); + LOGGER.info("Downloaded and untarred segment: {} for table: {} from: {} attempts: {}", segmentName, + _tableNameWithType, uri, attempts.get()); + return ret; } catch (AttemptsExceededException e) { LOGGER.error("Attempts exceeded when stream download-untarring segment: {} for table: {} from: {} to: {}", - segmentName, _tableNameWithType, uri, tempRootDir); - _serverMetrics.addMeteredTableValue(_tableNameWithType, ServerMeter.SEGMENT_STREAMED_DOWNLOAD_UNTAR_FAILURES, 1L); Review Comment: We still need to add the count to the metric here even if an error is thrown, otherwise it only covers the (partially) successful case? ########## pinot-spi/src/main/java/org/apache/pinot/spi/utils/retry/RetryPolicy.java: ########## @@ -33,7 +33,8 @@ public interface RetryPolicy { * @param operation The operation to attempt, which returns true on success and false on failure. * @throws AttemptsExceededException * @throws RetriableOperationException + * @return the number of attempts used for the operation. 0 means the first try was successful. */ - void attempt(Callable<Boolean> operation) + int attempt(Callable<Boolean> operation) Review Comment: can we add some tests in RetryPolicyTest for this new return value? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org