hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From bha...@apache.org
Subject [1/2] hadoop git commit: HDDS-864. Use strongly typed codec implementations for the tables of the OmMetadataManager. Contributed by Elek Marton.
Date Thu, 06 Dec 2018 19:40:29 GMT
Repository: hadoop
Updated Branches:
  refs/heads/trunk c03024a53 -> 343aaea2d


http://git-wip-us.apache.org/repos/asf/hadoop/blob/343aaea2/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
index 59f2940..ff0c9a5 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
@@ -16,15 +16,24 @@
  */
 package org.apache.hadoop.ozone.om;
 
-import org.apache.hadoop.hdds.protocol.StorageType;
+import java.io.File;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.StorageType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.server.ServerUtils;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
 import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -33,11 +42,6 @@ import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.mockito.runners.MockitoJUnitRunner;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-
 /**
  * Tests BucketManagerImpl, mocks OMMetadataManager for testing.
  */
@@ -62,10 +66,16 @@ public class TestBucketManagerImpl {
   private OmMetadataManagerImpl createSampleVol() throws IOException {
     OzoneConfiguration conf = createNewTestPath();
     OmMetadataManagerImpl metaMgr = new OmMetadataManagerImpl(conf);
-    byte[] volumeKey = metaMgr.getVolumeKey("sampleVol");
+    String volumeKey = metaMgr.getVolumeKey("sampleVol");
     // This is a simple hack for testing, we just test if the volume via a
     // null check, do not parse the value part. So just write some dummy value.
-    metaMgr.getVolumeTable().put(volumeKey, volumeKey);
+    OmVolumeArgs args =
+        OmVolumeArgs.newBuilder()
+            .setVolume("sampleVol")
+            .setAdminName("bilbo")
+            .setOwnerName("bilbo")
+            .build();
+    metaMgr.getVolumeTable().put(volumeKey, args);
     return metaMgr;
   }
 
@@ -344,12 +354,22 @@ public class TestBucketManagerImpl {
         .build();
     bucketManager.createBucket(bucketInfo);
     //Create keys in bucket
-    metaMgr.getKeyTable().put(DFSUtil.string2Bytes("/sampleVol/bucketOne" +
-            "/key_one"),
-        DFSUtil.string2Bytes("value_one"));
-    metaMgr.getKeyTable().put(DFSUtil.string2Bytes("/sampleVol/bucketOne" +
-            "/key_two"),
-        DFSUtil.string2Bytes("value_two"));
+    metaMgr.getKeyTable().put("/sampleVol/bucketOne/key_one",
+        new OmKeyInfo.Builder()
+            .setBucketName("bucketOne")
+            .setVolumeName("sampleVol")
+            .setKeyName("key_one")
+            .setReplicationFactor(ReplicationFactor.ONE)
+            .setReplicationType(ReplicationType.STAND_ALONE)
+            .build());
+    metaMgr.getKeyTable().put("/sampleVol/bucketOne/key_two",
+        new OmKeyInfo.Builder()
+            .setBucketName("bucketOne")
+            .setVolumeName("sampleVol")
+            .setKeyName("key_two")
+            .setReplicationFactor(ReplicationFactor.ONE)
+            .setReplicationType(ReplicationType.STAND_ALONE)
+            .build());
     try {
       bucketManager.deleteBucket("sampleVol", "bucketOne");
     } catch (OMException omEx) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343aaea2/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyDeletingService.java
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyDeletingService.java
index d4612af..07f1600 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyDeletingService.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyDeletingService.java
@@ -19,30 +19,30 @@
 
 package org.apache.hadoop.ozone.om;
 
-import org.apache.commons.lang3.RandomStringUtils;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.utils.db.DBConfigFromFile;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static org.apache.hadoop.hdds.HddsConfigKeys
-    .HDDS_CONTAINER_REPORT_INTERVAL;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
-
 /**
  * Test Key Deleting Service.
  * <p>
@@ -166,18 +166,25 @@ public class TestKeyDeletingService {
           RandomStringUtils.randomAlphanumeric(5));
       String keyName = String.format("key%s",
           RandomStringUtils.randomAlphanumeric(5));
-      byte[] volumeBytes =
+      String volumeBytes =
           keyManager.getMetadataManager().getVolumeKey(volumeName);
-      byte[] bucketBytes =
+      String bucketBytes =
           keyManager.getMetadataManager().getBucketKey(volumeName, bucketName);
       // cheat here, just create a volume and bucket entry so that we can
       // create the keys, we put the same data for key and value since the
       // system does not decode the object
       keyManager.getMetadataManager().getVolumeTable().put(volumeBytes,
-          volumeBytes);
+          OmVolumeArgs.newBuilder()
+              .setOwnerName("o")
+              .setAdminName("a")
+              .setVolume(volumeName)
+              .build());
 
       keyManager.getMetadataManager().getBucketTable().put(bucketBytes,
-          bucketBytes);
+          OmBucketInfo.newBuilder()
+              .setVolumeName(volumeName)
+              .setBucketName(bucketName)
+              .build());
 
       OmKeyArgs arg =
           new OmKeyArgs.Builder()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343aaea2/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
index 55843e1..82ecd2e 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.ozone.om;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
-import org.apache.commons.lang3.RandomStringUtils;
+
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@@ -29,14 +29,22 @@ import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
 import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.ozone.om.codec.OmBucketInfoCodec;
+import org.apache.hadoop.ozone.om.codec.OmKeyInfoCodec;
+import org.apache.hadoop.ozone.om.codec.OmVolumeArgsCodec;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.util.Time;
+import org.apache.hadoop.utils.db.CodecRegistry;
 import org.apache.hadoop.utils.db.RDBStore;
 import org.apache.hadoop.utils.db.Table;
 import org.apache.hadoop.utils.db.TableConfig;
+
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -48,8 +56,6 @@ import org.rocksdb.RocksDB;
 import org.rocksdb.Statistics;
 import org.rocksdb.StatsLevel;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-
 /**
  * Test class for @{@link KeyManagerImpl}.
  * */
@@ -64,7 +70,9 @@ public class TestKeyManagerImpl {
   private static final String BUCKET_NAME = "bucket1";
   private static final String VOLUME_NAME = "vol1";
   private static RDBStore rdbStore = null;
-  private static Table rdbTable = null;
+  private static Table<String, OmKeyInfo> keyTable = null;
+  private static Table<String, OmBucketInfo> bucketTable = null;
+  private static Table<String, OmVolumeArgs> volumeTable = null;
   private static DBOptions options = null;
   private KeyInfo keyData;
   @Rule
@@ -88,17 +96,17 @@ public class TestKeyManagerImpl {
             new SCMException("ChillModePrecheck failed for allocateBlock",
                 ResultCodes.CHILL_MODE_EXCEPTION));
     setupRocksDb();
-    Mockito.when(metadataManager.getVolumeTable()).thenReturn(rdbTable);
-    Mockito.when(metadataManager.getBucketTable()).thenReturn(rdbTable);
-    Mockito.when(metadataManager.getOpenKeyTable()).thenReturn(rdbTable);
+    Mockito.when(metadataManager.getVolumeTable()).thenReturn(volumeTable);
+    Mockito.when(metadataManager.getBucketTable()).thenReturn(bucketTable);
+    Mockito.when(metadataManager.getOpenKeyTable()).thenReturn(keyTable);
     Mockito.when(metadataManager.getLock())
         .thenReturn(new OzoneManagerLock(conf));
     Mockito.when(metadataManager.getVolumeKey(VOLUME_NAME))
-        .thenReturn(VOLUME_NAME.getBytes(UTF_8));
+        .thenReturn(VOLUME_NAME);
     Mockito.when(metadataManager.getBucketKey(VOLUME_NAME, BUCKET_NAME))
-        .thenReturn(BUCKET_NAME.getBytes(UTF_8));
-    Mockito.when(metadataManager.getOpenKeyBytes(VOLUME_NAME, BUCKET_NAME,
-        KEY_NAME, 1)).thenReturn(KEY_NAME.getBytes(UTF_8));
+        .thenReturn(BUCKET_NAME);
+    Mockito.when(metadataManager.getOpenKey(VOLUME_NAME, BUCKET_NAME,
+        KEY_NAME, 1)).thenReturn(KEY_NAME);
   }
 
   private void setupRocksDb() throws Exception {
@@ -113,7 +121,7 @@ public class TestKeyManagerImpl {
     Set<TableConfig> configSet = new HashSet<>();
     for (String name : Arrays
         .asList(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
-            "testTable")) {
+            "testKeyTable", "testBucketTable", "testVolumeTable")) {
       TableConfig newConfig = new TableConfig(name, new ColumnFamilyOptions());
       configSet.add(newConfig);
     }
@@ -128,13 +136,39 @@ public class TestKeyManagerImpl {
         .setModificationTime(Time.now())
         .build();
 
-    rdbStore = new RDBStore(folder.newFolder(), options, configSet);
-    rdbTable = rdbStore.getTable("testTable");
-    rdbTable.put(VOLUME_NAME.getBytes(UTF_8),
-        RandomStringUtils.random(10).getBytes(UTF_8));
-    rdbTable.put(BUCKET_NAME.getBytes(UTF_8),
-        RandomStringUtils.random(10).getBytes(UTF_8));
-    rdbTable.put(KEY_NAME.getBytes(UTF_8), keyData.toByteArray());
+    CodecRegistry registry = new CodecRegistry();
+    registry.addCodec(OmKeyInfo.class, new OmKeyInfoCodec());
+    registry.addCodec(OmVolumeArgs.class, new OmVolumeArgsCodec());
+    registry.addCodec(OmBucketInfo.class, new OmBucketInfoCodec());
+    rdbStore = new RDBStore(folder.newFolder(), options, configSet, registry);
+
+    keyTable =
+        rdbStore.getTable("testKeyTable", String.class, OmKeyInfo.class);
+
+    bucketTable =
+        rdbStore.getTable("testBucketTable", String.class, OmBucketInfo.class);
+
+    volumeTable =
+        rdbStore.getTable("testVolumeTable", String.class, OmVolumeArgs.class);
+
+    volumeTable.put(VOLUME_NAME, OmVolumeArgs.newBuilder()
+        .setAdminName("a")
+        .setOwnerName("o")
+        .setVolume(VOLUME_NAME)
+        .build());
+
+    bucketTable.put(BUCKET_NAME,
+        new OmBucketInfo.Builder().setBucketName(BUCKET_NAME)
+            .setVolumeName(VOLUME_NAME).build());
+
+    keyTable.put(KEY_NAME, new OmKeyInfo.Builder()
+        .setVolumeName(VOLUME_NAME)
+        .setBucketName(BUCKET_NAME)
+        .setKeyName(KEY_NAME)
+        .setReplicationType(ReplicationType.STAND_ALONE)
+        .setReplicationFactor(ReplicationFactor.THREE)
+        .build());
+
   }
 
   @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


Mime
View raw message