瀏覽代碼

HDDS-1608. Support Ozone Prefix ACLs in OM metadata table. Contributed by Xiaoyu Yao. (#875)

Xiaoyu Yao 6 年之前
父節點
當前提交
219e286722

+ 7 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmPrefixInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
@@ -262,6 +263,12 @@ public interface OMMetadataManager {
 
   Table<String, String> getS3Table();
 
+  /**
+   * Gets the Ozone prefix path to its acl mapping table.
+   * @return Table.
+   */
+  Table<String, OmPrefixInfo> getPrefixTable();
+
   /**
    * Returns the DB key name of a multipart upload key in OM metadata store.
    *

+ 53 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/OmPrefixInfoCodec.java

@@ -0,0 +1,53 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.om.codec;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.ozone.om.helpers.OmPrefixInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PrefixInfo;
+
+import org.apache.hadoop.utils.db.Codec;
+
+import java.io.IOException;
+
+/**
+ * Codec to encode PrefixAcl as byte array.
+ */
+public class OmPrefixInfoCodec implements Codec<OmPrefixInfo> {
+
+  @Override
+  public byte[] toPersistedFormat(OmPrefixInfo object) throws IOException {
+    Preconditions
+        .checkNotNull(object, "Null object can't be converted to byte array.");
+    return object.getProtobuf().toByteArray();
+  }
+
+  @Override
+  public OmPrefixInfo fromPersistedFormat(byte[] rawData) throws IOException {
+    Preconditions
+        .checkNotNull(rawData,
+            "Null byte array can't converted to real object.");
+    try {
+      return OmPrefixInfo.getFromProtobuf(PrefixInfo.parseFrom(rawData));
+    } catch (InvalidProtocolBufferException e) {
+      throw new IllegalArgumentException(
+          "Can't encode the the raw data from the byte array", e);
+    }
+  }
+}

+ 170 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmPrefixInfo.java

@@ -0,0 +1,170 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.om.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PrefixInfo;
+import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
+
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Wrapper class for Ozone prefix path info, currently mainly target for ACL but
+ * can be extended for other OzFS optimizations in future.
+ */
+// TODO: support Auditable interface
+public final class OmPrefixInfo extends WithMetadata {
+
+  private String name;
+  private List<OzoneAcl> acls;
+
+  public OmPrefixInfo(String name, List<OzoneAcl> acls,
+      Map<String, String> metadata) {
+    this.name = name;
+    this.acls = acls;
+    this.metadata = metadata;
+  }
+
+  /**
+   * Returns the ACL's associated with this prefix.
+   * @return {@literal List<OzoneAcl>}
+   */
+  public List<OzoneAcl> getAcls() {
+    return acls;
+  }
+
+  /**
+   * Returns the name of the prefix path.
+   * @return name of the prefix path.
+   */
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Returns new builder class that builds a OmPrefixInfo.
+   *
+   * @return Builder
+   */
+  public static OmPrefixInfo.Builder newBuilder() {
+    return new OmPrefixInfo.Builder();
+  }
+
+  /**
+   * Builder for OmPrefixInfo.
+   */
+  public static class Builder {
+    private String name;
+    private List<OzoneAcl> acls;
+    private Map<String, String> metadata;
+
+    public Builder() {
+      //Default values
+      this.acls = new LinkedList<>();
+      this.metadata = new HashMap<>();
+    }
+
+    public Builder setAcls(List<OzoneAcl> listOfAcls) {
+      this.acls = listOfAcls;
+      return this;
+    }
+
+    public Builder setName(String n) {
+      this.name = n;
+      return this;
+    }
+
+    public OmPrefixInfo.Builder addMetadata(String key, String value) {
+      metadata.put(key, value);
+      return this;
+    }
+
+    public OmPrefixInfo.Builder addAllMetadata(
+        Map<String, String> additionalMetadata) {
+      if (additionalMetadata != null) {
+        metadata.putAll(additionalMetadata);
+      }
+      return this;
+    }
+
+    /**
+     * Constructs the OmPrefixInfo.
+     * @return instance of OmPrefixInfo.
+     */
+    public OmPrefixInfo build() {
+      Preconditions.checkNotNull(name);
+      Preconditions.checkNotNull(acls);
+      return new OmPrefixInfo(name, acls, metadata);
+    }
+  }
+
+  /**
+   * Creates PrefixInfo protobuf from OmPrefixInfo.
+   */
+  public PrefixInfo getProtobuf() {
+    PrefixInfo.Builder pib =  PrefixInfo.newBuilder().setName(name)
+        .addAllAcls(acls.stream().map(OMPBHelper::convertOzoneAcl)
+            .collect(Collectors.toList()))
+        .addAllMetadata(KeyValueUtil.toProtobuf(metadata));
+    return pib.build();
+  }
+
+  /**
+   * Parses PrefixInfo protobuf and creates OmPrefixInfo.
+   * @param prefixInfo
+   * @return instance of OmPrefixInfo
+   */
+  public static OmPrefixInfo getFromProtobuf(PrefixInfo prefixInfo) {
+    OmPrefixInfo.Builder opib = OmPrefixInfo.newBuilder()
+        .setName(prefixInfo.getName())
+        .setAcls(prefixInfo.getAclsList().stream().map(
+            OMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
+    if (prefixInfo.getMetadataList() != null) {
+      opib.addAllMetadata(KeyValueUtil
+          .getFromProtobuf(prefixInfo.getMetadataList()));
+    }
+    return opib.build();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    OmPrefixInfo that = (OmPrefixInfo) o;
+    return name.equals(that.name) &&
+        Objects.equals(acls, that.acls) &&
+        Objects.equals(metadata, that.metadata);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(name);
+  }
+}
+

+ 7 - 0
hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto

@@ -459,11 +459,18 @@ message BucketArgs {
     repeated hadoop.hdds.KeyValue metadata = 7;
 }
 
+message PrefixInfo {
+    required string name = 1;
+    repeated OzoneAclInfo acls = 2;
+    repeated hadoop.hdds.KeyValue metadata = 3;
+}
+
 message OzoneObj {
   enum ObjectType {
     VOLUME = 1;
     BUCKET = 2;
     KEY = 3;
+    PREFIX = 4;
   }
 
   enum StoreType {

+ 98 - 0
hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/codec/TestOmPrefixInfoCodec.java

@@ -0,0 +1,98 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.om.codec;
+
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.helpers.OmPrefixInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
+
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.LinkedList;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * This class test OmPrefixInfoCodec.
+ */
+public class TestOmPrefixInfoCodec {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+
+  private OmPrefixInfoCodec codec;
+
+  @Before
+  public void setUp() {
+    codec = new OmPrefixInfoCodec();
+  }
+
+  @Test
+  public void testCodecWithIncorrectValues() throws Exception {
+    try {
+      codec.fromPersistedFormat("random".getBytes(StandardCharsets.UTF_8));
+      fail("testCodecWithIncorrectValues failed");
+    } catch (IllegalArgumentException ex) {
+      GenericTestUtils.assertExceptionContains("Can't encode the the raw " +
+          "data from the byte array", ex);
+    }
+  }
+
+  @Test
+  public void testCodecWithNullDataFromTable() throws Exception {
+    thrown.expect(NullPointerException.class);
+    codec.fromPersistedFormat(null);
+  }
+
+
+  @Test
+  public void testCodecWithNullDataFromUser() throws Exception {
+    thrown.expect(NullPointerException.class);
+    codec.toPersistedFormat(null);
+  }
+
+  @Test
+  public void testToAndFromPersistedFormat() throws IOException {
+
+    List<OzoneAcl> acls = new LinkedList<>();
+    OzoneAcl ozoneAcl = new OzoneAcl(ACLIdentityType.USER,
+        "hive", ACLType.ALL);
+    acls.add(ozoneAcl);
+    OmPrefixInfo opiSave = OmPrefixInfo.newBuilder()
+        .setName("/user/hive/warehouse")
+        .setAcls(acls)
+        .addMetadata("id", "100")
+        .build();
+
+    OmPrefixInfo opiLoad = codec.fromPersistedFormat(
+        codec.toPersistedFormat(opiSave));
+
+    assertTrue("Load saved prefix info should match",
+        opiLoad.equals(opiSave));
+  }
+}

+ 7 - 7
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/TestCSMMetrics.java

@@ -58,13 +58,13 @@ import java.util.function.BiConsumer;
 import org.junit.Test;
 import org.junit.Assert;
 
-  /**
-   * This class tests the metrics of ContainerStateMachine.
-   */
-  public class TestCSMMetrics {
-    static final String TEST_DIR =
-        GenericTestUtils.getTestDir("dfs").getAbsolutePath()
-            + File.separator;
+/**
+ * This class tests the metrics of ContainerStateMachine.
+ */
+public class TestCSMMetrics {
+  static final String TEST_DIR =
+      GenericTestUtils.getTestDir("dfs").getAbsolutePath()
+          + File.separator;
   @FunctionalInterface
   interface CheckedBiFunction<LEFT, RIGHT, OUT, THROWABLE extends Throwable> {
     OUT apply(LEFT left, RIGHT right) throws THROWABLE;

+ 18 - 1
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.ozone.om.codec.OmBucketInfoCodec;
 import org.apache.hadoop.ozone.om.codec.OmKeyInfoCodec;
 import org.apache.hadoop.ozone.om.codec.OmMultipartKeyInfoCodec;
 import org.apache.hadoop.ozone.om.codec.OmVolumeArgsCodec;
+import org.apache.hadoop.ozone.om.codec.OmPrefixInfoCodec;
 import org.apache.hadoop.ozone.om.codec.S3SecretValueCodec;
 import org.apache.hadoop.ozone.om.codec.TokenIdentifierCodec;
 import org.apache.hadoop.ozone.om.codec.VolumeListCodec;
@@ -41,6 +42,7 @@ import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmPrefixInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
@@ -98,6 +100,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
    * |-------------------------------------------------------------------|
    * | dTokenTable        | s3g_access_key_id -> s3Secret                |
    * |-------------------------------------------------------------------|
+   * | prefixInfoTable     | prefix -> PrefixInfo                       |
+   * |-------------------------------------------------------------------|
    */
 
   public static final String USER_TABLE = "userTable";
@@ -110,6 +114,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   public static final String MULTIPARTINFO_TABLE = "multipartInfoTable";
   public static final String S3_SECRET_TABLE = "s3SecretTable";
   public static final String DELEGATION_TOKEN_TABLE = "dTokenTable";
+  public static final String PREFIX_TABLE = "prefixTable";
 
   private DBStore store;
 
@@ -126,6 +131,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   private Table<String, OmMultipartKeyInfo> multipartInfoTable;
   private Table s3SecretTable;
   private Table dTokenTable;
+  private Table prefixTable;
 
   public OmMetadataManagerImpl(OzoneConfiguration conf) throws IOException {
     this.lock = new OzoneManagerLock(conf);
@@ -183,6 +189,11 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
     return s3Table;
   }
 
+  @Override
+  public Table<String, OmPrefixInfo> getPrefixTable() {
+    return prefixTable;
+  }
+
   @Override
   public Table<String, OmMultipartKeyInfo> getMultipartInfoTable() {
     return multipartInfoTable;
@@ -230,13 +241,15 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         .addTable(MULTIPARTINFO_TABLE)
         .addTable(DELEGATION_TOKEN_TABLE)
         .addTable(S3_SECRET_TABLE)
+        .addTable(PREFIX_TABLE)
         .addCodec(OzoneTokenIdentifier.class, new TokenIdentifierCodec())
         .addCodec(OmKeyInfo.class, new OmKeyInfoCodec())
         .addCodec(OmBucketInfo.class, new OmBucketInfoCodec())
         .addCodec(OmVolumeArgs.class, new OmVolumeArgsCodec())
         .addCodec(VolumeList.class, new VolumeListCodec())
         .addCodec(OmMultipartKeyInfo.class, new OmMultipartKeyInfoCodec())
-        .addCodec(S3SecretValue.class, new S3SecretValueCodec());
+        .addCodec(S3SecretValue.class, new S3SecretValueCodec())
+        .addCodec(OmPrefixInfo.class, new OmPrefixInfoCodec());
   }
 
   /**
@@ -282,6 +295,10 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
     s3SecretTable = this.store.getTable(S3_SECRET_TABLE, String.class,
         S3SecretValue.class);
     checkTableStatus(s3SecretTable, S3_SECRET_TABLE);
+
+    prefixTable = this.store.getTable(PREFIX_TABLE, String.class,
+        OmPrefixInfo.class);
+    checkTableStatus(prefixTable, PREFIX_TABLE);
   }
 
   /**