Skip to content

Commit

Permalink
STORM-4085 - Ensure that DownloadOrUpdate exception is caught (#3704)
Browse files Browse the repository at this point in the history
  • Loading branch information
paxadax authored Oct 9, 2024
1 parent 1e8eee6 commit 2f6732c
Showing 1 changed file with 6 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -197,7 +197,7 @@ private LocalizedResource getUserArchive(String user, String key) {
throw new AssertionError("All user archives require a user present");
}
ConcurrentMap<String, LocalizedResource> keyToResource = userArchives.computeIfAbsent(user, (u) -> new ConcurrentHashMap<>());
return keyToResource.computeIfAbsent(key,
return keyToResource.computeIfAbsent(key,
(k) -> new LocalizedResource(key, localBaseDir, true, fsOps, conf, user, metricsRegistry));
}

Expand All @@ -206,7 +206,7 @@ private LocalizedResource getUserFile(String user, String key) {
throw new AssertionError("All user archives require a user present");
}
ConcurrentMap<String, LocalizedResource> keyToResource = userFiles.computeIfAbsent(user, (u) -> new ConcurrentHashMap<>());
return keyToResource.computeIfAbsent(key,
return keyToResource.computeIfAbsent(key,
(k) -> new LocalizedResource(key, localBaseDir, false, fsOps, conf, user, metricsRegistry));
}

Expand Down Expand Up @@ -335,13 +335,11 @@ void updateBlobs() {
}
}
for (CompletableFuture<?> f : futures) {
try {
f.get();
} catch (Exception e) {
updateBlobExceptions.mark();
LOG.warn("Could not update blob ({}), will retry again later." , e.getClass().getName());
}
f.get();
}
} catch (Exception e) {
updateBlobExceptions.mark();
LOG.warn("Could not update blob ({}), will retry again later.", e.getClass().getName());
}
}

Expand Down

0 comments on commit 2f6732c

Please sign in to comment.