Browse Source

HDDS-1482. Use strongly typed codec implementations for the S3Table

Closes #789
Bharat Viswanadham 6 năm trước cách đây
mục cha
commit
4605db369e

+ 3 - 2
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java

@@ -25,6 +25,7 @@ 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.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
 import org.apache.hadoop.utils.db.DBStore;
@@ -259,7 +260,7 @@ public interface OMMetadataManager {
    * @return Table.
    */
 
-  Table<byte[], byte[]> getS3Table();
+  Table<String, String> getS3Table();
 
   /**
    * Returns the DB key name of a multipart upload key in OM metadata store.
@@ -285,7 +286,7 @@ public interface OMMetadataManager {
    * Gets the S3 Secrets table.
    * @return Table
    */
-  Table<byte[], byte[]> getS3SecretTable();
+  Table<String, S3SecretValue> getS3SecretTable();
 
   /**
    * Returns number of rows in a table.  This should not be used for very

+ 7 - 14
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/S3SecretManagerImpl.java

@@ -23,7 +23,6 @@ import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.security.OzoneSecurityException;
 import org.apache.logging.log4j.util.Strings;
 import org.slf4j.Logger;
@@ -31,7 +30,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.hadoop.ozone.security.OzoneSecurityException.ResultCodes.S3_SECRET_NOT_FOUND;
 
 /**
@@ -61,20 +59,17 @@ public class S3SecretManagerImpl implements S3SecretManager {
   public S3SecretValue getS3Secret(String kerberosID) throws IOException {
     Preconditions.checkArgument(Strings.isNotBlank(kerberosID),
         "kerberosID cannot be null or empty.");
-    byte[] awsAccessKey = kerberosID.getBytes(UTF_8);
     S3SecretValue result = null;
     omMetadataManager.getLock().acquireS3SecretLock(kerberosID);
     try {
-      byte[] s3Secret =
-          omMetadataManager.getS3SecretTable().get(awsAccessKey);
+      S3SecretValue s3Secret =
+          omMetadataManager.getS3SecretTable().get(kerberosID);
       if(s3Secret == null) {
         byte[] secret = OmUtils.getSHADigest();
         result = new S3SecretValue(kerberosID, DigestUtils.sha256Hex(secret));
-        omMetadataManager.getS3SecretTable()
-            .put(awsAccessKey, result.getProtobuf().toByteArray());
+        omMetadataManager.getS3SecretTable().put(kerberosID, result);
       } else {
-        result = S3SecretValue.fromProtobuf(
-            OzoneManagerProtocolProtos.S3Secret.parseFrom(s3Secret));
+        return s3Secret;
       }
     } finally {
       omMetadataManager.getLock().releaseS3SecretLock(kerberosID);
@@ -90,11 +85,10 @@ public class S3SecretManagerImpl implements S3SecretManager {
         "awsAccessKeyId cannot be null or empty.");
     LOG.trace("Get secret for awsAccessKey:{}", kerberosID);
 
-    byte[] s3Secret;
+    S3SecretValue s3Secret;
     omMetadataManager.getLock().acquireS3SecretLock(kerberosID);
     try {
-      s3Secret = omMetadataManager.getS3SecretTable()
-          .get(kerberosID.getBytes(UTF_8));
+      s3Secret = omMetadataManager.getS3SecretTable().get(kerberosID);
       if (s3Secret == null) {
         throw new OzoneSecurityException("S3 secret not found for " +
             "awsAccessKeyId " + kerberosID, S3_SECRET_NOT_FOUND);
@@ -103,8 +97,7 @@ public class S3SecretManagerImpl implements S3SecretManager {
       omMetadataManager.getLock().releaseS3SecretLock(kerberosID);
     }
 
-    return OzoneManagerProtocolProtos.S3Secret.parseFrom(s3Secret)
-        .getAwsSecret();
+    return s3Secret.getAwsSecret();
   }
 
   public OMMetadataManager getOmMetadataManager() {

+ 57 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/S3SecretValueCodec.java

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

+ 20 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/S3SecretValue.java

@@ -19,6 +19,8 @@ package org.apache.hadoop.ozone.om.helpers;
 
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 
+import java.util.Objects;
+
 /**
  * S3Secret to be saved in database.
  */
@@ -67,4 +69,22 @@ public class S3SecretValue {
   public String toString() {
     return "awsAccessKey=" + kerberosID + "\nawsSecret=" + awsSecret;
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    S3SecretValue that = (S3SecretValue) o;
+    return kerberosID.equals(that.kerberosID) &&
+        awsSecret.equals(that.awsSecret);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(kerberosID, awsSecret);
+  }
 }

+ 88 - 0
hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/codec/TestS3SecretValueCodec.java

@@ -0,0 +1,88 @@
+/**
+ * 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 java.nio.charset.StandardCharsets;
+import java.util.UUID;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.test.GenericTestUtils;
+
+import static org.junit.Assert.fail;
+
+/**
+ * This class test S3SecretValueCodec.
+ */
+public class TestS3SecretValueCodec {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private S3SecretValueCodec codec;
+
+  @Before
+  public void initialize() {
+    codec = new S3SecretValueCodec();
+  }
+  @Test
+  public void testCodecWithCorrectData() throws Exception {
+
+    S3SecretValue s3SecretValue =
+        new S3SecretValue(UUID.randomUUID().toString(),
+            UUID.randomUUID().toString());
+
+    byte[] data = codec.toPersistedFormat(s3SecretValue);
+    Assert.assertNotNull(data);
+
+    S3SecretValue docdedS3Secret = codec.fromPersistedFormat(data);
+
+    Assert.assertEquals(s3SecretValue, docdedS3Secret);
+
+  }
+
+  @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);
+  }
+}

