ankitsultana commented on code in PR #12317:
URL: https://github.com/apache/pinot/pull/12317#discussion_r1478187523
##########
pinot-common/src/main/java/org/apache/pinot/common/utils/PeerServerSegmentFinder.java:
##########
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.pinot.core.util;
+package org.apache.pinot.common.utils;
Review Comment:
@walterddr : any concerns with this x-package move?
@chenboat : you can avoid this move by adding a method as follows to
`BaseSegmentFetcher`.
(though I don't have strong opinions about this approach or the current
approach)
```
public class BaseSegmentFetcher {
// This is the existing method.
@Override
public void fetchSegmentToLocal(List<URI> uris, File dest) throws
IOException {
...
}
// This can be the new method. It gets the URIs every time from the
supplier
@Override
public void fetchSegmentToLocal(Supplier<List<URI>> uriSupplier, File
dest) throws IOException {
int attempt =
RetryPolicies.exponentialBackoffRetryPolicy(_retryCount,
_retryWaitMs, _retryDelayScaleFactor).attempt(() -> {
// First find servers hosting the segment in ONLINE state.
List<URI> computedUris = uriSupplier.get();
// Next get through the list of URIs to fetch the segment until
success.
for (URI uri : computedUris) {
fetchSegmentToLocalWithoutRetry(uri, dest);
return true;
}
....
}
}
```
##########
pinot-common/src/main/java/org/apache/pinot/common/utils/fetcher/HttpSegmentFetcher.java:
##########
@@ -48,6 +48,19 @@ protected void doInit(PinotConfiguration config) {
_httpClient = new
FileUploadDownloadClient(HttpClientConfig.newBuilder(config).build());
}
+ public HttpSegmentFetcher() {
+ }
+
+ public HttpSegmentFetcher(FileUploadDownloadClient httpClient,
PinotConfiguration config) {
+ _httpClient = httpClient;
+ _retryCount = config.getProperty(RETRY_COUNT_CONFIG_KEY,
DEFAULT_RETRY_COUNT);
Review Comment:
Aren't these already set in `BaseSegmentFetcher#init`?
##########
pinot-common/src/main/java/org/apache/pinot/common/utils/fetcher/BaseSegmentFetcher.java:
##########
@@ -109,6 +112,38 @@ public File fetchUntarSegmentToLocalStreamed(URI uri, File
dest, long rateLimit,
throw new UnsupportedOperationException();
}
+ // Download segment to a local location with retries.
Review Comment:
use javadocs
##########
pinot-common/src/main/java/org/apache/pinot/common/utils/fetcher/SegmentFetcher.java:
##########
@@ -49,4 +50,16 @@ File fetchUntarSegmentToLocalStreamed(URI uri, File dest,
long rateLimit, Atomic
*/
void fetchSegmentToLocal(List<URI> uri, File dest)
throws Exception;
+
+ /**
+ *
Review Comment:
add description, particularly about how this method uses peer servers
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]