|
@@ -47,10 +47,14 @@ import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.LocalDirAllocator;
|
|
import org.apache.hadoop.fs.LocalDirAllocator;
|
|
|
|
+import org.apache.hadoop.fs.statistics.DurationTracker;
|
|
|
|
+import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
|
|
import org.apache.hadoop.util.Preconditions;
|
|
import org.apache.hadoop.util.Preconditions;
|
|
|
|
|
|
import static java.util.Objects.requireNonNull;
|
|
import static java.util.Objects.requireNonNull;
|
|
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull;
|
|
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull;
|
|
|
|
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTrackerFactory;
|
|
|
|
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_FILE_CACHE_EVICTION;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Provides functionality necessary for caching blocks of data read from FileSystem.
|
|
* Provides functionality necessary for caching blocks of data read from FileSystem.
|
|
@@ -99,6 +103,11 @@ public class SingleFilePerBlockCache implements BlockCache {
|
|
|
|
|
|
private final PrefetchingStatistics prefetchingStatistics;
|
|
private final PrefetchingStatistics prefetchingStatistics;
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Duration tracker factory required to track the duration of some operations.
|
|
|
|
+ */
|
|
|
|
+ private final DurationTrackerFactory trackerFactory;
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* File attributes attached to any intermediate temporary file created during index creation.
|
|
* File attributes attached to any intermediate temporary file created during index creation.
|
|
*/
|
|
*/
|
|
@@ -209,14 +218,19 @@ public class SingleFilePerBlockCache implements BlockCache {
|
|
*
|
|
*
|
|
* @param prefetchingStatistics statistics for this stream.
|
|
* @param prefetchingStatistics statistics for this stream.
|
|
* @param maxBlocksCount max blocks count to be kept in cache at any time.
|
|
* @param maxBlocksCount max blocks count to be kept in cache at any time.
|
|
|
|
+ * @param trackerFactory tracker with statistics to update
|
|
*/
|
|
*/
|
|
- public SingleFilePerBlockCache(PrefetchingStatistics prefetchingStatistics, int maxBlocksCount) {
|
|
|
|
|
|
+ public SingleFilePerBlockCache(PrefetchingStatistics prefetchingStatistics,
|
|
|
|
+ int maxBlocksCount,
|
|
|
|
+ DurationTrackerFactory trackerFactory) {
|
|
this.prefetchingStatistics = requireNonNull(prefetchingStatistics);
|
|
this.prefetchingStatistics = requireNonNull(prefetchingStatistics);
|
|
this.closed = new AtomicBoolean(false);
|
|
this.closed = new AtomicBoolean(false);
|
|
this.maxBlocksCount = maxBlocksCount;
|
|
this.maxBlocksCount = maxBlocksCount;
|
|
Preconditions.checkArgument(maxBlocksCount > 0, "maxBlocksCount should be more than 0");
|
|
Preconditions.checkArgument(maxBlocksCount > 0, "maxBlocksCount should be more than 0");
|
|
blocks = new ConcurrentHashMap<>();
|
|
blocks = new ConcurrentHashMap<>();
|
|
blocksLock = new ReentrantReadWriteLock();
|
|
blocksLock = new ReentrantReadWriteLock();
|
|
|
|
+ this.trackerFactory = trackerFactory != null
|
|
|
|
+ ? trackerFactory : stubDurationTrackerFactory();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -430,25 +444,28 @@ public class SingleFilePerBlockCache implements BlockCache {
|
|
* @param elementToPurge Block entry to evict.
|
|
* @param elementToPurge Block entry to evict.
|
|
*/
|
|
*/
|
|
private void deleteBlockFileAndEvictCache(Entry elementToPurge) {
|
|
private void deleteBlockFileAndEvictCache(Entry elementToPurge) {
|
|
- boolean lockAcquired = elementToPurge.takeLock(Entry.LockType.WRITE,
|
|
|
|
- PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT,
|
|
|
|
- PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT_UNIT);
|
|
|
|
- if (!lockAcquired) {
|
|
|
|
- LOG.error("Cache file {} deletion would not be attempted as write lock could not"
|
|
|
|
- + " be acquired within {} {}", elementToPurge.path,
|
|
|
|
|
|
+ try (DurationTracker ignored = trackerFactory.trackDuration(STREAM_FILE_CACHE_EVICTION)) {
|
|
|
|
+ boolean lockAcquired = elementToPurge.takeLock(Entry.LockType.WRITE,
|
|
PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT,
|
|
PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT,
|
|
PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT_UNIT);
|
|
PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT_UNIT);
|
|
- } else {
|
|
|
|
- try {
|
|
|
|
- if (Files.deleteIfExists(elementToPurge.path)) {
|
|
|
|
- entryListSize--;
|
|
|
|
- prefetchingStatistics.blockRemovedFromFileCache();
|
|
|
|
- blocks.remove(elementToPurge.blockNumber);
|
|
|
|
|
|
+ if (!lockAcquired) {
|
|
|
|
+ LOG.error("Cache file {} deletion would not be attempted as write lock could not"
|
|
|
|
+ + " be acquired within {} {}", elementToPurge.path,
|
|
|
|
+ PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT,
|
|
|
|
+ PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT_UNIT);
|
|
|
|
+ } else {
|
|
|
|
+ try {
|
|
|
|
+ if (Files.deleteIfExists(elementToPurge.path)) {
|
|
|
|
+ entryListSize--;
|
|
|
|
+ prefetchingStatistics.blockRemovedFromFileCache();
|
|
|
|
+ blocks.remove(elementToPurge.blockNumber);
|
|
|
|
+ prefetchingStatistics.blockEvictedFromFileCache();
|
|
|
|
+ }
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.warn("Failed to delete cache file {}", elementToPurge.path, e);
|
|
|
|
+ } finally {
|
|
|
|
+ elementToPurge.releaseLock(Entry.LockType.WRITE);
|
|
}
|
|
}
|
|
- } catch (IOException e) {
|
|
|
|
- LOG.warn("Failed to delete cache file {}", elementToPurge.path, e);
|
|
|
|
- } finally {
|
|
|
|
- elementToPurge.releaseLock(Entry.LockType.WRITE);
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|