瀏覽代碼

HADOOP-14756 S3Guard: expose capability query in MetadataStore and add tests of authoritative mode (Gabor Bota)

Aaron Fabbri 7 年之前
父節點
當前提交
989a3929a9

+ 2 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java

@@ -1154,6 +1154,8 @@ public class DynamoDBMetadataStore implements MetadataStore {
       map.put(READ_CAPACITY, throughput.getReadCapacityUnits().toString());
       map.put(WRITE_CAPACITY, throughput.getWriteCapacityUnits().toString());
       map.put(TABLE, desc.toString());
+      map.put(MetadataStoreCapabilities.PERSISTS_AUTHORITATIVE_BIT,
+          Boolean.toString(false));
     } else {
       map.put("name", "DynamoDB Metadata Store");
       map.put(TABLE, "none");

+ 2 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java

@@ -447,6 +447,8 @@ public class LocalMetadataStore implements MetadataStore {
     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;
   }
 

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java

@@ -238,7 +238,7 @@ public interface MetadataStore extends Closeable {
   /**
    * 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 only.
+   * These are for debugging and testing only.
    * @return a map of strings.
    * @throws IOException if there is an error
    */

+ 43 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreCapabilities.java

@@ -0,0 +1,43 @@
+/**
+ * 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";
+}

+ 30 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java

@@ -24,6 +24,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.Map;
 
 import com.google.common.collect.Sets;
 import org.junit.After;
@@ -503,7 +504,6 @@ public abstract class MetadataStoreTestBase extends Assert {
       assertListingsEqual(dirMeta.getListing(), "/a1/b1", "/a1/b2");
     }
 
-    // TODO HADOOP-14756 instrument MetadataStore for asserting & testing
     dirMeta = ms.listChildren(strToPath("/a1/b1"));
     if (!allowMissing() || dirMeta != null) {
       assertListingsEqual(dirMeta.getListing(), "/a1/b1/file1", "/a1/b1/file2",
@@ -511,6 +511,35 @@ public abstract class MetadataStoreTestBase extends Assert {
     }
   }
 
+  private boolean isMetadataStoreAuthoritative() throws IOException {
+    Map<String, String> diags = ms.getDiagnostics();
+    String isAuth =
+        diags.get(MetadataStoreCapabilities.PERSISTS_AUTHORITATIVE_BIT);
+    if(isAuth == null){
+      return false;
+    }
+    return Boolean.valueOf(isAuth);
+  }
+
+  @Test
+  public void testListChildrenAuthoritative() throws IOException {
+    Assume.assumeTrue("MetadataStore should be capable for authoritative "
+        + "storage of directories to run this test.",
+        isMetadataStoreAuthoritative());
+
+    setupListStatus();
+
+    DirListingMetadata dirMeta = ms.listChildren(strToPath("/a1/b1"));
+    dirMeta.setAuthoritative(true);
+    dirMeta.put(makeFileStatus("/a1/b1/file_new", 100));
+    ms.put(dirMeta);
+
+    dirMeta = ms.listChildren(strToPath("/a1/b1"));
+    assertListingsEqual(dirMeta.getListing(), "/a1/b1/file1", "/a1/b1/file2",
+        "/a1/b1/c1", "/a1/b1/file_new");
+    assertTrue(dirMeta.isAuthoritative());
+  }
+
   @Test
   public void testDirListingRoot() throws Exception {
     commonTestPutListStatus("/");