|
@@ -36,7 +36,9 @@ import org.apache.commons.io.FileUtils;
|
|
import org.apache.commons.lang.RandomStringUtils;
|
|
import org.apache.commons.lang.RandomStringUtils;
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
|
|
|
+import org.apache.hadoop.hdds.client.BlockID;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
|
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
import org.apache.hadoop.hdds.scm.TestUtils;
|
|
import org.apache.hadoop.hdds.scm.TestUtils;
|
|
@@ -44,7 +46,12 @@ import org.apache.hadoop.hdds.scm.container.MockNodeManager;
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
|
|
|
|
+import org.apache.hadoop.hdds.scm.net.NetworkTopology;
|
|
|
|
+import org.apache.hadoop.hdds.scm.net.NetworkTopologyImpl;
|
|
|
|
+import org.apache.hadoop.hdds.scm.net.NodeSchema;
|
|
|
|
+import org.apache.hadoop.hdds.scm.net.NodeSchemaManager;
|
|
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
|
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
|
|
|
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
|
|
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
|
|
import org.apache.hadoop.hdds.scm.server.SCMConfigurator;
|
|
import org.apache.hadoop.hdds.scm.server.SCMConfigurator;
|
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
|
@@ -53,6 +60,7 @@ import org.apache.hadoop.ozone.om.exceptions.OMException;
|
|
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
|
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
|
import org.apache.hadoop.ozone.om.helpers.OmPrefixInfo;
|
|
import org.apache.hadoop.ozone.om.helpers.OmPrefixInfo;
|
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
|
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
|
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
|
@@ -71,6 +79,7 @@ import org.apache.hadoop.test.LambdaTestUtils;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
import org.junit.AfterClass;
|
|
import org.junit.AfterClass;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
|
|
+import org.junit.Assume;
|
|
import org.junit.BeforeClass;
|
|
import org.junit.BeforeClass;
|
|
import org.junit.Rule;
|
|
import org.junit.Rule;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
@@ -80,6 +89,11 @@ import org.mockito.Mockito;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_BLOCKS_MAX;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_BLOCKS_MAX;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
|
|
|
|
+
|
|
|
|
+import static org.apache.hadoop.hdds.scm.net.NetConstants.LEAF_SCHEMA;
|
|
|
|
+import static org.apache.hadoop.hdds.scm.net.NetConstants.RACK_SCHEMA;
|
|
|
|
+import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
|
|
|
|
+
|
|
import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.ALL;
|
|
import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.ALL;
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -91,6 +105,7 @@ public class TestKeyManagerImpl {
|
|
private static KeyManagerImpl keyManager;
|
|
private static KeyManagerImpl keyManager;
|
|
private static VolumeManagerImpl volumeManager;
|
|
private static VolumeManagerImpl volumeManager;
|
|
private static BucketManagerImpl bucketManager;
|
|
private static BucketManagerImpl bucketManager;
|
|
|
|
+ private static NodeManager nodeManager;
|
|
private static StorageContainerManager scm;
|
|
private static StorageContainerManager scm;
|
|
private static ScmBlockLocationProtocol mockScmBlockLocationProtocol;
|
|
private static ScmBlockLocationProtocol mockScmBlockLocationProtocol;
|
|
private static OzoneConfiguration conf;
|
|
private static OzoneConfiguration conf;
|
|
@@ -113,9 +128,17 @@ public class TestKeyManagerImpl {
|
|
metadataManager = new OmMetadataManagerImpl(conf);
|
|
metadataManager = new OmMetadataManagerImpl(conf);
|
|
volumeManager = new VolumeManagerImpl(metadataManager, conf);
|
|
volumeManager = new VolumeManagerImpl(metadataManager, conf);
|
|
bucketManager = new BucketManagerImpl(metadataManager);
|
|
bucketManager = new BucketManagerImpl(metadataManager);
|
|
- NodeManager nodeManager = new MockNodeManager(true, 10);
|
|
|
|
|
|
+ nodeManager = new MockNodeManager(true, 10);
|
|
|
|
+ NodeSchema[] schemas = new NodeSchema[]
|
|
|
|
+ {ROOT_SCHEMA, RACK_SCHEMA, LEAF_SCHEMA};
|
|
|
|
+ NodeSchemaManager schemaManager = NodeSchemaManager.getInstance();
|
|
|
|
+ schemaManager.init(schemas, false);
|
|
|
|
+ NetworkTopology clusterMap = new NetworkTopologyImpl(schemaManager);
|
|
|
|
+ nodeManager.getAllNodes().stream().forEach(node -> clusterMap.add(node));
|
|
|
|
+ ((MockNodeManager)nodeManager).setNetworkTopology(clusterMap);
|
|
SCMConfigurator configurator = new SCMConfigurator();
|
|
SCMConfigurator configurator = new SCMConfigurator();
|
|
configurator.setScmNodeManager(nodeManager);
|
|
configurator.setScmNodeManager(nodeManager);
|
|
|
|
+ configurator.setNetworkTopology(clusterMap);
|
|
scm = TestUtils.getScm(conf, configurator);
|
|
scm = TestUtils.getScm(conf, configurator);
|
|
scm.start();
|
|
scm.start();
|
|
scm.exitSafeMode();
|
|
scm.exitSafeMode();
|
|
@@ -563,7 +586,7 @@ public class TestKeyManagerImpl {
|
|
|
|
|
|
// lookup for a non-existent file
|
|
// lookup for a non-existent file
|
|
try {
|
|
try {
|
|
- keyManager.lookupFile(keyArgs);
|
|
|
|
|
|
+ keyManager.lookupFile(keyArgs, null);
|
|
Assert.fail("Lookup file should fail for non existent file");
|
|
Assert.fail("Lookup file should fail for non existent file");
|
|
} catch (OMException ex) {
|
|
} catch (OMException ex) {
|
|
if (ex.getResult() != OMException.ResultCodes.FILE_NOT_FOUND) {
|
|
if (ex.getResult() != OMException.ResultCodes.FILE_NOT_FOUND) {
|
|
@@ -576,14 +599,15 @@ public class TestKeyManagerImpl {
|
|
keyArgs.setLocationInfoList(
|
|
keyArgs.setLocationInfoList(
|
|
keySession.getKeyInfo().getLatestVersionLocations().getLocationList());
|
|
keySession.getKeyInfo().getLatestVersionLocations().getLocationList());
|
|
keyManager.commitKey(keyArgs, keySession.getId());
|
|
keyManager.commitKey(keyArgs, keySession.getId());
|
|
- Assert.assertEquals(keyManager.lookupFile(keyArgs).getKeyName(), keyName);
|
|
|
|
|
|
+ Assert.assertEquals(keyManager.lookupFile(keyArgs, null).getKeyName(),
|
|
|
|
+ keyName);
|
|
|
|
|
|
// lookup for created file
|
|
// lookup for created file
|
|
keyArgs = createBuilder()
|
|
keyArgs = createBuilder()
|
|
.setKeyName("")
|
|
.setKeyName("")
|
|
.build();
|
|
.build();
|
|
try {
|
|
try {
|
|
- keyManager.lookupFile(keyArgs);
|
|
|
|
|
|
+ keyManager.lookupFile(keyArgs, null);
|
|
Assert.fail("Lookup file should fail for a directory");
|
|
Assert.fail("Lookup file should fail for a directory");
|
|
} catch (OMException ex) {
|
|
} catch (OMException ex) {
|
|
if (ex.getResult() != OMException.ResultCodes.NOT_A_FILE) {
|
|
if (ex.getResult() != OMException.ResultCodes.NOT_A_FILE) {
|
|
@@ -596,6 +620,81 @@ public class TestKeyManagerImpl {
|
|
return createBuilder().setKeyName(toKeyName).build();
|
|
return createBuilder().setKeyName(toKeyName).build();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testLookupKeyWithLocation() throws IOException {
|
|
|
|
+ String keyName = RandomStringUtils.randomAlphabetic(5);
|
|
|
|
+ OmKeyArgs keyArgs = createBuilder()
|
|
|
|
+ .setKeyName(keyName)
|
|
|
|
+ .build();
|
|
|
|
+
|
|
|
|
+ // lookup for a non-existent key
|
|
|
|
+ try {
|
|
|
|
+ keyManager.lookupKey(keyArgs, null);
|
|
|
|
+ Assert.fail("Lookup key should fail for non existent key");
|
|
|
|
+ } catch (OMException ex) {
|
|
|
|
+ if (ex.getResult() != OMException.ResultCodes.KEY_NOT_FOUND) {
|
|
|
|
+ throw ex;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // create a key
|
|
|
|
+ OpenKeySession keySession = keyManager.createFile(keyArgs, false, false);
|
|
|
|
+ // randomly select 3 datanodes
|
|
|
|
+ List<DatanodeDetails> nodeList = new ArrayList<>();
|
|
|
|
+ nodeList.add((DatanodeDetails)scm.getClusterMap().getNode(
|
|
|
|
+ 0, null, null, null, null, 0));
|
|
|
|
+ nodeList.add((DatanodeDetails)scm.getClusterMap().getNode(
|
|
|
|
+ 1, null, null, null, null, 0));
|
|
|
|
+ nodeList.add((DatanodeDetails)scm.getClusterMap().getNode(
|
|
|
|
+ 2, null, null, null, null, 0));
|
|
|
|
+ Assume.assumeFalse(nodeList.get(0).equals(nodeList.get(1)));
|
|
|
|
+ Assume.assumeFalse(nodeList.get(0).equals(nodeList.get(2)));
|
|
|
|
+ // create a pipeline using 3 datanodes
|
|
|
|
+ Pipeline pipeline = scm.getPipelineManager().createPipeline(
|
|
|
|
+ ReplicationType.RATIS, ReplicationFactor.THREE, nodeList);
|
|
|
|
+ List<OmKeyLocationInfo> locationInfoList = new ArrayList<>();
|
|
|
|
+ locationInfoList.add(
|
|
|
|
+ new OmKeyLocationInfo.Builder().setPipeline(pipeline)
|
|
|
|
+ .setBlockID(new BlockID(1L, 1L)).build());
|
|
|
|
+ keyArgs.setLocationInfoList(locationInfoList);
|
|
|
|
+
|
|
|
|
+ keyManager.commitKey(keyArgs, keySession.getId());
|
|
|
|
+
|
|
|
|
+ OmKeyInfo key = keyManager.lookupKey(keyArgs, null);
|
|
|
|
+ Assert.assertEquals(key.getKeyName(), keyName);
|
|
|
|
+ List<OmKeyLocationInfo> keyLocations =
|
|
|
|
+ key.getLatestVersionLocations().getLocationList();
|
|
|
|
+ DatanodeDetails leader =
|
|
|
|
+ keyLocations.get(0).getPipeline().getFirstNode();
|
|
|
|
+ DatanodeDetails follower1 =
|
|
|
|
+ keyLocations.get(0).getPipeline().getNodes().get(1);
|
|
|
|
+ DatanodeDetails follower2 =
|
|
|
|
+ keyLocations.get(0).getPipeline().getNodes().get(2);
|
|
|
|
+ Assert.assertNotEquals(leader, follower1);
|
|
|
|
+ Assert.assertNotEquals(follower1, follower2);
|
|
|
|
+
|
|
|
|
+ // lookup key, leader as client
|
|
|
|
+ OmKeyInfo key1 = keyManager.lookupKey(keyArgs, leader.getNetworkName());
|
|
|
|
+ Assert.assertEquals(leader, key1.getLatestVersionLocations()
|
|
|
|
+ .getLocationList().get(0).getPipeline().getClosestNode());
|
|
|
|
+
|
|
|
|
+ // lookup key, follower1 as client
|
|
|
|
+ OmKeyInfo key2 = keyManager.lookupKey(keyArgs, follower1.getNetworkName());
|
|
|
|
+ Assert.assertEquals(follower1, key2.getLatestVersionLocations()
|
|
|
|
+ .getLocationList().get(0).getPipeline().getClosestNode());
|
|
|
|
+
|
|
|
|
+ // lookup key, follower2 as client
|
|
|
|
+ OmKeyInfo key3 = keyManager.lookupKey(keyArgs, follower2.getNetworkName());
|
|
|
|
+ Assert.assertEquals(follower2, key3.getLatestVersionLocations()
|
|
|
|
+ .getLocationList().get(0).getPipeline().getClosestNode());
|
|
|
|
+
|
|
|
|
+ // lookup key, random node as client
|
|
|
|
+ OmKeyInfo key4 = keyManager.lookupKey(keyArgs,
|
|
|
|
+ "/d=default-drack/127.0.0.1");
|
|
|
|
+ Assert.assertEquals(leader, key4.getLatestVersionLocations()
|
|
|
|
+ .getLocationList().get(0).getPipeline().getClosestNode());
|
|
|
|
+ }
|
|
|
|
+
|
|
@Test
|
|
@Test
|
|
public void testListStatus() throws IOException {
|
|
public void testListStatus() throws IOException {
|
|
String superDir = RandomStringUtils.randomAlphabetic(5);
|
|
String superDir = RandomStringUtils.randomAlphabetic(5);
|