|
@@ -21,7 +21,10 @@ package org.apache.hadoop.fs.s3a.s3guard;
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
|
|
|
+import com.google.common.cache.Cache;
|
|
|
+import com.google.common.cache.CacheBuilder;
|
|
|
import org.apache.commons.lang.StringUtils;
|
|
|
+import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
@@ -37,6 +40,7 @@ import java.util.HashMap;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.LinkedList;
|
|
|
import java.util.Map;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
/**
|
|
|
* This is a local, in-memory, implementation of MetadataStore.
|
|
@@ -51,24 +55,35 @@ import java.util.Map;
|
|
|
* This MetadataStore does not enforce filesystem rules such as disallowing
|
|
|
* non-recursive removal of non-empty directories. It is assumed the caller
|
|
|
* already has to perform these sorts of checks.
|
|
|
+ *
|
|
|
+ * Contains cache internally with time based eviction.
|
|
|
*/
|
|
|
public class LocalMetadataStore implements MetadataStore {
|
|
|
|
|
|
public static final Logger LOG = LoggerFactory.getLogger(MetadataStore.class);
|
|
|
- // TODO HADOOP-13649: use time instead of capacity for eviction.
|
|
|
public static final int DEFAULT_MAX_RECORDS = 128;
|
|
|
+ public static final int DEFAULT_CACHE_ENTRY_TTL_MSEC = 10 * 1000;
|
|
|
|
|
|
/**
|
|
|
* Maximum number of records.
|
|
|
*/
|
|
|
+ @InterfaceStability.Evolving
|
|
|
public static final String CONF_MAX_RECORDS =
|
|
|
"fs.metadatastore.local.max_records";
|
|
|
|
|
|
+ /**
|
|
|
+ * Time to live in milliseconds. If zero, time-based expiration is
|
|
|
+ * disabled.
|
|
|
+ */
|
|
|
+ @InterfaceStability.Evolving
|
|
|
+ public static final String CONF_CACHE_ENTRY_TTL =
|
|
|
+ "fs.metadatastore.local.ttl";
|
|
|
+
|
|
|
/** Contains directories and files. */
|
|
|
- private LruHashMap<Path, PathMetadata> fileHash;
|
|
|
+ private Cache<Path, PathMetadata> fileCache;
|
|
|
|
|
|
/** Contains directory listings. */
|
|
|
- private LruHashMap<Path, DirListingMetadata> dirHash;
|
|
|
+ private Cache<Path, DirListingMetadata> dirCache;
|
|
|
|
|
|
private FileSystem fs;
|
|
|
/* Null iff this FS does not have an associated URI host. */
|
|
@@ -94,9 +109,15 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
if (maxRecords < 4) {
|
|
|
maxRecords = 4;
|
|
|
}
|
|
|
- // Start w/ less than max capacity. Space / time trade off.
|
|
|
- fileHash = new LruHashMap<>(maxRecords/2, maxRecords);
|
|
|
- dirHash = new LruHashMap<>(maxRecords/4, maxRecords);
|
|
|
+ int ttl = conf.getInt(CONF_CACHE_ENTRY_TTL, DEFAULT_CACHE_ENTRY_TTL_MSEC);
|
|
|
+
|
|
|
+ CacheBuilder builder = CacheBuilder.newBuilder().maximumSize(maxRecords);
|
|
|
+ if (ttl >= 0) {
|
|
|
+ builder.expireAfterAccess(ttl, TimeUnit.MILLISECONDS);
|
|
|
+ }
|
|
|
+
|
|
|
+ fileCache = builder.build();
|
|
|
+ dirCache = builder.build();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -130,12 +151,12 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
|
|
|
// Delete entry from file cache, then from cached parent directory, if any
|
|
|
|
|
|
- deleteHashEntries(path, tombstone);
|
|
|
+ deleteCacheEntries(path, tombstone);
|
|
|
|
|
|
if (recursive) {
|
|
|
// Remove all entries that have this dir as path prefix.
|
|
|
- deleteHashByAncestor(path, dirHash, tombstone);
|
|
|
- deleteHashByAncestor(path, fileHash, tombstone);
|
|
|
+ deleteEntryByAncestor(path, dirCache, tombstone);
|
|
|
+ deleteEntryByAncestor(path, fileCache, tombstone);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -149,7 +170,7 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
throws IOException {
|
|
|
Path path = standardize(p);
|
|
|
synchronized (this) {
|
|
|
- PathMetadata m = fileHash.mruGet(path);
|
|
|
+ PathMetadata m = fileCache.getIfPresent(path);
|
|
|
|
|
|
if (wantEmptyDirectoryFlag && m != null &&
|
|
|
m.getFileStatus().isDirectory()) {
|
|
@@ -170,7 +191,7 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
* @return TRUE / FALSE if known empty / not-empty, UNKNOWN otherwise.
|
|
|
*/
|
|
|
private Tristate isEmptyDirectory(Path p) {
|
|
|
- DirListingMetadata dirMeta = dirHash.get(p);
|
|
|
+ DirListingMetadata dirMeta = dirCache.getIfPresent(p);
|
|
|
return dirMeta.withoutTombstones().isEmpty();
|
|
|
}
|
|
|
|
|
@@ -178,7 +199,7 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
public synchronized DirListingMetadata listChildren(Path p) throws
|
|
|
IOException {
|
|
|
Path path = standardize(p);
|
|
|
- DirListingMetadata listing = dirHash.mruGet(path);
|
|
|
+ DirListingMetadata listing = dirCache.getIfPresent(path);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("listChildren({}) -> {}", path,
|
|
|
listing == null ? "null" : listing.prettyPrint());
|
|
@@ -237,10 +258,10 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("put {} -> {}", path, meta.prettyPrint());
|
|
|
}
|
|
|
- fileHash.put(path, meta);
|
|
|
+ fileCache.put(path, meta);
|
|
|
|
|
|
/* Directory case:
|
|
|
- * We also make sure we have an entry in the dirHash, so subsequent
|
|
|
+ * We also make sure we have an entry in the dirCache, so subsequent
|
|
|
* listStatus(path) at least see the directory.
|
|
|
*
|
|
|
* If we had a boolean flag argument "isNew", we would know whether this
|
|
@@ -251,9 +272,9 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
*/
|
|
|
|
|
|
if (status.isDirectory()) {
|
|
|
- DirListingMetadata dir = dirHash.mruGet(path);
|
|
|
+ DirListingMetadata dir = dirCache.getIfPresent(path);
|
|
|
if (dir == null) {
|
|
|
- dirHash.put(path, new DirListingMetadata(path, DirListingMetadata
|
|
|
+ dirCache.put(path, new DirListingMetadata(path, DirListingMetadata
|
|
|
.EMPTY_DIR, false));
|
|
|
}
|
|
|
}
|
|
@@ -261,14 +282,14 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
/* Update cached parent dir. */
|
|
|
Path parentPath = path.getParent();
|
|
|
if (parentPath != null) {
|
|
|
- DirListingMetadata parent = dirHash.mruGet(parentPath);
|
|
|
+ DirListingMetadata parent = dirCache.getIfPresent(parentPath);
|
|
|
if (parent == null) {
|
|
|
/* Track this new file's listing in parent. Parent is not
|
|
|
* authoritative, since there may be other items in it we don't know
|
|
|
* about. */
|
|
|
parent = new DirListingMetadata(parentPath,
|
|
|
DirListingMetadata.EMPTY_DIR, false);
|
|
|
- dirHash.put(parentPath, parent);
|
|
|
+ dirCache.put(parentPath, parent);
|
|
|
}
|
|
|
parent.put(status);
|
|
|
}
|
|
@@ -280,7 +301,7 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("put dirMeta {}", meta.prettyPrint());
|
|
|
}
|
|
|
- dirHash.put(standardize(meta.getPath()), meta);
|
|
|
+ dirCache.put(standardize(meta.getPath()), meta);
|
|
|
put(meta.getListing());
|
|
|
}
|
|
|
|
|
@@ -298,8 +319,8 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
|
|
|
@Override
|
|
|
public void destroy() throws IOException {
|
|
|
- if (dirHash != null) {
|
|
|
- dirHash.clear();
|
|
|
+ if (dirCache != null) {
|
|
|
+ dirCache.invalidateAll();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -312,7 +333,7 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
public synchronized void prune(long modTime, String keyPrefix)
|
|
|
throws IOException {
|
|
|
Iterator<Map.Entry<Path, PathMetadata>> files =
|
|
|
- fileHash.entrySet().iterator();
|
|
|
+ fileCache.asMap().entrySet().iterator();
|
|
|
while (files.hasNext()) {
|
|
|
Map.Entry<Path, PathMetadata> entry = files.next();
|
|
|
if (expired(entry.getValue().getFileStatus(), modTime, keyPrefix)) {
|
|
@@ -320,7 +341,7 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
}
|
|
|
Iterator<Map.Entry<Path, DirListingMetadata>> dirs =
|
|
|
- dirHash.entrySet().iterator();
|
|
|
+ dirCache.asMap().entrySet().iterator();
|
|
|
while (dirs.hasNext()) {
|
|
|
Map.Entry<Path, DirListingMetadata> entry = dirs.next();
|
|
|
Path path = entry.getKey();
|
|
@@ -335,9 +356,10 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
}
|
|
|
if (newChildren.size() != oldChildren.size()) {
|
|
|
- dirHash.put(path, new DirListingMetadata(path, newChildren, false));
|
|
|
+ dirCache.put(path, new DirListingMetadata(path, newChildren, false));
|
|
|
if (!path.isRoot()) {
|
|
|
- DirListingMetadata parent = dirHash.get(path.getParent());
|
|
|
+ DirListingMetadata parent = null;
|
|
|
+ parent = dirCache.getIfPresent(path.getParent());
|
|
|
if (parent != null) {
|
|
|
parent.setAuthoritative(false);
|
|
|
}
|
|
@@ -354,9 +376,9 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- static <T> void deleteHashByAncestor(Path ancestor, Map<Path, T> hash,
|
|
|
+ static <T> void deleteEntryByAncestor(Path ancestor, Cache<Path, T> cache,
|
|
|
boolean tombstone) {
|
|
|
- for (Iterator<Map.Entry<Path, T>> it = hash.entrySet().iterator();
|
|
|
+ for (Iterator<Map.Entry<Path, T>> it = cache.asMap().entrySet().iterator();
|
|
|
it.hasNext();) {
|
|
|
Map.Entry<Path, T> entry = it.next();
|
|
|
Path f = entry.getKey();
|
|
@@ -364,11 +386,11 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
if (isAncestorOf(ancestor, f)) {
|
|
|
if (tombstone) {
|
|
|
if (meta instanceof PathMetadata) {
|
|
|
- entry.setValue((T) PathMetadata.tombstone(f));
|
|
|
+ cache.put(f, (T) PathMetadata.tombstone(f));
|
|
|
} else if (meta instanceof DirListingMetadata) {
|
|
|
it.remove();
|
|
|
} else {
|
|
|
- throw new IllegalStateException("Unknown type in hash");
|
|
|
+ throw new IllegalStateException("Unknown type in cache");
|
|
|
}
|
|
|
} else {
|
|
|
it.remove();
|
|
@@ -391,17 +413,17 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Update fileHash and dirHash to reflect deletion of file 'f'. Call with
|
|
|
+ * Update fileCache and dirCache to reflect deletion of file 'f'. Call with
|
|
|
* lock held.
|
|
|
*/
|
|
|
- private void deleteHashEntries(Path path, boolean tombstone) {
|
|
|
+ private void deleteCacheEntries(Path path, boolean tombstone) {
|
|
|
|
|
|
// Remove target file/dir
|
|
|
LOG.debug("delete file entry for {}", path);
|
|
|
if (tombstone) {
|
|
|
- fileHash.put(path, PathMetadata.tombstone(path));
|
|
|
+ fileCache.put(path, PathMetadata.tombstone(path));
|
|
|
} else {
|
|
|
- fileHash.remove(path);
|
|
|
+ fileCache.invalidate(path);
|
|
|
}
|
|
|
|
|
|
// Update this and parent dir listing, if any
|
|
@@ -409,12 +431,13 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
/* If this path is a dir, remove its listing */
|
|
|
LOG.debug("removing listing of {}", path);
|
|
|
|
|
|
- dirHash.remove(path);
|
|
|
+ dirCache.invalidate(path);
|
|
|
|
|
|
/* Remove this path from parent's dir listing */
|
|
|
Path parent = path.getParent();
|
|
|
if (parent != null) {
|
|
|
- DirListingMetadata dir = dirHash.get(parent);
|
|
|
+ DirListingMetadata dir = null;
|
|
|
+ dir = dirCache.getIfPresent(parent);
|
|
|
if (dir != null) {
|
|
|
LOG.debug("removing parent's entry for {} ", path);
|
|
|
if (tombstone) {
|