|
@@ -37,13 +37,12 @@ import java.io.IOException;
|
|
|
import java.net.URI;
|
|
|
import java.util.Collection;
|
|
|
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.
|
|
|
+ * This is a local, in-memory implementation of MetadataStore.
|
|
|
* This is <i>not</i> a coherent cache across processes. It is only
|
|
|
* locally-coherent.
|
|
|
*
|
|
@@ -56,12 +55,12 @@ import java.util.concurrent.TimeUnit;
|
|
|
* 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.
|
|
|
+ * Contains one cache internally with time based eviction.
|
|
|
*/
|
|
|
public class LocalMetadataStore implements MetadataStore {
|
|
|
|
|
|
public static final Logger LOG = LoggerFactory.getLogger(MetadataStore.class);
|
|
|
- public static final int DEFAULT_MAX_RECORDS = 128;
|
|
|
+ public static final int DEFAULT_MAX_RECORDS = 256;
|
|
|
public static final int DEFAULT_CACHE_ENTRY_TTL_MSEC = 10 * 1000;
|
|
|
|
|
|
/**
|
|
@@ -79,11 +78,8 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
public static final String CONF_CACHE_ENTRY_TTL =
|
|
|
"fs.metadatastore.local.ttl";
|
|
|
|
|
|
- /** Contains directories and files. */
|
|
|
- private Cache<Path, PathMetadata> fileCache;
|
|
|
-
|
|
|
- /** Contains directory listings. */
|
|
|
- private Cache<Path, DirListingMetadata> dirCache;
|
|
|
+ /** Contains directory and file listings. */
|
|
|
+ private Cache<Path, LocalMetadataEntry> localCache;
|
|
|
|
|
|
private FileSystem fs;
|
|
|
/* Null iff this FS does not have an associated URI host. */
|
|
@@ -116,8 +112,7 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
builder.expireAfterAccess(ttl, TimeUnit.MILLISECONDS);
|
|
|
}
|
|
|
|
|
|
- fileCache = builder.build();
|
|
|
- dirCache = builder.build();
|
|
|
+ localCache = builder.build();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -155,8 +150,7 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
|
|
|
if (recursive) {
|
|
|
// Remove all entries that have this dir as path prefix.
|
|
|
- deleteEntryByAncestor(path, dirCache, tombstone);
|
|
|
- deleteEntryByAncestor(path, fileCache, tombstone);
|
|
|
+ deleteEntryByAncestor(path, localCache, tombstone);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -170,7 +164,7 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
throws IOException {
|
|
|
Path path = standardize(p);
|
|
|
synchronized (this) {
|
|
|
- PathMetadata m = fileCache.getIfPresent(path);
|
|
|
+ PathMetadata m = getFileMeta(path);
|
|
|
|
|
|
if (wantEmptyDirectoryFlag && m != null &&
|
|
|
m.getFileStatus().isDirectory()) {
|
|
@@ -191,15 +185,15 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
* @return TRUE / FALSE if known empty / not-empty, UNKNOWN otherwise.
|
|
|
*/
|
|
|
private Tristate isEmptyDirectory(Path p) {
|
|
|
- DirListingMetadata dirMeta = dirCache.getIfPresent(p);
|
|
|
- return dirMeta.withoutTombstones().isEmpty();
|
|
|
+ DirListingMetadata dlm = getDirListingMeta(p);
|
|
|
+ return dlm.withoutTombstones().isEmpty();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public synchronized DirListingMetadata listChildren(Path p) throws
|
|
|
IOException {
|
|
|
Path path = standardize(p);
|
|
|
- DirListingMetadata listing = dirCache.getIfPresent(path);
|
|
|
+ DirListingMetadata listing = getDirListingMeta(path);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("listChildren({}) -> {}", path,
|
|
|
listing == null ? "null" : listing.prettyPrint());
|
|
@@ -211,6 +205,7 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
@Override
|
|
|
public void move(Collection<Path> pathsToDelete,
|
|
|
Collection<PathMetadata> pathsToCreate) throws IOException {
|
|
|
+ LOG.info("Move {} to {}", pathsToDelete, pathsToCreate);
|
|
|
|
|
|
Preconditions.checkNotNull(pathsToDelete, "pathsToDelete is null");
|
|
|
Preconditions.checkNotNull(pathsToCreate, "pathsToCreate is null");
|
|
@@ -258,7 +253,12 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("put {} -> {}", path, meta.prettyPrint());
|
|
|
}
|
|
|
- fileCache.put(path, meta);
|
|
|
+ LocalMetadataEntry entry = localCache.getIfPresent(path);
|
|
|
+ if(entry == null){
|
|
|
+ entry = new LocalMetadataEntry(meta);
|
|
|
+ } else {
|
|
|
+ entry.setPathMetadata(meta);
|
|
|
+ }
|
|
|
|
|
|
/* Directory case:
|
|
|
* We also make sure we have an entry in the dirCache, so subsequent
|
|
@@ -271,27 +271,32 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
* saving round trips to underlying store for subsequent listStatus()
|
|
|
*/
|
|
|
|
|
|
- if (status.isDirectory()) {
|
|
|
- DirListingMetadata dir = dirCache.getIfPresent(path);
|
|
|
- if (dir == null) {
|
|
|
- dirCache.put(path, new DirListingMetadata(path, DirListingMetadata
|
|
|
- .EMPTY_DIR, false));
|
|
|
- }
|
|
|
+ // only create DirListingMetadata if the entry does not have one
|
|
|
+ if (status.isDirectory() && !entry.hasDirMeta()) {
|
|
|
+ DirListingMetadata dlm =
|
|
|
+ new DirListingMetadata(path, DirListingMetadata.EMPTY_DIR, false);
|
|
|
+ entry.setDirListingMetadata(dlm);
|
|
|
}
|
|
|
+ localCache.put(path, entry);
|
|
|
|
|
|
/* Update cached parent dir. */
|
|
|
Path parentPath = path.getParent();
|
|
|
if (parentPath != null) {
|
|
|
- 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);
|
|
|
- dirCache.put(parentPath, parent);
|
|
|
+ LocalMetadataEntry parentMeta = localCache.getIfPresent(parentPath);
|
|
|
+ DirListingMetadata parentDirMeta =
|
|
|
+ new DirListingMetadata(parentPath, DirListingMetadata.EMPTY_DIR,
|
|
|
+ false);
|
|
|
+ parentDirMeta.put(status);
|
|
|
+
|
|
|
+ getDirListingMeta(parentPath);
|
|
|
+
|
|
|
+ if (parentMeta == null){
|
|
|
+ localCache.put(parentPath, new LocalMetadataEntry(parentDirMeta));
|
|
|
+ } else if (!parentMeta.hasDirMeta()) {
|
|
|
+ parentMeta.setDirListingMetadata(parentDirMeta);
|
|
|
+ } else {
|
|
|
+ parentMeta.getDirListingMeta().put(status);
|
|
|
}
|
|
|
- parent.put(status);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -301,7 +306,13 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("put dirMeta {}", meta.prettyPrint());
|
|
|
}
|
|
|
- dirCache.put(standardize(meta.getPath()), meta);
|
|
|
+ LocalMetadataEntry entry =
|
|
|
+ localCache.getIfPresent(standardize(meta.getPath()));
|
|
|
+ if(entry == null){
|
|
|
+ localCache.put(standardize(meta.getPath()), new LocalMetadataEntry(meta));
|
|
|
+ } else {
|
|
|
+ entry.setDirListingMetadata(meta);
|
|
|
+ }
|
|
|
put(meta.getListing());
|
|
|
}
|
|
|
|
|
@@ -319,8 +330,8 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
|
|
|
@Override
|
|
|
public void destroy() throws IOException {
|
|
|
- if (dirCache != null) {
|
|
|
- dirCache.invalidateAll();
|
|
|
+ if (localCache != null) {
|
|
|
+ localCache.invalidateAll();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -330,42 +341,44 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized void prune(long modTime, String keyPrefix)
|
|
|
- throws IOException {
|
|
|
- Iterator<Map.Entry<Path, PathMetadata>> files =
|
|
|
- fileCache.asMap().entrySet().iterator();
|
|
|
- while (files.hasNext()) {
|
|
|
- Map.Entry<Path, PathMetadata> entry = files.next();
|
|
|
- if (expired(entry.getValue().getFileStatus(), modTime, keyPrefix)) {
|
|
|
- files.remove();
|
|
|
- }
|
|
|
- }
|
|
|
- Iterator<Map.Entry<Path, DirListingMetadata>> dirs =
|
|
|
- dirCache.asMap().entrySet().iterator();
|
|
|
- while (dirs.hasNext()) {
|
|
|
- Map.Entry<Path, DirListingMetadata> entry = dirs.next();
|
|
|
- Path path = entry.getKey();
|
|
|
- DirListingMetadata metadata = entry.getValue();
|
|
|
- Collection<PathMetadata> oldChildren = metadata.getListing();
|
|
|
- Collection<PathMetadata> newChildren = new LinkedList<>();
|
|
|
-
|
|
|
- for (PathMetadata child : oldChildren) {
|
|
|
- FileStatus status = child.getFileStatus();
|
|
|
- if (!expired(status, modTime, keyPrefix)) {
|
|
|
- newChildren.add(child);
|
|
|
- }
|
|
|
- }
|
|
|
- if (newChildren.size() != oldChildren.size()) {
|
|
|
- dirCache.put(path, new DirListingMetadata(path, newChildren, false));
|
|
|
- if (!path.isRoot()) {
|
|
|
- DirListingMetadata parent = null;
|
|
|
- parent = dirCache.getIfPresent(path.getParent());
|
|
|
- if (parent != null) {
|
|
|
- parent.setAuthoritative(false);
|
|
|
+ public synchronized void prune(long modTime, String keyPrefix) {
|
|
|
+ // prune files
|
|
|
+ // filter path_metadata (files), filter expired, remove expired
|
|
|
+ localCache.asMap().entrySet().stream()
|
|
|
+ .filter(entry -> entry.getValue().hasPathMeta())
|
|
|
+ .filter(entry -> expired(
|
|
|
+ entry.getValue().getFileMeta().getFileStatus(), modTime, keyPrefix))
|
|
|
+ .forEach(entry -> localCache.invalidate(entry.getKey()));
|
|
|
+
|
|
|
+
|
|
|
+ // prune dirs
|
|
|
+ // filter DIR_LISTING_METADATA, remove expired, remove authoritative bit
|
|
|
+ localCache.asMap().entrySet().stream()
|
|
|
+ .filter(entry -> entry.getValue().hasDirMeta())
|
|
|
+ .forEach(entry -> {
|
|
|
+ Path path = entry.getKey();
|
|
|
+ DirListingMetadata metadata = entry.getValue().getDirListingMeta();
|
|
|
+ Collection<PathMetadata> oldChildren = metadata.getListing();
|
|
|
+ Collection<PathMetadata> newChildren = new LinkedList<>();
|
|
|
+
|
|
|
+ for (PathMetadata child : oldChildren) {
|
|
|
+ FileStatus status = child.getFileStatus();
|
|
|
+ if (!expired(status, modTime, keyPrefix)) {
|
|
|
+ newChildren.add(child);
|
|
|
+ }
|
|
|
}
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
+ if (newChildren.size() != oldChildren.size()) {
|
|
|
+ DirListingMetadata dlm =
|
|
|
+ new DirListingMetadata(path, newChildren, false);
|
|
|
+ localCache.put(path, new LocalMetadataEntry(dlm));
|
|
|
+ if (!path.isRoot()) {
|
|
|
+ DirListingMetadata parent = getDirListingMeta(path.getParent());
|
|
|
+ if (parent != null) {
|
|
|
+ parent.setAuthoritative(false);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ });
|
|
|
}
|
|
|
|
|
|
private boolean expired(FileStatus status, long expiry, String keyPrefix) {
|
|
@@ -390,31 +403,26 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- static <T> void deleteEntryByAncestor(Path ancestor, Cache<Path, T> cache,
|
|
|
- boolean tombstone) {
|
|
|
- for (Iterator<Map.Entry<Path, T>> it = cache.asMap().entrySet().iterator();
|
|
|
- it.hasNext();) {
|
|
|
- Map.Entry<Path, T> entry = it.next();
|
|
|
- Path f = entry.getKey();
|
|
|
- T meta = entry.getValue();
|
|
|
- if (isAncestorOf(ancestor, f)) {
|
|
|
- if (tombstone) {
|
|
|
- if (meta instanceof PathMetadata) {
|
|
|
- cache.put(f, (T) PathMetadata.tombstone(f));
|
|
|
- } else if (meta instanceof DirListingMetadata) {
|
|
|
- it.remove();
|
|
|
+ static void deleteEntryByAncestor(Path ancestor,
|
|
|
+ Cache<Path, LocalMetadataEntry> cache, boolean tombstone) {
|
|
|
+
|
|
|
+ cache.asMap().entrySet().stream()
|
|
|
+ .filter(entry -> isAncestorOf(ancestor, entry.getKey()))
|
|
|
+ .forEach(entry -> {
|
|
|
+ LocalMetadataEntry meta = entry.getValue();
|
|
|
+ Path path = entry.getKey();
|
|
|
+ if(meta.hasDirMeta()){
|
|
|
+ cache.invalidate(path);
|
|
|
+ } else if(tombstone && meta.hasPathMeta()){
|
|
|
+ meta.setPathMetadata(PathMetadata.tombstone(path));
|
|
|
} else {
|
|
|
- throw new IllegalStateException("Unknown type in cache");
|
|
|
+ cache.invalidate(path);
|
|
|
}
|
|
|
- } else {
|
|
|
- it.remove();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
+ });
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * @return true iff 'ancestor' is ancestor dir in path 'f'.
|
|
|
+ * @return true if 'ancestor' is ancestor dir in path 'f'.
|
|
|
* All paths here are absolute. Dir does not count as its own ancestor.
|
|
|
*/
|
|
|
private static boolean isAncestorOf(Path ancestor, Path f) {
|
|
@@ -431,27 +439,41 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
* lock held.
|
|
|
*/
|
|
|
private void deleteCacheEntries(Path path, boolean tombstone) {
|
|
|
+ LocalMetadataEntry entry = localCache.getIfPresent(path);
|
|
|
+ // If there's no entry, delete should silently succeed
|
|
|
+ // (based on MetadataStoreTestBase#testDeleteNonExisting)
|
|
|
+ if(entry == null){
|
|
|
+ LOG.warn("Delete: path {} is missing from cache.", path);
|
|
|
+ return;
|
|
|
+ }
|
|
|
|
|
|
- // Remove target file/dir
|
|
|
+ // Remove target file entry
|
|
|
LOG.debug("delete file entry for {}", path);
|
|
|
- if (tombstone) {
|
|
|
- fileCache.put(path, PathMetadata.tombstone(path));
|
|
|
- } else {
|
|
|
- fileCache.invalidate(path);
|
|
|
+ if(entry.hasPathMeta()){
|
|
|
+ if (tombstone) {
|
|
|
+ PathMetadata pmd = PathMetadata.tombstone(path);
|
|
|
+ entry.setPathMetadata(pmd);
|
|
|
+ } else {
|
|
|
+ entry.setPathMetadata(null);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- // Update this and parent dir listing, if any
|
|
|
-
|
|
|
- /* If this path is a dir, remove its listing */
|
|
|
- LOG.debug("removing listing of {}", path);
|
|
|
+ // If this path is a dir, remove its listing
|
|
|
+ if(entry.hasDirMeta()) {
|
|
|
+ LOG.debug("removing listing of {}", path);
|
|
|
+ entry.setDirListingMetadata(null);
|
|
|
+ }
|
|
|
|
|
|
- dirCache.invalidate(path);
|
|
|
+ // If the entry is empty (contains no dirMeta or pathMeta) remove it from
|
|
|
+ // the cache.
|
|
|
+ if(!entry.hasDirMeta() && !entry.hasPathMeta()){
|
|
|
+ localCache.invalidate(entry);
|
|
|
+ }
|
|
|
|
|
|
/* Remove this path from parent's dir listing */
|
|
|
Path parent = path.getParent();
|
|
|
if (parent != null) {
|
|
|
- DirListingMetadata dir = null;
|
|
|
- dir = dirCache.getIfPresent(parent);
|
|
|
+ DirListingMetadata dir = getDirListingMeta(parent);
|
|
|
if (dir != null) {
|
|
|
LOG.debug("removing parent's entry for {} ", path);
|
|
|
if (tombstone) {
|
|
@@ -494,4 +516,23 @@ public class LocalMetadataStore implements MetadataStore {
|
|
|
public void updateParameters(Map<String, String> parameters)
|
|
|
throws IOException {
|
|
|
}
|
|
|
+
|
|
|
+ PathMetadata getFileMeta(Path p){
|
|
|
+ LocalMetadataEntry entry = localCache.getIfPresent(p);
|
|
|
+ if(entry != null && entry.hasPathMeta()){
|
|
|
+ return entry.getFileMeta();
|
|
|
+ } else {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ DirListingMetadata getDirListingMeta(Path p){
|
|
|
+ LocalMetadataEntry entry = localCache.getIfPresent(p);
|
|
|
+ if(entry != null && entry.hasDirMeta()){
|
|
|
+ return entry.getDirListingMeta();
|
|
|
+ } else {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
}
|