+ 9 - 5
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.S3SecretValueCodec;
 import org.apache.hadoop.ozone.om.codec.TokenIdentifierCodec;
 import org.apache.hadoop.ozone.om.codec.VolumeListCodec;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
@@ -41,6 +42,7 @@ 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.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
 import org.apache.hadoop.utils.db.DBStore;
@@ -176,7 +178,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   }
 
   @Override
-  public Table<byte[], byte[]> getS3Table() {
+  public Table<String, String> getS3Table() {
     return s3Table;
   }
 
@@ -232,7 +234,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         .addCodec(OmBucketInfo.class, new OmBucketInfoCodec())
         .addCodec(OmVolumeArgs.class, new OmVolumeArgsCodec())
         .addCodec(VolumeList.class, new VolumeListCodec())
-        .addCodec(OmMultipartKeyInfo.class, new OmMultipartKeyInfoCodec());
+        .addCodec(OmMultipartKeyInfo.class, new OmMultipartKeyInfoCodec())
+        .addCodec(S3SecretValue.class, new S3SecretValueCodec());
   }
 
   /**
@@ -265,7 +268,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         this.store.getTable(OPEN_KEY_TABLE, String.class, OmKeyInfo.class);
     checkTableStatus(openKeyTable, OPEN_KEY_TABLE);
 
-    s3Table = this.store.getTable(S3_TABLE);
+    s3Table = this.store.getTable(S3_TABLE, String.class, String.class);
     checkTableStatus(s3Table, S3_TABLE);
 
     multipartInfoTable = this.store.getTable(MULTIPARTINFO_TABLE,
@@ -276,7 +279,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         OzoneTokenIdentifier.class, Long.class);
     checkTableStatus(dTokenTable, DELEGATION_TOKEN_TABLE);
 
-    s3SecretTable = this.store.getTable(S3_SECRET_TABLE);
+    s3SecretTable = this.store.getTable(S3_SECRET_TABLE, String.class,
+        S3SecretValue.class);
     checkTableStatus(s3SecretTable, S3_SECRET_TABLE);
   }
 
@@ -701,7 +705,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   }
 
   @Override
-  public Table<byte[], byte[]> getS3SecretTable() {
+  public Table<String, S3SecretValue> getS3SecretTable() {
     return s3SecretTable;
   }
 

+ 7 - 14
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManagerImpl.java

@@ -32,7 +32,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.nio.charset.StandardCharsets;
 import java.util.Objects;
 
 import static org.apache.hadoop.ozone.OzoneConsts.OM_S3_VOLUME_PREFIX;
@@ -104,9 +103,8 @@ public class S3BucketManagerImpl implements S3BucketManager {
 
     omMetadataManager.getLock().acquireS3Lock(bucketName);
     try {
-      byte[] bucket =
-          omMetadataManager.getS3Table().get(
-              bucketName.getBytes(StandardCharsets.UTF_8));
+      String bucket =
+          omMetadataManager.getS3Table().get(bucketName);
 
       if (bucket != null) {
         LOG.debug("Bucket already exists. {}", bucketName);
@@ -119,9 +117,7 @@ public class S3BucketManagerImpl implements S3BucketManager {
       String finalName = String.format("%s/%s", ozoneVolumeName,
           ozoneBucketName);
 
-      omMetadataManager.getS3Table().put(
-              bucketName.getBytes(StandardCharsets.UTF_8),
-              finalName.getBytes(StandardCharsets.UTF_8));
+      omMetadataManager.getS3Table().put(bucketName, finalName);
     } finally {
       omMetadataManager.getLock().releaseS3Lock(bucketName);
     }
@@ -134,8 +130,7 @@ public class S3BucketManagerImpl implements S3BucketManager {
 
     omMetadataManager.getLock().acquireS3Lock(bucketName);
     try {
-      byte[] bucket = bucketName.getBytes(StandardCharsets.UTF_8);
-      byte[] map = omMetadataManager.getS3Table().get(bucket);
+      String map = omMetadataManager.getS3Table().get(bucketName);
 
       if (map == null) {
         throw new OMException("No such S3 bucket. " + bucketName,
@@ -149,7 +144,7 @@ public class S3BucketManagerImpl implements S3BucketManager {
       } else {
         bucketManager.deleteBucket(getOzoneVolumeName(bucketName), bucketName);
       }
-      omMetadataManager.getS3Table().delete(bucket);
+      omMetadataManager.getS3Table().delete(bucketName);
     } catch(IOException ex) {
       throw ex;
     } finally {
@@ -224,11 +219,9 @@ public class S3BucketManagerImpl implements S3BucketManager {
         "Length of the S3 Bucket is not correct.");
     omMetadataManager.getLock().acquireS3Lock(s3BucketName);
     try {
-      byte[] mapping =
-          omMetadataManager.getS3Table().get(
-              s3BucketName.getBytes(StandardCharsets.UTF_8));
+      String mapping = omMetadataManager.getS3Table().get(s3BucketName);
       if (mapping != null) {
-        return new String(mapping, StandardCharsets.UTF_8);
+        return mapping;
       }
       throw new OMException("No such S3 bucket.",
           OMException.ResultCodes.S3_BUCKET_NOT_FOUND);