kfaraz commented on code in PR #16475:
URL: https://github.com/apache/druid/pull/16475#discussion_r1609643174
##########
server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheManager.java:
##########
@@ -96,13 +104,15 @@ public SegmentLocalCacheManager(
List<StorageLocation> locations,
Review Comment:
Don't see why we need to inject `List<StorageLocation>` if we are already
injecting the `SegmentLoaderConfig`. In fact, I think we should probably just
get rid of the provider method `StorageNodeModule.provideStorageLocations()`
and just inject the `SegmentLoaderConfig` in all the relevant places.
##########
server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java:
##########
@@ -490,9 +416,8 @@ void removeSegment(
if (scheduleDrop) {
log.info(
- "Completely removing [%s] in [%,d] millis",
- segment.getId(),
- config.getDropSegmentDelayMillis()
+ "Completely removing segment[%s] in [%,dms].",
Review Comment:
Nit: Maybe keep `ms` outside the square brackets for readability.
`Did something in [100ms]`
vs
`Did something in [100]ms`
##########
server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java:
##########
@@ -209,99 +196,43 @@ public boolean isStarted()
return started;
}
- private void loadLocalCache() throws IOException
+ private void bootstrapCachedSegments() throws IOException
Review Comment:
We don't really need two separate methods anymore. The content here can be
merged into `addSegments()` itself. The final method can then be called
`bootstrapCachedSegments()`.
##########
server/src/main/java/org/apache/druid/server/SegmentManager.java:
##########
@@ -241,8 +240,9 @@ private TableDataSource
getTableDataSource(DataSourceAnalysis analysis)
.orElseThrow(() -> new ISE("Cannot handle datasource: %s",
analysis.getBaseDataSource()));
}
+ @VisibleForTesting
Review Comment:
Let's remove this method if it is only needed for testing. The tests can
call the actual method with `null`s for the 4th arg.
##########
server/src/main/java/org/apache/druid/server/SegmentManager.java:
##########
@@ -262,8 +262,12 @@ public boolean loadSegment(final DataSegment segment,
boolean lazy, SegmentLazyL
*
* @throws SegmentLoadingException if the segment cannot be loaded
*/
- public boolean loadSegment(final DataSegment segment, boolean lazy,
SegmentLazyLoadFailCallback loadFailed,
- ExecutorService loadSegmentIntoPageCacheExec)
throws SegmentLoadingException
+ public boolean loadSegment(
+ final DataSegment segment,
+ boolean lazy,
+ SegmentLazyLoadFailCallback loadFailed,
+ ExecutorService loadSegmentIntoPageCacheExec
+ ) throws SegmentLoadingException, IOException
Review Comment:
Rather than passing a boolean `lazy` and an executor, this class should have
two separate methods `loadSegment(segment, callback)` and
`loadSegmentOnBootstrap(segment, callback)` (or `bootstrapLoadedSegment`,
whichever seems better).
There are two executors for loading segments into page cache, one to be used
while bootstrapping and the other for later. Both these executors should belong
to the `SegmentLocalCacheManager`. You might need to update the
`SegmentCacheManager` interface too to have two separate methods
`loadSegmentIntoPageCache` and `loadSegmentIntoPageCacheOnBootrstrap`.
##########
server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java:
##########
@@ -209,99 +196,43 @@ public boolean isStarted()
return started;
}
- private void loadLocalCache() throws IOException
+ private void bootstrapCachedSegments() throws IOException
{
- final long start = System.currentTimeMillis();
- File baseDir = config.getInfoDir();
- FileUtils.mkdirp(baseDir);
-
- List<DataSegment> cachedSegments = new ArrayList<>();
- File[] segmentsToLoad = baseDir.listFiles();
- int ignored = 0;
- for (int i = 0; i < segmentsToLoad.length; i++) {
- File file = segmentsToLoad[i];
- log.info("Loading segment cache file [%d/%d][%s].", i + 1,
segmentsToLoad.length, file);
- try {
- final DataSegment segment = jsonMapper.readValue(file,
DataSegment.class);
-
- if (!segment.getId().toString().equals(file.getName())) {
- log.warn("Ignoring cache file[%s] for segment[%s].", file.getPath(),
segment.getId());
- ignored++;
- } else if (segmentCacheManager.isSegmentCached(segment)) {
- cachedSegments.add(segment);
- } else {
- log.warn("Unable to find cache file for %s. Deleting lookup entry",
segment.getId());
-
- File segmentInfoCacheFile = new File(baseDir,
segment.getId().toString());
- if (!segmentInfoCacheFile.delete()) {
- log.warn("Unable to delete segmentInfoCacheFile[%s]",
segmentInfoCacheFile);
- }
- }
- }
- catch (Exception e) {
- log.makeAlert(e, "Failed to load segment from segmentInfo file")
- .addData("file", file)
- .emit();
- }
- }
-
- if (ignored > 0) {
- log.makeAlert("Ignored misnamed segment cache files on startup.")
- .addData("numIgnored", ignored)
- .emit();
- }
-
+ final Stopwatch stopwatch = Stopwatch.createStarted();
+ final List<DataSegment> cachedSegments =
segmentManager.getCachedSegments();
addSegments(
cachedSegments,
- () -> log.info("Cache load took %,d ms", System.currentTimeMillis() -
start)
+ () -> {
+ log.info("Cache load of [%d] bootstrap segments completed.",
cachedSegments.size());
+ }
);
- }
-
- private void loadSegment(DataSegment segment, DataSegmentChangeCallback
callback, boolean lazy)
- throws SegmentLoadingException
- {
- loadSegment(segment, callback, lazy, null);
+ stopwatch.stop();
+ log.info("Cache load of [%d] bootstrap segments took [%,dms]",
cachedSegments.size(), stopwatch.millisElapsed());
}
/**
- * Load a single segment. If the segment is loaded successfully, this
function simply returns. Otherwise it will
- * throw a SegmentLoadingException
- *
- * @throws SegmentLoadingException if it fails to load the given segment
+ * Load a single segment. If the segment is loaded successfully, this
function simply returns. Otherwise, it will
Review Comment:
If you want to avoid repetition, I would rather we delete the second
sentence here but retain the `@throws` tag.
--
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]