listing = entry.getListing();
- int files = 0, subdirs = 0;
- for (PathMetadata e : listing) {
- if (isDirectory(e)) {
- // queue for auditing
- queue.add((DDBPathMetadata) e);
- subdirs++;
- } else {
- files++;
- }
- }
- if (verbose && files > 0 || subdirs > 0) {
- LOG.info(" files {}; directories {}", files, subdirs);
- }
- } else {
- LOG.info("Directory {} has been deleted", dir);
- }
- }
- }
- // end of scan
- if (dirs == 1 && isRoot) {
- LOG.info("The store has no directories to scan");
- } else {
- LOG.info("Scanned {} directories - {} were not marked as authoritative",
- dirs, nonauth);
- }
- return Pair.of(dirs, nonauth);
- }
-
- /**
- * A directory was found which was non-authoritative.
- * The exit code for this operation is
- * {@link LauncherExitCodes#EXIT_NOT_ACCEPTABLE} -This is what the S3Guard
- * will return.
- */
- public static final class NonAuthoritativeDirException
- extends PathIOException implements ExitCodeProvider {
-
- /**
- * Instantiate.
- * @param path the path which is non-authoritative.
- */
- private NonAuthoritativeDirException(final Path path) {
- super(path.toString(), E_NONAUTH);
- }
-
- @Override
- public int getExitCode() {
- return ERROR_ENTRY_NOT_AUTH_IN_DDB;
- }
-
- @Override
- public String toString() {
- return getMessage();
- }
- }
-
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java
deleted file mode 100644
index b4974b73565..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/**
- * This represents state which may be passed to bulk IO operations
- * to enable them to store information about the state of the ongoing
- * operation across invocations.
- *
- * A bulk operation state MUST only be be used for the single store
- * from which it was created, and MUSTonly for the duration of a single
- * bulk update operation.
- *
- * Passing in the state is to allow the stores to maintain state about
- * updates they have already made to their store during this single operation:
- * a cache of what has happened. It is not a list of operations to be applied.
- * If a list of operations to perform is built up (e.g. during rename)
- * that is the duty of the caller, not this state.
- *
- * After the operation has completed, it MUST be closed so
- * as to guarantee that all state is released.
- */
-public class BulkOperationState implements Closeable {
-
- private final OperationType operation;
-
- /**
- * Constructor.
- * @param operation the type of the operation.
- */
- public BulkOperationState(final OperationType operation) {
- this.operation = operation;
- }
-
- /**
- * Get the operation type.
- * @return the operation type.
- */
- public OperationType getOperation() {
- return operation;
- }
-
- @Override
- public void close() throws IOException {
-
- }
-
- /**
- * Enumeration of operations which can be performed in bulk.
- * This can be used by the stores however they want.
- * One special aspect: renames are to be done through a {@link RenameTracker}.
- * Callers will be blocked from initiating a rename through
- * {@code S3Guard#initiateBulkWrite()}
- */
- public enum OperationType {
- /** Writing data. */
- Put,
- /**
- * Rename: add and delete.
- * After the rename, the tree under the destination path
- * can be tagged as authoritative.
- */
- Rename,
- /** Pruning: deleting entries and updating parents. */
- Prune,
- /** Commit operation. */
- Commit,
- /** Deletion operation. */
- Delete,
- /** FSCK operation. */
- Fsck,
- /**
- * Bulk directory tree import.
- * After an import, the entire tree under the path has been
- * enumerated and should be tagged as authoritative.
- */
- Import,
- /**
- * Listing update.
- */
- Listing,
- /**
- * Mkdir operation.
- */
- Mkdir,
- /**
- * Multipart upload operation.
- */
- Upload
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DDBPathMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DDBPathMetadata.java
deleted file mode 100644
index 292c0161348..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DDBPathMetadata.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-import org.apache.hadoop.fs.s3a.Tristate;
-
-/**
- * {@code DDBPathMetadata} wraps {@link PathMetadata} and adds the
- * isAuthoritativeDir flag to provide support for authoritative directory
- * listings in {@link DynamoDBMetadataStore}.
- */
-public class DDBPathMetadata extends PathMetadata {
-
- private boolean isAuthoritativeDir;
-
- public DDBPathMetadata(PathMetadata pmd) {
- super(pmd.getFileStatus(), pmd.isEmptyDirectory(), pmd.isDeleted(),
- pmd.getLastUpdated());
- this.isAuthoritativeDir = false;
- this.setLastUpdated(pmd.getLastUpdated());
- }
-
- public DDBPathMetadata(S3AFileStatus fileStatus) {
- super(fileStatus);
- this.isAuthoritativeDir = false;
- }
-
- public DDBPathMetadata(S3AFileStatus fileStatus, Tristate isEmptyDir,
- boolean isDeleted, long lastUpdated) {
- super(fileStatus, isEmptyDir, isDeleted, lastUpdated);
- this.isAuthoritativeDir = false;
- }
-
- public DDBPathMetadata(S3AFileStatus fileStatus, Tristate isEmptyDir,
- boolean isDeleted, boolean isAuthoritativeDir, long lastUpdated) {
- super(fileStatus, isEmptyDir, isDeleted, lastUpdated);
- this.isAuthoritativeDir = isAuthoritativeDir;
- }
-
- public boolean isAuthoritativeDir() {
- return isAuthoritativeDir;
- }
-
- public void setAuthoritativeDir(boolean authoritativeDir) {
- isAuthoritativeDir = authoritativeDir;
- }
-
- @Override
- public boolean equals(Object o) {
- return super.equals(o);
- }
-
- @Override public int hashCode() {
- return super.hashCode();
- }
-
- @Override public String toString() {
- return "DDBPathMetadata{" +
- "isAuthoritativeDir=" + isAuthoritativeDir +
- ", PathMetadata=" + super.toString() +
- '}';
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java
deleted file mode 100644
index 51a9821ace1..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import com.amazonaws.SdkBaseException;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
-import org.apache.hadoop.fs.s3a.Tristate;
-import org.apache.hadoop.fs.s3a.impl.StoreContext;
-import org.apache.hadoop.util.DurationInfo;
-
-import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveAncestors;
-import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveDir;
-import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveFile;
-
-/**
- * This is the rename updating strategy originally used:
- * a collection of source paths and a list of destinations are created,
- * then updated at the end (possibly slow).
- *
- * It is not currently instantiated by any of the active trackers,
- * but is preserved to show that the original rename strategy
- * can be implemented via the tracker model.
- */
-public class DelayedUpdateRenameTracker extends RenameTracker {
-
- private final MetadataStore metadataStore;
-
- private final Collection sourcePaths = new HashSet<>();
-
- private final List destMetas = new ArrayList<>();
-
- private final List deletedPaths = new ArrayList<>();
-
- public DelayedUpdateRenameTracker(
- final StoreContext storeContext,
- final MetadataStore metadataStore,
- final Path sourceRoot,
- final Path dest,
- final BulkOperationState operationState) {
- super("DelayedUpdateRenameTracker", storeContext, metadataStore,
- sourceRoot, dest, operationState);
- this.metadataStore = storeContext.getMetadataStore();
- }
-
- @Override
- public synchronized void fileCopied(
- final Path sourcePath,
- final S3ObjectAttributes sourceAttributes,
- final S3ObjectAttributes destAttributes,
- final Path destPath,
- final long blockSize,
- final boolean addAncestors) throws IOException {
-
- addMoveFile(metadataStore,
- sourcePaths,
- destMetas,
- sourcePath,
- destPath,
- sourceAttributes.getLen(),
- blockSize,
- getOwner(),
- destAttributes.getETag(),
- destAttributes.getVersionId());
- // Ancestor directories may not be listed, so we explicitly add them
- if (addAncestors) {
- addMoveAncestors(metadataStore,
- sourcePaths,
- destMetas,
- getSourceRoot(),
- sourcePath,
- destPath,
- getOwner());
- }
- }
-
- @Override
- public synchronized void directoryMarkerCopied(final Path sourcePath,
- final Path destPath,
- final boolean addAncestors) throws IOException {
- addMoveDir(metadataStore, sourcePaths, destMetas,
- sourcePath,
- destPath, getOwner());
- // Ancestor directories may not be listed, so we explicitly add them
- if (addAncestors) {
- addMoveAncestors(metadataStore,
- sourcePaths,
- destMetas,
- getSourceRoot(),
- sourcePath,
- destPath,
- getOwner());
- }
- }
-
- @Override
- public synchronized void moveSourceDirectory() throws IOException {
- if (!sourcePaths.contains(getSourceRoot())) {
- addMoveDir(metadataStore, sourcePaths, destMetas,
- getSourceRoot(),
- getDest(), getOwner());
- }
- }
-
- @Override
- public synchronized void sourceObjectsDeleted(
- final Collection paths) throws IOException {
- // add to the list of deleted paths.
- deletedPaths.addAll(paths);
- }
-
- @Override
- public void completeRename() throws IOException {
- metadataStore.move(sourcePaths, destMetas, getOperationState());
- super.completeRename();
- }
-
- @Override
- public IOException renameFailed(final Exception ex) {
- LOG.warn("Rename has failed; updating s3guard with destination state");
- try (DurationInfo ignored = new DurationInfo(LOG,
- "Cleaning up deleted paths")) {
- // the destination paths are updated; the source is left alone.
- metadataStore.move(new ArrayList<>(0), destMetas, getOperationState());
- for (Path deletedPath : deletedPaths) {
- // this is not ideal in that it may leave parent stuff around.
- metadataStore.delete(deletedPath, getOperationState());
- }
- deleteParentPaths();
- } catch (IOException | SdkBaseException e) {
- LOG.warn("Ignoring error raised in AWS SDK ", e);
- }
-
- return super.renameFailed(ex);
- }
-
- /**
- * Delete all the parent paths we know to be empty (by walking up the tree
- * deleting as appropriate).
- * @throws IOException failure
- */
- private void deleteParentPaths() throws IOException {
- Set parentPaths = new HashSet<>();
- for (Path deletedPath : deletedPaths) {
- Path parent = deletedPath.getParent();
- if (!parent.equals(getSourceRoot())) {
- parentPaths.add(parent);
- }
- }
- // now there's a set of parent paths. We now want to
- // get them ordered by depth, so that deeper entries come first
- // that way: when we check for a parent path existing we can
- // see if it really is empty.
- List parents = new ArrayList<>(parentPaths);
- parents.sort(PathOrderComparators.TOPMOST_PATH_LAST);
- for (Path parent : parents) {
- PathMetadata md = metadataStore.get(parent, true);
- if (md != null && md.isEmptyDirectory() == Tristate.TRUE) {
- // if were confident that this is empty: delete it.
- metadataStore.delete(parent, getOperationState());
- }
- }
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java
deleted file mode 100644
index a69109c9c88..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.LinkedList;
-import java.util.NoSuchElementException;
-import java.util.Queue;
-
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-
-/**
- * {@code DescendantsIterator} is a {@link RemoteIterator} that implements
- * pre-ordering breadth-first traversal (BFS) of a path and all of its
- * descendants recursively. After visiting each path, that path's direct
- * children are discovered by calling {@link MetadataStore#listChildren(Path)}.
- * Each iteration returns the next direct child, and if that child is a
- * directory, also pushes it onto a queue to discover its children later.
- *
- * For example, assume the consistent store contains metadata representing this
- * file system structure:
- *
- *
- * /dir1
- * |-- dir2
- * | |-- file1
- * | `-- file2
- * `-- dir3
- * |-- dir4
- * | `-- file3
- * |-- dir5
- * | `-- file4
- * `-- dir6
- *
- *
- * Consider this code sample:
- *
- * final PathMetadata dir1 = get(new Path("/dir1"));
- * for (DescendantsIterator descendants = new DescendantsIterator(dir1);
- * descendants.hasNext(); ) {
- * final FileStatus status = descendants.next().getFileStatus();
- * System.out.printf("%s %s%n", status.isDirectory() ? 'D' : 'F',
- * status.getPath());
- * }
- *
- *
- * The output is:
- *
- * D /dir1
- * D /dir1/dir2
- * D /dir1/dir3
- * F /dir1/dir2/file1
- * F /dir1/dir2/file2
- * D /dir1/dir3/dir4
- * D /dir1/dir3/dir5
- * F /dir1/dir3/dir4/file3
- * F /dir1/dir3/dir5/file4
- * D /dir1/dir3/dir6
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class DescendantsIterator implements RemoteIterator {
-
- private final MetadataStore metadataStore;
- private final Queue queue = new LinkedList<>();
-
- /**
- * Creates a new {@code DescendantsIterator}.
- *
- * @param ms the associated {@link MetadataStore}
- * @param meta base path for descendants iteration, which will be the first
- * returned during iteration (except root). Null makes empty iterator.
- * @throws IOException if errors happen during metadata store listing
- */
- public DescendantsIterator(MetadataStore ms, PathMetadata meta)
- throws IOException {
- Preconditions.checkNotNull(ms);
- this.metadataStore = ms;
-
- if (meta != null) {
- final Path path = meta.getFileStatus().getPath();
- if (path.isRoot()) {
- DirListingMetadata rootListing = ms.listChildren(path);
- if (rootListing != null) {
- rootListing = rootListing.withoutTombstones();
- queue.addAll(rootListing.getListing());
- }
- } else {
- queue.add(meta);
- }
- }
- }
-
- @Override
- public boolean hasNext() throws IOException {
- return !queue.isEmpty();
- }
-
- @Override
- public S3AFileStatus next() throws IOException {
- if (!hasNext()) {
- throw new NoSuchElementException("No more descendants.");
- }
- PathMetadata next;
- next = queue.poll();
- if (next.getFileStatus().isDirectory()) {
- final Path path = next.getFileStatus().getPath();
- DirListingMetadata meta = metadataStore.listChildren(path);
- if (meta != null) {
- Collection more = meta.withoutTombstones().getListing();
- if (!more.isEmpty()) {
- queue.addAll(more);
- }
- }
- }
- return next.getFileStatus();
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java
deleted file mode 100644
index 63600b3a942..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java
+++ /dev/null
@@ -1,372 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-import org.apache.hadoop.fs.s3a.Tristate;
-
-/**
- * {@code DirListingMetadata} models a directory listing stored in a
- * {@link MetadataStore}. Instances of this class are mutable and thread-safe.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class DirListingMetadata extends ExpirableMetadata {
-
- /**
- * Convenience parameter for passing into constructor.
- */
- public static final Collection EMPTY_DIR =
- Collections.emptyList();
-
- private final Path path;
-
- /** Using a map for fast find / remove with large directories. */
- private Map listMap = new ConcurrentHashMap<>();
-
- private boolean isAuthoritative;
-
- /**
- * Create a directory listing metadata container.
- *
- * @param path Path of the directory. If this path has a host component, then
- * all paths added later via {@link #put(PathMetadata)} must also have
- * the same host.
- * @param listing Entries in the directory.
- * @param isAuthoritative true iff listing is the full contents of the
- * directory, and the calling client reports that this may be cached as
- * the full and authoritative listing of all files in the directory.
- * @param lastUpdated last updated time on which expiration is based.
- */
- public DirListingMetadata(Path path, Collection listing,
- boolean isAuthoritative, long lastUpdated) {
-
- checkPathAbsolute(path);
- this.path = path;
-
- if (listing != null) {
- for (PathMetadata entry : listing) {
- Path childPath = entry.getFileStatus().getPath();
- checkChildPath(childPath);
- listMap.put(childPath, entry);
- }
- }
- this.isAuthoritative = isAuthoritative;
- this.setLastUpdated(lastUpdated);
- }
-
- public DirListingMetadata(Path path, Collection listing,
- boolean isAuthoritative) {
- this(path, listing, isAuthoritative, 0);
- }
-
- /**
- * Copy constructor.
- * @param d the existing {@link DirListingMetadata} object.
- */
- public DirListingMetadata(DirListingMetadata d) {
- path = d.path;
- isAuthoritative = d.isAuthoritative;
- this.setLastUpdated(d.getLastUpdated());
- listMap = new ConcurrentHashMap<>(d.listMap);
- }
-
- /**
- * @return {@code Path} of the directory that contains this listing.
- */
- public Path getPath() {
- return path;
- }
-
- /**
- * @return entries in the directory
- */
- public Collection getListing() {
- return Collections.unmodifiableCollection(listMap.values());
- }
-
- /**
- * List all tombstones.
- * @return all tombstones in the listing.
- */
- public Set listTombstones() {
- Set tombstones = new HashSet<>();
- for (PathMetadata meta : listMap.values()) {
- if (meta.isDeleted()) {
- tombstones.add(meta.getFileStatus().getPath());
- }
- }
- return tombstones;
- }
-
- /**
- * Get the directory listing excluding tombstones.
- * Returns a new DirListingMetadata instances, without the tombstones -the
- * lastUpdated field is copied from this instance.
- * @return a new DirListingMetadata without the tombstones.
- */
- public DirListingMetadata withoutTombstones() {
- Collection filteredList = new ArrayList<>();
- for (PathMetadata meta : listMap.values()) {
- if (!meta.isDeleted()) {
- filteredList.add(meta);
- }
- }
- return new DirListingMetadata(path, filteredList, isAuthoritative,
- this.getLastUpdated());
- }
-
- /**
- * @return number of entries tracked. This is not the same as the number
- * of entries in the actual directory unless {@link #isAuthoritative()} is
- * true.
- * It will also include any tombstones.
- */
- public int numEntries() {
- return listMap.size();
- }
-
- /**
- * @return true iff this directory listing is full and authoritative within
- * the scope of the {@code MetadataStore} that returned it.
- */
- public boolean isAuthoritative() {
- return isAuthoritative;
- }
-
-
- /**
- * Is the underlying directory known to be empty?
- * @return FALSE if directory is known to have a child entry, TRUE if
- * directory is known to be empty, UNKNOWN otherwise.
- */
- public Tristate isEmpty() {
- if (getListing().isEmpty()) {
- if (isAuthoritative()) {
- return Tristate.TRUE;
- } else {
- // This listing is empty, but may not be full list of underlying dir.
- return Tristate.UNKNOWN;
- }
- } else { // not empty listing
- // There exists at least one child, dir not empty.
- return Tristate.FALSE;
- }
- }
-
- /**
- * Marks this directory listing as full and authoritative.
- * @param authoritative see {@link #isAuthoritative()}.
- */
- public void setAuthoritative(boolean authoritative) {
- this.isAuthoritative = authoritative;
- }
-
- /**
- * Lookup entry within this directory listing. This may return null if the
- * {@code MetadataStore} only tracks a partial set of the directory entries.
- * In the case where {@link #isAuthoritative()} is true, however, this
- * function returns null iff the directory is known not to contain the listing
- * at given path (within the scope of the {@code MetadataStore} that returned
- * it).
- *
- * @param childPath path of entry to look for.
- * @return entry, or null if it is not present or not being tracked.
- */
- public PathMetadata get(Path childPath) {
- checkChildPath(childPath);
- return listMap.get(childPath);
- }
-
- /**
- * Replace an entry with a tombstone.
- * @param childPath path of entry to replace.
- */
- public void markDeleted(Path childPath, long lastUpdated) {
- checkChildPath(childPath);
- listMap.put(childPath, PathMetadata.tombstone(childPath, lastUpdated));
- }
-
- /**
- * Remove entry from this directory.
- *
- * @param childPath path of entry to remove.
- */
- public void remove(Path childPath) {
- checkChildPath(childPath);
- listMap.remove(childPath);
- }
-
- /**
- * Add an entry to the directory listing. If this listing already contains a
- * {@code FileStatus} with the same path, it will be replaced.
- *
- * @param childPathMetadata entry to add to this directory listing.
- * @return true if the status was added or replaced with a new value. False
- * if the same FileStatus value was already present.
- */
- public boolean put(PathMetadata childPathMetadata) {
- Preconditions.checkNotNull(childPathMetadata,
- "childPathMetadata must be non-null");
- final S3AFileStatus fileStatus = childPathMetadata.getFileStatus();
- Path childPath = childStatusToPathKey(fileStatus);
- PathMetadata newValue = childPathMetadata;
- PathMetadata oldValue = listMap.put(childPath, childPathMetadata);
- return oldValue == null || !oldValue.equals(newValue);
- }
-
- @Override
- public String toString() {
- return "DirListingMetadata{" +
- "path=" + path +
- ", listMap=" + listMap +
- ", isAuthoritative=" + isAuthoritative +
- ", lastUpdated=" + this.getLastUpdated() +
- '}';
- }
-
- /**
- * Remove expired entries from the listing based on TTL.
- * @param ttl the ttl time
- * @param now the current time
- * @return the expired values.
- */
- public synchronized List removeExpiredEntriesFromListing(
- long ttl, long now) {
- List expired = new ArrayList<>();
- final Iterator> iterator =
- listMap.entrySet().iterator();
- while (iterator.hasNext()) {
- final Map.Entry entry = iterator.next();
- // we filter iff the lastupdated is not 0 and the entry is expired
- PathMetadata metadata = entry.getValue();
- if (metadata.getLastUpdated() != 0
- && (metadata.getLastUpdated() + ttl) <= now) {
- expired.add(metadata);
- iterator.remove();
- }
- }
- return expired;
- }
-
- /**
- * Log contents to supplied StringBuilder in a pretty fashion.
- * @param sb target StringBuilder
- */
- public void prettyPrint(StringBuilder sb) {
- sb.append(String.format("DirMeta %-20s %-18s",
- path.toString(),
- isAuthoritative ? "Authoritative" : "Not Authoritative"));
- for (Map.Entry entry : listMap.entrySet()) {
- sb.append("\n key: ").append(entry.getKey()).append(": ");
- entry.getValue().prettyPrint(sb);
- }
- sb.append("\n");
- }
-
- public String prettyPrint() {
- StringBuilder sb = new StringBuilder();
- prettyPrint(sb);
- return sb.toString();
- }
-
- /**
- * Checks that child path is valid.
- * @param childPath path to check.
- */
- private void checkChildPath(Path childPath) {
- checkPathAbsolute(childPath);
-
- // If this dir's path has host (and thus scheme), so must its children
- URI parentUri = path.toUri();
- URI childUri = childPath.toUri();
- if (parentUri.getHost() != null) {
- Preconditions.checkNotNull(childUri.getHost(), "Expected non-null URI " +
- "host: %s", childUri);
- Preconditions.checkArgument(
- childUri.getHost().equals(parentUri.getHost()),
- "childUri %s and parentUri %s must have the same host",
- childUri, parentUri);
- Preconditions.checkNotNull(childUri.getScheme(), "No scheme in path %s",
- childUri);
- }
- Preconditions.checkArgument(!childPath.isRoot(),
- "childPath cannot be the root path: %s", childPath);
- Preconditions.checkArgument(parentUri.getPath().equals(
- childPath.getParent().toUri().getPath()),
- "childPath %s must be a child of %s", childPath, path);
- }
-
- /**
- * For Paths that are handed in directly, we assert they are in consistent
- * format with checkPath(). For paths that are supplied embedded in
- * FileStatus, we attempt to fill in missing scheme and host, when this
- * DirListingMetadata is associated with one.
- *
- * @return Path suitable for consistent hashtable lookups
- * @throws NullPointerException null status argument
- * @throws IllegalArgumentException bad status values or failure to
- * create a URI.
- */
- private Path childStatusToPathKey(FileStatus status) {
- Path p = status.getPath();
- Preconditions.checkNotNull(p, "Child status' path cannot be null");
- Preconditions.checkArgument(!p.isRoot(),
- "childPath cannot be the root path: %s", p);
- Preconditions.checkArgument(p.getParent().equals(path),
- "childPath %s must be a child of %s", p, path);
- URI uri = p.toUri();
- URI parentUri = path.toUri();
- // If FileStatus' path is missing host, but should have one, add it.
- if (uri.getHost() == null && parentUri.getHost() != null) {
- try {
- return new Path(new URI(parentUri.getScheme(), parentUri.getHost(),
- uri.getPath(), uri.getFragment()));
- } catch (URISyntaxException e) {
- throw new IllegalArgumentException("FileStatus path invalid with" +
- " added " + parentUri.getScheme() + "://" + parentUri.getHost() +
- " added", e);
- }
- }
- return p;
- }
-
- private void checkPathAbsolute(Path p) {
- Preconditions.checkNotNull(p, "path must be non-null");
- Preconditions.checkArgument(p.isAbsolute(), "path must be absolute: %s", p);
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java
deleted file mode 100644
index 2a7cb4c1b72..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java
+++ /dev/null
@@ -1,793 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import javax.annotation.Nullable;
-import java.io.Closeable;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.net.URI;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Deque;
-import java.util.List;
-
-import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder;
-import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathIOException;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.s3a.Listing;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
-import org.apache.hadoop.fs.s3a.S3ListRequest;
-import org.apache.hadoop.fs.store.audit.AuditSpan;
-import org.apache.hadoop.service.Service;
-import org.apache.hadoop.service.launcher.LauncherExitCodes;
-import org.apache.hadoop.service.launcher.ServiceLaunchException;
-import org.apache.hadoop.service.launcher.ServiceLauncher;
-import org.apache.hadoop.util.DurationInfo;
-import org.apache.hadoop.util.ExitUtil;
-
-import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.hadoop.fs.s3a.S3AUtils.ACCEPT_ALL;
-
-/**
- * This is a low-level diagnostics entry point which does a CVE/TSV dump of
- * the DDB state.
- * As it also lists the filesystem, it actually changes the state of the store
- * during the operation.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class DumpS3GuardDynamoTable extends AbstractS3GuardDynamoDBDiagnostic {
-
- private static final Logger LOG =
- LoggerFactory.getLogger(DumpS3GuardDynamoTable.class);
-
- /**
- * Application name.
- */
- public static final String NAME = "DumpS3GuardDynamoTable";
-
- /**
- * Usage.
- */
- private static final String USAGE_MESSAGE = NAME
- + " ";
-
- /**
- * Suffix for the flat list: {@value}.
- */
- public static final String FLAT_CSV = "-flat.csv";
-
- /**
- * Suffix for the raw S3 dump: {@value}.
- */
- public static final String RAW_CSV = "-s3.csv";
-
- /**
- * Suffix for the DDB scan: {@value}.
- */
- public static final String SCAN_CSV = "-scan.csv";
-
- /**
- * Suffix for the second DDB scan: : {@value}.
- */
- public static final String SCAN2_CSV = "-scan-2.csv";
-
- /**
- * Suffix for the treewalk scan of the S3A Filesystem: {@value}.
- */
- public static final String TREE_CSV = "-tree.csv";
-
- /**
- * Suffix for a recursive treewalk through the metastore: {@value}.
- */
- public static final String STORE_CSV = "-store.csv";
-
- /**
- * Path in the local filesystem to save the data.
- */
- private String destPath;
-
- private Pair scanEntryResult;
-
- private Pair secondScanResult;
-
- private long rawObjectStoreCount;
-
- private long listStatusCount;
-
- private long treewalkCount;
-
- /**
- * Instantiate.
- * @param name application name.
- */
- public DumpS3GuardDynamoTable(final String name) {
- super(name);
- }
-
- /**
- * Instantiate with default name.
- */
- public DumpS3GuardDynamoTable() {
- this(NAME);
- }
-
- /**
- * Bind to a specific FS + store.
- * @param fs filesystem
- * @param store metastore to use
- * @param destFile the base filename for output
- * @param uri URI of store -only needed if FS is null.
- */
- public DumpS3GuardDynamoTable(
- final S3AFileSystem fs,
- final DynamoDBMetadataStore store,
- final File destFile,
- final URI uri) {
- super(NAME, fs, store, uri);
- this.destPath = destFile.getAbsolutePath();
- }
-
- /**
- * Bind to the argument list, including validating the CLI.
- * @throws Exception failure.
- */
- @Override
- protected void serviceStart() throws Exception {
- if (getStore() == null) {
- List arg = getArgumentList(2, 2, USAGE_MESSAGE);
- bindFromCLI(arg.get(0));
- destPath = arg.get(1);
- }
- }
-
- /**
- * Dump the filesystem and the metastore.
- * @return the exit code.
- * @throws ServiceLaunchException on failure.
- * @throws IOException IO failure.
- */
- @Override
- public int execute() throws ServiceLaunchException, IOException {
-
- try {
- final File scanFile = new File(
- destPath + SCAN_CSV).getCanonicalFile();
- File parentDir = scanFile.getParentFile();
- if (!parentDir.mkdirs() && !parentDir.isDirectory()) {
- throw new PathIOException(parentDir.toString(),
- "Could not create destination directory");
- }
-
- try (CsvFile csv = new CsvFile(scanFile);
- DurationInfo ignored = new DurationInfo(LOG,
- "scanFile dump to %s", scanFile)) {
- scanEntryResult = scanMetastore(csv);
- }
-
- if (getFilesystem() != null) {
-
- Path basePath = getFilesystem().qualify(new Path(getUri()));
-
- final File destFile = new File(destPath + STORE_CSV)
- .getCanonicalFile();
- LOG.info("Writing Store details to {}", destFile);
- try (CsvFile csv = new CsvFile(destFile);
- DurationInfo ignored = new DurationInfo(LOG, "List metastore")) {
-
- LOG.info("Base path: {}", basePath);
- dumpMetastore(csv, basePath);
- }
-
- // these operations all update the metastore as they list,
- // that is: they are side-effecting.
- final File treewalkFile = new File(destPath + TREE_CSV)
- .getCanonicalFile();
-
- try (CsvFile csv = new CsvFile(treewalkFile);
- DurationInfo ignored = new DurationInfo(LOG,
- "Treewalk to %s", treewalkFile)) {
- treewalkCount = treewalkFilesystem(csv, basePath);
- }
- final File flatlistFile = new File(
- destPath + FLAT_CSV).getCanonicalFile();
-
- try (CsvFile csv = new CsvFile(flatlistFile);
- DurationInfo ignored = new DurationInfo(LOG,
- "Flat list to %s", flatlistFile)) {
- listStatusCount = listStatusFilesystem(csv, basePath);
- }
- final File rawFile = new File(
- destPath + RAW_CSV).getCanonicalFile();
-
- try (CsvFile csv = new CsvFile(rawFile);
- DurationInfo ignored = new DurationInfo(LOG,
- "Raw dump to %s", rawFile)) {
- rawObjectStoreCount = dumpRawS3ObjectStore(csv);
- }
- final File scanFile2 = new File(
- destPath + SCAN2_CSV).getCanonicalFile();
-
- try (CsvFile csv = new CsvFile(scanFile);
- DurationInfo ignored = new DurationInfo(LOG,
- "scanFile dump to %s", scanFile2)) {
- secondScanResult = scanMetastore(csv);
- }
- }
-
- return LauncherExitCodes.EXIT_SUCCESS;
- } catch (IOException | RuntimeException e) {
- LOG.error("failure", e);
- throw e;
- }
- }
-
- /**
- * Push all elements of a list to a queue, such that the first entry
- * on the list becomes the head of the queue.
- * @param queue queue to update
- * @param entries list of entries to add.
- * @param type of queue
- */
- private void pushAll(Deque queue, List entries) {
- List reversed = Lists.reverse(entries);
- for (T t : reversed) {
- queue.push(t);
- }
- }
-
- /**
- * Dump the filesystem via a treewalk.
- * If metastore entries mark directories as deleted, this
- * walk will not explore them.
- * @param csv destination.
- * @param base base path.
- * @return number of entries found.
- * @throws IOException IO failure.
- */
- protected long treewalkFilesystem(
- final CsvFile csv,
- final Path base) throws IOException {
- ArrayDeque queue = new ArrayDeque<>();
- queue.add(base);
- long count = 0;
- while (!queue.isEmpty()) {
- Path path = queue.pop();
- count++;
- FileStatus[] fileStatuses;
- try {
- fileStatuses = getFilesystem().listStatus(path);
- } catch (FileNotFoundException e) {
- LOG.warn("File {} was not found", path);
- continue;
- }
- // entries
- for (FileStatus fileStatus : fileStatuses) {
- csv.entry((S3AFileStatus) fileStatus);
- }
- // scan through the list, building up a reverse list of all directories
- // found.
- List dirs = new ArrayList<>(fileStatuses.length);
- for (FileStatus fileStatus : fileStatuses) {
- if (fileStatus.isDirectory()
- && !(fileStatus.getPath().equals(path))) {
- // directory: add to the end of the queue.
- dirs.add(fileStatus.getPath());
- } else {
- // file: just increment the count
- count++;
- }
- // now push the dirs list in reverse
- // so that they have been added in the sort order as returned.
- pushAll(queue, dirs);
- }
- }
- return count;
- }
-
- /**
- * Dump the filesystem via a recursive listStatus call.
- * @param csv destination.
- * @return number of entries found.
- * @throws IOException IO failure.
- */
- protected long listStatusFilesystem(
- final CsvFile csv,
- final Path path) throws IOException {
- long count = 0;
- RemoteIterator iterator = getFilesystem()
- .listFilesAndEmptyDirectories(path, true);
- while (iterator.hasNext()) {
- S3ALocatedFileStatus status = iterator.next();
- csv.entry(status.toS3AFileStatus());
- }
- return count;
- }
-
- /**
- * Dump the raw S3 Object Store.
- * @param csv destination.
- * @return number of entries found.
- * @throws IOException IO failure.
- */
- protected long dumpRawS3ObjectStore(
- final CsvFile csv) throws IOException {
- S3AFileSystem fs = getFilesystem();
- long count = 0;
- Path rootPath = fs.qualify(new Path("/"));
- try (AuditSpan span = fs.createSpan("DumpS3GuardDynamoTable",
- rootPath.toString(), null)) {
- Listing listing = fs.getListing();
- S3ListRequest request = listing.createListObjectsRequest("", null, span);
- count = 0;
- RemoteIterator st =
- listing.createFileStatusListingIterator(rootPath, request,
- ACCEPT_ALL,
- new Listing.AcceptAllButSelfAndS3nDirs(rootPath),
- span);
- while (st.hasNext()) {
- count++;
- S3AFileStatus next = st.next();
- LOG.debug("[{}] {}", count, next);
- csv.entry(next);
- }
- LOG.info("entry count: {}", count);
- }
- return count;
- }
-
- /**
- * list children under the metastore from a base path, through
- * a recursive query + walk strategy.
- * @param csv dest
- * @param basePath base path
- * @throws IOException failure.
- */
- protected void dumpMetastore(final CsvFile csv,
- final Path basePath) throws IOException {
- dumpStoreEntries(csv, getStore().listChildren(basePath));
- }
-
- /**
- * Recursive Store Dump.
- * @param csv open CSV file.
- * @param dir directory listing
- * @return (directories, files)
- * @throws IOException failure
- */
- private Pair dumpStoreEntries(
- CsvFile csv,
- DirListingMetadata dir) throws IOException {
- ArrayDeque queue = new ArrayDeque<>();
- queue.add(dir);
- long files = 0, dirs = 1;
- while (!queue.isEmpty()) {
- DirListingMetadata next = queue.pop();
- List childDirs = new ArrayList<>();
- Collection listing = next.getListing();
- // sort by name
- List sorted = new ArrayList<>(listing);
- sorted.sort(new PathOrderComparators.PathMetadataComparator(
- (l, r) -> l.compareTo(r)));
-
- for (PathMetadata pmd : sorted) {
- DDBPathMetadata ddbMd = (DDBPathMetadata) pmd;
- dumpEntry(csv, ddbMd);
- if (ddbMd.getFileStatus().isDirectory()) {
- childDirs.add(ddbMd);
- } else {
- files++;
- }
- }
- List childMD = new ArrayList<>(childDirs.size());
- for (DDBPathMetadata childDir : childDirs) {
- childMD.add(getStore().listChildren(
- childDir.getFileStatus().getPath()));
- }
- pushAll(queue, childMD);
- }
-
- return Pair.of(dirs, files);
- }
-
-
- /**
- * Dump a single entry, and log it.
- * @param csv CSV output file.
- * @param md metadata to log.
- */
- private void dumpEntry(CsvFile csv, DDBPathMetadata md) {
- LOG.debug("{}", md.prettyPrint());
- csv.entry(md);
- }
-
- /**
- * Scan the metastore for all entries and dump them.
- * There's no attempt to sort the output.
- * @param csv file
- * @return tuple of (live entries, tombstones).
- */
- private Pair scanMetastore(CsvFile csv) {
- S3GuardTableAccess tableAccess = new S3GuardTableAccess(getStore());
- ExpressionSpecBuilder builder = new ExpressionSpecBuilder();
- Iterable results =
- getStore().wrapWithRetries(tableAccess.scanMetadata(builder));
- long live = 0;
- long tombstone = 0;
- for (DDBPathMetadata md : results) {
- if (!(md instanceof S3GuardTableAccess.VersionMarker)) {
- // print it
- csv.entry(md);
- if (md.isDeleted()) {
- tombstone++;
- } else {
- live++;
- }
-
- }
- }
- return Pair.of(live, tombstone);
- }
-
- public Pair getScanEntryResult() {
- return scanEntryResult;
- }
-
- public Pair getSecondScanResult() {
- return secondScanResult;
- }
-
- public long getRawObjectStoreCount() {
- return rawObjectStoreCount;
- }
-
- public long getListStatusCount() {
- return listStatusCount;
- }
-
- public long getTreewalkCount() {
- return treewalkCount;
- }
-
- /**
- * Convert a timestamp in milliseconds to a human string.
- * @param millis epoch time in millis
- * @return a string for the CSV file.
- */
- private static String stringify(long millis) {
- return new Date(millis).toString();
- }
-
- /**
- * This is the JVM entry point for the service launcher.
- *
- * Converts the arguments to a list, then invokes
- * {@link #serviceMain(List, AbstractS3GuardDynamoDBDiagnostic)}.
- * @param args command line arguments.
- */
- public static void main(String[] args) {
- try {
- serviceMain(Arrays.asList(args), new DumpS3GuardDynamoTable());
- } catch (ExitUtil.ExitException e) {
- ExitUtil.terminate(e);
- }
- }
-
- /**
- * The real main function, which takes the arguments as a list.
- * Argument 0 MUST be the service classname
- * @param argsList the list of arguments
- * @param service service to launch.
- */
- static void serviceMain(
- final List argsList,
- final AbstractS3GuardDynamoDBDiagnostic service) {
- ServiceLauncher serviceLauncher =
- new ServiceLauncher<>(service.getName());
-
- ExitUtil.ExitException ex = serviceLauncher.launchService(
- new Configuration(),
- service,
- argsList,
- false,
- true);
- if (ex != null) {
- throw ex;
- }
- }
-
- /**
- * Entry point to dump the metastore and s3 store world views
- *
- * Both the FS and the store will be dumped: the store is scanned
- * before and after the sequence to show what changes were made to
- * the store during the list operation.
- * @param fs fs to dump. If null a store must be provided.
- * @param store store to dump (fallback to FS)
- * @param conf configuration to use (fallback to fs)
- * @param destFile base name of the output files.
- * @param uri URI of store -only needed if FS is null.
- * @throws ExitUtil.ExitException failure.
- * @return the store
- */
- public static DumpS3GuardDynamoTable dumpStore(
- @Nullable final S3AFileSystem fs,
- @Nullable DynamoDBMetadataStore store,
- @Nullable Configuration conf,
- final File destFile,
- @Nullable URI uri) throws ExitUtil.ExitException {
- ServiceLauncher serviceLauncher =
- new ServiceLauncher<>(NAME);
-
- if (conf == null) {
- conf = checkNotNull(fs, "No filesystem").getConf();
- }
- if (store == null) {
- store = (DynamoDBMetadataStore) checkNotNull(fs, "No filesystem")
- .getMetadataStore();
- }
- DumpS3GuardDynamoTable dump = new DumpS3GuardDynamoTable(fs,
- store,
- destFile,
- uri);
- ExitUtil.ExitException ex = serviceLauncher.launchService(
- conf,
- dump,
- Collections.emptyList(),
- false,
- true);
- if (ex != null && ex.getExitCode() != 0) {
- throw ex;
- }
- LOG.info("Results:");
- Pair r = dump.getScanEntryResult();
- LOG.info("Metastore entries: {}", r);
- LOG.info("Metastore scan total {}, entries {}, tombstones {}",
- r.getLeft() + r.getRight(),
- r.getLeft(),
- r.getRight());
- LOG.info("S3 count {}", dump.getRawObjectStoreCount());
- LOG.info("Treewalk Count {}", dump.getTreewalkCount());
- LOG.info("List Status Count {}", dump.getListStatusCount());
- r = dump.getSecondScanResult();
- if (r != null) {
- LOG.info("Second metastore scan total {}, entries {}, tombstones {}",
- r.getLeft() + r.getRight(),
- r.getLeft(),
- r.getRight());
- }
- return dump;
- }
-
- /**
- * Writer for generating test CSV files.
- *
- * Quotes are manged by passing in a long whose specific bits control
- * whether or not a row is quoted, bit 0 for column 0, etc.
- *
- * There is no escaping of values here.
- */
- private static final class CsvFile implements Closeable {
-
-
- /** constant to quote all columns. */
- public static final long ALL_QUOTES = 0x7fffffff;
-
- /** least significant bit is used for first column; 1 mean 'quote'. */
- public static final int ROW_QUOTE_MAP = 0b1110_1001_1111;
-
- /** quote nothing: {@value}. */
- public static final long NO_QUOTES = 0;
-
- private final Path path;
-
- private final PrintWriter out;
-
- private final String separator;
-
- private final String eol;
-
- private final String quote;
-
- /**
- * Create.
- * @param path filesystem path.
- * @param out output write.
- * @param separator separator of entries.
- * @param eol EOL marker.
- * @param quote quote marker.
- * @throws IOException failure.
- */
- private CsvFile(
- final Path path,
- final PrintWriter out,
- final String separator,
- final String eol,
- final String quote) throws IOException {
- this.separator = checkNotNull(separator);
- this.eol = checkNotNull(eol);
- this.quote = checkNotNull(quote);
- this.path = path;
- this.out = checkNotNull(out);
- header();
- }
-
- /**
- * Create to a file, with UTF-8 output and the standard
- * options of the TSV file.
- * @param file destination file.
- * @throws IOException failure.
- */
- private CsvFile(File file) throws IOException {
- this(null,
- new PrintWriter(file, "UTF-8"), "\t", "\n", "\"");
- }
-
- /**
- * Close the file, if not already done.
- * @throws IOException on a failure.
- */
- @Override
- public synchronized void close() throws IOException {
- if (out != null) {
- out.close();
- }
- }
-
- public Path getPath() {
- return path;
- }
-
- public String getSeparator() {
- return separator;
- }
-
- public String getEol() {
- return eol;
- }
-
- /**
- * Write a row.
- * Entries are quoted if the bit for that column is true.
- * @param quotes quote policy: every bit defines the rule for that element
- * @param columns columns to write
- * @return self for ease of chaining.
- */
- public CsvFile row(long quotes, Object... columns) {
- checkNotNull(out);
- for (int i = 0; i < columns.length; i++) {
- if (i != 0) {
- out.write(separator);
- }
- boolean toQuote = (quotes & 1) == 1;
- // unsigned right shift to make next column flag @ position 0
- quotes = quotes >>> 1;
- if (toQuote) {
- out.write(quote);
- }
- Object column = columns[i];
- out.write(column != null ? column.toString() : "");
- if (toQuote) {
- out.write(quote);
- }
- }
- out.write(eol);
- return this;
- }
-
- /**
- * Write a line.
- * @param line line to print
- * @return self for ease of chaining.
- */
- public CsvFile line(String line) {
- out.write(line);
- out.write(eol);
- return this;
- }
-
- /**
- * Get the output stream.
- * @return the stream.
- */
- public PrintWriter getOut() {
- return out;
- }
-
- /**
- * Print the header.
- */
- void header() {
- row(CsvFile.ALL_QUOTES,
- "type",
- "deleted",
- "path",
- "is_auth_dir",
- "is_empty_dir",
- "len",
- "updated",
- "updated_s",
- "last_modified",
- "last_modified_s",
- "etag",
- "version");
- }
-
- /**
- * Add a metadata entry.
- * @param md metadata.
- */
- void entry(DDBPathMetadata md) {
- S3AFileStatus fileStatus = md.getFileStatus();
- row(ROW_QUOTE_MAP,
- fileStatus.isDirectory() ? "dir" : "file",
- md.isDeleted(),
- fileStatus.getPath().toString(),
- md.isAuthoritativeDir(),
- md.isEmptyDirectory().name(),
- fileStatus.getLen(),
- md.getLastUpdated(),
- stringify(md.getLastUpdated()),
- fileStatus.getModificationTime(),
- stringify(fileStatus.getModificationTime()),
- fileStatus.getETag(),
- fileStatus.getVersionId());
- }
-
- /**
- * filesystem entry: no metadata.
- * @param fileStatus file status
- */
- void entry(S3AFileStatus fileStatus) {
- row(ROW_QUOTE_MAP,
- fileStatus.isDirectory() ? "dir" : "file",
- "false",
- fileStatus.getPath().toString(),
- "",
- fileStatus.isEmptyDirectory().name(),
- fileStatus.getLen(),
- "",
- "",
- fileStatus.getModificationTime(),
- stringify(fileStatus.getModificationTime()),
- fileStatus.getETag(),
- fileStatus.getVersionId());
- }
- }
-
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
deleted file mode 100644
index e4275ccb51c..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import java.io.IOException;
-
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.regions.Regions;
-import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
-import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClientBuilder;
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.s3a.Constants;
-import org.apache.hadoop.fs.s3a.S3AUtils;
-
-import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_REGION_KEY;
-
-/**
- * Interface to create a DynamoDB client.
- *
- * Implementation should be configured for setting and getting configuration.
- */
-@InterfaceAudience.Private
-public interface DynamoDBClientFactory extends Configurable {
- Logger LOG = LoggerFactory.getLogger(DynamoDBClientFactory.class);
-
- /**
- * Create a DynamoDB client object from configuration.
- *
- * The DynamoDB client to create does not have to relate to any S3 buckets.
- * All information needed to create a DynamoDB client is from the hadoop
- * configuration. Specially, if the region is not configured, it will use the
- * provided region parameter. If region is neither configured nor provided,
- * it will indicate an error.
- *
- * @param defaultRegion the default region of the AmazonDynamoDB client
- * @param bucket Optional bucket to use to look up per-bucket proxy secrets
- * @param credentials credentials to use for authentication.
- * @return a new DynamoDB client
- * @throws IOException if any IO error happens
- */
- AmazonDynamoDB createDynamoDBClient(final String defaultRegion,
- final String bucket,
- final AWSCredentialsProvider credentials) throws IOException;
-
- /**
- * The default implementation for creating an AmazonDynamoDB.
- */
- class DefaultDynamoDBClientFactory extends Configured
- implements DynamoDBClientFactory {
- @Override
- public AmazonDynamoDB createDynamoDBClient(String defaultRegion,
- final String bucket,
- final AWSCredentialsProvider credentials)
- throws IOException {
- Preconditions.checkNotNull(getConf(),
- "Should have been configured before usage");
-
- final Configuration conf = getConf();
- final ClientConfiguration awsConf = S3AUtils
- .createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_DDB);
-
- final String region = getRegion(conf, defaultRegion);
- LOG.debug("Creating DynamoDB client in region {}", region);
-
- return AmazonDynamoDBClientBuilder.standard()
- .withCredentials(credentials)
- .withClientConfiguration(awsConf)
- .withRegion(region)
- .build();
- }
-
- /**
- * Helper method to get and validate the AWS region for DynamoDBClient.
- *
- * @param conf configuration
- * @param defaultRegion the default region
- * @return configured region or else the provided default region
- * @throws IOException if the region is not valid
- */
- static String getRegion(Configuration conf, String defaultRegion)
- throws IOException {
- String region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
- if (StringUtils.isEmpty(region)) {
- region = defaultRegion;
- }
- try {
- Regions.fromName(region);
- } catch (IllegalArgumentException | NullPointerException e) {
- throw new IOException("Invalid region specified: " + region + "; " +
- "Region can be configured with " + S3GUARD_DDB_REGION_KEY + ": " +
- validRegionsString());
- }
- return region;
- }
-
- private static String validRegionsString() {
- final String delimiter = ", ";
- Regions[] regions = Regions.values();
- StringBuilder sb = new StringBuilder();
- for (int i = 0; i < regions.length; i++) {
- if (i > 0) {
- sb.append(delimiter);
- }
- sb.append(regions[i].getName());
- }
- return sb.toString();
-
- }
- }
-
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
deleted file mode 100644
index 3c991b35453..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
+++ /dev/null
@@ -1,2534 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import javax.annotation.Nullable;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.io.UncheckedIOException;
-import java.net.URI;
-import java.nio.file.AccessDeniedException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-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.CompletableFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.stream.Collectors;
-
-import com.amazonaws.AmazonServiceException;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
-import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
-import com.amazonaws.services.dynamodbv2.document.DynamoDB;
-import com.amazonaws.services.dynamodbv2.document.Item;
-import com.amazonaws.services.dynamodbv2.document.ItemCollection;
-import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
-import com.amazonaws.services.dynamodbv2.document.PutItemOutcome;
-import com.amazonaws.services.dynamodbv2.document.QueryOutcome;
-import com.amazonaws.services.dynamodbv2.document.ScanOutcome;
-import com.amazonaws.services.dynamodbv2.document.Table;
-import com.amazonaws.services.dynamodbv2.document.TableWriteItems;
-import com.amazonaws.services.dynamodbv2.document.internal.IteratorSupport;
-import com.amazonaws.services.dynamodbv2.document.spec.GetItemSpec;
-import com.amazonaws.services.dynamodbv2.document.spec.QuerySpec;
-import com.amazonaws.services.dynamodbv2.document.utils.ValueMap;
-import com.amazonaws.services.dynamodbv2.model.AmazonDynamoDBException;
-import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
-import com.amazonaws.services.dynamodbv2.model.TableDescription;
-import com.amazonaws.services.dynamodbv2.model.WriteRequest;
-
-import org.apache.hadoop.fs.s3a.impl.InternalConstants;
-import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
-import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService;
-import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathIOException;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.util.functional.CallableRaisingIOE;
-import org.apache.hadoop.util.functional.RemoteIterators;
-import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
-import org.apache.hadoop.fs.s3a.AWSServiceThrottledException;
-import org.apache.hadoop.fs.s3a.Constants;
-import org.apache.hadoop.fs.s3a.Invoker;
-import org.apache.hadoop.fs.s3a.Retries;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.S3AUtils;
-import org.apache.hadoop.fs.s3a.Tristate;
-import org.apache.hadoop.fs.s3a.auth.RoleModel;
-import org.apache.hadoop.fs.s3a.auth.RolePolicies;
-import org.apache.hadoop.fs.s3a.auth.delegation.AWSPolicyProvider;
-import org.apache.hadoop.fs.s3a.impl.StoreContext;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
-import org.apache.hadoop.util.DurationInfo;
-import org.apache.hadoop.util.ReflectionUtils;
-
-import static org.apache.hadoop.fs.s3a.Constants.*;
-import static org.apache.hadoop.fs.s3a.S3AUtils.*;
-import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowAllDynamoDBOperations;
-import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3GuardClientOperations;
-import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
-import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion;
-import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
-import static org.apache.hadoop.fs.s3a.s3guard.PathOrderComparators.TOPMOST_PM_LAST;
-import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*;
-
-/**
- * DynamoDBMetadataStore is a {@link MetadataStore} that persists
- * file system metadata to DynamoDB.
- *
- * The current implementation uses a schema consisting of a single table. The
- * name of the table can be configured by config key
- * {@link org.apache.hadoop.fs.s3a.Constants#S3GUARD_DDB_TABLE_NAME_KEY}.
- * By default, it matches the name of the S3 bucket. Each item in the table
- * represents a single directory or file. Its path is split into separate table
- * attributes:
- *
- * - parent (absolute path of the parent, with bucket name inserted as
- * first path component).
- * - child (path of that specific child, relative to parent).
- * - optional boolean attribute tracking whether the path is a directory.
- * Absence or a false value indicates the path is a file.
- * - optional long attribute revealing modification time of file.
- * This attribute is meaningful only to file items.
- * - optional long attribute revealing file length.
- * This attribute is meaningful only to file items.
- * - optional long attribute revealing block size of the file.
- * This attribute is meaningful only to file items.
- * - optional string attribute tracking the s3 eTag of the file.
- * May be absent if the metadata was entered with a version of S3Guard
- * before this was tracked.
- * This attribute is meaningful only to file items.
- * - optional string attribute tracking the s3 versionId of the file.
- * May be absent if the metadata was entered with a version of S3Guard
- * before this was tracked.
- * This attribute is meaningful only to file items.
- *
- *
- * The DynamoDB partition key is the parent, and the range key is the child.
- *
- * To allow multiple buckets to share the same DynamoDB table, the bucket
- * name is treated as the root directory.
- *
- * For example, assume the consistent store contains metadata representing this
- * file system structure:
- *
- *
- * s3a://bucket/dir1
- * |-- dir2
- * | |-- file1
- * | `-- file2
- * `-- dir3
- * |-- dir4
- * | `-- file3
- * |-- dir5
- * | `-- file4
- * `-- dir6
- *
- *
- * This is persisted to a single DynamoDB table as:
- *
- *
- * ====================================================================================
- * | parent | child | is_dir | mod_time | len | etag | ver_id | ... |
- * ====================================================================================
- * | /bucket | dir1 | true | | | | | |
- * | /bucket/dir1 | dir2 | true | | | | | |
- * | /bucket/dir1 | dir3 | true | | | | | |
- * | /bucket/dir1/dir2 | file1 | | 100 | 111 | abc | mno | |
- * | /bucket/dir1/dir2 | file2 | | 200 | 222 | def | pqr | |
- * | /bucket/dir1/dir3 | dir4 | true | | | | | |
- * | /bucket/dir1/dir3 | dir5 | true | | | | | |
- * | /bucket/dir1/dir3/dir4 | file3 | | 300 | 333 | ghi | stu | |
- * | /bucket/dir1/dir3/dir5 | file4 | | 400 | 444 | jkl | vwx | |
- * | /bucket/dir1/dir3 | dir6 | true | | | | | |
- * ====================================================================================
- *
- *
- * This choice of schema is efficient for read access patterns.
- * {@link #get(Path)} can be served from a single item lookup.
- * {@link #listChildren(Path)} can be served from a query against all rows
- * matching the parent (the partition key) and the returned list is guaranteed
- * to be sorted by child (the range key). Tracking whether or not a path is a
- * directory helps prevent unnecessary queries during traversal of an entire
- * sub-tree.
- *
- * Some mutating operations, notably
- * {@link MetadataStore#deleteSubtree(Path, BulkOperationState)} and
- * {@link MetadataStore#move(Collection, Collection, BulkOperationState)}
- * are less efficient with this schema.
- * They require mutating multiple items in the DynamoDB table.
- *
- * By default, DynamoDB access is performed within the same AWS region as
- * the S3 bucket that hosts the S3A instance. During initialization, it checks
- * the location of the S3 bucket and creates a DynamoDB client connected to the
- * same region. The region may also be set explicitly by setting the config
- * parameter {@code fs.s3a.s3guard.ddb.region} to the corresponding region.
- */
-@SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class DynamoDBMetadataStore implements MetadataStore,
- AWSPolicyProvider {
- public static final Logger LOG = LoggerFactory.getLogger(
- DynamoDBMetadataStore.class);
-
- /**
- * Name of the operations log.
- */
- public static final String OPERATIONS_LOG_NAME =
- "org.apache.hadoop.fs.s3a.s3guard.Operations";
-
- /**
- * A log of all state changing operations to the store;
- * only updated at debug level.
- */
- public static final Logger OPERATIONS_LOG = LoggerFactory.getLogger(
- OPERATIONS_LOG_NAME);
-
- /** parent/child name to use in the version marker. */
- public static final String VERSION_MARKER_ITEM_NAME = "../VERSION";
-
- /** parent/child name to use in the version marker. */
- public static final String VERSION_MARKER_TAG_NAME = "s3guard_version";
-
- /** Current version number. */
- public static final int VERSION = 100;
-
- @VisibleForTesting
- static final String BILLING_MODE
- = "billing-mode";
-
- @VisibleForTesting
- static final String BILLING_MODE_PER_REQUEST
- = "per-request";
-
- @VisibleForTesting
- static final String BILLING_MODE_PROVISIONED
- = "provisioned";
-
- @VisibleForTesting
- static final String DESCRIPTION
- = "S3Guard metadata store in DynamoDB";
- @VisibleForTesting
- static final String READ_CAPACITY = "read-capacity";
- @VisibleForTesting
- static final String WRITE_CAPACITY = "write-capacity";
- @VisibleForTesting
- static final String STATUS = "status";
- @VisibleForTesting
- static final String TABLE = "table";
-
- @VisibleForTesting
- static final String HINT_DDB_IOPS_TOO_LOW
- = " This may be because the write threshold of DynamoDB is set too low.";
-
- @VisibleForTesting
- static final String THROTTLING = "Throttling";
-
- public static final String E_ON_DEMAND_NO_SET_CAPACITY
- = "Neither ReadCapacityUnits nor WriteCapacityUnits can be specified when BillingMode is PAY_PER_REQUEST";
-
- @VisibleForTesting
- static final String E_INCONSISTENT_UPDATE
- = "Duplicate and inconsistent entry in update operation";
-
- private static final ValueMap DELETE_TRACKING_VALUE_MAP =
- new ValueMap().withBoolean(":false", false);
-
- /**
- * The maximum number of outstanding operations to submit
- * before blocking to await completion of all the executors.
- * Paging work like this is less efficient, but it ensures that
- * failure (auth, network, etc) are picked up before many more
- * operations are submitted.
- *
- * Arbitrary Choice.
- * Value: {@value}.
- */
- private static final int S3GUARD_DDB_SUBMITTED_TASK_LIMIT = 50;
-
- private AmazonDynamoDB amazonDynamoDB;
- private DynamoDB dynamoDB;
- private AWSCredentialProviderList credentials;
- private String region;
- private Table table;
- private String tableName;
- private Configuration conf;
- private String username;
-
- /**
- * This policy is mostly for batched writes, not for processing
- * exceptions in invoke() calls.
- * It also has a role purpose in
- * {@link DynamoDBMetadataStoreTableManager#getVersionMarkerItem()};
- * look at that method for the details.
- */
- private RetryPolicy batchWriteRetryPolicy;
-
- /**
- * The instrumentation is never null -if/when bound to an owner file system
- * That filesystem statistics will be updated as appropriate.
- */
- private MetastoreInstrumentation instrumentation
- = new MetastoreInstrumentationImpl();
-
- /** Owner FS: only valid if configured with an owner FS. */
- private S3AFileSystem owner;
-
- /** Invoker for IO. Until configured properly, use try-once. */
- private Invoker invoker = new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL,
- Invoker.NO_OP
- );
-
- /** Invoker for read operations. */
- private Invoker readOp;
-
- /** Invoker for write operations. */
- private Invoker writeOp;
-
- /** Invoker for scan operations. */
- private Invoker scanOp;
-
- private final AtomicLong readThrottleEvents = new AtomicLong(0);
- private final AtomicLong writeThrottleEvents = new AtomicLong(0);
- private final AtomicLong scanThrottleEvents = new AtomicLong(0);
- private final AtomicLong batchWriteCapacityExceededEvents = new AtomicLong(0);
-
- /**
- * Total limit on the number of throttle events after which
- * we stop warning in the log. Keeps the noise down.
- */
- private static final int THROTTLE_EVENT_LOG_LIMIT = 100;
-
- /**
- * Count of the total number of throttle events; used to crank back logging.
- */
- private AtomicInteger throttleEventCount = new AtomicInteger(0);
-
- /**
- * Executor for submitting operations.
- */
- private ListeningExecutorService executor;
-
- /**
- * Time source. This is used during writes when parent
- * entries need to be created.
- */
- private ITtlTimeProvider ttlTimeProvider;
-
- private DynamoDBMetadataStoreTableManager tableHandler;
-
- /**
- * A utility function to create DynamoDB instance.
- * @param conf the file system configuration
- * @param s3Region region of the associated S3 bucket (if any).
- * @param bucket Optional bucket to use to look up per-bucket proxy secrets
- * @param credentials credentials.
- * @return DynamoDB instance.
- * @throws IOException I/O error.
- */
- private DynamoDB createDynamoDB(
- final Configuration conf,
- final String s3Region,
- final String bucket,
- final AWSCredentialsProvider credentials)
- throws IOException {
- if (amazonDynamoDB == null) {
- Preconditions.checkNotNull(conf);
- final Class extends DynamoDBClientFactory> cls =
- conf.getClass(S3GUARD_DDB_CLIENT_FACTORY_IMPL,
- S3GUARD_DDB_CLIENT_FACTORY_IMPL_DEFAULT, DynamoDBClientFactory.class);
- LOG.debug("Creating DynamoDB client {} with S3 region {}", cls, s3Region);
- amazonDynamoDB = ReflectionUtils.newInstance(cls, conf)
- .createDynamoDBClient(s3Region, bucket, credentials);
- }
- return new DynamoDB(amazonDynamoDB);
- }
-
- /**
- * {@inheritDoc}.
- * The credentials for authenticating with S3 are requested from the
- * FS via {@link S3AFileSystem#shareCredentials(String)}; this will
- * increment the reference counter of these credentials.
- * @param fs {@code S3AFileSystem} associated with the MetadataStore
- * @param ttlTp the time provider to use for metadata expiry
- * @throws IOException on a failure
- */
- @Override
- @Retries.OnceRaw
- public void initialize(FileSystem fs, ITtlTimeProvider ttlTp)
- throws IOException {
- Preconditions.checkNotNull(fs, "Null filesystem");
- Preconditions.checkArgument(fs instanceof S3AFileSystem,
- "DynamoDBMetadataStore only supports S3A filesystem - not %s",
- fs);
- bindToOwnerFilesystem((S3AFileSystem) fs);
- final String bucket = owner.getBucket();
- String confRegion = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
- if (!StringUtils.isEmpty(confRegion)) {
- region = confRegion;
- LOG.debug("Overriding S3 region with configured DynamoDB region: {}",
- region);
- } else {
- try {
- region = owner.getBucketLocation();
- } catch (AccessDeniedException e) {
- // access denied here == can't call getBucket. Report meaningfully
- URI uri = owner.getUri();
- String message =
- "Failed to get bucket location as client lacks permission "
- + RolePolicies.S3_GET_BUCKET_LOCATION + " for " + uri;
- LOG.error(message);
- throw (IOException)new AccessDeniedException(message).initCause(e);
- }
- LOG.debug("Inferring DynamoDB region from S3 bucket: {}", region);
- }
- credentials = owner.shareCredentials("s3guard");
- dynamoDB = createDynamoDB(conf, region, bucket, credentials);
-
- // use the bucket as the DynamoDB table name if not specified in config
- tableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY, bucket);
- initDataAccessRetries(conf);
-
- this.ttlTimeProvider = ttlTp;
-
- tableHandler = new DynamoDBMetadataStoreTableManager(
- dynamoDB, tableName, region, amazonDynamoDB, conf, readOp,
- batchWriteRetryPolicy);
- this.table = tableHandler.initTable();
-
- instrumentation.initialized();
- }
-
- /**
- * Declare that this table is owned by the specific S3A FS instance.
- * This will bind some fields to the values provided by the owner,
- * including wiring up the instrumentation.
- * @param fs owner filesystem
- */
- @VisibleForTesting
- void bindToOwnerFilesystem(final S3AFileSystem fs) {
- owner = fs;
- conf = owner.getConf();
- StoreContext context = owner.createStoreContext();
- instrumentation = context.getInstrumentation()
- .getS3GuardInstrumentation();
- username = context.getUsername();
- executor = MoreExecutors.listeningDecorator(
- context.createThrottledExecutor());
- ttlTimeProvider = Preconditions.checkNotNull(
- context.getTimeProvider(),
- "ttlTimeProvider must not be null");
- }
-
- /**
- * Performs one-time initialization of the metadata store via configuration.
- *
- * This initialization depends on the configuration object to get AWS
- * credentials, DynamoDBFactory implementation class, DynamoDB endpoints,
- * DynamoDB table names etc. After initialization, this metadata store does
- * not explicitly relate to any S3 bucket, which be nonexistent.
- *
- * This is used to operate the metadata store directly beyond the scope of the
- * S3AFileSystem integration, e.g. command line tools.
- * Generally, callers should use
- * {@link MetadataStore#initialize(FileSystem, ITtlTimeProvider)}
- * with an initialized {@code S3AFileSystem} instance.
- *
- * Without a filesystem to act as a reference point, the configuration itself
- * must declare the table name and region in the
- * {@link Constants#S3GUARD_DDB_TABLE_NAME_KEY} and
- * {@link Constants#S3GUARD_DDB_REGION_KEY} respectively.
- * It also creates a new credential provider list from the configuration,
- * using the base fs.s3a.* options, as there is no bucket to infer per-bucket
- * settings from.
- *
- * @see MetadataStore#initialize(FileSystem, ITtlTimeProvider)
- * @throws IOException if there is an error
- * @throws IllegalArgumentException if the configuration is incomplete
- */
- @Override
- @Retries.OnceRaw
- public void initialize(Configuration config,
- ITtlTimeProvider ttlTp) throws IOException {
- conf = config;
- // use the bucket as the DynamoDB table name if not specified in config
- tableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY);
-
- Preconditions.checkArgument(!StringUtils.isEmpty(tableName),
- "No DynamoDB table name configured");
- region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
- Preconditions.checkArgument(!StringUtils.isEmpty(region),
- "No DynamoDB region configured");
- // there's no URI here, which complicates life: you cannot
- // create AWS providers here which require one.
- credentials = createAWSCredentialProviderSet(null, conf);
- dynamoDB = createDynamoDB(conf, region, null, credentials);
-
- username = UserGroupInformation.getCurrentUser().getShortUserName();
- // without an executor from the owner FS, create one using
- // the executor capacity for work.
- int executorCapacity = intOption(conf,
- EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1);
- executor = MoreExecutors.listeningDecorator(
- BlockingThreadPoolExecutorService.newInstance(
- executorCapacity,
- executorCapacity * 2,
- longOption(conf, KEEPALIVE_TIME,
- DEFAULT_KEEPALIVE_TIME, 0),
- TimeUnit.SECONDS,
- "s3a-ddb-" + tableName));
- initDataAccessRetries(conf);
- this.ttlTimeProvider = ttlTp;
-
- tableHandler = new DynamoDBMetadataStoreTableManager(
- dynamoDB, tableName, region, amazonDynamoDB, conf, readOp,
- batchWriteRetryPolicy);
- this.table = tableHandler.initTable();
- }
-
- /**
- * Set retry policy. This is driven by the value of
- * {@link Constants#S3GUARD_DDB_MAX_RETRIES} with an exponential backoff
- * between each attempt of {@link Constants#S3GUARD_DDB_THROTTLE_RETRY_INTERVAL}
- * milliseconds.
- * @param config configuration for data access
- */
- private void initDataAccessRetries(Configuration config) {
- batchWriteRetryPolicy = RetryPolicies
- .exponentialBackoffRetry(
- config.getInt(S3GUARD_DDB_MAX_RETRIES,
- S3GUARD_DDB_MAX_RETRIES_DEFAULT),
- conf.getTimeDuration(S3GUARD_DDB_THROTTLE_RETRY_INTERVAL,
- S3GUARD_DDB_THROTTLE_RETRY_INTERVAL_DEFAULT,
- TimeUnit.MILLISECONDS),
- TimeUnit.MILLISECONDS);
- final RetryPolicy throttledRetryRetryPolicy
- = new S3GuardDataAccessRetryPolicy(config);
- readOp = new Invoker(throttledRetryRetryPolicy, this::readRetryEvent);
- writeOp = new Invoker(throttledRetryRetryPolicy, this::writeRetryEvent);
- scanOp = new Invoker(throttledRetryRetryPolicy, this::scanRetryEvent);
- }
-
- @Override
- @Retries.RetryTranslated
- public void delete(Path path,
- final BulkOperationState operationState)
- throws IOException {
- innerDelete(path, true,
- extractOrCreate(operationState,
- BulkOperationState.OperationType.Delete));
- }
-
- @Override
- @Retries.RetryTranslated
- public void forgetMetadata(Path path) throws IOException {
- LOG.debug("Forget metadata for {}", path);
- innerDelete(path, false, null);
- }
-
- /**
- * Inner delete option, action based on the {@code tombstone} flag.
- * No tombstone: delete the entry. Tombstone: create a tombstone entry.
- * There is no check as to whether the entry exists in the table first.
- * @param path path to delete
- * @param tombstone flag to create a tombstone marker
- * @param ancestorState ancestor state for context.
- * @throws IOException I/O error.
- */
- @Retries.RetryTranslated
- private void innerDelete(final Path path,
- final boolean tombstone,
- final AncestorState ancestorState)
- throws IOException {
- checkPath(path);
- LOG.debug("Deleting from table {} in region {}: {}",
- tableName, region, path);
-
- // deleting nonexistent item consumes 1 write capacity; skip it
- if (path.isRoot()) {
- LOG.debug("Skip deleting root directory as it does not exist in table");
- return;
- }
- // the policy on whether repeating delete operations is based
- // on that of S3A itself
- boolean idempotent = InternalConstants.DELETE_CONSIDERED_IDEMPOTENT;
- if (tombstone) {
- Preconditions.checkArgument(ttlTimeProvider != null, "ttlTimeProvider "
- + "must not be null");
- final PathMetadata pmTombstone = PathMetadata.tombstone(path,
- ttlTimeProvider.getNow());
- Item item = PathMetadataDynamoDBTranslation.pathMetadataToItem(
- new DDBPathMetadata(pmTombstone));
- writeOp.retry(
- "Put tombstone",
- path.toString(),
- idempotent,
- () -> {
- logPut(ancestorState, item);
- recordsWritten(1);
- table.putItem(item);
- });
- } else {
- PrimaryKey key = pathToKey(path);
- writeOp.retry(
- "Delete key",
- path.toString(),
- idempotent,
- () -> {
- // record the attempt so even on retry the counter goes up.
- logDelete(ancestorState, key);
- recordsDeleted(1);
- table.deleteItem(key);
- });
- }
- }
-
- @Override
- @Retries.RetryTranslated
- public void deleteSubtree(Path path,
- final BulkOperationState operationState)
- throws IOException {
- checkPath(path);
- LOG.debug("Deleting subtree from table {} in region {}: {}",
- tableName, region, path);
-
- final PathMetadata meta = get(path);
- if (meta == null) {
- LOG.debug("Subtree path {} does not exist; this will be a no-op", path);
- return;
- }
- if (meta.isDeleted()) {
- LOG.debug("Subtree path {} is deleted; this will be a no-op", path);
- return;
- }
- deleteEntries(RemoteIterators.mappingRemoteIterator(
- new DescendantsIterator(this, meta),
- FileStatus::getPath),
- operationState);
- }
-
- @Override
- @Retries.RetryTranslated
- public void deletePaths(Collection paths,
- final BulkOperationState operationState)
- throws IOException {
- deleteEntries(RemoteIterators.remoteIteratorFromIterable(paths),
- operationState);
- }
-
- /**
- * Delete the entries under an iterator.
- * There's no attempt to order the paths: they are
- * deleted in the order passed in.
- * @param entries entries to delete.
- * @param operationState Nullable operation state
- * @throws IOException failure
- */
- @Retries.RetryTranslated
- private void deleteEntries(RemoteIterator entries,
- final BulkOperationState operationState)
- throws IOException {
- final List> futures = new ArrayList<>();
- AncestorState state = extractOrCreate(operationState,
- BulkOperationState.OperationType.Delete);
-
- while (entries.hasNext()) {
- final Path pathToDelete = entries.next();
- futures.add(submit(executor, () -> {
- innerDelete(pathToDelete, true, state);
- return null;
- }));
- if (futures.size() > S3GUARD_DDB_SUBMITTED_TASK_LIMIT) {
- // first batch done; block for completion.
- waitForCompletion(futures);
- futures.clear();
- }
- }
- // now wait for the final set.
- waitForCompletion(futures);
- }
-
- /**
- * Get a consistent view of an item.
- * @param path path to look up in the database
- * @return the result
- * @throws IOException failure
- */
- @Retries.RetryTranslated
- private Item getConsistentItem(final Path path) throws IOException {
- PrimaryKey key = pathToKey(path);
- final GetItemSpec spec = new GetItemSpec()
- .withPrimaryKey(key)
- .withConsistentRead(true); // strictly consistent read
- return readOp.retry("get",
- path.toString(),
- true,
- () -> {
- recordsRead(1);
- return table.getItem(spec);
- });
- }
-
- @Override
- @Retries.RetryTranslated
- public DDBPathMetadata get(Path path) throws IOException {
- return get(path, false);
- }
-
- @Override
- @Retries.RetryTranslated
- public DDBPathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
- throws IOException {
- checkPath(path);
- LOG.debug("Get from table {} in region {}: {} ; wantEmptyDirectory={}",
- tableName, region, path, wantEmptyDirectoryFlag);
- DDBPathMetadata result = innerGet(path, wantEmptyDirectoryFlag);
- LOG.debug("result of get {} is: {}", path, result);
- return result;
- }
-
- /**
- * Inner get operation, as invoked in the retry logic.
- * @param path the path to get
- * @param wantEmptyDirectoryFlag Set to true to give a hint to the
- * MetadataStore that it should try to compute the empty directory flag.
- * @return metadata for {@code path}, {@code null} if not found
- * @throws IOException IO problem
- */
- @Retries.RetryTranslated
- private DDBPathMetadata innerGet(Path path, boolean wantEmptyDirectoryFlag)
- throws IOException {
- final DDBPathMetadata meta;
- if (path.isRoot()) {
- // Root does not persist in the table
- meta =
- new DDBPathMetadata(makeDirStatus(username, path));
- } else {
- final Item item = getConsistentItem(path);
- meta = itemToPathMetadata(item, username);
- LOG.debug("Get from table {} in region {} returning for {}: {}",
- tableName, region, path, meta);
- }
-
- if (wantEmptyDirectoryFlag && meta != null && !meta.isDeleted()) {
- final FileStatus status = meta.getFileStatus();
- // for a non-deleted directory, we query its direct undeleted children
- // to determine the isEmpty bit. There's no TTL checking going on here.
- if (status.isDirectory()) {
- final QuerySpec spec = new QuerySpec()
- .withHashKey(pathToParentKeyAttribute(path))
- .withConsistentRead(true)
- .withFilterExpression(IS_DELETED + " = :false")
- .withValueMap(DELETE_TRACKING_VALUE_MAP);
- boolean hasChildren = readOp.retry("get/hasChildren",
- path.toString(),
- true,
- () -> {
- // issue the query
- final IteratorSupport- it = table.query(
- spec).iterator();
- // if non empty, log the result to aid with some debugging
- if (it.hasNext()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Dir {} is non-empty", status.getPath());
- while(it.hasNext()) {
- LOG.debug("{}", itemToPathMetadata(it.next(), username));
- }
- }
- return true;
- } else {
- return false;
- }
- });
-
- // If directory is authoritative, we can set the empty directory flag
- // to TRUE or FALSE. Otherwise FALSE, or UNKNOWN.
- if (meta.isAuthoritativeDir()) {
- meta.setIsEmptyDirectory(
- hasChildren ? Tristate.FALSE : Tristate.TRUE);
- } else {
- meta.setIsEmptyDirectory(
- hasChildren ? Tristate.FALSE : Tristate.UNKNOWN);
- }
- }
- }
-
- return meta;
- }
-
- /**
- * Make a S3AFileStatus object for a directory at given path.
- * The FileStatus only contains what S3A needs, and omits mod time
- * since S3A uses its own implementation which returns current system time.
- * @param dirOwner username of owner
- * @param path path to dir
- * @return new S3AFileStatus
- */
- private S3AFileStatus makeDirStatus(String dirOwner, Path path) {
- return new S3AFileStatus(Tristate.UNKNOWN, path, dirOwner);
- }
-
- @Override
- @Retries.RetryTranslated
- public DirListingMetadata listChildren(final Path path) throws IOException {
- checkPath(path);
- LOG.debug("Listing table {} in region {}: {}", tableName, region, path);
-
- final QuerySpec spec = new QuerySpec()
- .withHashKey(pathToParentKeyAttribute(path))
- .withConsistentRead(true); // strictly consistent read
- final List metas = new ArrayList<>();
- // find the children in the table
- final ItemCollection items = scanOp.retry(
- "listChildren",
- path.toString(),
- true,
- () -> table.query(spec));
- // now wrap the result with retry logic
- try {
- for (Item item : wrapWithRetries(items)) {
- metas.add(itemToPathMetadata(item, username));
- }
- } catch (UncheckedIOException e) {
- // failure in the iterators; unwrap.
- throw e.getCause();
- }
-
- // Minor race condition here - if the path is deleted between
- // getting the list of items and the directory metadata we might
- // get a null in DDBPathMetadata.
- return getDirListingMetadataFromDirMetaAndList(path, metas,
- get(path));
- }
-
- DirListingMetadata getDirListingMetadataFromDirMetaAndList(Path path,
- List metas, DDBPathMetadata dirPathMeta) {
- boolean isAuthoritative = false;
- if (dirPathMeta != null) {
- isAuthoritative = dirPathMeta.isAuthoritativeDir();
- }
-
- LOG.trace("Listing table {} in region {} for {} returning {}",
- tableName, region, path, metas);
-
- if (!metas.isEmpty() && dirPathMeta == null) {
- // We handle this case as the directory is deleted.
- LOG.warn("Directory marker is deleted, but the list of the directory "
- + "elements is not empty: {}. This case is handled as if the "
- + "directory was deleted.", metas);
- return null;
- }
-
- if(metas.isEmpty() && dirPathMeta == null) {
- return null;
- }
-
- return new DirListingMetadata(path, metas, isAuthoritative,
- dirPathMeta.getLastUpdated());
- }
-
- /**
- * Origin of entries in the ancestor map built up in
- * {@link #completeAncestry(Collection, AncestorState)}.
- * This is done to stop generated ancestor entries to overwriting those
- * in the store, while allowing those requested in the API call to do this.
- */
- private enum EntryOrigin {
- Requested, // requested in method call
- Retrieved, // retrieved from DDB: do not resubmit
- Generated // generated ancestor.
- }
-
- /**
- * Build the list of all parent entries.
- *
- * Thread safety: none. Callers must synchronize access.
- *
- * Callers are required to synchronize on ancestorState.
- * @param pathsToCreate paths to create
- * @param ancestorState ongoing ancestor state.
- * @return the full ancestry paths
- */
- private Collection completeAncestry(
- final Collection pathsToCreate,
- final AncestorState ancestorState) throws IOException {
- // Key on path to allow fast lookup
- Map> ancestry = new HashMap<>();
- LOG.debug("Completing ancestry for {} paths", pathsToCreate.size());
- // we sort the inputs to guarantee that the topmost entries come first.
- // that way if the put request contains both parents and children
- // then the existing parents will not be re-created -they will just
- // be added to the ancestor list first.
- List sortedPaths = new ArrayList<>(pathsToCreate);
- sortedPaths.sort(PathOrderComparators.TOPMOST_PM_FIRST);
- // iterate through the paths.
- for (DDBPathMetadata entry : sortedPaths) {
- Preconditions.checkArgument(entry != null);
- Path path = entry.getFileStatus().getPath();
- LOG.debug("Adding entry {}", path);
- if (path.isRoot()) {
- // this is a root entry: do not add it.
- break;
- }
- // add it to the ancestor state, failing if it is already there and
- // of a different type.
- DDBPathMetadata oldEntry = ancestorState.put(path, entry);
- boolean addAncestors = true;
- if (oldEntry != null) {
- // check for and warn if the existing bulk operation has an inconsistent
- // entry.
- // two directories or two files are both allowed.
- // file-over-file can happen in multipart uploaders when the same
- // uploader is overwriting file entries to the same destination as
- // part of its bulk operation.
- boolean oldWasDir = oldEntry.getFileStatus().isDirectory();
- boolean newIsDir = entry.getFileStatus().isDirectory();
- if ((oldWasDir && !newIsDir)
- || (!oldWasDir && newIsDir)) {
- LOG.warn("Overwriting a S3Guard file created in the operation: {}",
- oldEntry);
- LOG.warn("With new entry: {}", entry);
- // restore the old state
- ancestorState.put(path, oldEntry);
- // then raise an exception
- throw new PathIOException(path.toString(),
- String.format("%s old %s new %s",
- E_INCONSISTENT_UPDATE,
- oldEntry,
- entry));
- } else {
- // a directory is already present. Log and continue.
- LOG.debug("Directory at {} being updated with value {}",
- path, entry);
- // and we skip the the subsequent parent scan as we've already been
- // here
- addAncestors = false;
- }
- }
- // add the entry to the ancestry map as an explicitly requested entry.
- ancestry.put(path, Pair.of(EntryOrigin.Requested, entry));
- // now scan up the ancestor tree to see if there are any
- // immediately missing entries.
- Path parent = path.getParent();
- while (addAncestors
- && !parent.isRoot() && !ancestry.containsKey(parent)) {
- if (!ancestorState.findEntry(parent, true)) {
- // there is no entry in the ancestor state.
- // look in the store
- DDBPathMetadata md;
- Pair newEntry;
- final Item item = getConsistentItem(parent);
- if (item != null && !itemToPathMetadata(item, username).isDeleted()) {
- // This is an undeleted entry found in the database.
- // register it in ancestor state and in the map of entries to create
- // as a retrieved entry
- md = itemToPathMetadata(item, username);
- LOG.debug("Found existing entry for parent: {}", md);
- newEntry = Pair.of(EntryOrigin.Retrieved, md);
- // and we break, assuming that if there is an entry, its parents
- // are valid too.
- addAncestors = false;
- } else {
- // A directory entry was not found in the DB. Create one.
- LOG.debug("auto-create ancestor path {} for child path {}",
- parent, path);
- final S3AFileStatus status = makeDirStatus(parent, username);
- md = new DDBPathMetadata(status, Tristate.FALSE,
- false, false, ttlTimeProvider.getNow());
- // declare to be a generated entry
- newEntry = Pair.of(EntryOrigin.Generated, md);
- }
- // insert into the ancestor state to avoid further checks
- ancestorState.put(parent, md);
- ancestry.put(parent, newEntry);
- }
- parent = parent.getParent();
- }
- }
- // we now have a list of entries which were not in the operation state.
- // Filter out those which were retrieved, to produce a list of those
- // which must be written to the database.
- // TODO sort in reverse order of existence
- return ancestry.values().stream()
- .filter(p -> p.getLeft() != EntryOrigin.Retrieved)
- .map(Pair::getRight)
- .collect(Collectors.toList());
- }
-
- /**
- * {@inheritDoc}
- *
- * The implementation scans all up the directory tree and does a get()
- * for each entry; at each level one is found it is added to the ancestor
- * state.
- *
- * The original implementation would stop on finding the first non-empty
- * parent. This (re) implementation issues a GET for every parent entry
- * and so detects and recovers from a tombstone marker further up the tree
- * (i.e. an inconsistent store is corrected for).
- *
- * if {@code operationState} is not null, when this method returns the
- * operation state will be updated with all new entries created.
- * This ensures that subsequent operations with the same store will not
- * trigger new updates.
- * @param qualifiedPath path to update
- * @param operationState (nullable) operational state for a bulk update
- * @throws IOException on failure.
- */
- @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
- @Override
- @Retries.RetryTranslated
- public void addAncestors(final Path qualifiedPath,
- @Nullable final BulkOperationState operationState) throws IOException {
-
- Collection newDirs = new ArrayList<>();
- final AncestorState ancestorState = extractOrCreate(operationState,
- BulkOperationState.OperationType.Put);
- Path parent = qualifiedPath.getParent();
- boolean entryFound = false;
-
- // Iterate up the parents.
- // note that only ancestorState get/set operations are synchronized;
- // the DDB read between them is not. As a result, more than one
- // thread may probe the state, find the entry missing, do the database
- // query and add the entry.
- // This is done to avoid making the remote dynamo query part of the
- // synchronized block.
- // If a race does occur, the cost is simply one extra GET and potentially
- // one extra PUT.
- while (!parent.isRoot()) {
- synchronized (ancestorState) {
- if (ancestorState.contains(parent)) {
- // the ancestry map contains the key, so no need to even look for it.
- break;
- }
- }
- // we don't worry about tombstone expiry here as expired or not,
- // a directory entry will go in.
- PathMetadata directory = get(parent);
- if (directory == null || directory.isDeleted()) {
- if (entryFound) {
- LOG.warn("Inconsistent S3Guard table: adding directory {}", parent);
- }
- S3AFileStatus status = makeDirStatus(username, parent);
- LOG.debug("Adding new ancestor entry {}", status);
- DDBPathMetadata meta = new DDBPathMetadata(status, Tristate.FALSE,
- false, ttlTimeProvider.getNow());
- newDirs.add(meta);
- // Do not update ancestor state here, as it
- // will happen in the innerPut() call. Were we to add it
- // here that put operation would actually (mistakenly) skip
- // creating the entry.
- } else {
- // an entry was found. Check its type
- entryFound = true;
- if (directory.getFileStatus().isFile()) {
- throw new PathIOException(parent.toString(),
- "Cannot overwrite parent file: metastore is"
- + " in an inconsistent state");
- }
- // the directory exists. Add it to the ancestor state for next time.
- synchronized (ancestorState) {
- ancestorState.put(parent, new DDBPathMetadata(directory));
- }
- }
- parent = parent.getParent();
- }
- // the listing of directories to put is all those parents which we know
- // are not in the store or BulkOperationState.
- if (!newDirs.isEmpty()) {
- // patch up the time.
- patchLastUpdated(newDirs, ttlTimeProvider);
- innerPut(newDirs, operationState);
- }
- }
-
- /**
- * {@inheritDoc}.
- *
- * The DDB implementation sorts all the paths such that new items
- * are ordered highest level entry first; deleted items are ordered
- * lowest entry first.
- *
- * This is to ensure that if a client failed partway through the update,
- * there will no entries in the table which lack parent entries.
- * @param pathsToDelete Collection of all paths that were removed from the
- * source directory tree of the move.
- * @param pathsToCreate Collection of all PathMetadata for the new paths
- * that were created at the destination of the rename
- * ().
- * @param operationState Any ongoing state supplied to the rename tracker
- * which is to be passed in with each move operation.
- * @throws IOException if there is an error
- */
- @Override
- @Retries.RetryTranslated
- public void move(@Nullable Collection pathsToDelete,
- @Nullable Collection pathsToCreate,
- @Nullable final BulkOperationState operationState) throws IOException {
- if (pathsToDelete == null && pathsToCreate == null) {
- return;
- }
-
- LOG.debug("Moving paths of table {} in region {}: {} paths to delete and {}"
- + " paths to create", tableName, region,
- pathsToDelete == null ? 0 : pathsToDelete.size(),
- pathsToCreate == null ? 0 : pathsToCreate.size());
- LOG.trace("move: pathsToDelete = {}, pathsToCreate = {}", pathsToDelete,
- pathsToCreate);
-
- // In DynamoDBMetadataStore implementation, we assume that if a path
- // exists, all its ancestors will also exist in the table.
- // 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
- AncestorState ancestorState = extractOrCreate(operationState,
- BulkOperationState.OperationType.Rename);
- List newItems = new ArrayList<>();
- if (pathsToCreate != null) {
- // create all parent entries.
- // this is synchronized on the move state so that across both serialized
- // and parallelized renames, duplicate ancestor entries are not created.
- synchronized (ancestorState) {
- newItems.addAll(
- completeAncestry(
- pathMetaToDDBPathMeta(pathsToCreate),
- ancestorState));
- }
- }
- // sort all the new items topmost first.
- newItems.sort(PathOrderComparators.TOPMOST_PM_FIRST);
-
- // now process the deletions.
- if (pathsToDelete != null) {
- List tombstones = new ArrayList<>(pathsToDelete.size());
- for (Path meta : pathsToDelete) {
- Preconditions.checkArgument(ttlTimeProvider != null, "ttlTimeProvider"
- + " must not be null");
- final PathMetadata pmTombstone = PathMetadata.tombstone(meta,
- ttlTimeProvider.getNow());
- tombstones.add(new DDBPathMetadata(pmTombstone));
- }
- // sort all the tombstones lowest first.
- tombstones.sort(TOPMOST_PM_LAST);
- newItems.addAll(tombstones);
- }
-
- processBatchWriteRequest(ancestorState,
- null, pathMetadataToItem(newItems));
- }
-
- /**
- * Helper method to issue a batch write request to DynamoDB.
- *
- * - Keys to delete are processed ahead of writing new items.
- * - No attempt is made to sort the input: the caller must do that
- *
- * As well as retrying on the operation invocation, incomplete
- * batches are retried until all have been processed.
- *
- * @param ancestorState ancestor state for logging
- * @param keysToDelete primary keys to be deleted; can be null
- * @param itemsToPut new items to be put; can be null
- * @return the number of iterations needed to complete the call.
- */
- @Retries.RetryTranslated("Outstanding batch items are updated with backoff")
- private int processBatchWriteRequest(
- @Nullable AncestorState ancestorState,
- PrimaryKey[] keysToDelete,
- Item[] itemsToPut) throws IOException {
- final int totalToDelete = (keysToDelete == null ? 0 : keysToDelete.length);
- final int totalToPut = (itemsToPut == null ? 0 : itemsToPut.length);
- if (totalToPut == 0 && totalToDelete == 0) {
- LOG.debug("Ignoring empty batch write request");
- return 0;
- }
- int count = 0;
- int batches = 0;
- while (count < totalToDelete + totalToPut) {
- final TableWriteItems writeItems = new TableWriteItems(tableName);
- int numToDelete = 0;
- if (keysToDelete != null
- && count < totalToDelete) {
- numToDelete = Math.min(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT,
- totalToDelete - count);
- PrimaryKey[] toDelete = Arrays.copyOfRange(keysToDelete,
- count, count + numToDelete);
- LOG.debug("Deleting {} entries: {}", toDelete.length, toDelete);
- writeItems.withPrimaryKeysToDelete(toDelete);
- count += numToDelete;
- }
-
- if (numToDelete < S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT
- && itemsToPut != null
- && count < totalToDelete + totalToPut) {
- final int numToPut = Math.min(
- S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT - numToDelete,
- totalToDelete + totalToPut - count);
- final int index = count - totalToDelete;
- writeItems.withItemsToPut(
- Arrays.copyOfRange(itemsToPut, index, index + numToPut));
- count += numToPut;
- }
-
- // if there's a retry and another process updates things then it's not
- // quite idempotent, but this was the case anyway
- batches++;
- BatchWriteItemOutcome res = writeOp.retry(
- "batch write",
- "",
- true,
- () -> dynamoDB.batchWriteItem(writeItems));
- // Check for unprocessed keys in case of exceeding provisioned throughput
- Map> unprocessed = res.getUnprocessedItems();
- int retryCount = 0;
- while (!unprocessed.isEmpty()) {
- batchWriteCapacityExceededEvents.incrementAndGet();
- batches++;
- retryBackoffOnBatchWrite(retryCount++);
- // use a different reference to keep the compiler quiet
- final Map> upx = unprocessed;
- res = writeOp.retry(
- "batch write",
- "",
- true,
- () -> dynamoDB.batchWriteItemUnprocessed(upx));
- unprocessed = res.getUnprocessedItems();
- }
- }
- if (itemsToPut != null) {
- recordsWritten(itemsToPut.length);
- logPut(ancestorState, itemsToPut);
- }
- if (keysToDelete != null) {
- recordsDeleted(keysToDelete.length);
- logDelete(ancestorState, keysToDelete);
-
- }
- return batches;
- }
-
- /**
- * Put the current thread to sleep to implement exponential backoff
- * depending on retryCount. If max retries are exceeded, throws an
- * exception instead.
- *
- * @param retryCount number of retries so far
- * @throws IOException when max retryCount is exceeded.
- */
- private void retryBackoffOnBatchWrite(int retryCount) throws IOException {
- try {
- // Our RetryPolicy ignores everything but retryCount here.
- RetryPolicy.RetryAction action = batchWriteRetryPolicy.shouldRetry(
- null,
- retryCount, 0, true);
- if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
- // Create an AWSServiceThrottledException, with a fake inner cause
- // which we fill in to look like a real exception so
- // error messages look sensible
- AmazonServiceException cause = new AmazonServiceException(
- "Throttling");
- cause.setServiceName("S3Guard");
- cause.setStatusCode(AWSServiceThrottledException.STATUS_CODE);
- cause.setErrorCode(THROTTLING); // used in real AWS errors
- cause.setErrorType(AmazonServiceException.ErrorType.Service);
- cause.setErrorMessage(THROTTLING);
- cause.setRequestId("n/a");
- throw new AWSServiceThrottledException(
- String.format("Max retries during batch write exceeded"
- + " (%d) for DynamoDB."
- + HINT_DDB_IOPS_TOO_LOW,
- retryCount),
- cause);
- } else {
- LOG.debug("Sleeping {} msec before next retry", action.delayMillis);
- Thread.sleep(action.delayMillis);
- }
- } catch (InterruptedException e) {
- throw (IOException)new InterruptedIOException(e.toString()).initCause(e);
- } catch (IOException e) {
- throw e;
- } catch (Exception e) {
- throw new IOException("Unexpected exception " + e, e);
- }
- }
-
- @Override
- @Retries.RetryTranslated
- public void put(final PathMetadata meta) throws IOException {
- put(meta, null);
- }
-
- @Override
- @Retries.RetryTranslated
- public void put(
- final PathMetadata meta,
- @Nullable final BulkOperationState operationState) throws IOException {
- // For a deeply nested path, this method will automatically create the full
- // ancestry and save respective item in DynamoDB table.
- // So after put operation, we maintain the invariant that if a path exists,
- // all its ancestors will also exist in the table.
- // For performance purpose, we generate the full paths to put and use batch
- // write item request to save the items.
- LOG.debug("Saving to table {} in region {}: {}", tableName, region, meta);
-
- Collection wrapper = new ArrayList<>(1);
- wrapper.add(meta);
- put(wrapper, operationState);
- }
-
- @Override
- @Retries.RetryTranslated
- public void put(
- final Collection extends PathMetadata> metas,
- @Nullable final BulkOperationState operationState) throws IOException {
- innerPut(pathMetaToDDBPathMeta(metas), operationState);
- }
-
- /**
- * Internal put operation.
- *
- * The ancestors to all entries are added to the set of entries to write,
- * provided they are not already stored in any supplied operation state.
- * Both the supplied metadata entries and ancestor entries are sorted
- * so that the topmost entries are written first.
- * This is to ensure that a failure partway through the operation will not
- * create entries in the table without parents.
- * @param metas metadata entries to write.
- * @param operationState (nullable) operational state for a bulk update
- * @throws IOException failure.
- */
- @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
- @Retries.RetryTranslated
- private void innerPut(
- final Collection metas,
- @Nullable final BulkOperationState operationState) throws IOException {
- if (metas.isEmpty()) {
- // Happens when someone calls put() with an empty list.
- LOG.debug("Ignoring empty list of entries to put");
- return;
- }
- // always create or retrieve an ancestor state instance, so it can
- // always be used for synchronization.
- final AncestorState ancestorState = extractOrCreate(operationState,
- BulkOperationState.OperationType.Put);
-
- Item[] items;
- synchronized (ancestorState) {
- items = pathMetadataToItem(
- completeAncestry(metas, ancestorState));
- }
- LOG.debug("Saving batch of {} items to table {}, region {}", items.length,
- tableName, region);
- processBatchWriteRequest(ancestorState, null, items);
- }
-
- /**
- * Get full path of ancestors that are nonexistent in table.
- *
- * This queries DDB when looking for parents which are not in
- * any supplied ongoing operation state.
- * Updates the operation state with found entries to reduce further checks.
- *
- * @param meta metadata to put
- * @param operationState ongoing bulk state
- * @return a possibly empty list of entries to put.
- * @throws IOException failure
- */
- @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
- @VisibleForTesting
- @Retries.RetryTranslated
- List fullPathsToPut(DDBPathMetadata meta,
- @Nullable BulkOperationState operationState)
- throws IOException {
- checkPathMetadata(meta);
- final List metasToPut = new ArrayList<>();
- // root path is not persisted
- if (!meta.getFileStatus().getPath().isRoot()) {
- metasToPut.add(meta);
- }
-
- // put all its ancestors if not present; as an optimization we return at its
- // first existent ancestor
- final AncestorState ancestorState = extractOrCreate(operationState,
- BulkOperationState.OperationType.Put);
- Path path = meta.getFileStatus().getPath().getParent();
- while (path != null && !path.isRoot()) {
- synchronized (ancestorState) {
- if (ancestorState.findEntry(path, true)) {
- break;
- }
- }
- final Item item = getConsistentItem(path);
- if (!itemExists(item)) {
- final S3AFileStatus status = makeDirStatus(path, username);
- metasToPut.add(new DDBPathMetadata(status, Tristate.FALSE, false,
- meta.isAuthoritativeDir(), meta.getLastUpdated()));
- path = path.getParent();
- } else {
- // found the entry in the table, so add it to the ancestor state
- synchronized (ancestorState) {
- ancestorState.put(path, itemToPathMetadata(item, username));
- }
- // then break out of the loop.
- break;
- }
- }
- return metasToPut;
- }
-
- /**
- * Does an item represent an object which exists?
- * @param item item retrieved in a query.
- * @return true iff the item isn't null and, if there is an is_deleted
- * column, that its value is false.
- */
- private static boolean itemExists(Item item) {
- if (item == null) {
- return false;
- }
- if (item.hasAttribute(IS_DELETED) &&
- item.getBoolean(IS_DELETED)) {
- return false;
- }
- return true;
- }
-
- /**
- * Get the value of an optional boolean attribute, falling back to the
- * default value if the attribute is absent.
- * @param item Item
- * @param attrName Attribute name
- * @param defVal Default value
- * @return The value or the default
- */
- private static boolean getBoolAttribute(Item item,
- String attrName,
- boolean defVal) {
- return item.hasAttribute(attrName) ? item.getBoolean(attrName) : defVal;
- }
-
- /** Create a directory FileStatus using 0 for the lastUpdated time. */
- static S3AFileStatus makeDirStatus(Path f, String owner) {
- return new S3AFileStatus(Tristate.UNKNOWN, f, owner);
- }
-
- /**
- * {@inheritDoc}.
- * There is retry around building the list of paths to update, but
- * the call to
- * {@link #processBatchWriteRequest(DynamoDBMetadataStore.AncestorState, PrimaryKey[], Item[])}
- * is only tried once.
- * @param meta Directory listing metadata.
- * @param unchangedEntries unchanged child entry paths
- * @param operationState operational state for a bulk update
- * @throws IOException IO problem
- */
- @Override
- @Retries.RetryTranslated
- public void put(
- final DirListingMetadata meta,
- final List unchangedEntries,
- @Nullable final BulkOperationState operationState) throws IOException {
- LOG.debug("Saving {} dir meta for {} to table {} in region {}: {}",
- meta.isAuthoritative() ? "auth" : "nonauth",
- meta.getPath(),
- tableName, region, meta);
- // directory path
- Path path = meta.getPath();
- DDBPathMetadata ddbPathMeta =
- new DDBPathMetadata(makeDirStatus(path, username), meta.isEmpty(),
- false, meta.isAuthoritative(), meta.getLastUpdated());
- // put all its ancestors if not present
- final AncestorState ancestorState = extractOrCreate(operationState,
- BulkOperationState.OperationType.Put);
- // First add any missing ancestors...
- final List metasToPut = fullPathsToPut(ddbPathMeta,
- ancestorState);
-
- // next add all changed children of the directory
- // ones that came from the previous listing are left as-is
- final Collection children = meta.getListing()
- .stream()
- .filter(e -> !unchangedEntries.contains(e.getFileStatus().getPath()))
- .collect(Collectors.toList());
-
- metasToPut.addAll(pathMetaToDDBPathMeta(children));
-
- // sort so highest-level entries are written to the store first.
- // if a sequence fails, no orphan entries will have been written.
- metasToPut.sort(PathOrderComparators.TOPMOST_PM_FIRST);
- processBatchWriteRequest(ancestorState,
- null,
- pathMetadataToItem(metasToPut));
- // and add the ancestors
- synchronized (ancestorState) {
- metasToPut.forEach(ancestorState::put);
- }
- }
-
- @Override
- public synchronized void close() {
- instrumentation.storeClosed();
- try {
- if (dynamoDB != null) {
- LOG.debug("Shutting down {}", this);
- dynamoDB.shutdown();
- dynamoDB = null;
- }
- } finally {
- closeAutocloseables(LOG, credentials);
- credentials = null;
- }
- }
-
- @Override
- @Retries.RetryTranslated
- public void destroy() throws IOException {
- tableHandler.destroy();
- }
-
- @Retries.RetryTranslated
- private ItemCollection expiredFiles(PruneMode pruneMode,
- long cutoff, String keyPrefix) throws IOException {
-
- String filterExpression;
- String projectionExpression;
- ValueMap map;
-
- switch (pruneMode) {
- case ALL_BY_MODTIME:
- // filter all files under the given parent older than the modtime.
- // this implicitly skips directories, because they lack a modtime field.
- // however we explicitly exclude directories to make clear that
- // directories are to be excluded and avoid any confusion
- // see: HADOOP-16725.
- // note: files lack the is_dir field entirely, so we use a `not` to
- // filter out the directories.
- filterExpression =
- "mod_time < :mod_time and begins_with(parent, :parent)"
- + " and not is_dir = :is_dir";
- projectionExpression = "parent,child";
- map = new ValueMap()
- .withLong(":mod_time", cutoff)
- .withString(":parent", keyPrefix)
- .withBoolean(":is_dir", true);
- break;
- case TOMBSTONES_BY_LASTUPDATED:
- filterExpression =
- "last_updated < :last_updated and begins_with(parent, :parent) "
- + "and is_deleted = :is_deleted";
- projectionExpression = "parent,child,is_deleted";
- map = new ValueMap()
- .withLong(":last_updated", cutoff)
- .withString(":parent", keyPrefix)
- .withBoolean(":is_deleted", true);
- break;
- default:
- throw new UnsupportedOperationException("Unsupported prune mode: "
- + pruneMode);
- }
-
- return readOp.retry(
- "scan",
- keyPrefix,
- true,
- () -> table.scan(filterExpression, projectionExpression, null, map));
- }
-
- @Override
- @Retries.RetryTranslated
- public void prune(PruneMode pruneMode, long cutoff) throws IOException {
- prune(pruneMode, cutoff, "/");
- }
-
- /**
- * Prune files, in batches. There's optionally a sleep between each batch.
- *
- * @param pruneMode The mode of operation for the prune For details see
- * {@link MetadataStore#prune(PruneMode, long)}
- * @param cutoff Oldest modification time to allow
- * @param keyPrefix The prefix for the keys that should be removed
- * @throws IOException Any IO/DDB failure.
- * @throws InterruptedIOException if the prune was interrupted
- * @return count of pruned items.
- */
- @Override
- @Retries.RetryTranslated
- public long prune(PruneMode pruneMode, long cutoff, String keyPrefix)
- throws IOException {
- LOG.debug("Prune {} under {} with age {}",
- pruneMode == PruneMode.ALL_BY_MODTIME
- ? "files and tombstones" : "tombstones",
- keyPrefix, cutoff);
- final ItemCollection items =
- expiredFiles(pruneMode, cutoff, keyPrefix);
- return innerPrune(pruneMode, cutoff, keyPrefix, items);
- }
-
- /**
- * Prune files, in batches. There's optionally a sleep between each batch.
- *
- * @param pruneMode The mode of operation for the prune For details see
- * {@link MetadataStore#prune(PruneMode, long)}
- * @param cutoff Oldest modification time to allow
- * @param keyPrefix The prefix for the keys that should be removed
- * @param items expired items
- * @return count of pruned items.
- * @throws IOException Any IO/DDB failure.
- * @throws InterruptedIOException if the prune was interrupted
- */
- private int innerPrune(
- final PruneMode pruneMode, final long cutoff, final String keyPrefix,
- final ItemCollection items)
- throws IOException {
- int itemCount = 0;
- try (AncestorState state = initiateBulkWrite(
- BulkOperationState.OperationType.Prune, null);
- DurationInfo ignored =
- new DurationInfo(LOG, "Pruning DynamoDB Store")) {
- ArrayList deletionBatch =
- new ArrayList<>(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
- long delay = conf.getTimeDuration(
- S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY,
- S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT,
- TimeUnit.MILLISECONDS);
- Set parentPathSet = new HashSet<>();
- Set clearedParentPathSet = new HashSet<>();
- // declare the operation to delete a batch as a function so
- // as to keep the code consistent across multiple uses.
- CallableRaisingIOE deleteBatchOperation =
- () -> {
- // lowest path entries get deleted first.
- deletionBatch.sort(PathOrderComparators.TOPMOST_PATH_LAST);
- processBatchWriteRequest(state, pathToKey(deletionBatch), null);
-
- // set authoritative false for each pruned dir listing
- // if at least one entry was not a tombstone
- removeAuthoritativeDirFlag(parentPathSet, state);
- // already cleared parent paths.
- clearedParentPathSet.addAll(parentPathSet);
- parentPathSet.clear();
- return null;
- };
- for (Item item : items) {
- DDBPathMetadata md = PathMetadataDynamoDBTranslation
- .itemToPathMetadata(item, username);
- Path path = md.getFileStatus().getPath();
- boolean tombstone = md.isDeleted();
- LOG.debug("Prune entry {}", path);
- deletionBatch.add(path);
-
- // add parent path of item so it can be marked as non-auth.
- // this is only done if
- // * it has not already been processed
- // * the entry pruned is not a tombstone (no need to update)
- // * the file is not in the root dir
- Path parentPath = path.getParent();
- if (!tombstone
- && parentPath != null
- && !parentPath.isRoot()
- && !clearedParentPathSet.contains(parentPath)) {
- parentPathSet.add(parentPath);
- }
-
- itemCount++;
- if (deletionBatch.size() == S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT) {
- deleteBatchOperation.apply();
- deletionBatch.clear();
- if (delay > 0) {
- Thread.sleep(delay);
- }
- }
- }
- // final batch of deletes
- if (!deletionBatch.isEmpty()) {
- deleteBatchOperation.apply();
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new InterruptedIOException("Pruning was interrupted");
- } catch (AmazonDynamoDBException e) {
- throw translateDynamoDBException(keyPrefix,
- "Prune of " + keyPrefix + " failed", e);
- }
- LOG.info("Finished pruning {} items in batches of {}", itemCount,
- S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
- return itemCount;
- }
-
- /**
- * Remove the Authoritative Directory Marker from a set of paths, if
- * those paths are in the store.
- *
- * This operation is onlyfor pruning; it does not raise an error
- * if, during the prune phase, the table appears inconsistent.
- * This is not unusual as it can happen in a number of ways
- *
- * - The state of the table changes during a slow prune operation which
- * deliberately inserts pauses to avoid overloading prepaid IO capacity.
- *
- * - Tombstone markers have been left in the table after many other
- * operations have taken place, including deleting/replacing
- * parents.
- *
- *
- *
- * If an exception is raised in the get/update process, then the exception
- * is caught and only rethrown after all the other paths are processed.
- * This is to ensure a best-effort attempt to update the store.
- * @param pathSet set of paths.
- * @param state ongoing operation state.
- * @throws IOException only after a best effort is made to update the store.
- */
- private void removeAuthoritativeDirFlag(
- final Set pathSet,
- final AncestorState state) throws IOException {
-
- AtomicReference rIOException = new AtomicReference<>();
-
- Set metas = pathSet.stream().map(path -> {
- try {
- if (path.isRoot()) {
- LOG.debug("ignoring root path");
- return null;
- }
- if (state != null && state.get(path) != null) {
- // there's already an entry for this path
- LOG.debug("Ignoring update of entry already in the state map");
- return null;
- }
- DDBPathMetadata ddbPathMetadata = get(path);
- if (ddbPathMetadata == null) {
- // there is no entry.
- LOG.debug("No parent {}; skipping", path);
- return null;
- }
- if (ddbPathMetadata.isDeleted()) {
- // the parent itself is deleted
- LOG.debug("Parent has been deleted {}; skipping", path);
- return null;
- }
- if (!ddbPathMetadata.getFileStatus().isDirectory()) {
- // the parent itself is deleted
- LOG.debug("Parent is not a directory {}; skipping", path);
- return null;
- }
- LOG.debug("Setting isAuthoritativeDir==false on {}", ddbPathMetadata);
- ddbPathMetadata.setAuthoritativeDir(false);
- ddbPathMetadata.setLastUpdated(ttlTimeProvider.getNow());
- 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);
- if (!metas.isEmpty()) {
- innerPut(metas, state);
- }
- } 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();
- }
- }
-
- @VisibleForTesting
- public AmazonDynamoDB getAmazonDynamoDB() {
- return amazonDynamoDB;
- }
-
- @Override
- public String toString() {
- return getClass().getSimpleName() + '{'
- + "region=" + region
- + ", tableName=" + tableName
- + ", tableArn=" + tableHandler.getTableArn()
- + '}';
- }
-
- /**
- * The administrative policy includes all DDB table operations;
- * application access is restricted to those operations S3Guard operations
- * require when working with data in a guarded bucket.
- * @param access access level desired.
- * @return a possibly empty list of statements.
- */
- @Override
- public List listAWSPolicyRules(
- final Set access) {
- Preconditions.checkState(tableHandler.getTableArn() != null,
- "TableARN not known");
- if (access.isEmpty()) {
- return Collections.emptyList();
- }
- RoleModel.Statement stat;
- if (access.contains(AccessLevel.ADMIN)) {
- stat = allowAllDynamoDBOperations(tableHandler.getTableArn());
- } else {
- stat = allowS3GuardClientOperations(tableHandler.getTableArn());
- }
- return Lists.newArrayList(stat);
- }
-
- /**
- * PUT a single item to the table.
- * @param item item to put
- * @return the outcome.
- */
- @Retries.OnceRaw
- private PutItemOutcome putItem(Item item) {
- LOG.debug("Putting item {}", item);
- return table.putItem(item);
- }
-
- @VisibleForTesting
- Table getTable() {
- return table;
- }
-
- String getRegion() {
- return region;
- }
-
- @VisibleForTesting
- public String getTableName() {
- return tableName;
- }
-
- @VisibleForTesting
- DynamoDB getDynamoDB() {
- return dynamoDB;
- }
-
- /**
- * Validates a path object; it must be absolute, have an s3a:/// scheme
- * and contain a host (bucket) component.
- * @param path path to check
- * @return the path passed in
- */
- private Path checkPath(Path path) {
- Preconditions.checkNotNull(path);
- Preconditions.checkArgument(path.isAbsolute(), "Path %s is not absolute",
- path);
- URI uri = path.toUri();
- Preconditions.checkNotNull(uri.getScheme(), "Path %s missing scheme", path);
- Preconditions.checkArgument(uri.getScheme().equals(Constants.FS_S3A),
- "Path %s scheme must be %s", path, Constants.FS_S3A);
- Preconditions.checkArgument(!StringUtils.isEmpty(uri.getHost()), "Path %s" +
- " is missing bucket.", path);
- return path;
- }
-
- /**
- * Validates a path meta-data object.
- */
- private static void checkPathMetadata(PathMetadata meta) {
- Preconditions.checkNotNull(meta);
- Preconditions.checkNotNull(meta.getFileStatus());
- Preconditions.checkNotNull(meta.getFileStatus().getPath());
- }
-
- @Override
- @Retries.OnceRaw
- public Map getDiagnostics() throws IOException {
- Map map = new TreeMap<>();
- if (table != null) {
- TableDescription desc = getTableDescription(true);
- map.put("name", desc.getTableName());
- map.put(STATUS, desc.getTableStatus());
- map.put("ARN", desc.getTableArn());
- map.put("size", desc.getTableSizeBytes().toString());
- map.put(TABLE, desc.toString());
- ProvisionedThroughputDescription throughput
- = desc.getProvisionedThroughput();
- map.put(READ_CAPACITY, throughput.getReadCapacityUnits().toString());
- map.put(WRITE_CAPACITY, throughput.getWriteCapacityUnits().toString());
- map.put(BILLING_MODE,
- throughput.getWriteCapacityUnits() == 0
- ? BILLING_MODE_PER_REQUEST
- : BILLING_MODE_PROVISIONED);
- map.put("sse", desc.getSSEDescription() == null
- ? "DISABLED"
- : desc.getSSEDescription().toString());
- map.put(MetadataStoreCapabilities.PERSISTS_AUTHORITATIVE_BIT,
- Boolean.toString(true));
- } else {
- map.put("name", "DynamoDB Metadata Store");
- map.put(TABLE, "none");
- map.put(STATUS, "undefined");
- }
- map.put("description", DESCRIPTION);
- map.put("region", region);
- if (batchWriteRetryPolicy != null) {
- map.put("retryPolicy", batchWriteRetryPolicy.toString());
- }
- return map;
- }
-
- @Retries.OnceRaw
- private TableDescription getTableDescription(boolean forceUpdate) {
- TableDescription desc = table.getDescription();
- if (desc == null || forceUpdate) {
- desc = table.describe();
- }
- return desc;
- }
-
- @Override
- @Retries.OnceRaw
- public void updateParameters(Map parameters)
- throws IOException {
- Preconditions.checkNotNull(table, "Not initialized");
- TableDescription desc = getTableDescription(true);
- ProvisionedThroughputDescription current
- = desc.getProvisionedThroughput();
-
- long currentRead = current.getReadCapacityUnits();
- long newRead = getLongParam(parameters,
- S3GUARD_DDB_TABLE_CAPACITY_READ_KEY,
- currentRead);
- long currentWrite = current.getWriteCapacityUnits();
- long newWrite = getLongParam(parameters,
- S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY,
- currentWrite);
-
- if (currentRead == 0 || currentWrite == 0) {
- // table is pay on demand
- throw new IOException(E_ON_DEMAND_NO_SET_CAPACITY);
- }
-
- if (newRead != currentRead || newWrite != currentWrite) {
- LOG.info("Current table capacity is read: {}, write: {}",
- currentRead, currentWrite);
- LOG.info("Changing capacity of table to read: {}, write: {}",
- newRead, newWrite);
- tableHandler.provisionTableBlocking(newRead, newWrite);
- } else {
- LOG.info("Table capacity unchanged at read: {}, write: {}",
- newRead, newWrite);
- }
- }
-
- private long getLongParam(Map parameters,
- String key,
- long defVal) {
- String k = parameters.get(key);
- if (k != null) {
- return Long.parseLong(k);
- } else {
- return defVal;
- }
- }
-
- /**
- * Callback on a read operation retried.
- * @param text text of the operation
- * @param ex exception
- * @param attempts number of attempts
- * @param idempotent is the method idempotent (this is assumed to be true)
- */
- void readRetryEvent(
- String text,
- IOException ex,
- int attempts,
- boolean idempotent) {
- readThrottleEvents.incrementAndGet();
- retryEvent(text, ex, attempts, true);
- }
-
- /**
- * Callback on a write operation retried.
- * @param text text of the operation
- * @param ex exception
- * @param attempts number of attempts
- * @param idempotent is the method idempotent (this is assumed to be true)
- */
- void writeRetryEvent(
- String text,
- IOException ex,
- int attempts,
- boolean idempotent) {
- writeThrottleEvents.incrementAndGet();
- retryEvent(text, ex, attempts, idempotent);
- }
-
- /**
- * Callback on a scan operation retried.
- * @param text text of the operation
- * @param ex exception
- * @param attempts number of attempts
- * @param idempotent is the method idempotent (this is assumed to be true)
- */
- void scanRetryEvent(
- String text,
- IOException ex,
- int attempts,
- boolean idempotent) {
- scanThrottleEvents.incrementAndGet();
- retryEvent(text, ex, attempts, idempotent);
- }
-
- /**
- * Callback from {@link Invoker} when an operation is retried.
- * @param text text of the operation
- * @param ex exception
- * @param attempts number of attempts
- * @param idempotent is the method idempotent
- */
- void retryEvent(
- String text,
- IOException ex,
- int attempts,
- boolean idempotent) {
- if (S3AUtils.isThrottleException(ex)) {
- // throttled
- instrumentation.throttled();
- int eventCount = throttleEventCount.addAndGet(1);
- if (attempts == 1 && eventCount < THROTTLE_EVENT_LOG_LIMIT) {
- LOG.warn("DynamoDB IO limits reached in {};"
- + " consider increasing capacity: {}", text, ex.toString());
- LOG.debug("Throttled", ex);
- } else {
- // user has been warned already, log at debug only.
- LOG.debug("DynamoDB IO limits reached in {};"
- + " consider increasing capacity: {}", text, ex.toString());
- }
- } else if (attempts == 1) {
- // not throttled. Log on the first attempt only
- LOG.info("Retrying {}: {}", text, ex.toString());
- LOG.debug("Retrying {}", text, ex);
- }
-
- // note a retry
- instrumentation.retrying();
- if (owner != null) {
- owner.metastoreOperationRetried(ex, attempts, idempotent);
- }
- }
-
- /**
- * Get the count of read throttle events.
- * @return the current count of read throttle events.
- */
- @VisibleForTesting
- public long getReadThrottleEventCount() {
- return readThrottleEvents.get();
- }
-
- /**
- * Get the count of write throttle events.
- * @return the current count of write throttle events.
- */
- @VisibleForTesting
- public long getWriteThrottleEventCount() {
- return writeThrottleEvents.get();
- }
-
- /**
- * Get the count of scan throttle events.
- * @return the current count of scan throttle events.
- */
- @VisibleForTesting
- public long getScanThrottleEventCount() {
- return scanThrottleEvents.get();
- }
-
- @VisibleForTesting
- public long getBatchWriteCapacityExceededCount() {
- return batchWriteCapacityExceededEvents.get();
- }
-
- /**
- * Get the operation invoker for write operations.
- * @return an invoker for retrying mutating operations on a store.
- */
- public Invoker getInvoker() {
- return writeOp;
- }
-
- /**
- * Wrap an iterator returned from any scan with a retrying one.
- * This includes throttle handling.
- * Retries will update the relevant counters/metrics for scan operations.
- * @param source source iterator
- * @return a retrying iterator.
- */
- public Iterable wrapWithRetries(
- final Iterable source) {
- return new RetryingCollection<>("scan dynamoDB table", scanOp, source);
- }
-
- /**
- * Record the number of records written.
- * @param count count of records.
- */
- private void recordsWritten(final int count) {
- instrumentation.recordsWritten(count);
- }
-
- /**
- * Record the number of records read.
- * @param count count of records.
- */
- private void recordsRead(final int count) {
- instrumentation.recordsRead(count);
- }
- /**
- * Record the number of records deleted.
- * @param count count of records.
- */
- private void recordsDeleted(final int count) {
- instrumentation.recordsDeleted(count);
- }
-
- /**
- * Initiate the rename operation by creating the tracker for the filesystem
- * to keep up to date with state changes in the S3A bucket.
- * @param storeContext store context.
- * @param source source path
- * @param sourceStatus status of the source file/dir
- * @param dest destination path.
- * @return the rename tracker
- */
- @Override
- public RenameTracker initiateRenameOperation(
- final StoreContext storeContext,
- final Path source,
- final S3AFileStatus sourceStatus,
- final Path dest) {
- return new ProgressiveRenameTracker(storeContext, this, source, dest,
- new AncestorState(this, BulkOperationState.OperationType.Rename, dest));
- }
-
- /**
- * Mark the directories instantiated under the destination path
- * as authoritative. That is: all entries in the
- * operationState (which must be an AncestorState instance),
- * that are under the destination path.
- *
- * The database update synchronized on the operationState, so all other
- * threads trying to update that state will be blocked until completion.
- *
- * This operation is only used in import and at the end of a rename,
- * so this is not considered an issue.
- * @param dest destination path.
- * @param operationState active state.
- * @throws IOException failure.
- * @return the number of directories marked.
- */
- @Override
- public int markAsAuthoritative(
- final Path dest,
- final BulkOperationState operationState) throws IOException {
- if (operationState == null) {
- return 0;
- }
- Preconditions.checkArgument(operationState instanceof AncestorState,
- "Not an AncestorState %s", operationState);
- final AncestorState state = (AncestorState)operationState;
- // only mark paths under the dest as auth
- final String simpleDestKey = pathToParentKey(dest);
- final String destPathKey = simpleDestKey + "/";
- final String opId = AncestorState.stateAsString(state);
- LOG.debug("{}: marking directories under {} as authoritative",
- opId, destPathKey);
-
- // the list of dirs to build up.
- final List dirsToUpdate = new ArrayList<>();
- synchronized (state) {
- for (Map.Entry entry :
- state.getAncestry().entrySet()) {
- final Path path = entry.getKey();
- final DDBPathMetadata md = entry.getValue();
- final String key = pathToParentKey(path);
- if (md.getFileStatus().isDirectory()
- && (key.equals(simpleDestKey) || key.startsWith(destPathKey))) {
- // the updated entry is under the destination.
- md.setAuthoritativeDir(true);
- md.setLastUpdated(ttlTimeProvider.getNow());
- LOG.debug("{}: added {}", opId, key);
- dirsToUpdate.add(md);
- }
- }
- processBatchWriteRequest(state,
- null, pathMetadataToItem(dirsToUpdate));
- }
- return dirsToUpdate.size();
- }
-
- @Override
- public AncestorState initiateBulkWrite(
- final BulkOperationState.OperationType operation,
- final Path dest) {
- return new AncestorState(this, operation, dest);
- }
-
- @Override
- public void setTtlTimeProvider(ITtlTimeProvider ttlTimeProvider) {
- this.ttlTimeProvider = ttlTimeProvider;
- }
-
- /**
- * Username.
- * @return the current username
- */
- String getUsername() {
- return username;
- }
-
- /**
- * Log a PUT into the operations log at debug level.
- * @param state optional ancestor state.
- * @param items items which have been PUT
- */
- private static void logPut(
- @Nullable AncestorState state,
- Item[] items) {
- if (OPERATIONS_LOG.isDebugEnabled()) {
- // log the operations
- String stateStr = AncestorState.stateAsString(state);
- for (Item item : items) {
- boolean tombstone = !itemExists(item);
- boolean isDir = getBoolAttribute(item, IS_DIR, false);
- boolean auth = getBoolAttribute(item, IS_AUTHORITATIVE, false);
- OPERATIONS_LOG.debug("{} {} {}{}{}",
- stateStr,
- tombstone ? "TOMBSTONE" : "PUT",
- itemPrimaryKeyToString(item),
- auth ? " [auth]" : "",
- isDir ? " directory" : "");
- }
- }
- }
-
- /**
- * Log a PUT into the operations log at debug level.
- * @param state optional ancestor state.
- * @param item item PUT.
- */
- private static void logPut(
- @Nullable AncestorState state,
- Item item) {
- if (OPERATIONS_LOG.isDebugEnabled()) {
- // log the operations
- logPut(state, new Item[]{item});
- }
- }
-
- /**
- * Log a DELETE into the operations log at debug level.
- * @param state optional ancestor state.
- * @param keysDeleted keys which were deleted.
- */
- private static void logDelete(
- @Nullable AncestorState state,
- PrimaryKey[] keysDeleted) {
- if (OPERATIONS_LOG.isDebugEnabled()) {
- // log the operations
- String stateStr = AncestorState.stateAsString(state);
- for (PrimaryKey key : keysDeleted) {
- OPERATIONS_LOG.debug("{} DELETE {}",
- stateStr, primaryKeyToString(key));
- }
- }
- }
-
- /**
- * Log a DELETE into the operations log at debug level.
- * @param state optional ancestor state.
- * @param key Deleted key
- */
- private static void logDelete(
- @Nullable AncestorState state,
- PrimaryKey key) {
- if (OPERATIONS_LOG.isDebugEnabled()) {
- logDelete(state, new PrimaryKey[]{key});
- }
- }
-
- /**
- * Get the move state passed in; create a new one if needed.
- * @param state state.
- * @param operation the type of the operation to use if the state is created.
- * @return the cast or created state.
- */
- private AncestorState extractOrCreate(@Nullable BulkOperationState state,
- BulkOperationState.OperationType operation) {
- if (state != null) {
- return (AncestorState) state;
- } else {
- return new AncestorState(this, operation, null);
- }
- }
-
- @Override
- public MetastoreInstrumentation getInstrumentation() {
- return instrumentation;
- }
-
- /**
- * This tracks all the ancestors created,
- * across multiple move/write operations.
- * This is to avoid duplicate creation of ancestors during bulk commits
- * and rename operations managed by a rename tracker.
- *
- * There is no thread safety: callers must synchronize as appropriate.
- */
- @VisibleForTesting
- static final class AncestorState extends BulkOperationState {
-
- /**
- * Counter of IDs issued.
- */
- private static final AtomicLong ID_COUNTER = new AtomicLong(0);
-
- /** Owning store. */
- private final DynamoDBMetadataStore store;
-
- /** The ID of the state; for logging. */
- private final long id;
-
- /**
- * Map of ancestors.
- */
- private final Map ancestry = new HashMap<>();
-
- /**
- * Destination path.
- */
- private final Path dest;
-
- /**
- * Create the state.
- * @param store the store, for use in validation.
- * If null: no validation (test only operation)
- * @param operation the type of the operation.
- * @param dest destination path.
- */
- AncestorState(
- @Nullable final DynamoDBMetadataStore store,
- final OperationType operation,
- @Nullable final Path dest) {
- super(operation);
- this.store = store;
- this.dest = dest;
- this.id = ID_COUNTER.addAndGet(1);
- }
-
- int size() {
- return ancestry.size();
- }
-
- /**
- * Get the ancestry. Not thread safe.
- * @return the map of ancestors.
- */
- Map getAncestry() {
- return ancestry;
- }
-
- public Path getDest() {
- return dest;
- }
-
- long getId() {
- return id;
- }
-
- @Override
- public String toString() {
- final StringBuilder sb = new StringBuilder(
- "AncestorState{");
- sb.append("operation=").append(getOperation());
- sb.append("id=").append(id);
- sb.append("; dest=").append(dest);
- sb.append("; size=").append(size());
- sb.append("; paths={")
- .append(StringUtils.join(ancestry.keySet(), " "))
- .append('}');
- sb.append('}');
- return sb.toString();
- }
-
- /**
- * Does the ancestor state contain a path?
- * @param p path to check
- * @return true if the state has an entry
- */
- boolean contains(Path p) {
- return get(p) != null;
- }
-
- DDBPathMetadata put(Path p, DDBPathMetadata md) {
- return ancestry.put(p, md);
- }
-
- DDBPathMetadata put(DDBPathMetadata md) {
- return ancestry.put(md.getFileStatus().getPath(), md);
- }
-
- DDBPathMetadata get(Path p) {
- return ancestry.get(p);
- }
-
- /**
- * Find an entry in the ancestor state, warning and optionally
- * raising an exception if there is a file at the path.
- * @param path path to look up
- * @param failOnFile fail if a file was found.
- * @return true iff a directory was found in the ancestor state.
- * @throws PathIOException if there was a file at the path.
- */
- boolean findEntry(
- final Path path,
- final boolean failOnFile) throws PathIOException {
- final DDBPathMetadata ancestor = get(path);
- if (ancestor != null) {
- // there's an entry in the ancestor state
- if (!ancestor.getFileStatus().isDirectory()) {
- // but: its a file, which means this update is now inconsistent.
- final String message = E_INCONSISTENT_UPDATE + " entry is " + ancestor
- .getFileStatus();
- LOG.error(message);
- if (failOnFile) {
- // errors trigger failure
- throw new PathIOException(path.toString(), message);
- }
- }
- return true;
- } else {
- return false;
- }
- }
-
- /**
- * If debug logging is enabled, this does an audit of the store state.
- * it only logs this; the error messages are created so as they could
- * be turned into exception messages.
- * Audit failures aren't being turned into IOEs is that
- * rename operations delete the source entry and that ends up in the
- * ancestor state as present
- * @throws IOException failure
- */
- @Override
- public void close() throws IOException {
- if (LOG.isDebugEnabled() && store != null) {
- LOG.debug("Auditing {}", stateAsString(this));
- for (Map.Entry entry : ancestry
- .entrySet()) {
- Path path = entry.getKey();
- DDBPathMetadata expected = entry.getValue();
- if (expected.isDeleted()) {
- // file was deleted in bulk op; we don't care about it
- // any more
- continue;
- }
- DDBPathMetadata actual;
- try {
- actual = store.get(path);
- } catch (IOException e) {
- LOG.debug("Retrieving {}", path, e);
- // this is for debug; don't be ambitious
- return;
- }
- if (actual == null || actual.isDeleted()) {
- String message = "Metastore entry for path "
- + path + " deleted during bulk "
- + getOperation() + " operation";
- LOG.debug(message);
- } else {
- if (actual.getFileStatus().isDirectory() !=
- expected.getFileStatus().isDirectory()) {
- // the type of the entry has changed
- String message = "Metastore entry for path "
- + path + " changed during bulk "
- + getOperation() + " operation"
- + " from " + expected
- + " to " + actual;
- LOG.debug(message);
- }
- }
-
- }
- }
- }
-
- /**
- * Create a string from the state including operation and ID.
- * @param state state to use -may be null
- * @return a string for logging.
- */
- private static String stateAsString(@Nullable AncestorState state) {
- String stateStr;
- if (state != null) {
- stateStr = String.format("#(%s-%04d)",
- state.getOperation(),
- state.getId());
- } else {
- stateStr = "#()";
- }
- return stateStr;
- }
- }
-
- protected DynamoDBMetadataStoreTableManager getTableHandler() {
- Preconditions.checkNotNull(tableHandler, "Not initialized");
- return tableHandler;
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStoreTableManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStoreTableManager.java
deleted file mode 100644
index d04ea3ebdc7..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStoreTableManager.java
+++ /dev/null
@@ -1,756 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.nio.file.AccessDeniedException;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-import com.amazonaws.AmazonClientException;
-import com.amazonaws.SdkBaseException;
-import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
-import com.amazonaws.services.dynamodbv2.document.DynamoDB;
-import com.amazonaws.services.dynamodbv2.document.Item;
-import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
-import com.amazonaws.services.dynamodbv2.document.PutItemOutcome;
-import com.amazonaws.services.dynamodbv2.document.Table;
-import com.amazonaws.services.dynamodbv2.model.AmazonDynamoDBException;
-import com.amazonaws.services.dynamodbv2.model.BillingMode;
-import com.amazonaws.services.dynamodbv2.model.CreateTableRequest;
-import com.amazonaws.services.dynamodbv2.model.ListTagsOfResourceRequest;
-import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput;
-import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
-import com.amazonaws.services.dynamodbv2.model.ResourceInUseException;
-import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
-import com.amazonaws.services.dynamodbv2.model.SSESpecification;
-import com.amazonaws.services.dynamodbv2.model.ScanRequest;
-import com.amazonaws.services.dynamodbv2.model.ScanResult;
-import com.amazonaws.services.dynamodbv2.model.TableDescription;
-import com.amazonaws.services.dynamodbv2.model.Tag;
-import com.amazonaws.services.dynamodbv2.model.TagResourceRequest;
-import com.amazonaws.waiters.WaiterTimedOutException;
-import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.s3a.AWSClientIOException;
-import org.apache.hadoop.fs.s3a.Invoker;
-import org.apache.hadoop.fs.s3a.Retries;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.io.retry.RetryPolicy;
-
-import static java.lang.String.valueOf;
-
-import static org.apache.commons.lang3.StringUtils.isEmpty;
-
-import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_CAPACITY_READ_DEFAULT;
-import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_CAPACITY_READ_KEY;
-import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_CAPACITY_WRITE_DEFAULT;
-import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY;
-import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_CREATE_KEY;
-import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_SSE_CMK;
-import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_SSE_ENABLED;
-import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_TAG;
-import static org.apache.hadoop.fs.s3a.S3AUtils.lookupPassword;
-import static org.apache.hadoop.fs.s3a.S3AUtils.translateDynamoDBException;
-import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
-import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.E_ON_DEMAND_NO_SET_CAPACITY;
-import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.VERSION;
-import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.VERSION_MARKER_ITEM_NAME;
-import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.VERSION_MARKER_TAG_NAME;
-import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.attributeDefinitions;
-import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.createVersionMarker;
-import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.createVersionMarkerPrimaryKey;
-import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.extractCreationTimeFromMarker;
-import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.extractVersionFromMarker;
-import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.keySchema;
-
-/**
- * Managing dynamo tables for S3Guard dynamodb based metadatastore.
- * Factored out from DynamoDBMetadataStore.
- */
-public class DynamoDBMetadataStoreTableManager {
- public static final Logger LOG = LoggerFactory.getLogger(
- DynamoDBMetadataStoreTableManager.class);
-
- /** Error: version marker not found in table but the table is not empty. */
- public static final String E_NO_VERSION_MARKER_AND_NOT_EMPTY
- = "S3Guard table lacks version marker, and it is not empty.";
-
- /** Error: version mismatch. */
- public static final String E_INCOMPATIBLE_TAG_VERSION
- = "Database table is from an incompatible S3Guard version based on table TAG.";
-
- /** Error: version mismatch. */
- public static final String E_INCOMPATIBLE_ITEM_VERSION
- = "Database table is from an incompatible S3Guard version based on table ITEM.";
-
- /** The AWS managed CMK for DynamoDB server side encryption. */
- public static final String SSE_DEFAULT_MASTER_KEY = "alias/aws/dynamodb";
-
- /** Invoker for IO. Until configured properly, use try-once. */
- private Invoker invoker = new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL,
- Invoker.NO_OP
- );
-
- final private AmazonDynamoDB amazonDynamoDB;
- final private DynamoDB dynamoDB;
- final private String tableName;
- final private String region;
- final private Configuration conf;
- final private Invoker readOp;
- final private RetryPolicy batchWriteRetryPolicy;
-
- private Table table;
- private String tableArn;
-
- public DynamoDBMetadataStoreTableManager(DynamoDB dynamoDB,
- String tableName,
- String region,
- AmazonDynamoDB amazonDynamoDB,
- Configuration conf,
- Invoker readOp,
- RetryPolicy batchWriteCapacityExceededEvents) {
- this.dynamoDB = dynamoDB;
- this.amazonDynamoDB = amazonDynamoDB;
- this.tableName = tableName;
- this.region = region;
- this.conf = conf;
- this.readOp = readOp;
- this.batchWriteRetryPolicy = batchWriteCapacityExceededEvents;
- }
-
- /**
- * Create a table if it does not exist and wait for it to become active.
- *
- * If a table with the intended name already exists, then it uses that table.
- * Otherwise, it will automatically create the table if the config
- * {@link org.apache.hadoop.fs.s3a.Constants#S3GUARD_DDB_TABLE_CREATE_KEY} is
- * enabled. The DynamoDB table creation API is asynchronous. This method wait
- * for the table to become active after sending the creation request, so
- * overall, this method is synchronous, and the table is guaranteed to exist
- * after this method returns successfully.
- *
- * The wait for a table becoming active is Retry+Translated; it can fail
- * while a table is not yet ready.
- *
- * @throws IOException if table does not exist and auto-creation is disabled;
- * or table is being deleted, or any other I/O exception occurred.
- */
- @VisibleForTesting
- @Retries.RetryTranslated
- Table initTable() throws IOException {
- table = dynamoDB.getTable(tableName);
- try {
- try {
- LOG.debug("Binding to table {}", tableName);
- TableDescription description = table.describe();
- LOG.debug("Table state: {}", description);
- tableArn = description.getTableArn();
- final String status = description.getTableStatus();
- switch (status) {
- case "CREATING":
- LOG.debug("Table {} in region {} is being created/updated. This may"
- + " indicate that the table is being operated by another "
- + "concurrent thread or process. Waiting for active...",
- tableName, region);
- waitForTableActive(table);
- break;
- case "DELETING":
- throw new FileNotFoundException("DynamoDB table "
- + "'" + tableName + "' is being "
- + "deleted in region " + region);
- case "UPDATING":
- // table being updated; it can still be used.
- LOG.debug("Table is being updated.");
- break;
- case "ACTIVE":
- break;
- default:
- throw new IOException("Unknown DynamoDB table status " + status
- + ": tableName='" + tableName + "', region=" + region);
- }
-
- verifyVersionCompatibility();
- final Item versionMarker = getVersionMarkerItem();
- Long created = extractCreationTimeFromMarker(versionMarker);
- LOG.debug("Using existing DynamoDB table {} in region {} created {}",
- tableName, region, (created != null) ? new Date(created) : null);
- } catch (ResourceNotFoundException rnfe) {
- if (conf.getBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, false)) {
- long readCapacity = conf.getLong(S3GUARD_DDB_TABLE_CAPACITY_READ_KEY,
- S3GUARD_DDB_TABLE_CAPACITY_READ_DEFAULT);
- long writeCapacity = conf.getLong(
- S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY,
- S3GUARD_DDB_TABLE_CAPACITY_WRITE_DEFAULT);
- ProvisionedThroughput capacity;
- if (readCapacity > 0 && writeCapacity > 0) {
- capacity = new ProvisionedThroughput(
- readCapacity,
- writeCapacity);
- } else {
- // at least one capacity value is <= 0
- // verify they are both exactly zero
- Preconditions.checkArgument(
- readCapacity == 0 && writeCapacity == 0,
- "S3Guard table read capacity %d and and write capacity %d"
- + " are inconsistent", readCapacity, writeCapacity);
- // and set the capacity to null for per-request billing.
- capacity = null;
- }
-
- createTable(capacity);
- } else {
- throw (FileNotFoundException) new FileNotFoundException(
- "DynamoDB table '" + tableName + "' does not "
- + "exist in region " + region +
- "; auto-creation is turned off")
- .initCause(rnfe);
- }
- }
-
- } catch (AmazonClientException e) {
- throw translateException("initTable", tableName, e);
- }
-
- return table;
- }
-
- protected void tagTableWithVersionMarker() throws AmazonDynamoDBException {
- try {
- TagResourceRequest tagResourceRequest = new TagResourceRequest()
- .withResourceArn(table.getDescription().getTableArn())
- .withTags(newVersionMarkerTag());
- amazonDynamoDB.tagResource(tagResourceRequest);
- } catch (AmazonDynamoDBException e) {
- LOG.debug("Exception during tagging table: {}", e.getMessage(), e);
- }
- }
-
- protected static Item getVersionMarkerFromTags(Table table,
- AmazonDynamoDB addb) throws IOException {
- List tags = null;
- try {
- final TableDescription description = table.describe();
- ListTagsOfResourceRequest listTagsOfResourceRequest =
- new ListTagsOfResourceRequest()
- .withResourceArn(description.getTableArn());
- tags = addb.listTagsOfResource(listTagsOfResourceRequest).getTags();
- } catch (ResourceNotFoundException e) {
- LOG.error("Table: {} not found.", table.getTableName());
- throw e;
- } catch (AmazonDynamoDBException e) {
- LOG.debug("Exception while getting tags from the dynamo table: {}",
- e.getMessage(), e);
- throw translateDynamoDBException(table.getTableName(),
- "Retrieving tags.", e);
- }
-
- if (tags == null) {
- return null;
- }
-
- final Optional first = tags.stream()
- .filter(tag -> tag.getKey().equals(VERSION_MARKER_TAG_NAME)).findFirst();
- if (first.isPresent()) {
- final Tag vmTag = first.get();
- return createVersionMarker(
- vmTag.getKey(), Integer.parseInt(vmTag.getValue()), 0
- );
- } else {
- return null;
- }
- }
-
- /**
- * Create a table, wait for it to become active, then add the version
- * marker.
- * Creating an setting up the table isn't wrapped by any retry operations;
- * the wait for a table to become available is RetryTranslated.
- * The tags are added to the table during creation, not after creation.
- * We can assume that tagging and creating the table is a single atomic
- * operation.
- *
- * @param capacity capacity to provision. If null: create a per-request
- * table.
- * @throws IOException on any failure.
- * @throws InterruptedIOException if the wait was interrupted
- */
- @Retries.OnceMixed
- private void createTable(ProvisionedThroughput capacity) throws IOException {
- try {
- String mode;
- CreateTableRequest request = new CreateTableRequest()
- .withTableName(tableName)
- .withKeySchema(keySchema())
- .withAttributeDefinitions(attributeDefinitions())
- .withSSESpecification(getSseSpecFromConfig())
- .withTags(getTableTagsFromConfig());
- if (capacity != null) {
- mode = String.format("with provisioned read capacity %d and"
- + " write capacity %s",
- capacity.getReadCapacityUnits(), capacity.getWriteCapacityUnits());
- request.withProvisionedThroughput(capacity);
- } else {
- mode = "with pay-per-request billing";
- request.withBillingMode(BillingMode.PAY_PER_REQUEST);
- }
- LOG.info("Creating non-existent DynamoDB table {} in region {} {}",
- tableName, region, mode);
- table = dynamoDB.createTable(request);
- LOG.debug("Awaiting table becoming active");
- } catch (ResourceInUseException e) {
- LOG.warn("ResourceInUseException while creating DynamoDB table {} "
- + "in region {}. This may indicate that the table was "
- + "created by another concurrent thread or process.",
- tableName, region);
- }
- waitForTableActive(table);
- putVersionMarkerItemToTable();
- }
-
- /**
- * Get DynamoDB table server side encryption (SSE) settings from configuration.
- */
- private SSESpecification getSseSpecFromConfig() {
- final SSESpecification sseSpecification = new SSESpecification();
- boolean enabled = conf.getBoolean(S3GUARD_DDB_TABLE_SSE_ENABLED, false);
- if (!enabled) {
- // Do not set other options if SSE is disabled. Otherwise it will throw
- // ValidationException.
- return sseSpecification;
- }
- sseSpecification.setEnabled(Boolean.TRUE);
- String cmk = null;
- try {
- // Get DynamoDB table SSE CMK from a configuration/credential provider.
- cmk = lookupPassword("", conf, S3GUARD_DDB_TABLE_SSE_CMK);
- } catch (IOException e) {
- LOG.error("Cannot retrieve " + S3GUARD_DDB_TABLE_SSE_CMK, e);
- }
- if (isEmpty(cmk)) {
- // Using Amazon managed default master key for DynamoDB table
- return sseSpecification;
- }
- if (SSE_DEFAULT_MASTER_KEY.equals(cmk)) {
- LOG.warn("Ignoring default DynamoDB table KMS Master Key {}",
- SSE_DEFAULT_MASTER_KEY);
- } else {
- sseSpecification.setSSEType("KMS");
- sseSpecification.setKMSMasterKeyId(cmk);
- }
- return sseSpecification;
- }
-
- /**
- * Return tags from configuration and the version marker for adding to
- * dynamo table during creation.
- */
- @Retries.OnceRaw
- public List getTableTagsFromConfig() {
- List tags = new ArrayList<>();
-
- // from configuration
- Map tagProperties =
- conf.getPropsWithPrefix(S3GUARD_DDB_TABLE_TAG);
- for (Map.Entry tagMapEntry : tagProperties.entrySet()) {
- Tag tag = new Tag().withKey(tagMapEntry.getKey())
- .withValue(tagMapEntry.getValue());
- tags.add(tag);
- }
- // add the version marker
- tags.add(newVersionMarkerTag());
- return tags;
- }
-
- /**
- * Create a new version marker tag.
- * @return a new version marker tag
- */
- private static Tag newVersionMarkerTag() {
- return new Tag().withKey(VERSION_MARKER_TAG_NAME).withValue(valueOf(VERSION));
- }
-
- /**
- * Verify that a table version is compatible with this S3Guard client.
- *
- * Checks for consistency between the version marker as the item and tag.
- *
- *
- * 1. If the table lacks both version markers AND it's empty,
- * both markers will be added.
- * If the table is not empty the check throws IOException
- * 2. If there's no version marker ITEM, the compatibility with the TAG
- * will be checked, and the version marker ITEM will be added if the
- * TAG version is compatible.
- * If the TAG version is not compatible, the check throws OException
- * 3. If there's no version marker TAG, the compatibility with the ITEM
- * version marker will be checked, and the version marker ITEM will be
- * added if the ITEM version is compatible.
- * If the ITEM version is not compatible, the check throws IOException
- * 4. If the TAG and ITEM versions are both present then both will be checked
- * for compatibility. If the ITEM or TAG version marker is not compatible,
- * the check throws IOException
- *
- *
- * @throws IOException on any incompatibility
- */
- @VisibleForTesting
- protected void verifyVersionCompatibility() throws IOException {
- final Item versionMarkerItem = getVersionMarkerItem();
- Item versionMarkerFromTag = null;
- boolean canReadDdbTags = true;
-
- try {
- versionMarkerFromTag = getVersionMarkerFromTags(table, amazonDynamoDB);
- } catch (AccessDeniedException e) {
- LOG.debug("Can not read tags of table.");
- canReadDdbTags = false;
- }
-
- LOG.debug("versionMarkerItem: {}; versionMarkerFromTag: {}",
- versionMarkerItem, versionMarkerFromTag);
-
- if (versionMarkerItem == null && versionMarkerFromTag == null) {
- if (!isEmptyTable(tableName, amazonDynamoDB)) {
- LOG.error("Table is not empty but missing the version maker. Failing.");
- throw new IOException(E_NO_VERSION_MARKER_AND_NOT_EMPTY
- + " Table: " + tableName);
- }
-
- if (canReadDdbTags) {
- LOG.info("Table {} contains no version marker item and tag. " +
- "The table is empty, so the version marker will be added " +
- "as TAG and ITEM.", tableName);
- putVersionMarkerItemToTable();
- tagTableWithVersionMarker();
- }
-
- if (!canReadDdbTags) {
- LOG.info("Table {} contains no version marker item and the tags are not readable. " +
- "The table is empty, so the ITEM version marker will be added .", tableName);
- putVersionMarkerItemToTable();
- }
- }
-
- if (versionMarkerItem == null && versionMarkerFromTag != null) {
- final int tagVersionMarker =
- extractVersionFromMarker(versionMarkerFromTag);
- throwExceptionOnVersionMismatch(tagVersionMarker, tableName,
- E_INCOMPATIBLE_TAG_VERSION);
-
- LOG.info("Table {} contains no version marker ITEM but contains " +
- "compatible version marker TAG. Restoring the version marker " +
- "item from tag.", tableName);
-
- putVersionMarkerItemToTable();
- }
-
- if (versionMarkerItem != null && versionMarkerFromTag == null
- && canReadDdbTags) {
- final int itemVersionMarker =
- extractVersionFromMarker(versionMarkerItem);
- throwExceptionOnVersionMismatch(itemVersionMarker, tableName,
- E_INCOMPATIBLE_ITEM_VERSION);
-
- LOG.info("Table {} contains no version marker TAG but contains " +
- "compatible version marker ITEM. Restoring the version marker " +
- "item from item.", tableName);
-
- tagTableWithVersionMarker();
- }
-
- if (versionMarkerItem != null && versionMarkerFromTag != null) {
- final int tagVersionMarker =
- extractVersionFromMarker(versionMarkerFromTag);
- final int itemVersionMarker =
- extractVersionFromMarker(versionMarkerItem);
-
- throwExceptionOnVersionMismatch(tagVersionMarker, tableName,
- E_INCOMPATIBLE_TAG_VERSION);
- throwExceptionOnVersionMismatch(itemVersionMarker, tableName,
- E_INCOMPATIBLE_ITEM_VERSION);
-
- LOG.debug("Table {} contains correct version marker TAG and ITEM.",
- tableName);
- }
- }
-
- private static boolean isEmptyTable(String tableName, AmazonDynamoDB aadb) {
- final ScanRequest req = new ScanRequest().withTableName(
- tableName).withLimit(1);
- final ScanResult result = aadb.scan(req);
- return result.getCount() == 0;
- }
-
- private static void throwExceptionOnVersionMismatch(int actual,
- String tableName,
- String exMsg) throws IOException {
-
- if (VERSION != actual) {
- throw new IOException(exMsg + " Table " + tableName
- + " Expected version: " + VERSION + " actual tag version: " +
- actual);
- }
- }
-
- /**
- * Add version marker to the dynamo table.
- */
- @Retries.OnceRaw
- private void putVersionMarkerItemToTable() {
- final Item marker = createVersionMarker(VERSION_MARKER_ITEM_NAME, VERSION,
- System.currentTimeMillis());
- putItem(marker);
- }
-
- /**
- * Wait for table being active.
- * @param t table to block on.
- * @throws IOException IO problems
- * @throws InterruptedIOException if the wait was interrupted
- * @throws IllegalArgumentException if an exception was raised in the waiter
- */
- @Retries.RetryTranslated
- private void waitForTableActive(Table t) throws IOException {
- invoker.retry("Waiting for active state of table " + tableName,
- null,
- true,
- () -> {
- try {
- t.waitForActive();
- } catch (IllegalArgumentException ex) {
- throw translateTableWaitFailure(tableName, ex);
- } catch (InterruptedException e) {
- LOG.warn("Interrupted while waiting for table {} in region {}"
- + " active",
- tableName, region, e);
- Thread.currentThread().interrupt();
- throw (InterruptedIOException)
- new InterruptedIOException("DynamoDB table '"
- + tableName + "' is not active yet in region " + region)
- .initCause(e);
- }
- });
- }
-
- /**
- * Handle a table wait failure by extracting any inner cause and
- * converting it, or, if unconvertable by wrapping
- * the IllegalArgumentException in an IOE.
- *
- * @param name name of the table
- * @param e exception
- * @return an IOE to raise.
- */
- @VisibleForTesting
- static IOException translateTableWaitFailure(
- final String name, IllegalArgumentException e) {
- final SdkBaseException ex = extractInnerException(e);
- if (ex != null) {
- if (ex instanceof WaiterTimedOutException) {
- // a timeout waiting for state change: extract the
- // message from the outer exception, but translate
- // the inner one for the throttle policy.
- return new AWSClientIOException(e.getMessage(), ex);
- } else {
- return translateException(e.getMessage(), name, ex);
- }
- } else {
- return new IOException(e);
- }
- }
-
- /**
- * Take an {@code IllegalArgumentException} raised by a DDB operation
- * and if it contains an inner SDK exception, unwrap it.
- * @param ex exception.
- * @return the inner AWS exception or null.
- */
- public static SdkBaseException extractInnerException(
- IllegalArgumentException ex) {
- if (ex.getCause() instanceof SdkBaseException) {
- return (SdkBaseException) ex.getCause();
- } else {
- return null;
- }
- }
-
- /**
- * Get the version mark item in the existing DynamoDB table.
- *
- * As the version marker item may be created by another concurrent thread or
- * process, we sleep and retry a limited number times if the lookup returns
- * with a null value.
- * DDB throttling is always retried.
- */
- @VisibleForTesting
- @Retries.RetryTranslated
- protected Item getVersionMarkerItem() throws IOException {
- final PrimaryKey versionMarkerKey =
- createVersionMarkerPrimaryKey(VERSION_MARKER_ITEM_NAME);
- int retryCount = 0;
- // look for a version marker, with usual throttling/failure retries.
- Item versionMarker = queryVersionMarker(versionMarkerKey);
- while (versionMarker == null) {
- // The marker was null.
- // Two possibilities
- // 1. This isn't a S3Guard table.
- // 2. This is a S3Guard table in construction; another thread/process
- // is about to write/actively writing the version marker.
- // So that state #2 is handled, batchWriteRetryPolicy is used to manage
- // retries.
- // This will mean that if the cause is actually #1, failure will not
- // be immediate. As this will ultimately result in a failure to
- // init S3Guard and the S3A FS, this isn't going to be a performance
- // bottleneck -simply a slightly slower failure report than would otherwise
- // be seen.
- // "if your settings are broken, performance is not your main issue"
- try {
- RetryPolicy.RetryAction action = batchWriteRetryPolicy.shouldRetry(null,
- retryCount, 0, true);
- if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
- break;
- } else {
- LOG.warn("No version marker found in the DynamoDB table: {}. " +
- "Sleeping {} ms before next retry", tableName, action.delayMillis);
- Thread.sleep(action.delayMillis);
- }
- } catch (Exception e) {
- throw new IOException("initTable: Unexpected exception " + e, e);
- }
- retryCount++;
- versionMarker = queryVersionMarker(versionMarkerKey);
- }
- return versionMarker;
- }
-
- /**
- * Issue the query to get the version marker, with throttling for overloaded
- * DDB tables.
- * @param versionMarkerKey key to look up
- * @return the marker
- * @throws IOException failure
- */
- @Retries.RetryTranslated
- private Item queryVersionMarker(final PrimaryKey versionMarkerKey)
- throws IOException {
- return readOp.retry("getVersionMarkerItem",
- VERSION_MARKER_ITEM_NAME, true,
- () -> table.getItem(versionMarkerKey));
- }
-
- /**
- * PUT a single item to the table.
- * @param item item to put
- * @return the outcome.
- */
- @Retries.OnceRaw
- private PutItemOutcome putItem(Item item) {
- LOG.debug("Putting item {}", item);
- return table.putItem(item);
- }
-
- /**
- * Provision the table with given read and write capacity units.
- * Call will fail if the table is busy, or the new values match the current
- * ones.
- *
- * Until the AWS SDK lets us switch a table to on-demand, an attempt to
- * set the I/O capacity to zero will fail.
- * @param readCapacity read units: must be greater than zero
- * @param writeCapacity write units: must be greater than zero
- * @throws IOException on a failure
- */
- @Retries.RetryTranslated
- void provisionTable(Long readCapacity, Long writeCapacity)
- throws IOException {
-
- if (readCapacity == 0 || writeCapacity == 0) {
- // table is pay on demand
- throw new IOException(E_ON_DEMAND_NO_SET_CAPACITY);
- }
- final ProvisionedThroughput toProvision = new ProvisionedThroughput()
- .withReadCapacityUnits(readCapacity)
- .withWriteCapacityUnits(writeCapacity);
- invoker.retry("ProvisionTable", tableName, true,
- () -> {
- final ProvisionedThroughputDescription p =
- table.updateTable(toProvision).getProvisionedThroughput();
- LOG.info("Provision table {} in region {}: readCapacityUnits={}, "
- + "writeCapacityUnits={}",
- tableName, region, p.getReadCapacityUnits(),
- p.getWriteCapacityUnits());
- });
- }
-
- @Retries.RetryTranslated
- public void destroy() throws IOException {
- if (table == null) {
- LOG.info("In destroy(): no table to delete");
- return;
- }
- LOG.info("Deleting DynamoDB table {} in region {}", tableName, region);
- Preconditions.checkNotNull(dynamoDB, "Not connected to DynamoDB");
- try {
- invoker.retry("delete", null, true,
- () -> table.delete());
- table.waitForDelete();
- } catch (IllegalArgumentException ex) {
- throw new TableDeleteTimeoutException(tableName,
- "Timeout waiting for the table " + getTableArn()
- + " to be deleted", ex);
- } catch (FileNotFoundException rnfe) {
- LOG.info("FileNotFoundException while deleting DynamoDB table {} in "
- + "region {}. This may indicate that the table does not exist, "
- + "or has been deleted by another concurrent thread or process.",
- tableName, region);
- } catch (InterruptedException ie) {
- Thread.currentThread().interrupt();
- LOG.warn("Interrupted while waiting for DynamoDB table {} being deleted",
- tableName, ie);
- throw new InterruptedIOException("Table " + tableName
- + " in region " + region + " has not been deleted");
- }
- }
-
- @Retries.RetryTranslated
- @VisibleForTesting
- void provisionTableBlocking(Long readCapacity, Long writeCapacity)
- throws IOException {
- provisionTable(readCapacity, writeCapacity);
- waitForTableActive(table);
- }
-
- public Table getTable() {
- return table;
- }
-
- public String getTableArn() {
- return tableArn;
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ExpirableMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ExpirableMetadata.java
deleted file mode 100644
index 5776bd4d345..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ExpirableMetadata.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-/**
- * Expirable Metadata abstract class is for storing the field needed for
- * metadata classes in S3Guard that could be expired with TTL.
- */
-public abstract class ExpirableMetadata {
- private long lastUpdated = 0;
-
- public long getLastUpdated() {
- return lastUpdated;
- }
-
- public void setLastUpdated(long lastUpdated) {
- this.lastUpdated = lastUpdated;
- }
-
- public boolean isExpired(long ttl, long now) {
- return (lastUpdated + ttl) <= now;
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ITtlTimeProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ITtlTimeProvider.java
deleted file mode 100644
index aa7fc4721b4..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ITtlTimeProvider.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-/**
- * This interface is defined for handling TTL expiry of metadata in S3Guard.
- *
- * TTL can be tested by implementing this interface and setting is as
- * {@code S3Guard.ttlTimeProvider}. By doing this, getNow() can return any
- * value preferred and flaky tests could be avoided. By default getNow()
- * returns the EPOCH in runtime.
- *
- * Time is measured in milliseconds,
- */
-public interface ITtlTimeProvider {
-
- /**
- * The current time in milliseconds.
- * Assuming this calls System.currentTimeMillis(), this is a native iO call
- * and so should be invoked sparingly (i.e. evaluate before any loop, rather
- * than inside).
- * @return the current time.
- */
- long getNow();
-
- /**
- * The TTL of the metadata.
- * @return time in millis after which metadata is considered out of date.
- */
- long getMetadataTtl();
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ImportOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ImportOperation.java
deleted file mode 100644
index 9c8b7022286..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ImportOperation.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import javax.annotation.Nullable;
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
-import org.apache.hadoop.fs.s3a.impl.ExecutingStoreOperation;
-import org.apache.hadoop.util.DurationInfo;
-
-/**
- * Import a directory tree into the metastore.
- * This code was moved from S3GuardTool and enhanced to mark
- * the destination tree as authoritative.
- */
-class ImportOperation extends ExecutingStoreOperation {
-
- private static final Logger LOG = LoggerFactory.getLogger(
- ImportOperation.class);
-
- /**
- * Source file system: must not be guarded.
- */
- private final S3AFileSystem filesystem;
-
- /**
- * Destination metadata store.
- */
- private final MetadataStore store;
-
- /**
- * Source entry: File or directory.
- */
- private final S3AFileStatus status;
-
- /**
- * If importing the directory tree -should it be marked
- * authoritative afterwards?
- */
- private final boolean authoritative;
-
- private final boolean verbose;
-
- /**
- * For DDB the BulkOperation tracking eliminates the need for this cache,
- * but it is retained here for the local store and to allow for
- * ease of moving to operations which may update the store in parallel with
- * writing.
- */
- private final Set dirCache = new HashSet<>();
-
- /**
- * Import.
- * @param filesystem Unguarded FS to scan.
- * @param store store to update
- * @param status source status
- * @param authoritative should the imported tree be marked as authoritative
- * @param verbose Verbose output
- */
- ImportOperation(final S3AFileSystem filesystem,
- final MetadataStore store,
- final S3AFileStatus status,
- final boolean authoritative,
- final boolean verbose) {
- super(filesystem.createStoreContext());
- this.verbose = verbose;
- Preconditions.checkState(!filesystem.hasMetadataStore(),
- "Source filesystem for import has a metadata store");
- this.filesystem = filesystem;
- this.store = store;
- this.status = status;
- this.authoritative = authoritative;
- }
-
- private S3AFileSystem getFilesystem() {
- return filesystem;
- }
-
- private MetadataStore getStore() {
- return store;
- }
-
- private FileStatus getStatus() {
- return status;
- }
-
- @Override
- public Long execute() throws IOException {
- final long items;
- if (status.isFile()) {
- PathMetadata meta = new PathMetadata(status);
- getStore().put(meta, null);
- items = 1;
- } else {
- try (DurationInfo ignored =
- new DurationInfo(LOG, "Importing %s", getStatus().getPath())) {
- items = importDir();
- }
- }
- return items;
- }
-
- /**
- * Recursively import every path under path.
- * @return number of items inserted into MetadataStore
- * @throws IOException on I/O errors.
- */
- private long importDir() throws IOException {
- Preconditions.checkArgument(status.isDirectory());
- long totalCountOfEntriesWritten = 0;
- final Path basePath = status.getPath();
- final MetadataStore ms = getStore();
- LOG.info("Importing directory {}", basePath);
- try (BulkOperationState operationState = ms
- .initiateBulkWrite(
- BulkOperationState.OperationType.Import,
- basePath)) {
- long countOfFilesWritten = 0;
- long countOfDirsWritten = 0;
- RemoteIterator it = getFilesystem()
- .listFilesAndEmptyDirectoriesForceNonAuth(basePath, true);
- while (it.hasNext()) {
- S3ALocatedFileStatus located = it.next();
- S3AFileStatus child;
- final Path path = located.getPath();
- final boolean isDirectory = located.isDirectory();
- if (isDirectory) {
- child = DynamoDBMetadataStore.makeDirStatus(path,
- located.getOwner());
- dirCache.add(path);
- // and update the dir count
- countOfDirsWritten++;
- } else {
- child = located.toS3AFileStatus();
- }
-
- int parentsWritten = putParentsIfNotPresent(child, operationState);
- LOG.debug("Wrote {} parent entries", parentsWritten);
-
- // We don't blindly overwrite any existing file entry in S3Guard with a
- // new one, Because that may lose the version information.
- // instead we merge them
- if (!isDirectory) {
- final PathMetadata existingEntry = S3Guard.getWithTtl(ms, path, null,
- false, true);
- if (existingEntry != null) {
- final S3AFileStatus existingStatus = existingEntry.getFileStatus();
- if (existingStatus.isFile()) {
- // source is also a file.
- // we only worry about an update if the timestamp is different,
- final String existingEtag = existingStatus.getETag();
- final String childEtag = child.getETag();
- if (child.getModificationTime()
- != existingStatus.getModificationTime()
- || existingStatus.getLen() != child.getLen()
- || existingEtag == null
- || !existingEtag.equals(childEtag)) {
- // files are potentially different, though a modtime change
- // can just be a clock skew problem
- // so if the etag is unchanged, we propagate any versionID
- if (childEtag.equals(existingEtag)) {
- // copy over any version ID.
- child.setVersionId(existingStatus.getVersionId());
- }
- } else {
- // the entry modtimes match
- child = null;
- }
- }
- }
- if (child != null) {
- countOfFilesWritten++;
- }
- }
- if (child != null) {
- // there's an entry to add.
-
- // log entry spaced to same width
- String t = isDirectory ? "Dir " : "File";
- if (verbose) {
- LOG.info("{} {}", t, path);
- } else {
- LOG.debug("{} {}", t, path);
- }
- S3Guard.putWithTtl(
- ms,
- new PathMetadata(child),
- getFilesystem().getTtlTimeProvider(),
- operationState);
- totalCountOfEntriesWritten++;
- }
- }
- LOG.info("Updated S3Guard with {} files and {} directory entries",
- countOfFilesWritten, countOfDirsWritten);
-
- // here all entries are imported.
- // tell the store that everything should be marked as auth
- if (authoritative) {
- LOG.info("Marking directory tree {} as authoritative",
- basePath);
- ms.markAsAuthoritative(basePath, operationState);
- }
- }
- return totalCountOfEntriesWritten;
- }
-
- /**
- * Put parents into metastore and cache if the parents are not present.
- *
- * There's duplication here with S3Guard DDB ancestor state, but this
- * is designed to work across implementations.
- * @param fileStatus the file or an empty directory.
- * @param operationState store's bulk update state.
- * @return number of entries written.
- * @throws IOException on I/O errors.
- */
- private int putParentsIfNotPresent(FileStatus fileStatus,
- @Nullable BulkOperationState operationState) throws IOException {
- Preconditions.checkNotNull(fileStatus);
- Path parent = fileStatus.getPath().getParent();
- int count = 0;
- while (parent != null) {
- if (dirCache.contains(parent)) {
- return count;
- }
- final ITtlTimeProvider timeProvider
- = getFilesystem().getTtlTimeProvider();
- final PathMetadata pmd = S3Guard.getWithTtl(getStore(), parent,
- timeProvider, false, true);
- if (pmd == null || pmd.isDeleted()) {
- S3AFileStatus dir = DynamoDBMetadataStore.makeDirStatus(parent,
- fileStatus.getOwner());
- S3Guard.putWithTtl(getStore(), new PathMetadata(dir),
- timeProvider,
- operationState);
- count++;
- }
- dirCache.add(parent);
- parent = parent.getParent();
- }
- return count;
- }
-
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataEntry.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataEntry.java
deleted file mode 100644
index 54050749758..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataEntry.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import javax.annotation.Nullable;
-
-/**
- * LocalMetadataEntry is used to store entries in the cache of
- * LocalMetadataStore. PathMetadata or dirListingMetadata can be null. The
- * entry is not immutable.
- */
-public final class LocalMetadataEntry {
- @Nullable
- private PathMetadata pathMetadata;
- @Nullable
- private DirListingMetadata dirListingMetadata;
-
- LocalMetadataEntry() {
- }
-
- LocalMetadataEntry(PathMetadata pmd){
- pathMetadata = pmd;
- dirListingMetadata = null;
- }
-
- LocalMetadataEntry(DirListingMetadata dlm){
- pathMetadata = null;
- dirListingMetadata = dlm;
- }
-
- public PathMetadata getFileMeta() {
- return pathMetadata;
- }
-
- public DirListingMetadata getDirListingMeta() {
- return dirListingMetadata;
- }
-
-
- public boolean hasPathMeta() {
- return this.pathMetadata != null;
- }
-
- public boolean hasDirMeta() {
- return this.dirListingMetadata != null;
- }
-
- public void setPathMetadata(PathMetadata pathMetadata) {
- this.pathMetadata = pathMetadata;
- }
-
- public void setDirListingMetadata(DirListingMetadata dirListingMetadata) {
- this.dirListingMetadata = dirListingMetadata;
- }
-
- @Override public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append("LocalMetadataEntry{");
- if(pathMetadata != null) {
- sb.append("pathMetadata=" + pathMetadata.getFileStatus().getPath());
- }
- if(dirListingMetadata != null){
- sb.append("; dirListingMetadata=" + dirListingMetadata.getPath());
- }
- sb.append("}");
- return sb.toString();
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
deleted file mode 100644
index 9d0ba9171ab..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
+++ /dev/null
@@ -1,651 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import javax.annotation.Nullable;
-
-import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-
-import org.apache.hadoop.thirdparty.com.google.common.cache.Cache;
-import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-import org.apache.hadoop.fs.s3a.Tristate;
-import org.apache.hadoop.fs.s3a.impl.StoreContext;
-import org.apache.hadoop.security.UserGroupInformation;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static org.apache.hadoop.fs.s3a.Constants.*;
-
-/**
- * This is a local, in-memory implementation of MetadataStore.
- * This is not a coherent cache across processes. It is only
- * locally-coherent.
- *
- * The purpose of this is for unit and integration testing.
- * It could also be used to accelerate local-only operations where only one
- * process is operating on a given object store, or multiple processes are
- * accessing a read-only storage bucket.
- *
- * 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 one cache internally with time based eviction.
- */
-public class LocalMetadataStore implements MetadataStore {
-
- public static final Logger LOG = LoggerFactory.getLogger(MetadataStore.class);
-
- /** Contains directory and file listings. */
- private Cache localCache;
-
- private FileSystem fs;
- /* Null iff this FS does not have an associated URI host. */
- private String uriHost;
-
- private String username;
-
- private ITtlTimeProvider ttlTimeProvider;
-
- @Override
- public void initialize(FileSystem fileSystem,
- ITtlTimeProvider ttlTp) throws IOException {
- Preconditions.checkNotNull(fileSystem);
- fs = fileSystem;
- URI fsURI = fs.getUri();
- uriHost = fsURI.getHost();
- if (uriHost != null && uriHost.equals("")) {
- uriHost = null;
- }
-
- initialize(fs.getConf(), ttlTp);
- }
-
- @Override
- public void initialize(Configuration conf, ITtlTimeProvider ttlTp)
- throws IOException {
- Preconditions.checkNotNull(conf);
- int maxRecords = conf.getInt(S3GUARD_METASTORE_LOCAL_MAX_RECORDS,
- DEFAULT_S3GUARD_METASTORE_LOCAL_MAX_RECORDS);
- if (maxRecords < 4) {
- maxRecords = 4;
- }
- int ttl = conf.getInt(S3GUARD_METASTORE_LOCAL_ENTRY_TTL,
- DEFAULT_S3GUARD_METASTORE_LOCAL_ENTRY_TTL);
-
- CacheBuilder builder = CacheBuilder.newBuilder().maximumSize(maxRecords);
- if (ttl >= 0) {
- builder.expireAfterAccess(ttl, TimeUnit.MILLISECONDS);
- }
-
- localCache = builder.build();
- username = UserGroupInformation.getCurrentUser().getShortUserName();
- this.ttlTimeProvider = ttlTp;
- }
-
- @Override
- public String toString() {
- final StringBuilder sb = new StringBuilder(
- "LocalMetadataStore{");
- sb.append("uriHost='").append(uriHost).append('\'');
- sb.append('}');
- return sb.toString();
- }
-
- @Override
- public void delete(Path p,
- final BulkOperationState operationState)
- throws IOException {
- doDelete(p, false, true);
- }
-
- @Override
- public void forgetMetadata(Path p) throws IOException {
- doDelete(p, false, false);
- }
-
- @Override
- public void deleteSubtree(Path path,
- final BulkOperationState operationState)
- throws IOException {
- doDelete(path, true, true);
- }
-
- private synchronized void doDelete(Path p, boolean recursive,
- boolean tombstone) {
-
- Path path = standardize(p);
-
- // Delete entry from file cache, then from cached parent directory, if any
- deleteCacheEntries(path, tombstone);
-
- if (recursive) {
- // Remove all entries that have this dir as path prefix.
- deleteEntryByAncestor(path, localCache, tombstone, ttlTimeProvider);
- }
- }
-
- @Override
- public void deletePaths(final Collection paths,
- @Nullable final BulkOperationState operationState) throws IOException {
- for (Path path : paths) {
- doDelete(path, false, true);
- }
- }
-
- @Override
- public synchronized PathMetadata get(Path p) throws IOException {
- return get(p, false);
- }
-
- @Override
- public PathMetadata get(Path p, boolean wantEmptyDirectoryFlag)
- throws IOException {
- Path path = standardize(p);
- synchronized (this) {
- PathMetadata m = getFileMeta(path);
-
- if (wantEmptyDirectoryFlag && m != null &&
- m.getFileStatus().isDirectory()) {
- m.setIsEmptyDirectory(isEmptyDirectory(p));
- }
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("get({}) -> {}", path, m == null ? "null" : m.prettyPrint());
- }
- return m;
- }
- }
-
- /**
- * Determine if directory is empty.
- * Call with lock held.
- * @param p a Path, already filtered through standardize()
- * @return TRUE / FALSE if known empty / not-empty, UNKNOWN otherwise.
- */
- private Tristate isEmptyDirectory(Path p) {
- DirListingMetadata dlm = getDirListingMeta(p);
- return dlm.withoutTombstones().isEmpty();
- }
-
- @Override
- public synchronized DirListingMetadata listChildren(Path p) throws
- IOException {
- Path path = standardize(p);
- DirListingMetadata listing = getDirListingMeta(path);
- if (LOG.isDebugEnabled()) {
- LOG.debug("listChildren({}) -> {}", path,
- listing == null ? "null" : listing.prettyPrint());
- }
-
- if (listing != null) {
- // Make a copy so callers can mutate without affecting our state
- return new DirListingMetadata(listing);
- }
- return null;
- }
-
- @Override
- public void move(@Nullable Collection pathsToDelete,
- @Nullable Collection pathsToCreate,
- @Nullable final BulkOperationState operationState) throws IOException {
- LOG.info("Move {} to {}", pathsToDelete, pathsToCreate);
-
- if (pathsToCreate == null) {
- pathsToCreate = Collections.emptyList();
- }
- if (pathsToDelete == null) {
- pathsToDelete = Collections.emptyList();
- }
-
- // I feel dirty for using reentrant lock. :-|
- synchronized (this) {
-
- // 1. Delete pathsToDelete
- for (Path meta : pathsToDelete) {
- LOG.debug("move: deleting metadata {}", meta);
- delete(meta, null);
- }
-
- // 2. Create new destination path metadata
- for (PathMetadata meta : pathsToCreate) {
- LOG.debug("move: adding metadata {}", meta);
- put(meta, null);
- }
-
- // 3. We now know full contents of all dirs in destination subtree
- for (PathMetadata meta : pathsToCreate) {
- FileStatus status = meta.getFileStatus();
- if (status == null || status.isDirectory()) {
- continue;
- }
- DirListingMetadata dir = listChildren(status.getPath());
- if (dir != null) { // could be evicted already
- dir.setAuthoritative(true);
- }
- }
- }
- }
-
- @Override
- public void put(final PathMetadata meta) throws IOException {
- put(meta, null);
- }
-
- @Override
- public void put(PathMetadata meta,
- final BulkOperationState operationState) throws IOException {
-
- Preconditions.checkNotNull(meta);
- S3AFileStatus status = meta.getFileStatus();
- Path path = standardize(status.getPath());
- synchronized (this) {
-
- /* Add entry for this file. */
- if (LOG.isDebugEnabled()) {
- LOG.debug("put {} -> {}", path, meta.prettyPrint());
- }
- 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
- * listStatus(path) at least see the directory.
- *
- * If we had a boolean flag argument "isNew", we would know whether this
- * is an existing directory the client discovered via getFileStatus(),
- * or if it is a newly-created directory. In the latter case, we would
- * be able to mark the directory as authoritative (fully-cached),
- * saving round trips to underlying store for subsequent listStatus()
- */
-
- // 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) {
- LocalMetadataEntry parentMeta = localCache.getIfPresent(parentPath);
-
- // Create empty parent LocalMetadataEntry if it doesn't exist
- if (parentMeta == null){
- parentMeta = new LocalMetadataEntry();
- localCache.put(parentPath, parentMeta);
- }
-
- // If there is no directory metadata on the parent entry, create
- // an empty one
- if (!parentMeta.hasDirMeta()) {
- DirListingMetadata parentDirMeta =
- new DirListingMetadata(parentPath, DirListingMetadata.EMPTY_DIR,
- false);
- parentDirMeta.setLastUpdated(meta.getLastUpdated());
- parentMeta.setDirListingMetadata(parentDirMeta);
- }
-
- // Add the child pathMetadata to the listing
- parentMeta.getDirListingMeta().put(meta);
-
- // Mark the listing entry as deleted if the meta is set to deleted
- if(meta.isDeleted()) {
- parentMeta.getDirListingMeta().markDeleted(path,
- ttlTimeProvider.getNow());
- }
- }
- }
- }
-
- @Override
- public synchronized void put(DirListingMetadata meta,
- final List unchangedEntries,
- final BulkOperationState operationState) throws IOException {
- if (LOG.isDebugEnabled()) {
- LOG.debug("put dirMeta {}", meta.prettyPrint());
- }
- 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(), null);
- }
-
- public synchronized void put(Collection extends PathMetadata> metas,
- final BulkOperationState operationState) throws
- IOException {
- for (PathMetadata meta : metas) {
- put(meta, operationState);
- }
- }
-
- @Override
- public void close() throws IOException {
-
- }
-
- @Override
- public void destroy() throws IOException {
- if (localCache != null) {
- localCache.invalidateAll();
- }
- }
-
- @Override
- public void prune(PruneMode pruneMode, long cutoff) throws IOException{
- prune(pruneMode, cutoff, "");
- }
-
- @Override
- public synchronized long prune(PruneMode pruneMode, long cutoff,
- String keyPrefix) {
- // prune files
- AtomicLong count = new AtomicLong();
- // filter path_metadata (files), filter expired, remove expired
- localCache.asMap().entrySet().stream()
- .filter(entry -> entry.getValue().hasPathMeta())
- .filter(entry -> expired(pruneMode,
- entry.getValue().getFileMeta(), cutoff, keyPrefix))
- .forEach(entry -> {
- localCache.invalidate(entry.getKey());
- count.incrementAndGet();
- });
-
-
- // 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 oldChildren = metadata.getListing();
- Collection newChildren = new LinkedList<>();
-
- for (PathMetadata child : oldChildren) {
- if (!expired(pruneMode, child, cutoff, keyPrefix)) {
- newChildren.add(child);
- } else {
- count.incrementAndGet();
- }
- }
- removeAuthoritativeFromParent(path, oldChildren, newChildren);
- });
- return count.get();
- }
-
- private void removeAuthoritativeFromParent(Path path,
- Collection oldChildren,
- Collection newChildren) {
- 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(PruneMode pruneMode, PathMetadata metadata,
- long cutoff, String keyPrefix) {
- final S3AFileStatus status = metadata.getFileStatus();
- final URI statusUri = status.getPath().toUri();
-
- // remove the protocol from path string to be able to compare
- String bucket = statusUri.getHost();
- String statusTranslatedPath = "";
- if(bucket != null && !bucket.isEmpty()){
- // if there's a bucket, (well defined host in Uri) the pathToParentKey
- // can be used to get the path from the status
- statusTranslatedPath =
- PathMetadataDynamoDBTranslation.pathToParentKey(status.getPath());
- } else {
- // if there's no bucket in the path the pathToParentKey will fail, so
- // this is the fallback to get the path from status
- statusTranslatedPath = statusUri.getPath();
- }
-
- boolean expired;
- switch (pruneMode) {
- case ALL_BY_MODTIME:
- // Note: S3 doesn't track modification time on directories, so for
- // consistency with the DynamoDB implementation we ignore that here
- expired = status.getModificationTime() < cutoff && !status.isDirectory()
- && statusTranslatedPath.startsWith(keyPrefix);
- break;
- case TOMBSTONES_BY_LASTUPDATED:
- expired = metadata.getLastUpdated() < cutoff && metadata.isDeleted()
- && statusTranslatedPath.startsWith(keyPrefix);
- break;
- default:
- throw new UnsupportedOperationException("Unsupported prune mode: "
- + pruneMode);
- }
-
- return expired;
- }
-
- @VisibleForTesting
- static void deleteEntryByAncestor(Path ancestor,
- Cache cache, boolean tombstone,
- ITtlTimeProvider ttlTimeProvider) {
-
- 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()){
- final PathMetadata pmTombstone = PathMetadata.tombstone(path,
- ttlTimeProvider.getNow());
- meta.setPathMetadata(pmTombstone);
- } else {
- cache.invalidate(path);
- }
- });
- }
-
- /**
- * @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) {
- String aStr = ancestor.toString();
- if (!ancestor.isRoot()) {
- aStr += "/";
- }
- String fStr = f.toString();
- return (fStr.startsWith(aStr));
- }
-
- /**
- * Update fileCache and dirCache to reflect deletion of file 'f'. Call with
- * 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 entry
- LOG.debug("delete file entry for {}", path);
- if(entry.hasPathMeta()){
- if (tombstone) {
- PathMetadata pmd = PathMetadata.tombstone(path,
- ttlTimeProvider.getNow());
- entry.setPathMetadata(pmd);
- } else {
- entry.setPathMetadata(null);
- }
- }
-
- // If this path is a dir, remove its listing
- if(entry.hasDirMeta()) {
- LOG.debug("removing listing of {}", path);
- entry.setDirListingMetadata(null);
- }
-
- // 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 = getDirListingMeta(parent);
- if (dir != null) {
- LOG.debug("removing parent's entry for {} ", path);
- if (tombstone) {
- dir.markDeleted(path, ttlTimeProvider.getNow());
- } else {
- dir.remove(path);
- }
- }
- }
- }
-
- /**
- * Return a "standardized" version of a path so we always have a consistent
- * hash value. Also asserts the path is absolute, and contains host
- * component.
- * @param p input Path
- * @return standardized version of Path, suitable for hash key
- */
- private Path standardize(Path p) {
- Preconditions.checkArgument(p.isAbsolute(), "Path must be absolute");
- URI uri = p.toUri();
- if (uriHost != null) {
- Preconditions.checkArgument(StringUtils.isNotEmpty(uri.getHost()));
- }
- return p;
- }
-
- @Override
- public Map getDiagnostics() throws IOException {
- Map map = new HashMap<>();
- map.put("name", "local://metadata");
- map.put("uriHost", uriHost);
- map.put("description", "Local in-VM metadata store for testing");
- map.put(MetadataStoreCapabilities.PERSISTS_AUTHORITATIVE_BIT,
- Boolean.toString(true));
- return map;
- }
-
- @Override
- public void updateParameters(Map 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;
- }
- }
-
- @Override
- public RenameTracker initiateRenameOperation(final StoreContext storeContext,
- final Path source,
- final S3AFileStatus sourceStatus, final Path dest) throws IOException {
- return new ProgressiveRenameTracker(storeContext, this, source, dest,
- null);
- }
-
- @Override
- public synchronized void setTtlTimeProvider(ITtlTimeProvider ttlTimeProvider) {
- this.ttlTimeProvider = ttlTimeProvider;
- }
-
- @Override
- public synchronized void addAncestors(final Path qualifiedPath,
- @Nullable final BulkOperationState operationState) throws IOException {
-
- Collection newDirs = new ArrayList<>();
- Path parent = qualifiedPath.getParent();
- while (!parent.isRoot()) {
- PathMetadata directory = get(parent);
- if (directory == null || directory.isDeleted()) {
- S3AFileStatus status = new S3AFileStatus(Tristate.FALSE, parent,
- username);
- PathMetadata meta = new PathMetadata(status, Tristate.FALSE, false,
- ttlTimeProvider.getNow());
- newDirs.add(meta);
- } else {
- break;
- }
- parent = parent.getParent();
- }
- if (!newDirs.isEmpty()) {
- put(newDirs, operationState);
- }
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
deleted file mode 100644
index 94d80635371..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
+++ /dev/null
@@ -1,438 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import javax.annotation.Nullable;
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3a.Retries;
-import org.apache.hadoop.fs.s3a.Retries.RetryTranslated;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-import org.apache.hadoop.fs.s3a.impl.StoreContext;
-
-/**
- * {@code MetadataStore} defines the set of operations that any metadata store
- * implementation must provide. Note that all {@link Path} objects provided
- * to methods must be absolute, not relative paths.
- * Implementations must implement any retries needed internally, such that
- * transient errors are generally recovered from without throwing exceptions
- * from this API.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public interface MetadataStore extends Closeable {
-
- /**
- * Performs one-time initialization of the metadata store.
- *
- * @param fs {@code FileSystem} associated with the MetadataStore
- * @param ttlTimeProvider the time provider to use for metadata expiry
- * @throws IOException if there is an error
- */
- void initialize(FileSystem fs, ITtlTimeProvider ttlTimeProvider)
- throws IOException;
-
- /**
- * Performs one-time initialization of the metadata store via configuration.
- * @see #initialize(FileSystem, ITtlTimeProvider)
- * @param conf Configuration.
- * @param ttlTimeProvider the time provider to use for metadata expiry
- * @throws IOException if there is an error
- */
- void initialize(Configuration conf,
- ITtlTimeProvider ttlTimeProvider) throws IOException;
-
- /**
- * Deletes exactly one path, leaving a tombstone to prevent lingering,
- * inconsistent copies of it from being listed.
- *
- * Deleting an entry with a tombstone needs a
- * {@link org.apache.hadoop.fs.s3a.s3guard.S3Guard.TtlTimeProvider} because
- * the lastUpdated field of the record has to be updated to now
.
- *
- * @param path the path to delete
- * @param operationState (nullable) operational state for a bulk update
- * @throws IOException if there is an error
- */
- void delete(Path path,
- @Nullable BulkOperationState operationState)
- throws IOException;
-
- /**
- * Removes the record of exactly one path. Does not leave a tombstone (see
- * {@link MetadataStore#delete(Path, BulkOperationState)}. It is currently
- * intended for testing only, and a need to use it as part of normal
- * FileSystem usage is not anticipated.
- *
- * @param path the path to delete
- * @throws IOException if there is an error
- */
- @VisibleForTesting
- void forgetMetadata(Path path) throws IOException;
-
- /**
- * Deletes the entire sub-tree rooted at the given path, leaving tombstones
- * to prevent lingering, inconsistent copies of it from being listed.
- *
- * In addition to affecting future calls to {@link #get(Path)},
- * implementations must also update any stored {@code DirListingMetadata}
- * objects which track the parent of this file.
- *
- * Deleting a subtree with a tombstone needs a
- * {@link org.apache.hadoop.fs.s3a.s3guard.S3Guard.TtlTimeProvider} because
- * the lastUpdated field of all records have to be updated to now
.
- *
- * @param path the root of the sub-tree to delete
- * @param operationState (nullable) operational state for a bulk update
- * @throws IOException if there is an error
- */
- @Retries.RetryTranslated
- void deleteSubtree(Path path,
- @Nullable BulkOperationState operationState)
- throws IOException;
-
- /**
- * Delete the paths.
- * There's no attempt to order the paths: they are
- * deleted in the order passed in.
- * @param paths paths to delete.
- * @param operationState Nullable operation state
- * @throws IOException failure
- */
-
- @RetryTranslated
- void deletePaths(Collection paths,
- @Nullable BulkOperationState operationState)
- throws IOException;
-
- /**
- * Gets metadata for a path.
- *
- * @param path the path to get
- * @return metadata for {@code path}, {@code null} if not found
- * @throws IOException if there is an error
- */
- PathMetadata get(Path path) throws IOException;
-
- /**
- * Gets metadata for a path. Alternate method that includes a hint
- * whether or not the MetadataStore should do work to compute the value for
- * {@link PathMetadata#isEmptyDirectory()}. Since determining emptiness
- * may be an expensive operation, this can save wasted work.
- *
- * @param path the path to get
- * @param wantEmptyDirectoryFlag Set to true to give a hint to the
- * MetadataStore that it should try to compute the empty directory flag.
- * @return metadata for {@code path}, {@code null} if not found
- * @throws IOException if there is an error
- */
- PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
- throws IOException;
-
- /**
- * Lists metadata for all direct children of a path.
- *
- * @param path the path to list
- * @return metadata for all direct children of {@code path} which are being
- * tracked by the MetadataStore, or {@code null} if the path was not found
- * in the MetadataStore.
- * @throws IOException if there is an error
- */
- @Retries.RetryTranslated
- DirListingMetadata listChildren(Path path) throws IOException;
-
- /**
- * This adds all new ancestors of a path as directories.
- *
- * Important: to propagate TTL information, any new ancestors added
- * must have their last updated timestamps set through
- * {@link S3Guard#patchLastUpdated(Collection, ITtlTimeProvider)}.
- * @param qualifiedPath path to update
- * @param operationState (nullable) operational state for a bulk update
- * @throws IOException failure
- */
- @RetryTranslated
- void addAncestors(Path qualifiedPath,
- @Nullable BulkOperationState operationState) throws IOException;
-
- /**
- * Record the effects of a {@link FileSystem#rename(Path, Path)} in the
- * MetadataStore. Clients provide explicit enumeration of the affected
- * paths (recursively), before and after the rename.
- *
- * This operation is not atomic, unless specific implementations claim
- * otherwise.
- *
- * On the need to provide an enumeration of directory trees instead of just
- * source and destination paths:
- * Since a MetadataStore does not have to track all metadata for the
- * underlying storage system, and a new MetadataStore may be created on an
- * existing underlying filesystem, this move() may be the first time the
- * MetadataStore sees the affected paths. Therefore, simply providing src
- * and destination paths may not be enough to record the deletions (under
- * src path) and creations (at destination) that are happening during the
- * rename().
- *
- * @param pathsToDelete Collection of all paths that were removed from the
- * source directory tree of the move.
- * @param pathsToCreate Collection of all PathMetadata for the new paths
- * that were created at the destination of the rename().
- * @param operationState Any ongoing state supplied to the rename tracker
- * which is to be passed in with each move operation.
- * @throws IOException if there is an error
- */
- void move(@Nullable Collection pathsToDelete,
- @Nullable Collection pathsToCreate,
- @Nullable BulkOperationState operationState) throws IOException;
-
- /**
- * Saves metadata for exactly one path.
- *
- * Implementations may pre-create all the path's ancestors automatically.
- * Implementations must update any {@code DirListingMetadata} objects which
- * track the immediate parent of this file.
- *
- * @param meta the metadata to save
- * @throws IOException if there is an error
- */
- @RetryTranslated
- void put(PathMetadata meta) throws IOException;
-
- /**
- * Saves metadata for exactly one path, potentially
- * using any bulk operation state to eliminate duplicate work.
- *
- * Implementations may pre-create all the path's ancestors automatically.
- * Implementations must update any {@code DirListingMetadata} objects which
- * track the immediate parent of this file.
- *
- * @param meta the metadata to save
- * @param operationState operational state for a bulk update
- * @throws IOException if there is an error
- */
- @RetryTranslated
- void put(PathMetadata meta,
- @Nullable BulkOperationState operationState) throws IOException;
-
- /**
- * Saves metadata for any number of paths.
- *
- * Semantics are otherwise the same as single-path puts.
- *
- * @param metas the metadata to save
- * @param operationState (nullable) operational state for a bulk update
- * @throws IOException if there is an error
- */
- void put(Collection extends PathMetadata> metas,
- @Nullable BulkOperationState operationState) throws IOException;
-
- /**
- * Save directory listing metadata. Callers may save a partial directory
- * listing for a given path, or may store a complete and authoritative copy
- * of the directory listing. {@code MetadataStore} implementations may
- * subsequently keep track of all modifications to the directory contents at
- * this path, and return authoritative results from subsequent calls to
- * {@link #listChildren(Path)}. See {@link DirListingMetadata}.
- *
- * Any authoritative results returned are only authoritative for the scope
- * of the {@code MetadataStore}: A per-process {@code MetadataStore}, for
- * example, would only show results visible to that process, potentially
- * missing metadata updates (create, delete) made to the same path by
- * another process.
- *
- * To optimize updates and avoid overwriting existing entries which
- * may contain extra data, entries in the list of unchangedEntries may
- * be excluded. That is: the listing metadata has the full list of
- * what it believes are children, but implementations can opt to ignore
- * some.
- * @param meta Directory listing metadata.
- * @param unchangedEntries list of entries in the dir listing which have
- * not changed since the directory was list scanned on s3guard.
- * @param operationState operational state for a bulk update
- * @throws IOException if there is an error
- */
- void put(DirListingMetadata meta,
- final List unchangedEntries,
- @Nullable BulkOperationState operationState) throws IOException;
-
- /**
- * Destroy all resources associated with the metadata store.
- *
- * The destroyed resources can be DynamoDB tables, MySQL databases/tables, or
- * HDFS directories. Any operations after calling this method may possibly
- * fail.
- *
- * This operation is idempotent.
- *
- * @throws IOException if there is an error
- */
- void destroy() throws IOException;
-
- /**
- * Prune method with two modes of operation:
- *
- * -
- * {@link PruneMode#ALL_BY_MODTIME}
- * Clear any metadata older than a specified mod_time from the store.
- * Note that this modification time is the S3 modification time from the
- * object's metadata - from the object store.
- * Implementations MUST clear file metadata, and MAY clear directory
- * metadata (s3a itself does not track modification time for directories).
- * Implementations may also choose to throw UnsupportedOperationException
- * instead. Note that modification times must be in UTC, as returned by
- * System.currentTimeMillis at the time of modification.
- *
- *
- *
- *
- * -
- * {@link PruneMode#TOMBSTONES_BY_LASTUPDATED}
- * Clear any tombstone updated earlier than a specified time from the
- * store. Note that this last_updated is the time when the metadata
- * entry was last updated and maintained by the metadata store.
- * Implementations MUST clear file metadata, and MAY clear directory
- * metadata (s3a itself does not track modification time for directories).
- * Implementations may also choose to throw UnsupportedOperationException
- * instead. Note that last_updated must be in UTC, as returned by
- * System.currentTimeMillis at the time of modification.
- *
- *
- *
- * @param pruneMode Prune Mode
- * @param cutoff Oldest time to allow (UTC)
- * @throws IOException if there is an error
- * @throws UnsupportedOperationException if not implemented
- */
- void prune(PruneMode pruneMode, long cutoff) throws IOException,
- UnsupportedOperationException;
-
- /**
- * Same as {@link MetadataStore#prune(PruneMode, long)}, but with an
- * additional keyPrefix parameter to filter the pruned keys with a prefix.
- *
- * @param pruneMode Prune Mode
- * @param cutoff Oldest time in milliseconds to allow (UTC)
- * @param keyPrefix The prefix for the keys that should be removed
- * @throws IOException if there is an error
- * @throws UnsupportedOperationException if not implemented
- * @return the number of pruned entries
- */
- long prune(PruneMode pruneMode, long cutoff, String keyPrefix)
- throws IOException, UnsupportedOperationException;
-
- /**
- * Get any diagnostics information from a store, as a list of (key, value)
- * tuples for display. Arbitrary values; no guarantee of stability.
- * These are for debugging and testing only.
- * @return a map of strings.
- * @throws IOException if there is an error
- */
- Map getDiagnostics() throws IOException;
-
- /**
- * Tune/update parameters for an existing table.
- * @param parameters map of params to change.
- * @throws IOException if there is an error
- */
- void updateParameters(Map parameters) throws IOException;
-
- /**
- * Mark all directories created/touched in an operation as authoritative.
- * The metastore can now update that path with any authoritative
- * flags it chooses.
- * The store may assume that therefore the operation state is complete.
- * This holds for rename and needs to be documented for import.
- * @param dest destination path.
- * @param operationState active state.
- * @throws IOException failure.
- * @return the number of directories marked.
- */
- default int markAsAuthoritative(Path dest,
- BulkOperationState operationState)
- throws IOException {
- return 0;
- }
-
- /**
- * Modes of operation for prune.
- * For details see {@link MetadataStore#prune(PruneMode, long)}
- */
- enum PruneMode {
- ALL_BY_MODTIME,
- TOMBSTONES_BY_LASTUPDATED
- }
-
- /**
- * Start a rename operation.
- *
- * @param storeContext store context.
- * @param source source path
- * @param sourceStatus status of the source file/dir
- * @param dest destination path.
- * @return the rename tracker
- * @throws IOException Failure.
- */
- RenameTracker initiateRenameOperation(
- StoreContext storeContext,
- Path source,
- S3AFileStatus sourceStatus,
- Path dest)
- throws IOException;
-
- /**
- * Initiate a bulk update and create an operation state for it.
- * This may then be passed into put operations.
- * @param operation the type of the operation.
- * @param dest path under which updates will be explicitly put.
- * @return null or a store-specific state to pass into the put operations.
- * @throws IOException failure
- */
- default BulkOperationState initiateBulkWrite(
- BulkOperationState.OperationType operation,
- Path dest) throws IOException {
- return new BulkOperationState(operation);
- }
-
- /**
- * The TtlTimeProvider has to be set during the initialization for the
- * metadatastore, but this method can be used for testing, and change the
- * instance during runtime.
- *
- * @param ttlTimeProvider
- */
- void setTtlTimeProvider(ITtlTimeProvider ttlTimeProvider);
-
- /**
- * Get any S3GuardInstrumentation for this store...must not be null.
- * @return any store instrumentation.
- */
- default MetastoreInstrumentation getInstrumentation() {
- return new MetastoreInstrumentationImpl();
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreCapabilities.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreCapabilities.java
deleted file mode 100644
index c1464402246..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreCapabilities.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * All the capability constants used for the
- * {@link MetadataStore} implementations.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public final class MetadataStoreCapabilities {
-
- private MetadataStoreCapabilities(){
- }
-
- /**
- * This capability tells if the metadata store supports authoritative
- * directories. Used in {@link MetadataStore#getDiagnostics()} as a key
- * for this capability. The value can be boolean true or false.
- * If the Map.get() returns null for this key, that is interpreted as false.
- */
- public static final String PERSISTS_AUTHORITATIVE_BIT =
- "persist.authoritative.bit";
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java
deleted file mode 100644
index a92aaeb0560..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.Queue;
-import java.util.Set;
-
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-
-/**
- * {@code MetadataStoreListFilesIterator} is a {@link RemoteIterator} that
- * is similar to {@code DescendantsIterator} but does not return directories
- * that have (or may have) children, and will also provide access to the set of
- * tombstones to allow recently deleted S3 objects to be filtered out from a
- * corresponding request. In other words, it returns tombstones and the same
- * set of objects that should exist in S3: empty directories, and files, and not
- * other directories whose existence is inferred therefrom.
- *
- * For example, assume the consistent store contains metadata representing this
- * file system structure:
- *
- *
- * /dir1
- * |-- dir2
- * | |-- file1
- * | `-- file2
- * `-- dir3
- * |-- dir4
- * | `-- file3
- * |-- dir5
- * | `-- file4
- * `-- dir6
- *
- *
- * Consider this code sample:
- *
- * final PathMetadata dir1 = get(new Path("/dir1"));
- * for (MetadataStoreListFilesIterator files =
- * new MetadataStoreListFilesIterator(dir1); files.hasNext(); ) {
- * final FileStatus status = files.next().getFileStatus();
- * System.out.printf("%s %s%n", status.isDirectory() ? 'D' : 'F',
- * status.getPath());
- * }
- *
- *
- * The output is:
- *
- * F /dir1/dir2/file1
- * F /dir1/dir2/file2
- * F /dir1/dir3/dir4/file3
- * F /dir1/dir3/dir5/file4
- * D /dir1/dir3/dir6
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class MetadataStoreListFilesIterator implements
- RemoteIterator {
- public static final Logger LOG = LoggerFactory.getLogger(
- MetadataStoreListFilesIterator.class);
-
- private final boolean allowAuthoritative;
- private final MetadataStore metadataStore;
- private final Set tombstones = new HashSet<>();
- private final boolean recursivelyAuthoritative;
- private Iterator leafNodesIterator = null;
-
- public MetadataStoreListFilesIterator(MetadataStore ms, PathMetadata meta,
- boolean allowAuthoritative) throws IOException {
- Preconditions.checkNotNull(ms);
- this.metadataStore = ms;
- this.allowAuthoritative = allowAuthoritative;
- this.recursivelyAuthoritative = prefetch(meta);
- }
-
- /**
- * Walks the listing tree, starting from given metadata path. All
- * encountered files and empty directories are added to
- * {@link leafNodesIterator} unless a directory seems to be empty
- * and at least one of the following conditions hold:
- *
- * -
- * The directory listing is not marked authoritative
- *
- * -
- * Authoritative mode is not allowed
- *
- *
- * @param meta starting point for tree walk
- * @return {@code true} if all encountered directory listings
- * are marked as authoritative
- * @throws IOException
- */
- private boolean prefetch(PathMetadata meta) throws IOException {
- final Queue queue = new LinkedList<>();
- final Collection leafNodes = new ArrayList<>();
-
- boolean allListingsAuthoritative = true;
- if (meta != null) {
- final Path path = meta.getFileStatus().getPath();
- if (path.isRoot()) {
- DirListingMetadata rootListing = metadataStore.listChildren(path);
- if (rootListing != null) {
- if (!rootListing.isAuthoritative()) {
- allListingsAuthoritative = false;
- }
- tombstones.addAll(rootListing.listTombstones());
- queue.addAll(rootListing.withoutTombstones().getListing());
- }
- } else {
- queue.add(meta);
- }
- } else {
- allListingsAuthoritative = false;
- }
-
- while(!queue.isEmpty()) {
- PathMetadata nextMetadata = queue.poll();
- S3AFileStatus nextStatus = nextMetadata.getFileStatus();
- if (nextStatus.isFile()) {
- // All files are leaf nodes by definition
- leafNodes.add(nextStatus);
- continue;
- }
- if (nextStatus.isDirectory()) {
- final Path path = nextStatus.getPath();
- DirListingMetadata children = metadataStore.listChildren(path);
- if (children != null) {
- if (!children.isAuthoritative()) {
- allListingsAuthoritative = false;
- }
- tombstones.addAll(children.listTombstones());
- Collection liveChildren =
- children.withoutTombstones().getListing();
- if (!liveChildren.isEmpty()) {
- // If it's a directory, has children, not all deleted, then we
- // add the children to the queue and move on to the next node
- queue.addAll(liveChildren);
- continue;
- } else if (allowAuthoritative && children.isAuthoritative()) {
- leafNodes.add(nextStatus);
- }
- } else {
- // we do not have a listing, so directory definitely non-authoritative
- allListingsAuthoritative = false;
- }
- }
- // Directories that *might* be empty are ignored for now, since we
- // cannot confirm that they are empty without incurring other costs.
- // Users of this class can still discover empty directories via S3's
- // fake directories, subject to the same consistency semantics as before.
- // The only other possibility is a symlink, which is unsupported on S3A.
- }
- leafNodesIterator = leafNodes.iterator();
- return allListingsAuthoritative;
- }
-
- @Override
- public boolean hasNext() {
- return leafNodesIterator.hasNext();
- }
-
- @Override
- public S3AFileStatus next() {
- return leafNodesIterator.next();
- }
-
- public boolean isRecursivelyAuthoritative() {
- return recursivelyAuthoritative;
- }
-
- public Set listTombstones() {
- return tombstones;
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetastoreInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetastoreInstrumentation.java
deleted file mode 100644
index b5ac00eafe9..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetastoreInstrumentation.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-/**
- * Instrumentation exported to S3Guard.
- */
-public interface MetastoreInstrumentation {
-
- /** Initialized event. */
- void initialized();
-
- /** Store has been closed. */
- void storeClosed();
-
- /**
- * Throttled request.
- */
- void throttled();
-
- /**
- * S3Guard is retrying after a (retryable) failure.
- */
- void retrying();
-
- /**
- * Records have been deleted.
- * @param count the number of records deleted.
- */
- void recordsDeleted(int count);
-
- /**
- * Records have been read.
- * @param count the number of records read
- */
- void recordsRead(int count);
-
- /**
- * records have been written (including tombstones).
- * @param count number of records written.
- */
- void recordsWritten(int count);
-
- /**
- * A directory has been tagged as authoritative.
- */
- void directoryMarkedAuthoritative();
-
- /**
- * An entry was added.
- * @param durationNanos time to add
- */
- void entryAdded(long durationNanos);
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetastoreInstrumentationImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetastoreInstrumentationImpl.java
deleted file mode 100644
index 7884d8e830f..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetastoreInstrumentationImpl.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-/**
- * A no-op implementation of {@link MetastoreInstrumentation}
- * which allows metastores to always return an instance
- * when requested.
- */
-public class MetastoreInstrumentationImpl implements MetastoreInstrumentation {
-
- @Override
- public void initialized() {
-
- }
-
- @Override
- public void storeClosed() {
-
- }
-
- @Override
- public void throttled() {
-
- }
-
- @Override
- public void retrying() {
-
- }
-
- @Override
- public void recordsDeleted(final int count) {
-
- }
-
- @Override
- public void recordsRead(final int count) {
-
- }
-
- @Override
- public void recordsWritten(final int count) {
-
- }
-
- @Override
- public void directoryMarkedAuthoritative() {
-
- }
-
- @Override
- public void entryAdded(final long durationNanos) {
-
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
deleted file mode 100644
index 722f42176ef..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import javax.annotation.Nullable;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
-import org.apache.hadoop.fs.s3a.impl.StoreContext;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A no-op implementation of MetadataStore. Clients that use this
- * implementation should behave the same as they would without any
- * MetadataStore.
- */
-public class NullMetadataStore implements MetadataStore {
-
- @Override
- public void initialize(FileSystem fs, ITtlTimeProvider ttlTimeProvider)
- throws IOException {
- }
-
- @Override
- public void initialize(Configuration conf, ITtlTimeProvider ttlTimeProvider)
- throws IOException {
- }
-
- @Override
- public void close() throws IOException {
- }
-
- @Override
- public void delete(Path path,
- final BulkOperationState operationState)
- throws IOException {
- }
-
- @Override
- public void forgetMetadata(Path path) throws IOException {
- }
-
- @Override
- public void deleteSubtree(Path path,
- final BulkOperationState operationState)
- throws IOException {
- }
-
- @Override
- public void deletePaths(final Collection paths,
- @Nullable final BulkOperationState operationState) throws IOException {
-
- }
-
- @Override
- public PathMetadata get(Path path) throws IOException {
- return null;
- }
-
- @Override
- public PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
- throws IOException {
- return null;
- }
-
- @Override
- public DirListingMetadata listChildren(Path path) throws IOException {
- return null;
- }
-
- @Override
- public void move(Collection pathsToDelete,
- Collection pathsToCreate,
- final BulkOperationState operationState) throws IOException {
- }
-
- @Override
- public void put(final PathMetadata meta) throws IOException {
- }
-
- @Override
- public void put(PathMetadata meta,
- final BulkOperationState operationState) throws IOException {
- }
-
- @Override
- public void put(Collection extends PathMetadata> meta,
- final BulkOperationState operationState) throws IOException {
- }
-
- @Override
- public void put(DirListingMetadata meta,
- final List unchangedEntries,
- final BulkOperationState operationState) throws IOException {
- }
-
- @Override
- public void destroy() throws IOException {
- }
-
- @Override
- public void prune(PruneMode pruneMode, long cutoff) {
- }
-
- @Override
- public long prune(PruneMode pruneMode, long cutoff, String keyPrefix) {
- return 0;
- }
-
- @Override
- public String toString() {
- return "NullMetadataStore";
- }
-
- @Override
- public Map getDiagnostics() throws IOException {
- Map map = new HashMap<>();
- map.put("name", "Null Metadata Store");
- map.put("description", "This is not a real metadata store");
- return map;
- }
-
- @Override
- public void updateParameters(Map parameters)
- throws IOException {
- }
-
- @Override
- public RenameTracker initiateRenameOperation(final StoreContext storeContext,
- final Path source,
- final S3AFileStatus sourceStatus,
- final Path dest)
- throws IOException {
- return new NullRenameTracker(storeContext, source, dest, this);
- }
-
- @Override
- public void setTtlTimeProvider(ITtlTimeProvider ttlTimeProvider) {
- }
-
- @Override
- public void addAncestors(final Path qualifiedPath,
- @Nullable final BulkOperationState operationState) throws IOException {
- }
-
- private static final class NullRenameTracker extends RenameTracker {
-
- private NullRenameTracker(
- final StoreContext storeContext,
- final Path source,
- final Path dest,
- MetadataStore metadataStore) {
- super("NullRenameTracker", storeContext, metadataStore, source, dest,
- null);
- }
-
- @Override
- public void fileCopied(final Path childSource,
- final S3ObjectAttributes sourceAttributes,
- final S3ObjectAttributes destAttributes,
- final Path destPath,
- final long blockSize,
- final boolean addAncestors) throws IOException {
-
- }
-
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java
deleted file mode 100644
index 09297056f65..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-import org.apache.hadoop.fs.s3a.Tristate;
-
-/**
- * {@code PathMetadata} models path metadata stored in the
- * {@link MetadataStore}. The lastUpdated field is implicitly set to 0 in the
- * constructors without that parameter to show that it will be initialized
- * with 0 if not set otherwise.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class PathMetadata extends ExpirableMetadata {
-
- private S3AFileStatus fileStatus;
- private Tristate isEmptyDirectory;
- private boolean isDeleted;
-
- /**
- * Create a tombstone from the current time.
- * It is mandatory to set the lastUpdated field to update when the
- * tombstone state has changed to set when the entry got deleted.
- *
- * @param path path to tombstone
- * @param lastUpdated last updated time on which expiration is based.
- * @return the entry.
- */
- public static PathMetadata tombstone(Path path, long lastUpdated) {
- S3AFileStatus s3aStatus = new S3AFileStatus(0,
- System.currentTimeMillis(), path, 0, null,
- null, null);
- return new PathMetadata(s3aStatus, Tristate.UNKNOWN, true, lastUpdated);
- }
-
- /**
- * Creates a new {@code PathMetadata} containing given {@code FileStatus}.
- * lastUpdated field will be updated to 0 implicitly in this constructor.
- *
- * @param fileStatus file status containing an absolute path.
- */
- public PathMetadata(S3AFileStatus fileStatus) {
- this(fileStatus, Tristate.UNKNOWN, false, 0);
- }
-
- /**
- * Creates a new {@code PathMetadata} containing given {@code FileStatus}.
- *
- * @param fileStatus file status containing an absolute path.
- * @param lastUpdated last updated time on which expiration is based.
- */
- public PathMetadata(S3AFileStatus fileStatus, long lastUpdated) {
- this(fileStatus, Tristate.UNKNOWN, false, lastUpdated);
- }
-
- /**
- * Creates a new {@code PathMetadata}.
- * lastUpdated field will be updated to 0 implicitly in this constructor.
- *
- * @param fileStatus file status containing an absolute path.
- * @param isEmptyDir empty directory {@link Tristate}
- */
- public PathMetadata(S3AFileStatus fileStatus, Tristate isEmptyDir) {
- this(fileStatus, isEmptyDir, false, 0);
- }
-
- /**
- * Creates a new {@code PathMetadata}.
- * lastUpdated field will be updated to 0 implicitly in this constructor.
- *
- * @param fileStatus file status containing an absolute path.
- * @param isEmptyDir empty directory {@link Tristate}
- * @param isDeleted deleted / tombstoned flag
- */
- public PathMetadata(S3AFileStatus fileStatus, Tristate isEmptyDir,
- boolean isDeleted) {
- this(fileStatus, isEmptyDir, isDeleted, 0);
- }
-
- /**
- * Creates a new {@code PathMetadata}.
- *
- * @param fileStatus file status containing an absolute path.
- * @param isEmptyDir empty directory {@link Tristate}
- * @param isDeleted deleted / tombstoned flag
- * @param lastUpdated last updated time on which expiration is based.
- */
- public PathMetadata(S3AFileStatus fileStatus, Tristate isEmptyDir, boolean
- isDeleted, long lastUpdated) {
- Preconditions.checkNotNull(fileStatus, "fileStatus must be non-null");
- Preconditions.checkNotNull(fileStatus.getPath(), "fileStatus path must be" +
- " non-null");
- Preconditions.checkArgument(fileStatus.getPath().isAbsolute(), "path must" +
- " be absolute");
- Preconditions.checkArgument(lastUpdated >=0, "lastUpdated parameter must "
- + "be greater or equal to 0.");
- this.fileStatus = fileStatus;
- this.isEmptyDirectory = isEmptyDir;
- this.isDeleted = isDeleted;
- this.setLastUpdated(lastUpdated);
- }
-
- /**
- * @return {@code FileStatus} contained in this {@code PathMetadata}.
- */
- public final S3AFileStatus getFileStatus() {
- return fileStatus;
- }
-
- /**
- * Query if a directory is empty.
- * @return Tristate.TRUE if this is known to be an empty directory,
- * Tristate.FALSE if known to not be empty, and Tristate.UNKNOWN if the
- * MetadataStore does have enough information to determine either way.
- */
- public Tristate isEmptyDirectory() {
- return isEmptyDirectory;
- }
-
- void setIsEmptyDirectory(Tristate isEmptyDirectory) {
- this.isEmptyDirectory = isEmptyDirectory;
- fileStatus.setIsEmptyDirectory(isEmptyDirectory);
- }
-
- public boolean isDeleted() {
- return isDeleted;
- }
-
- void setIsDeleted(boolean isDeleted) {
- this.isDeleted = isDeleted;
- }
-
- @Override
- public boolean equals(Object o) {
- if (!(o instanceof PathMetadata)) {
- return false;
- }
- return this.fileStatus.equals(((PathMetadata)o).fileStatus);
- }
-
- @Override
- public int hashCode() {
- return fileStatus.hashCode();
- }
-
- @Override
- public String toString() {
- return "PathMetadata{" +
- "fileStatus=" + fileStatus +
- "; isEmptyDirectory=" + isEmptyDirectory +
- "; isDeleted=" + isDeleted +
- "; lastUpdated=" + super.getLastUpdated() +
- '}';
- }
-
- /**
- * Log contents to supplied StringBuilder in a pretty fashion.
- * @param sb target StringBuilder
- */
- public void prettyPrint(StringBuilder sb) {
- sb.append(String.format("%-5s %-20s %-7d %-8s %-6s %-20s %-20s",
- fileStatus.isDirectory() ? "dir" : "file",
- fileStatus.getPath().toString(), fileStatus.getLen(),
- isEmptyDirectory.name(), isDeleted,
- fileStatus.getETag(), fileStatus.getVersionId()));
- sb.append(fileStatus);
- }
-
- public String prettyPrint() {
- StringBuilder sb = new StringBuilder();
- prettyPrint(sb);
- return sb.toString();
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
deleted file mode 100644
index 413ecf1f422..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
+++ /dev/null
@@ -1,425 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import com.amazonaws.services.dynamodbv2.document.Item;
-import com.amazonaws.services.dynamodbv2.document.KeyAttribute;
-import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
-import com.amazonaws.services.dynamodbv2.model.AttributeDefinition;
-import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
-import com.amazonaws.services.dynamodbv2.model.KeyType;
-import com.amazonaws.services.dynamodbv2.model.ScalarAttributeType;
-import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3a.Constants;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-import org.apache.hadoop.fs.s3a.Tristate;
-
-/**
- * Defines methods for translating between domain model objects and their
- * representations in the DynamoDB schema.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-@VisibleForTesting
-public final class PathMetadataDynamoDBTranslation {
-
- /** The HASH key name of each item. */
- @VisibleForTesting
- static final String PARENT = "parent";
- /** The RANGE key name of each item. */
- @VisibleForTesting
- static final String CHILD = "child";
- @VisibleForTesting
- static final String IS_DIR = "is_dir";
- @VisibleForTesting
- static final String MOD_TIME = "mod_time";
- @VisibleForTesting
- static final String FILE_LENGTH = "file_length";
- @VisibleForTesting
- static final String BLOCK_SIZE = "block_size";
- static final String IS_DELETED = "is_deleted";
- static final String IS_AUTHORITATIVE = "is_authoritative";
- static final String LAST_UPDATED = "last_updated";
- static final String ETAG = "etag";
- static final String VERSION_ID = "version_id";
-
- /** Used while testing backward compatibility. */
- @VisibleForTesting
- static final Set IGNORED_FIELDS = new HashSet<>();
-
- /** Table version field {@value} in version marker item. */
- @VisibleForTesting
- static final String TABLE_VERSION = "table_version";
-
- /** Table creation timestampfield {@value} in version marker item. */
- @VisibleForTesting
- static final String TABLE_CREATED = "table_created";
-
- /** The version marker field is invalid. */
- static final String E_NOT_VERSION_MARKER = "Not a version marker: ";
-
- /**
- * Returns the key schema for the DynamoDB table.
- *
- * @return DynamoDB key schema
- */
- static Collection keySchema() {
- return Arrays.asList(
- new KeySchemaElement(PARENT, KeyType.HASH),
- new KeySchemaElement(CHILD, KeyType.RANGE));
- }
-
- /**
- * Returns the attribute definitions for the DynamoDB table.
- *
- * @return DynamoDB attribute definitions
- */
- static Collection attributeDefinitions() {
- return Arrays.asList(
- new AttributeDefinition(PARENT, ScalarAttributeType.S),
- new AttributeDefinition(CHILD, ScalarAttributeType.S));
- }
-
- /**
- * Converts a DynamoDB item to a {@link DDBPathMetadata}.
- *
- * @param item DynamoDB item to convert
- * @return {@code item} converted to a {@link DDBPathMetadata}
- */
- static DDBPathMetadata itemToPathMetadata(Item item, String username) {
- if (item == null) {
- return null;
- }
-
- String parentStr = item.getString(PARENT);
- Preconditions.checkNotNull(parentStr, "No parent entry in item %s", item);
- String childStr = item.getString(CHILD);
- Preconditions.checkNotNull(childStr, "No child entry in item %s", item);
-
- // Skip table version markers, which are only non-absolute paths stored.
- Path rawPath = new Path(parentStr, childStr);
- if (!rawPath.isAbsoluteAndSchemeAuthorityNull()) {
- return null;
- }
-
- Path parent = new Path(Constants.FS_S3A + ":/" + parentStr + "/");
- Path path = new Path(parent, childStr);
-
- boolean isDir = item.hasAttribute(IS_DIR) && item.getBoolean(IS_DIR);
- boolean isAuthoritativeDir = false;
- final S3AFileStatus fileStatus;
- long lastUpdated = 0;
- if (isDir) {
- isAuthoritativeDir = !IGNORED_FIELDS.contains(IS_AUTHORITATIVE)
- && item.hasAttribute(IS_AUTHORITATIVE)
- && item.getBoolean(IS_AUTHORITATIVE);
- fileStatus = DynamoDBMetadataStore.makeDirStatus(path, username);
- } else {
- long len = item.hasAttribute(FILE_LENGTH) ? item.getLong(FILE_LENGTH) : 0;
- long modTime = item.hasAttribute(MOD_TIME) ? item.getLong(MOD_TIME) : 0;
- long block = item.hasAttribute(BLOCK_SIZE) ? item.getLong(BLOCK_SIZE) : 0;
- String eTag = item.getString(ETAG);
- String versionId = item.getString(VERSION_ID);
- fileStatus = new S3AFileStatus(
- len, modTime, path, block, username, eTag, versionId);
- }
- lastUpdated =
- !IGNORED_FIELDS.contains(LAST_UPDATED)
- && item.hasAttribute(LAST_UPDATED)
- ? item.getLong(LAST_UPDATED) : 0;
-
- boolean isDeleted =
- item.hasAttribute(IS_DELETED) && item.getBoolean(IS_DELETED);
-
- return new DDBPathMetadata(fileStatus, Tristate.UNKNOWN, isDeleted,
- isAuthoritativeDir, lastUpdated);
- }
-
- /**
- * Converts a {@link DDBPathMetadata} to a DynamoDB item.
- *
- * Can ignore {@code IS_AUTHORITATIVE} flag if {@code ignoreIsAuthFlag} is
- * true.
- *
- * @param meta {@link DDBPathMetadata} to convert
- * @return {@code meta} converted to DynamoDB item
- */
- static Item pathMetadataToItem(DDBPathMetadata meta) {
- Preconditions.checkNotNull(meta);
- final S3AFileStatus status = meta.getFileStatus();
- final Item item = new Item().withPrimaryKey(pathToKey(status.getPath()));
- if (status.isDirectory()) {
- item.withBoolean(IS_DIR, true);
- if (!IGNORED_FIELDS.contains(IS_AUTHORITATIVE)) {
- item.withBoolean(IS_AUTHORITATIVE, meta.isAuthoritativeDir());
- }
- } else {
- item.withLong(FILE_LENGTH, status.getLen())
- .withLong(MOD_TIME, status.getModificationTime())
- .withLong(BLOCK_SIZE, status.getBlockSize());
- if (status.getETag() != null) {
- item.withString(ETAG, status.getETag());
- }
- if (status.getVersionId() != null) {
- item.withString(VERSION_ID, status.getVersionId());
- }
- }
- item.withBoolean(IS_DELETED, meta.isDeleted());
-
- if(!IGNORED_FIELDS.contains(LAST_UPDATED)) {
- item.withLong(LAST_UPDATED, meta.getLastUpdated());
- }
-
- return item;
- }
-
- /**
- * The version marker has a primary key whose PARENT is {@code name};
- * this MUST NOT be a value which represents an absolute path.
- * @param name name of the version marker
- * @param version version number
- * @param timestamp creation timestamp
- * @return an item representing a version marker.
- */
- static Item createVersionMarker(String name, int version, long timestamp) {
- return new Item().withPrimaryKey(createVersionMarkerPrimaryKey(name))
- .withInt(TABLE_VERSION, version)
- .withLong(TABLE_CREATED, timestamp);
- }
-
- /**
- * Create the primary key of the version marker.
- * @param name key name
- * @return the key to use when registering or resolving version markers
- */
- static PrimaryKey createVersionMarkerPrimaryKey(String name) {
- return new PrimaryKey(PARENT, name, CHILD, name);
- }
-
- /**
- * Extract the version from a version marker item.
- * @param marker version marker item
- * @return the extracted version field
- * @throws IOException if the item is not a version marker
- */
- static int extractVersionFromMarker(Item marker) throws IOException {
- if (marker.hasAttribute(TABLE_VERSION)) {
- return marker.getInt(TABLE_VERSION);
- } else {
- throw new IOException(E_NOT_VERSION_MARKER + marker);
- }
- }
-
- /**
- * Extract the creation time, if present.
- * @param marker version marker item
- * @return the creation time, or null
- * @throws IOException if the item is not a version marker
- */
- static Long extractCreationTimeFromMarker(Item marker) {
- if (marker.hasAttribute(TABLE_CREATED)) {
- return marker.getLong(TABLE_CREATED);
- } else {
- return null;
- }
- }
-
- /**
- * Converts a collection {@link DDBPathMetadata} to a collection DynamoDB
- * items.
- *
- * @see #pathMetadataToItem(DDBPathMetadata)
- */
- static Item[] pathMetadataToItem(Collection metas) {
- if (metas == null) {
- return null;
- }
-
- final Item[] items = new Item[metas.size()];
- int i = 0;
- for (DDBPathMetadata meta : metas) {
- items[i++] = pathMetadataToItem(meta);
- }
- return items;
- }
-
- /**
- * Converts a {@link Path} to a DynamoDB equality condition on that path as
- * parent, suitable for querying all direct children of the path.
- *
- * @param path the path; can not be null
- * @return DynamoDB equality condition on {@code path} as parent
- */
- static KeyAttribute pathToParentKeyAttribute(Path path) {
- return new KeyAttribute(PARENT, pathToParentKey(path));
- }
-
- /**
- * e.g. {@code pathToParentKey(s3a://bucket/path/a) -> /bucket/path/a}
- * @param path path to convert
- * @return string for parent key
- */
- @VisibleForTesting
- public static String pathToParentKey(Path path) {
- Preconditions.checkNotNull(path);
- Preconditions.checkArgument(path.isUriPathAbsolute(),
- "Path not absolute: '%s'", path);
- URI uri = path.toUri();
- String bucket = uri.getHost();
- Preconditions.checkArgument(!StringUtils.isEmpty(bucket),
- "Path missing bucket %s", path);
- String pKey = "/" + bucket + uri.getPath();
-
- // Strip trailing slash
- if (pKey.endsWith("/")) {
- pKey = pKey.substring(0, pKey.length() - 1);
- }
- return pKey;
- }
-
- /**
- * Converts a {@link Path} to a DynamoDB key, suitable for getting the item
- * matching the path.
- *
- * @param path the path; can not be null
- * @return DynamoDB key for item matching {@code path}
- */
- static PrimaryKey pathToKey(Path path) {
- Preconditions.checkArgument(!path.isRoot(),
- "Root path is not mapped to any PrimaryKey");
- String childName = path.getName();
- PrimaryKey key = new PrimaryKey(PARENT,
- pathToParentKey(path.getParent()), CHILD,
- childName);
- for (KeyAttribute attr : key.getComponents()) {
- String name = attr.getName();
- Object v = attr.getValue();
- Preconditions.checkNotNull(v,
- "Null value for DynamoDB attribute \"%s\"", name);
- Preconditions.checkState(!((String)v).isEmpty(),
- "Empty string value for DynamoDB attribute \"%s\"", name);
- }
- return key;
-
- }
-
- /**
- * Converts a collection of {@link Path} to a collection of DynamoDB keys.
- *
- * @see #pathToKey(Path)
- */
- static PrimaryKey[] pathToKey(Collection paths) {
- if (paths == null) {
- return null;
- }
-
- final PrimaryKey[] keys = new PrimaryKey[paths.size()];
- int i = 0;
- for (Path p : paths) {
- keys[i++] = pathToKey(p);
- }
- return keys;
- }
-
- /**
- * There is no need to instantiate this class.
- */
- private PathMetadataDynamoDBTranslation() {
- }
-
- /**
- * Convert a collection of metadata entries to a list
- * of DDBPathMetadata entries.
- * If the sources are already DDBPathMetadata instances, they
- * are copied directly into the new list, otherwise new
- * instances are created.
- * @param pathMetadatas source data
- * @return the converted list.
- */
- static List