|
@@ -47,6 +47,7 @@ import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.LocalDirAllocator;
|
|
|
+import org.apache.hadoop.util.Preconditions;
|
|
|
|
|
|
import static java.util.Objects.requireNonNull;
|
|
|
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull;
|
|
@@ -61,27 +62,42 @@ public class SingleFilePerBlockCache implements BlockCache {
|
|
|
/**
|
|
|
* Blocks stored in this cache.
|
|
|
*/
|
|
|
- private final Map<Integer, Entry> blocks = new ConcurrentHashMap<>();
|
|
|
+ private final Map<Integer, Entry> blocks;
|
|
|
|
|
|
/**
|
|
|
- * Number of times a block was read from this cache.
|
|
|
- * Used for determining cache utilization factor.
|
|
|
+ * Total max blocks count, to be considered as baseline for LRU cache eviction.
|
|
|
*/
|
|
|
- private int numGets = 0;
|
|
|
+ private final int maxBlocksCount;
|
|
|
|
|
|
- private final AtomicBoolean closed;
|
|
|
+ /**
|
|
|
+ * The lock to be shared by LRU based linked list updates.
|
|
|
+ */
|
|
|
+ private final ReentrantReadWriteLock blocksLock;
|
|
|
|
|
|
- private final PrefetchingStatistics prefetchingStatistics;
|
|
|
+ /**
|
|
|
+ * Head of the linked list.
|
|
|
+ */
|
|
|
+ private Entry head;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tail of the linked list.
|
|
|
+ */
|
|
|
+ private Entry tail;
|
|
|
|
|
|
/**
|
|
|
- * Timeout to be used by close, while acquiring prefetch block write lock.
|
|
|
+ * Total size of the linked list.
|
|
|
*/
|
|
|
- private static final int PREFETCH_WRITE_LOCK_TIMEOUT = 5;
|
|
|
+ private int entryListSize;
|
|
|
|
|
|
/**
|
|
|
- * Lock timeout unit to be used by the thread while acquiring prefetch block write lock.
|
|
|
+ * Number of times a block was read from this cache.
|
|
|
+ * Used for determining cache utilization factor.
|
|
|
*/
|
|
|
- private static final TimeUnit PREFETCH_WRITE_LOCK_TIMEOUT_UNIT = TimeUnit.SECONDS;
|
|
|
+ private int numGets = 0;
|
|
|
+
|
|
|
+ private final AtomicBoolean closed;
|
|
|
+
|
|
|
+ private final PrefetchingStatistics prefetchingStatistics;
|
|
|
|
|
|
/**
|
|
|
* File attributes attached to any intermediate temporary file created during index creation.
|
|
@@ -103,6 +119,8 @@ public class SingleFilePerBlockCache implements BlockCache {
|
|
|
READ,
|
|
|
WRITE
|
|
|
}
|
|
|
+ private Entry previous;
|
|
|
+ private Entry next;
|
|
|
|
|
|
Entry(int blockNumber, Path path, int size, long checksum) {
|
|
|
this.blockNumber = blockNumber;
|
|
@@ -110,6 +128,8 @@ public class SingleFilePerBlockCache implements BlockCache {
|
|
|
this.size = size;
|
|
|
this.checksum = checksum;
|
|
|
this.lock = new ReentrantReadWriteLock();
|
|
|
+ this.previous = null;
|
|
|
+ this.next = null;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -166,16 +186,37 @@ public class SingleFilePerBlockCache implements BlockCache {
|
|
|
}
|
|
|
return false;
|
|
|
}
|
|
|
+
|
|
|
+ private Entry getPrevious() {
|
|
|
+ return previous;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setPrevious(Entry previous) {
|
|
|
+ this.previous = previous;
|
|
|
+ }
|
|
|
+
|
|
|
+ private Entry getNext() {
|
|
|
+ return next;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setNext(Entry next) {
|
|
|
+ this.next = next;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Constructs an instance of a {@code SingleFilePerBlockCache}.
|
|
|
*
|
|
|
* @param prefetchingStatistics statistics for this stream.
|
|
|
+ * @param maxBlocksCount max blocks count to be kept in cache at any time.
|
|
|
*/
|
|
|
- public SingleFilePerBlockCache(PrefetchingStatistics prefetchingStatistics) {
|
|
|
+ public SingleFilePerBlockCache(PrefetchingStatistics prefetchingStatistics, int maxBlocksCount) {
|
|
|
this.prefetchingStatistics = requireNonNull(prefetchingStatistics);
|
|
|
this.closed = new AtomicBoolean(false);
|
|
|
+ this.maxBlocksCount = maxBlocksCount;
|
|
|
+ Preconditions.checkArgument(maxBlocksCount > 0, "maxBlocksCount should be more than 0");
|
|
|
+ blocks = new ConcurrentHashMap<>();
|
|
|
+ blocksLock = new ReentrantReadWriteLock();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -247,9 +288,60 @@ public class SingleFilePerBlockCache implements BlockCache {
|
|
|
throw new IllegalStateException(String.format("block %d not found in cache", blockNumber));
|
|
|
}
|
|
|
numGets++;
|
|
|
+ addToLinkedListHead(entry);
|
|
|
return entry;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Helper method to add the given entry to the head of the linked list.
|
|
|
+ *
|
|
|
+ * @param entry Block entry to add.
|
|
|
+ */
|
|
|
+ private void addToLinkedListHead(Entry entry) {
|
|
|
+ blocksLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ addToHeadOfLinkedList(entry);
|
|
|
+ } finally {
|
|
|
+ blocksLock.writeLock().unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add the given entry to the head of the linked list.
|
|
|
+ *
|
|
|
+ * @param entry Block entry to add.
|
|
|
+ */
|
|
|
+ private void addToHeadOfLinkedList(Entry entry) {
|
|
|
+ if (head == null) {
|
|
|
+ head = entry;
|
|
|
+ tail = entry;
|
|
|
+ }
|
|
|
+ LOG.debug(
|
|
|
+ "Block num {} to be added to the head. Current head block num: {} and tail block num: {}",
|
|
|
+ entry.blockNumber, head.blockNumber, tail.blockNumber);
|
|
|
+ if (entry != head) {
|
|
|
+ Entry prev = entry.getPrevious();
|
|
|
+ Entry nxt = entry.getNext();
|
|
|
+ // no-op if the block is already evicted
|
|
|
+ if (!blocks.containsKey(entry.blockNumber)) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ if (prev != null) {
|
|
|
+ prev.setNext(nxt);
|
|
|
+ }
|
|
|
+ if (nxt != null) {
|
|
|
+ nxt.setPrevious(prev);
|
|
|
+ }
|
|
|
+ entry.setPrevious(null);
|
|
|
+ entry.setNext(head);
|
|
|
+ head.setPrevious(entry);
|
|
|
+ head = entry;
|
|
|
+ if (prev != null && prev.getNext() == null) {
|
|
|
+ tail = prev;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Puts the given block in this cache.
|
|
|
*
|
|
@@ -278,6 +370,7 @@ public class SingleFilePerBlockCache implements BlockCache {
|
|
|
} finally {
|
|
|
entry.releaseLock(Entry.LockType.READ);
|
|
|
}
|
|
|
+ addToLinkedListHead(entry);
|
|
|
return;
|
|
|
}
|
|
|
|
|
@@ -299,9 +392,65 @@ public class SingleFilePerBlockCache implements BlockCache {
|
|
|
// Update stream_read_blocks_in_cache stats only after blocks map is updated with new file
|
|
|
// entry to avoid any discrepancy related to the value of stream_read_blocks_in_cache.
|
|
|
// If stream_read_blocks_in_cache is updated before updating the blocks map here, closing of
|
|
|
- // the input stream can lead to the removal of the cache file even before blocks is added with
|
|
|
- // the new cache file, leading to incorrect value of stream_read_blocks_in_cache.
|
|
|
+ // the input stream can lead to the removal of the cache file even before blocks is added
|
|
|
+ // with the new cache file, leading to incorrect value of stream_read_blocks_in_cache.
|
|
|
prefetchingStatistics.blockAddedToFileCache();
|
|
|
+ addToLinkedListAndEvictIfRequired(entry);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add the given entry to the head of the linked list and if the LRU cache size
|
|
|
+ * exceeds the max limit, evict tail of the LRU linked list.
|
|
|
+ *
|
|
|
+ * @param entry Block entry to add.
|
|
|
+ */
|
|
|
+ private void addToLinkedListAndEvictIfRequired(Entry entry) {
|
|
|
+ blocksLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ addToHeadOfLinkedList(entry);
|
|
|
+ entryListSize++;
|
|
|
+ if (entryListSize > maxBlocksCount && !closed.get()) {
|
|
|
+ Entry elementToPurge = tail;
|
|
|
+ tail = tail.getPrevious();
|
|
|
+ if (tail == null) {
|
|
|
+ tail = head;
|
|
|
+ }
|
|
|
+ tail.setNext(null);
|
|
|
+ elementToPurge.setPrevious(null);
|
|
|
+ deleteBlockFileAndEvictCache(elementToPurge);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ blocksLock.writeLock().unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Delete cache file as part of the block cache LRU eviction.
|
|
|
+ *
|
|
|
+ * @param elementToPurge Block entry to evict.
|
|
|
+ */
|
|
|
+ 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,
|
|
|
+ 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);
|
|
|
+ }
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Failed to delete cache file {}", elementToPurge.path, e);
|
|
|
+ } finally {
|
|
|
+ elementToPurge.releaseLock(Entry.LockType.WRITE);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private static final Set<? extends OpenOption> CREATE_OPTIONS =
|
|
@@ -337,30 +486,38 @@ public class SingleFilePerBlockCache implements BlockCache {
|
|
|
public void close() throws IOException {
|
|
|
if (closed.compareAndSet(false, true)) {
|
|
|
LOG.debug(getStats());
|
|
|
- int numFilesDeleted = 0;
|
|
|
-
|
|
|
- for (Entry entry : blocks.values()) {
|
|
|
- boolean lockAcquired = entry.takeLock(Entry.LockType.WRITE, PREFETCH_WRITE_LOCK_TIMEOUT,
|
|
|
- PREFETCH_WRITE_LOCK_TIMEOUT_UNIT);
|
|
|
- if (!lockAcquired) {
|
|
|
- LOG.error("Cache file {} deletion would not be attempted as write lock could not"
|
|
|
- + " be acquired within {} {}", entry.path, PREFETCH_WRITE_LOCK_TIMEOUT,
|
|
|
- PREFETCH_WRITE_LOCK_TIMEOUT_UNIT);
|
|
|
- continue;
|
|
|
- }
|
|
|
- try {
|
|
|
- Files.deleteIfExists(entry.path);
|
|
|
+ deleteCacheFiles();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Delete cache files as part of the close call.
|
|
|
+ */
|
|
|
+ private void deleteCacheFiles() {
|
|
|
+ int numFilesDeleted = 0;
|
|
|
+ for (Entry entry : blocks.values()) {
|
|
|
+ boolean lockAcquired =
|
|
|
+ entry.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 {} {}", entry.path,
|
|
|
+ PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT,
|
|
|
+ PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT_UNIT);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ if (Files.deleteIfExists(entry.path)) {
|
|
|
prefetchingStatistics.blockRemovedFromFileCache();
|
|
|
numFilesDeleted++;
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.warn("Failed to delete cache file {}", entry.path, e);
|
|
|
- } finally {
|
|
|
- entry.releaseLock(Entry.LockType.WRITE);
|
|
|
}
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Failed to delete cache file {}", entry.path, e);
|
|
|
+ } finally {
|
|
|
+ entry.releaseLock(Entry.LockType.WRITE);
|
|
|
}
|
|
|
-
|
|
|
- LOG.debug("Prefetch cache close: Deleted {} cache files", numFilesDeleted);
|
|
|
}
|
|
|
+ LOG.debug("Prefetch cache close: Deleted {} cache files", numFilesDeleted);
|
|
|
}
|
|
|
|
|
|
@Override
|