|
@@ -28,11 +28,16 @@ import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
import java.util.Date;
|
|
|
import java.util.HashMap;
|
|
|
+import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
+import java.util.Objects;
|
|
|
+import java.util.Set;
|
|
|
import java.util.TreeMap;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
+import java.util.concurrent.atomic.AtomicReference;
|
|
|
+import java.util.stream.Collectors;
|
|
|
|
|
|
import com.amazonaws.AmazonClientException;
|
|
|
import com.amazonaws.auth.AWSCredentialsProvider;
|
|
@@ -422,7 +427,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
boolean idempotent = S3AFileSystem.DELETE_CONSIDERED_IDEMPOTENT;
|
|
|
if (tombstone) {
|
|
|
Item item = PathMetadataDynamoDBTranslation.pathMetadataToItem(
|
|
|
- PathMetadata.tombstone(path));
|
|
|
+ new DDBPathMetadata(PathMetadata.tombstone(path)));
|
|
|
invoker.retry("Put tombstone", path.toString(), idempotent,
|
|
|
() -> table.putItem(item));
|
|
|
} else {
|
|
@@ -461,13 +466,13 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
|
|
|
@Override
|
|
|
@Retries.OnceTranslated
|
|
|
- public PathMetadata get(Path path) throws IOException {
|
|
|
+ public DDBPathMetadata get(Path path) throws IOException {
|
|
|
return get(path, false);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
@Retries.OnceTranslated
|
|
|
- public PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
|
|
|
+ public DDBPathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
|
|
|
throws IOException {
|
|
|
checkPath(path);
|
|
|
LOG.debug("Get from table {} in region {}: {}", tableName, region, path);
|
|
@@ -485,12 +490,13 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
* @throws AmazonClientException dynamo DB level problem
|
|
|
*/
|
|
|
@Retries.OnceRaw
|
|
|
- private PathMetadata innerGet(Path path, boolean wantEmptyDirectoryFlag)
|
|
|
+ private DDBPathMetadata innerGet(Path path, boolean wantEmptyDirectoryFlag)
|
|
|
throws IOException {
|
|
|
- final PathMetadata meta;
|
|
|
+ final DDBPathMetadata meta;
|
|
|
if (path.isRoot()) {
|
|
|
// Root does not persist in the table
|
|
|
- meta = new PathMetadata(makeDirStatus(username, path));
|
|
|
+ meta =
|
|
|
+ new DDBPathMetadata(makeDirStatus(username, path));
|
|
|
} else {
|
|
|
final Item item = getConsistentItem(pathToKey(path));
|
|
|
meta = itemToPathMetadata(item, username);
|
|
@@ -550,15 +556,22 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
|
|
|
final List<PathMetadata> metas = new ArrayList<>();
|
|
|
for (Item item : items) {
|
|
|
- PathMetadata meta = itemToPathMetadata(item, username);
|
|
|
+ DDBPathMetadata meta = itemToPathMetadata(item, username);
|
|
|
metas.add(meta);
|
|
|
}
|
|
|
+
|
|
|
+ DDBPathMetadata dirPathMeta = get(path);
|
|
|
+ boolean isAuthoritative = false;
|
|
|
+ if(dirPathMeta != null) {
|
|
|
+ isAuthoritative = dirPathMeta.isAuthoritativeDir();
|
|
|
+ }
|
|
|
+
|
|
|
LOG.trace("Listing table {} in region {} for {} returning {}",
|
|
|
tableName, region, path, metas);
|
|
|
|
|
|
- return (metas.isEmpty() && get(path) == null)
|
|
|
+ return (metas.isEmpty() && dirPathMeta == null)
|
|
|
? null
|
|
|
- : new DirListingMetadata(path, metas, false);
|
|
|
+ : new DirListingMetadata(path, metas, isAuthoritative);
|
|
|
});
|
|
|
}
|
|
|
|
|
@@ -567,24 +580,25 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
* @param pathsToCreate paths to create
|
|
|
* @return the full ancestry paths
|
|
|
*/
|
|
|
- Collection<PathMetadata> completeAncestry(
|
|
|
- Collection<PathMetadata> pathsToCreate) {
|
|
|
+ Collection<DDBPathMetadata> completeAncestry(
|
|
|
+ Collection<DDBPathMetadata> pathsToCreate) {
|
|
|
// Key on path to allow fast lookup
|
|
|
- Map<Path, PathMetadata> ancestry = new HashMap<>();
|
|
|
+ Map<Path, DDBPathMetadata> ancestry = new HashMap<>();
|
|
|
|
|
|
- for (PathMetadata meta : pathsToCreate) {
|
|
|
+ for (DDBPathMetadata meta : pathsToCreate) {
|
|
|
Preconditions.checkArgument(meta != null);
|
|
|
Path path = meta.getFileStatus().getPath();
|
|
|
if (path.isRoot()) {
|
|
|
break;
|
|
|
}
|
|
|
- ancestry.put(path, meta);
|
|
|
+ ancestry.put(path, new DDBPathMetadata(meta));
|
|
|
Path parent = path.getParent();
|
|
|
while (!parent.isRoot() && !ancestry.containsKey(parent)) {
|
|
|
LOG.debug("auto-create ancestor path {} for child path {}",
|
|
|
parent, path);
|
|
|
final FileStatus status = makeDirStatus(parent, username);
|
|
|
- ancestry.put(parent, new PathMetadata(status, Tristate.FALSE, false));
|
|
|
+ ancestry.put(parent, new DDBPathMetadata(status, Tristate.FALSE,
|
|
|
+ false));
|
|
|
parent = parent.getParent();
|
|
|
}
|
|
|
}
|
|
@@ -611,13 +625,13 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
// Following code is to maintain this invariant by putting all ancestor
|
|
|
// directories of the paths to create.
|
|
|
// ancestor paths that are not explicitly added to paths to create
|
|
|
- Collection<PathMetadata> newItems = new ArrayList<>();
|
|
|
+ Collection<DDBPathMetadata> newItems = new ArrayList<>();
|
|
|
if (pathsToCreate != null) {
|
|
|
- newItems.addAll(completeAncestry(pathsToCreate));
|
|
|
+ newItems.addAll(completeAncestry(pathMetaToDDBPathMeta(pathsToCreate)));
|
|
|
}
|
|
|
if (pathsToDelete != null) {
|
|
|
for (Path meta : pathsToDelete) {
|
|
|
- newItems.add(PathMetadata.tombstone(meta));
|
|
|
+ newItems.add(new DDBPathMetadata(PathMetadata.tombstone(meta)));
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -725,7 +739,11 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
@Override
|
|
|
@Retries.OnceRaw
|
|
|
public void put(Collection<PathMetadata> metas) throws IOException {
|
|
|
+ innerPut(pathMetaToDDBPathMeta(metas));
|
|
|
+ }
|
|
|
|
|
|
+ @Retries.OnceRaw
|
|
|
+ private void innerPut(Collection<DDBPathMetadata> metas) throws IOException {
|
|
|
Item[] items = pathMetadataToItem(completeAncestry(metas));
|
|
|
LOG.debug("Saving batch of {} items to table {}, region {}", items.length,
|
|
|
tableName, region);
|
|
@@ -736,10 +754,10 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
* Helper method to get full path of ancestors that are nonexistent in table.
|
|
|
*/
|
|
|
@Retries.OnceRaw
|
|
|
- private Collection<PathMetadata> fullPathsToPut(PathMetadata meta)
|
|
|
+ private Collection<DDBPathMetadata> fullPathsToPut(DDBPathMetadata meta)
|
|
|
throws IOException {
|
|
|
checkPathMetadata(meta);
|
|
|
- final Collection<PathMetadata> metasToPut = new ArrayList<>();
|
|
|
+ final Collection<DDBPathMetadata> metasToPut = new ArrayList<>();
|
|
|
// root path is not persisted
|
|
|
if (!meta.getFileStatus().getPath().isRoot()) {
|
|
|
metasToPut.add(meta);
|
|
@@ -752,7 +770,8 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
final Item item = getConsistentItem(pathToKey(path));
|
|
|
if (!itemExists(item)) {
|
|
|
final FileStatus status = makeDirStatus(path, username);
|
|
|
- metasToPut.add(new PathMetadata(status, Tristate.FALSE, false));
|
|
|
+ metasToPut.add(new DDBPathMetadata(status, Tristate.FALSE, false,
|
|
|
+ meta.isAuthoritativeDir()));
|
|
|
path = path.getParent();
|
|
|
} else {
|
|
|
break;
|
|
@@ -793,16 +812,17 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
|
|
|
// directory path
|
|
|
Path path = meta.getPath();
|
|
|
- PathMetadata p = new PathMetadata(makeDirStatus(path, username),
|
|
|
- meta.isEmpty(), false);
|
|
|
+ DDBPathMetadata ddbPathMeta =
|
|
|
+ new DDBPathMetadata(makeDirStatus(path, username), meta.isEmpty(),
|
|
|
+ false, meta.isAuthoritative());
|
|
|
|
|
|
// First add any missing ancestors...
|
|
|
- final Collection<PathMetadata> metasToPut = invoker.retry(
|
|
|
+ final Collection<DDBPathMetadata> metasToPut = invoker.retry(
|
|
|
"paths to put", path.toString(), true,
|
|
|
- () -> fullPathsToPut(p));
|
|
|
+ () -> fullPathsToPut(ddbPathMeta));
|
|
|
|
|
|
// next add all children of the directory
|
|
|
- metasToPut.addAll(meta.getListing());
|
|
|
+ metasToPut.addAll(pathMetaToDDBPathMeta(meta.getListing()));
|
|
|
|
|
|
Invoker.once("put", path.toString(),
|
|
|
() -> processBatchWriteRequest(null, pathMetadataToItem(metasToPut)));
|
|
@@ -880,21 +900,38 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
new ArrayList<>(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
|
|
|
int delay = conf.getInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY,
|
|
|
S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT);
|
|
|
+ Set<Path> parentPathSet = new HashSet<>();
|
|
|
for (Item item : expiredFiles(modTime, keyPrefix)) {
|
|
|
- PathMetadata md = PathMetadataDynamoDBTranslation
|
|
|
+ DDBPathMetadata md = PathMetadataDynamoDBTranslation
|
|
|
.itemToPathMetadata(item, username);
|
|
|
Path path = md.getFileStatus().getPath();
|
|
|
deletionBatch.add(path);
|
|
|
+
|
|
|
+ // add parent path of what we remove
|
|
|
+ Path parentPath = path.getParent();
|
|
|
+ if (parentPath != null) {
|
|
|
+ parentPathSet.add(parentPath);
|
|
|
+ }
|
|
|
+
|
|
|
itemCount++;
|
|
|
if (deletionBatch.size() == S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT) {
|
|
|
Thread.sleep(delay);
|
|
|
processBatchWriteRequest(pathToKey(deletionBatch), null);
|
|
|
+
|
|
|
+ // set authoritative false for each pruned dir listing
|
|
|
+ removeAuthoritativeDirFlag(parentPathSet);
|
|
|
+ parentPathSet.clear();
|
|
|
+
|
|
|
deletionBatch.clear();
|
|
|
}
|
|
|
}
|
|
|
if (deletionBatch.size() > 0) {
|
|
|
Thread.sleep(delay);
|
|
|
processBatchWriteRequest(pathToKey(deletionBatch), null);
|
|
|
+
|
|
|
+ // set authoritative false for each pruned dir listing
|
|
|
+ removeAuthoritativeDirFlag(parentPathSet);
|
|
|
+ parentPathSet.clear();
|
|
|
}
|
|
|
} catch (InterruptedException e) {
|
|
|
Thread.currentThread().interrupt();
|
|
@@ -904,6 +941,43 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
|
|
|
}
|
|
|
|
|
|
+ private void removeAuthoritativeDirFlag(Set<Path> pathSet)
|
|
|
+ throws IOException {
|
|
|
+ AtomicReference<IOException> rIOException = new AtomicReference<>();
|
|
|
+
|
|
|
+ Set<DDBPathMetadata> metas = pathSet.stream().map(path -> {
|
|
|
+ try {
|
|
|
+ DDBPathMetadata ddbPathMetadata = get(path);
|
|
|
+ if(ddbPathMetadata == null) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ LOG.debug("Setting false isAuthoritativeDir on {}", ddbPathMetadata);
|
|
|
+ ddbPathMetadata.setAuthoritativeDir(false);
|
|
|
+ return ddbPathMetadata;
|
|
|
+ } catch (IOException e) {
|
|
|
+ String msg = String.format("IOException while getting PathMetadata "
|
|
|
+ + "on path: %s.", path);
|
|
|
+ LOG.error(msg, e);
|
|
|
+ rIOException.set(e);
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }).filter(Objects::nonNull).collect(Collectors.toSet());
|
|
|
+
|
|
|
+ try {
|
|
|
+ LOG.debug("innerPut on metas: {}", metas);
|
|
|
+ innerPut(metas);
|
|
|
+ } catch (IOException e) {
|
|
|
+ String msg = String.format("IOException while setting false "
|
|
|
+ + "authoritative directory flag on: %s.", metas);
|
|
|
+ LOG.error(msg, e);
|
|
|
+ rIOException.set(e);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (rIOException.get() != null) {
|
|
|
+ throw rIOException.get();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
return getClass().getSimpleName() + '{'
|
|
@@ -1197,7 +1271,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
map.put(WRITE_CAPACITY, throughput.getWriteCapacityUnits().toString());
|
|
|
map.put(TABLE, desc.toString());
|
|
|
map.put(MetadataStoreCapabilities.PERSISTS_AUTHORITATIVE_BIT,
|
|
|
- Boolean.toString(false));
|
|
|
+ Boolean.toString(true));
|
|
|
} else {
|
|
|
map.put("name", "DynamoDB Metadata Store");
|
|
|
map.put(TABLE, "none");
|