From 852ca6b432a801df51f957a9b169483008e3c744 Mon Sep 17 00:00:00 2001 From: Nanda kumar Date: Fri, 29 Jun 2018 04:11:39 +0530 Subject: [PATCH 01/47] HDDS-185: TestCloseContainerByPipeline#testCloseContainerViaRatis fail intermittently. Contributed by Shashikant Banerjee. --- .../TestCloseContainerByPipeline.java | 65 +++++++++++++++++-- 1 file changed, 58 insertions(+), 7 deletions(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java index 9e8cb468bbe..265c82bdeb9 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java @@ -23,6 +23,8 @@ import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.MiniOzoneCluster; @@ -33,6 +35,7 @@ import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; +import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; @@ -45,6 +48,7 @@ import org.junit.Test; import java.io.IOException; import java.util.List; +import java.util.Random; import java.util.concurrent.TimeoutException; public class TestCloseContainerByPipeline { @@ -88,6 +92,59 @@ public class TestCloseContainerByPipeline { } } + @Test + public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception { + OzoneOutputStream key = objectStore.getVolume("test").getBucket("test") + .createKey("testCloseContainer", 1024, ReplicationType.STAND_ALONE, + ReplicationFactor.ONE); + key.write("standalone".getBytes()); + key.close(); + + //get the name of a valid container + KsmKeyArgs keyArgs = + new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test") + .setType(HddsProtos.ReplicationType.STAND_ALONE) + .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024) + .setKeyName("testCloseContainer").build(); + + KsmKeyLocationInfo ksmKeyLocationInfo = + cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions() + .get(0).getBlocksLatestVersionOnly().get(0); + + long containerID = ksmKeyLocationInfo.getContainerID(); + List datanodes = + cluster.getStorageContainerManager().getContainerInfo(containerID) + .getPipeline().getMachines(); + Assert.assertTrue(datanodes.size() == 1); + + DatanodeDetails datanodeDetails = datanodes.get(0); + HddsDatanodeService datanodeService = null; + Assert + .assertFalse(isContainerClosed(cluster, containerID, datanodeDetails)); + for (HddsDatanodeService datanodeServiceItr : cluster.getHddsDatanodes()) { + if (datanodeDetails.equals(datanodeServiceItr.getDatanodeDetails())) { + datanodeService = datanodeServiceItr; + break; + } + } + CommandHandler closeContainerHandler = + datanodeService.getDatanodeStateMachine().getCommandDispatcher() + .getCloseContainerHandler(); + int lastInvocationCount = closeContainerHandler.getInvocationCount(); + //send the order to close the container + cluster.getStorageContainerManager().getScmNodeManager() + .addDatanodeCommand(datanodeDetails.getUuid(), + new CloseContainerCommand(containerID, + HddsProtos.ReplicationType.STAND_ALONE)); + GenericTestUtils + .waitFor(() -> isContainerClosed(cluster, containerID, datanodeDetails), + 500, 5 * 1000); + // Make sure the closeContainerCommandHandler is Invoked + Assert.assertTrue( + closeContainerHandler.getInvocationCount() > lastInvocationCount); + + } + @Test public void testCloseContainerViaStandaAlone() throws IOException, TimeoutException, InterruptedException { @@ -205,13 +262,7 @@ public class TestCloseContainerByPipeline { containerData = datanodeService.getDatanodeStateMachine().getContainer() .getContainerManager().readContainer(containerID); - if (!containerData.isOpen()) { - // make sure the closeContainerHandler on the Datanode is invoked - Assert.assertTrue( - datanodeService.getDatanodeStateMachine().getCommandDispatcher() - .getCloseContainerHandler().getInvocationCount() > 0); - return true; - } + return !containerData.isOpen(); } } catch (StorageContainerException e) { throw new AssertionError(e); From e4d7227aad586f055b47bdc90c65361f9fb23146 Mon Sep 17 00:00:00 2001 From: Nanda kumar Date: Fri, 29 Jun 2018 04:37:16 +0530 Subject: [PATCH 02/47] HDDS-178: DN should update transactionId on block delete. Contributed by Lokesh Jain. --- .../common/impl/ContainerManagerImpl.java | 7 +- .../common/interfaces/ContainerManager.java | 2 + .../DeleteBlocksCommandHandler.java | 29 ++- .../commandhandler/TestBlockDeletion.java | 211 ++++++++++++++++++ 4 files changed, 243 insertions(+), 6 deletions(-) create mode 100644 hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java index 02572a8fa2b..e81f1c6c725 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java @@ -1111,5 +1111,10 @@ public class ContainerManagerImpl implements ContainerManager { return cData.getKeyCount(); } - + @Override + public void updateDeleteTransactionId(long containerId, + long deleteTransactionId) { + containerMap.get(containerId) + .updateDeleteTransactionId(deleteTransactionId); + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java index 49b68dc2a04..cf68b08a23f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java @@ -264,4 +264,6 @@ public interface ContainerManager extends RwLock { */ long getNumKeys(long containerId); + void updateDeleteTransactionId(long containerId, long deleteTransactionId); + } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java index f954d98f939..d215da98656 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java @@ -158,6 +158,13 @@ public class DeleteBlocksCommandHandler implements CommandHandler { containerInfo.getDBPath()); } + if (delTX.getTxID() < containerInfo.getDeleteTransactionId()) { + LOG.debug(String.format("Ignoring delete blocks for containerId: %d." + + " Outdated delete transactionId %d < %d", containerId, + delTX.getTxID(), containerInfo.getDeleteTransactionId())); + return; + } + int newDeletionBlocks = 0; MetadataStore containerDB = KeyUtils.getDB(containerInfo, config); for (Long blk : delTX.getLocalIDList()) { @@ -165,10 +172,20 @@ public class DeleteBlocksCommandHandler implements CommandHandler { byte[] blkBytes = Longs.toByteArray(blk); byte[] blkInfo = containerDB.get(blkBytes); if (blkInfo != null) { + byte[] deletingKeyBytes = + DFSUtil.string2Bytes(OzoneConsts.DELETING_KEY_PREFIX + blk); + byte[] deletedKeyBytes = + DFSUtil.string2Bytes(OzoneConsts.DELETED_KEY_PREFIX + blk); + if (containerDB.get(deletingKeyBytes) != null + || containerDB.get(deletedKeyBytes) != null) { + LOG.debug(String.format( + "Ignoring delete for block %d in container %d." + + " Entry already added.", blk, containerId)); + continue; + } // Found the block in container db, // use an atomic update to change its state to deleting. - batch.put(DFSUtil.string2Bytes(OzoneConsts.DELETING_KEY_PREFIX + blk), - blkInfo); + batch.put(deletingKeyBytes, blkInfo); batch.delete(blkBytes); try { containerDB.writeBatch(batch); @@ -186,11 +203,13 @@ public class DeleteBlocksCommandHandler implements CommandHandler { LOG.debug("Block {} not found or already under deletion in" + " container {}, skip deleting it.", blk, containerId); } - containerDB.put(DFSUtil.string2Bytes( - OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX + delTX.getContainerID()), - Longs.toByteArray(delTX.getTxID())); } + containerDB.put(DFSUtil.string2Bytes( + OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX + delTX.getContainerID()), + Longs.toByteArray(delTX.getTxID())); + containerManager + .updateDeleteTransactionId(delTX.getContainerID(), delTX.getTxID()); // update pending deletion blocks count in in-memory container status containerManager.incrPendingDeletionBlocks(newDeletionBlocks, containerId); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java new file mode 100644 index 00000000000..43e3f5095f3 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java @@ -0,0 +1,211 @@ +/** + * 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.ozone.container.common.statemachine.commandhandler; + +import com.google.common.primitives.Longs; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.client.ReplicationFactor; +import org.apache.hadoop.hdds.client.ReplicationType; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.server.StorageContainerManager; +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.client.ObjectStore; +import org.apache.hadoop.ozone.client.OzoneBucket; +import org.apache.hadoop.ozone.client.OzoneClientFactory; +import org.apache.hadoop.ozone.client.OzoneVolume; +import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.hadoop.ozone.container.common.helpers.ContainerData; +import org.apache.hadoop.ozone.container.common.helpers.KeyUtils; +import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl; +import org.apache.hadoop.ozone.ksm.KeySpaceManager; +import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; +import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; +import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup; +import org.apache.hadoop.ozone.ozShell.TestOzoneShell; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.utils.MetadataStore; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL; + +public class TestBlockDeletion { + private static OzoneConfiguration conf = null; + private static ObjectStore store; + private static ContainerManagerImpl dnContainerManager = null; + private static StorageContainerManager scm = null; + private static KeySpaceManager ksm = null; + private static Set containerIdsWithDeletedBlocks; + + @BeforeClass + public static void init() throws Exception { + conf = new OzoneConfiguration(); + + String path = + GenericTestUtils.getTempPath(TestOzoneShell.class.getSimpleName()); + File baseDir = new File(path); + baseDir.mkdirs(); + + path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, + OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT); + + conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path); + conf.setQuietMode(false); + conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 100, + TimeUnit.MILLISECONDS); + + MiniOzoneCluster cluster = + MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build(); + cluster.waitForClusterToBeReady(); + store = OzoneClientFactory.getRpcClient(conf).getObjectStore(); + dnContainerManager = + (ContainerManagerImpl) cluster.getHddsDatanodes().get(0) + .getDatanodeStateMachine().getContainer().getContainerManager(); + ksm = cluster.getKeySpaceManager(); + scm = cluster.getStorageContainerManager(); + containerIdsWithDeletedBlocks = new HashSet<>(); + } + + @Test(timeout = 60000) + public void testBlockDeletion() + throws IOException, InterruptedException { + String volumeName = UUID.randomUUID().toString(); + String bucketName = UUID.randomUUID().toString(); + + String value = RandomStringUtils.random(1000000); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); + + String keyName = UUID.randomUUID().toString(); + + OzoneOutputStream out = bucket.createKey(keyName, value.getBytes().length, + ReplicationType.STAND_ALONE, ReplicationFactor.ONE); + out.write(value.getBytes()); + out.close(); + + KsmKeyArgs keyArgs = new KsmKeyArgs.Builder().setVolumeName(volumeName) + .setBucketName(bucketName).setKeyName(keyName).setDataSize(0) + .setType(HddsProtos.ReplicationType.STAND_ALONE) + .setFactor(HddsProtos.ReplicationFactor.ONE).build(); + List ksmKeyLocationInfoGroupList = + ksm.lookupKey(keyArgs).getKeyLocationVersions(); + + // verify key blocks were created in DN. + Assert.assertTrue(verifyBlocksCreated(ksmKeyLocationInfoGroupList)); + // No containers with deleted blocks + Assert.assertTrue(containerIdsWithDeletedBlocks.isEmpty()); + // Delete transactionIds for the containers should be 0 + matchContainerTransactionIds(); + ksm.deleteKey(keyArgs); + Thread.sleep(5000); + // The blocks should be deleted in the DN. + Assert.assertTrue(verifyBlocksDeleted(ksmKeyLocationInfoGroupList)); + + // Few containers with deleted blocks + Assert.assertTrue(!containerIdsWithDeletedBlocks.isEmpty()); + // Containers in the DN and SCM should have same delete transactionIds + matchContainerTransactionIds(); + } + + private void matchContainerTransactionIds() throws IOException { + List containerDataList = new ArrayList<>(); + dnContainerManager.listContainer(0, 10000, containerDataList); + for (ContainerData containerData : containerDataList) { + long containerId = containerData.getContainerID(); + if (containerIdsWithDeletedBlocks.contains(containerId)) { + Assert.assertTrue( + scm.getContainerInfo(containerId).getDeleteTransactionId() > 0); + } else { + Assert.assertEquals( + scm.getContainerInfo(containerId).getDeleteTransactionId(), 0); + } + Assert.assertEquals(dnContainerManager.readContainer(containerId) + .getDeleteTransactionId(), + scm.getContainerInfo(containerId).getDeleteTransactionId()); + } + } + + private boolean verifyBlocksCreated( + List ksmKeyLocationInfoGroups) + throws IOException { + return performOperationOnKeyContainers((blockID) -> { + try { + MetadataStore db = KeyUtils.getDB( + dnContainerManager.getContainerMap().get(blockID.getContainerID()), + conf); + Assert.assertNotNull(db.get(Longs.toByteArray(blockID.getLocalID()))); + } catch (IOException e) { + e.printStackTrace(); + } + }, ksmKeyLocationInfoGroups); + } + + private boolean verifyBlocksDeleted( + List ksmKeyLocationInfoGroups) + throws IOException { + return performOperationOnKeyContainers((blockID) -> { + try { + MetadataStore db = KeyUtils.getDB( + dnContainerManager.getContainerMap().get(blockID.getContainerID()), + conf); + Assert.assertNull(db.get(Longs.toByteArray(blockID.getLocalID()))); + Assert.assertNull(db.get(DFSUtil.string2Bytes( + OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID()))); + Assert.assertNotNull(DFSUtil.string2Bytes( + OzoneConsts.DELETED_KEY_PREFIX + blockID.getLocalID())); + containerIdsWithDeletedBlocks.add(blockID.getContainerID()); + } catch (IOException e) { + e.printStackTrace(); + } + }, ksmKeyLocationInfoGroups); + } + + private boolean performOperationOnKeyContainers(Consumer consumer, + List ksmKeyLocationInfoGroups) + throws IOException { + + try { + for (KsmKeyLocationInfoGroup ksmKeyLocationInfoGroup : ksmKeyLocationInfoGroups) { + List ksmKeyLocationInfos = + ksmKeyLocationInfoGroup.getLocationList(); + for (KsmKeyLocationInfo ksmKeyLocationInfo : ksmKeyLocationInfos) { + BlockID blockID = ksmKeyLocationInfo.getBlockID(); + consumer.accept(blockID); + } + } + } catch (Error e) { + e.printStackTrace(); + return false; + } + return true; + } +} \ No newline at end of file From 73746c5da76d5e39df131534a1ec35dfc5d2529b Mon Sep 17 00:00:00 2001 From: Inigo Goiri Date: Fri, 29 Jun 2018 09:56:13 -0700 Subject: [PATCH 03/47] HDFS-13707. [PROVIDED Storage] Fix failing integration tests in ITestProvidedImplementation. Contributed by Virajith Jalaparti. --- .../common/blockaliasmap/impl/TextFileRegionAliasMap.java | 5 +++-- .../hdfs/server/namenode/ITestProvidedImplementation.java | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java index abe92e3edbd..4d65142c958 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java @@ -28,6 +28,7 @@ import java.io.OutputStream; import java.io.OutputStreamWriter; import java.nio.charset.Charset; import java.util.ArrayList; +import java.util.Base64; import java.util.Iterator; import java.util.Map; import java.util.Collections; @@ -359,7 +360,7 @@ public class TextFileRegionAliasMap } byte[] nonce = new byte[0]; if (f.length == 6) { - nonce = f[5].getBytes(Charset.forName("UTF-8")); + nonce = Base64.getDecoder().decode(f[5]); } return new FileRegion(Long.parseLong(f[0]), new Path(f[1]), Long.parseLong(f[2]), Long.parseLong(f[3]), Long.parseLong(f[4]), @@ -451,7 +452,7 @@ public class TextFileRegionAliasMap out.append(Long.toString(block.getGenerationStamp())); if (psl.getNonce().length > 0) { out.append(delim) - .append(new String(psl.getNonce(), Charset.forName("UTF-8"))); + .append(Base64.getEncoder().encodeToString(psl.getNonce())); } out.append("\n"); } diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java index 49c9bcfc0b9..7d3ab0ea8f9 100644 --- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java +++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java @@ -132,7 +132,7 @@ public class ITestProvidedImplementation { nnDirPath.toString()); conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE, new Path(nnDirPath, fileNameFromBlockPoolID(bpid)).toString()); - conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER, ","); + conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER, "\t"); conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR_PROVIDED, new File(providedPath.toUri()).toString()); From 469b29c0817b7bf1902c9195c4f8d031a909e1c9 Mon Sep 17 00:00:00 2001 From: Sunil G Date: Fri, 29 Jun 2018 10:02:53 -0700 Subject: [PATCH 04/47] YARN-8455. Add basic ACL check for all ATS v2 REST APIs. Contributed by Rohith Sharma K S. --- .../reader/TimelineFromIdConverter.java | 93 ++++++++ .../reader/TimelineReaderWebServices.java | 198 ++++++++++++++---- ...TestTimelineReaderWebServicesBasicAcl.java | 154 ++++++++++++++ 3 files changed, 407 insertions(+), 38 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineFromIdConverter.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesBasicAcl.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineFromIdConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineFromIdConverter.java new file mode 100644 index 00000000000..5f5f0b14d1b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineFromIdConverter.java @@ -0,0 +1,93 @@ +/** + * 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.yarn.server.timelineservice.reader; + +import java.util.List; + +/** + * Used for decoding FROM_ID + */ +enum TimelineFromIdConverter { + + APPLICATION_FROMID { + @Override TimelineReaderContext decodeUID(String fromId) throws Exception { + if (fromId == null) { + return null; + } + + List appTupleList = TimelineReaderUtils.split(fromId); + if (appTupleList == null || appTupleList.size() != 5) { + throw new IllegalArgumentException( + "Invalid row key for application table."); + } + + return new TimelineReaderContext(appTupleList.get(0), appTupleList.get(1), + appTupleList.get(2), Long.parseLong(appTupleList.get(3)), + appTupleList.get(4), null, null); + } + }, + + SUB_APPLICATION_ENTITY_FROMID { + @Override TimelineReaderContext decodeUID(String fromId) throws Exception { + if (fromId == null) { + return null; + } + List split = TimelineReaderUtils.split(fromId); + if (split == null || split.size() != 6) { + throw new IllegalArgumentException( + "Invalid row key for sub app table."); + } + + String subAppUserId = split.get(0); + String clusterId = split.get(1); + String entityType = split.get(2); + Long entityIdPrefix = Long.valueOf(split.get(3)); + String entityId = split.get(4); + String userId = split.get(5); + return new TimelineReaderContext(clusterId, userId, null, null, null, + entityType, entityIdPrefix, entityId, subAppUserId); + } + }, + + GENERIC_ENTITY_FROMID { + @Override TimelineReaderContext decodeUID(String fromId) throws Exception { + if (fromId == null) { + return null; + } + List split = TimelineReaderUtils.split(fromId); + if (split == null || split.size() != 8) { + throw new IllegalArgumentException("Invalid row key for entity table."); + } + Long flowRunId = Long.valueOf(split.get(3)); + Long entityIdPrefix = Long.valueOf(split.get(6)); + return new TimelineReaderContext(split.get(0), split.get(1), split.get(2), + flowRunId, split.get(4), split.get(5), entityIdPrefix, split.get(7)); + } + }; + + /** + * Decodes FROM_ID depending on FROM_ID implementation. + * + * @param fromId FROM_ID to be decoded. + * @return a {@link TimelineReaderContext} object if FROM_ID passed can be + * decoded, null otherwise. + * @throws Exception if any problem occurs while decoding. + */ + abstract TimelineReaderContext decodeUID(String fromId) throws Exception; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java index 7bf66b0bd76..7f96bfb6371 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java @@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.apache.hadoop.yarn.webapp.BadRequestException; +import org.apache.hadoop.yarn.webapp.ForbiddenException; import org.apache.hadoop.yarn.webapp.NotFoundException; import com.google.common.annotations.VisibleForTesting; @@ -188,6 +189,8 @@ public class TimelineReaderWebServices { "Filter Parsing failed." : e.getMessage()); } else if (e instanceof BadRequestException) { throw (BadRequestException)e; + } else if (e instanceof ForbiddenException) { + throw (ForbiddenException) e; } else { LOG.error("Error while processing REST request", e); throw new WebApplicationException(e, @@ -339,6 +342,7 @@ public class TimelineReaderWebServices { TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( confsToRetrieve, metricsToRetrieve, fields, metricsLimit, metricsTimeStart, metricsTimeEnd)); + checkAccessForGenericEntities(entities, callerUGI, entityType); } catch (Exception e) { handleException(e, url, startTime, "createdTime start/end or limit or flowrunid"); @@ -607,13 +611,15 @@ public class TimelineReaderWebServices { .createTimelineReaderContext(clusterId, userId, flowName, flowRunId, appId, entityType, null, null); entities = timelineReaderManager.getEntities(context, - TimelineReaderWebServicesUtils.createTimelineEntityFilters( - limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, - infofilters, conffilters, metricfilters, eventfilters, - fromId), - TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( - confsToRetrieve, metricsToRetrieve, fields, metricsLimit, - metricsTimeStart, metricsTimeEnd)); + TimelineReaderWebServicesUtils + .createTimelineEntityFilters(limit, createdTimeStart, + createdTimeEnd, relatesTo, isRelatedTo, infofilters, + conffilters, metricfilters, eventfilters, fromId), + TimelineReaderWebServicesUtils + .createTimelineDataToRetrieve(confsToRetrieve, metricsToRetrieve, + fields, metricsLimit, metricsTimeStart, metricsTimeEnd)); + + checkAccessForGenericEntities(entities, callerUGI, entityType); } catch (Exception e) { handleException(e, url, startTime, "createdTime start/end or limit or flowrunid"); @@ -704,6 +710,7 @@ public class TimelineReaderWebServices { TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( confsToRetrieve, metricsToRetrieve, fields, metricsLimit, metricsTimeStart, metricsTimeEnd)); + checkAccessForGenericEntity(entity, callerUGI); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -893,6 +900,7 @@ public class TimelineReaderWebServices { TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( confsToRetrieve, metricsToRetrieve, fields, metricsLimit, metricsTimeStart, metricsTimeEnd)); + checkAccessForGenericEntity(entity, callerUGI); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -956,6 +964,8 @@ public class TimelineReaderWebServices { if (context == null) { throw new BadRequestException("Incorrect UID " + uId); } + // TODO to be removed or modified once ACL story is played + checkAccess(timelineReaderManager, callerUGI, context.getUserId()); context.setEntityType(TimelineEntityType.YARN_FLOW_RUN.toString()); entity = timelineReaderManager.getEntity(context, TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( @@ -1063,12 +1073,16 @@ public class TimelineReaderWebServices { TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); TimelineEntity entity = null; try { - entity = timelineReaderManager.getEntity( - TimelineReaderWebServicesUtils.createTimelineReaderContext( - clusterId, userId, flowName, flowRunId, null, - TimelineEntityType.YARN_FLOW_RUN.toString(), null, null), - TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( - null, metricsToRetrieve, null, null, null, null)); + TimelineReaderContext context = TimelineReaderWebServicesUtils + .createTimelineReaderContext(clusterId, userId, flowName, flowRunId, + null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null); + // TODO to be removed or modified once ACL story is played + checkAccess(timelineReaderManager, callerUGI, context.getUserId()); + + entity = timelineReaderManager.getEntity(context, + TimelineReaderWebServicesUtils + .createTimelineDataToRetrieve(null, metricsToRetrieve, null, null, + null, null)); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -1156,6 +1170,8 @@ public class TimelineReaderWebServices { if (context == null) { throw new BadRequestException("Incorrect UID " + uId); } + // TODO to be removed or modified once ACL story is played + checkAccess(timelineReaderManager, callerUGI, context.getUserId()); context.setEntityType(TimelineEntityType.YARN_FLOW_RUN.toString()); entities = timelineReaderManager.getEntities(context, TimelineReaderWebServicesUtils.createTimelineEntityFilters( @@ -1304,15 +1320,21 @@ public class TimelineReaderWebServices { TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); Set entities = null; try { - entities = timelineReaderManager.getEntities( - TimelineReaderWebServicesUtils.createTimelineReaderContext( - clusterId, userId, flowName, null, null, - TimelineEntityType.YARN_FLOW_RUN.toString(), null, null), - TimelineReaderWebServicesUtils.createTimelineEntityFilters( - limit, createdTimeStart, createdTimeEnd, null, null, null, - null, null, null, fromId), - TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( - null, metricsToRetrieve, fields, null, null, null)); + TimelineReaderContext timelineReaderContext = TimelineReaderWebServicesUtils + .createTimelineReaderContext(clusterId, userId, flowName, null, + null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, + null); + // TODO to be removed or modified once ACL story is played + checkAccess(timelineReaderManager, callerUGI, + timelineReaderContext.getUserId()); + + entities = timelineReaderManager.getEntities(timelineReaderContext, + TimelineReaderWebServicesUtils + .createTimelineEntityFilters(limit, createdTimeStart, + createdTimeEnd, null, null, null, null, null, null, fromId), + TimelineReaderWebServicesUtils + .createTimelineDataToRetrieve(null, metricsToRetrieve, fields, + null, null, null)); } catch (Exception e) { handleException(e, url, startTime, "createdTime start/end or limit or fromId"); @@ -1435,7 +1457,6 @@ public class TimelineReaderWebServices { long startTime = Time.monotonicNow(); init(res); TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); - Configuration config = timelineReaderManager.getConfig(); Set entities = null; try { DateRange range = parseDateRange(dateRange); @@ -1455,19 +1476,9 @@ public class TimelineReaderWebServices { long endTime = Time.monotonicNow(); if (entities == null) { entities = Collections.emptySet(); - } else if (isDisplayEntityPerUserFilterEnabled(config)) { - Set userEntities = new LinkedHashSet<>(); - userEntities.addAll(entities); - for (TimelineEntity entity : userEntities) { - if (entity.getInfo() != null) { - String userId = - (String) entity.getInfo().get(FlowActivityEntity.USER_INFO_KEY); - if (!validateAuthUserWithEntityUser(timelineReaderManager, callerUGI, - userId)) { - entities.remove(entity); - } - } - } + } else { + checkAccess(timelineReaderManager, callerUGI, entities, + FlowActivityEntity.USER_INFO_KEY, true); } LOG.info("Processed URL " + url + " (Took " + (endTime - startTime) + " ms.)"); @@ -1552,6 +1563,7 @@ public class TimelineReaderWebServices { TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( confsToRetrieve, metricsToRetrieve, fields, metricsLimit, metricsTimeStart, metricsTimeEnd)); + checkAccessForAppEntity(entity, callerUGI); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -1722,6 +1734,7 @@ public class TimelineReaderWebServices { TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( confsToRetrieve, metricsToRetrieve, fields, metricsLimit, metricsTimeStart, metricsTimeEnd)); + checkAccessForAppEntity(entity, callerUGI); } catch (Exception e) { handleException(e, url, startTime, "flowrunid"); } @@ -1852,6 +1865,8 @@ public class TimelineReaderWebServices { if (context == null) { throw new BadRequestException("Incorrect UID " + uId); } + // TODO to be removed or modified once ACL story is played + checkAccess(timelineReaderManager, callerUGI, context.getUserId()); context.setEntityType(TimelineEntityType.YARN_APPLICATION.toString()); entities = timelineReaderManager.getEntities(context, TimelineReaderWebServicesUtils.createTimelineEntityFilters( @@ -3343,6 +3358,7 @@ public class TimelineReaderWebServices { TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( confsToRetrieve, metricsToRetrieve, fields, metricsLimit, metricsTimeStart, metricsTimeEnd)); + checkAccessForSubAppEntities(entities,callerUGI); } catch (Exception e) { handleException(e, url, startTime, "createdTime start/end or limit"); @@ -3410,6 +3426,7 @@ public class TimelineReaderWebServices { TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( confsToRetrieve, metricsToRetrieve, fields, metricsLimit, metricsTimeStart, metricsTimeEnd)); + checkAccessForSubAppEntities(entities,callerUGI); } catch (Exception e) { handleException(e, url, startTime, ""); } @@ -3422,7 +3439,7 @@ public class TimelineReaderWebServices { return entities; } - private boolean isDisplayEntityPerUserFilterEnabled(Configuration config) { + static boolean isDisplayEntityPerUserFilterEnabled(Configuration config) { return !config .getBoolean(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, YarnConfiguration.DEFAULT_TIMELINE_SERVICE_READ_AUTH_ENABLED) @@ -3430,8 +3447,76 @@ public class TimelineReaderWebServices { .getBoolean(YarnConfiguration.FILTER_ENTITY_LIST_BY_USER, false); } + // TODO to be removed or modified once ACL story is played + private void checkAccessForSubAppEntities(Set entities, + UserGroupInformation callerUGI) throws Exception { + if (entities != null && entities.size() > 0 + && isDisplayEntityPerUserFilterEnabled( + getTimelineReaderManager().getConfig())) { + TimelineReaderContext timelineReaderContext = null; + TimelineEntity entity = entities.iterator().next(); + String fromId = + (String) entity.getInfo().get(TimelineReaderUtils.FROMID_KEY); + timelineReaderContext = + TimelineFromIdConverter.SUB_APPLICATION_ENTITY_FROMID + .decodeUID(fromId); + checkAccess(getTimelineReaderManager(), callerUGI, + timelineReaderContext.getDoAsUser()); + } + } + + // TODO to be removed or modified once ACL story is played + private void checkAccessForAppEntity(TimelineEntity entity, + UserGroupInformation callerUGI) throws Exception { + if (entity != null && isDisplayEntityPerUserFilterEnabled( + getTimelineReaderManager().getConfig())) { + String fromId = + (String) entity.getInfo().get(TimelineReaderUtils.FROMID_KEY); + TimelineReaderContext timelineReaderContext = + TimelineFromIdConverter.APPLICATION_FROMID.decodeUID(fromId); + checkAccess(getTimelineReaderManager(), callerUGI, + timelineReaderContext.getUserId()); + } + } + + // TODO to be removed or modified once ACL story is played + private void checkAccessForGenericEntity(TimelineEntity entity, + UserGroupInformation callerUGI) throws Exception { + if (entity != null && isDisplayEntityPerUserFilterEnabled( + getTimelineReaderManager().getConfig())) { + String fromId = + (String) entity.getInfo().get(TimelineReaderUtils.FROMID_KEY); + TimelineReaderContext timelineReaderContext = + TimelineFromIdConverter.GENERIC_ENTITY_FROMID.decodeUID(fromId); + checkAccess(getTimelineReaderManager(), callerUGI, + timelineReaderContext.getUserId()); + } + } + + // TODO to be removed or modified once ACL story is played + private void checkAccessForGenericEntities(Set entities, + UserGroupInformation callerUGI, String entityType) throws Exception { + if (entities != null && entities.size() > 0 + && isDisplayEntityPerUserFilterEnabled( + getTimelineReaderManager().getConfig())) { + TimelineReaderContext timelineReaderContext = null; + TimelineEntity entity = entities.iterator().next(); + String uid = + (String) entity.getInfo().get(TimelineReaderUtils.FROMID_KEY); + if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) { + timelineReaderContext = + TimelineFromIdConverter.APPLICATION_FROMID.decodeUID(uid); + } else { + timelineReaderContext = + TimelineFromIdConverter.GENERIC_ENTITY_FROMID.decodeUID(uid); + } + checkAccess(getTimelineReaderManager(), callerUGI, + timelineReaderContext.getUserId()); + } + } + // TODO to be removed/modified once ACL story has played - private boolean validateAuthUserWithEntityUser( + static boolean validateAuthUserWithEntityUser( TimelineReaderManager readerManager, UserGroupInformation ugi, String entityUser) { String authUser = TimelineReaderWebServicesUtils.getUserName(ugi); @@ -3442,4 +3527,41 @@ public class TimelineReaderWebServices { } return (readerManager.checkAccess(ugi) || authUser.equals(requestedUser)); } + + // TODO to be removed/modified once ACL story has played + static boolean checkAccess(TimelineReaderManager readerManager, + UserGroupInformation ugi, String entityUser) { + if (isDisplayEntityPerUserFilterEnabled(readerManager.getConfig())) { + if (!validateAuthUserWithEntityUser(readerManager, ugi, entityUser)) { + String userName = ugi.getShortUserName(); + String msg = "User " + userName + + " is not allowed to read TimelineService V2 data."; + throw new ForbiddenException(msg); + } + } + return true; + } + + // TODO to be removed or modified once ACL story is played + static void checkAccess(TimelineReaderManager readerManager, + UserGroupInformation callerUGI, Set entities, + String entityUserKey, boolean verifyForAllEntity) { + if (entities.size() > 0 && isDisplayEntityPerUserFilterEnabled( + readerManager.getConfig())) { + Set userEntities = new LinkedHashSet<>(); + userEntities.addAll(entities); + for (TimelineEntity entity : userEntities) { + if (entity.getInfo() != null) { + String userId = (String) entity.getInfo().get(entityUserKey); + if (!validateAuthUserWithEntityUser(readerManager, callerUGI, + userId)) { + entities.remove(entity); + if (!verifyForAllEntity) { + break; + } + } + } + } + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesBasicAcl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesBasicAcl.java new file mode 100644 index 00000000000..4239bf0460d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesBasicAcl.java @@ -0,0 +1,154 @@ +/** + * 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.yarn.server.timelineservice.reader; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.webapp.ForbiddenException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.LinkedHashSet; +import java.util.Set; + +public class TestTimelineReaderWebServicesBasicAcl { + + private TimelineReaderManager manager; + private static String adminUser = "admin"; + private static UserGroupInformation adminUgi = + UserGroupInformation.createRemoteUser(adminUser); + private Configuration config; + + @Before public void setUp() throws Exception { + config = new YarnConfiguration(); + } + + @After public void tearDown() throws Exception { + if (manager != null) { + manager.stop(); + manager = null; + } + config = null; + } + + @Test public void testTimelineReaderManagerAclsWhenDisabled() + throws Exception { + config.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, false); + config.set(YarnConfiguration.YARN_ADMIN_ACL, adminUser); + manager = new TimelineReaderManager(null); + manager.init(config); + manager.start(); + + // when acls are disabled, always return true + Assert.assertTrue(manager.checkAccess(null)); + + // filter is disabled, so should return false + Assert.assertFalse( + TimelineReaderWebServices.isDisplayEntityPerUserFilterEnabled(config)); + } + + @Test public void testTimelineReaderManagerAclsWhenEnabled() + throws Exception { + Configuration config = new YarnConfiguration(); + config.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true); + config.setBoolean(YarnConfiguration.FILTER_ENTITY_LIST_BY_USER, true); + config.set(YarnConfiguration.YARN_ADMIN_ACL, adminUser); + manager = new TimelineReaderManager(null); + manager.init(config); + manager.start(); + + String user1 = "user1"; + String user2 = "user2"; + UserGroupInformation user1Ugi = + UserGroupInformation.createRemoteUser(user1); + UserGroupInformation user2Ugi = + UserGroupInformation.createRemoteUser(user2); + + // false because ugi is null + Assert.assertFalse(TimelineReaderWebServices + .validateAuthUserWithEntityUser(manager, null, user1)); + + // incoming ugi is admin asking for entity owner user1 + Assert.assertTrue( + TimelineReaderWebServices.checkAccess(manager, adminUgi, user1)); + + // incoming ugi is admin asking for entity owner user1 + Assert.assertTrue( + TimelineReaderWebServices.checkAccess(manager, adminUgi, user2)); + + // incoming ugi is non-admin i.e user1Ugi asking for entity owner user2 + try { + TimelineReaderWebServices.checkAccess(manager, user1Ugi, user2); + Assert.fail("user1Ugi is not allowed to view user2"); + } catch (ForbiddenException e) { + // expected + } + + // incoming ugi is non-admin i.e user2Ugi asking for entity owner user1 + try { + TimelineReaderWebServices.checkAccess(manager, user1Ugi, user2); + Assert.fail("user2Ugi is not allowed to view user1"); + } catch (ForbiddenException e) { + // expected + } + + String userKey = "user"; + // incoming ugi is admin asking for entities + Set entities = createEntities(10, userKey); + TimelineReaderWebServices + .checkAccess(manager, adminUgi, entities, userKey, true); + // admin is allowed to view other entities + Assert.assertTrue(entities.size() == 10); + + // incoming ugi is user1Ugi asking for entities + // only user1 entities are allowed to view + entities = createEntities(5, userKey); + TimelineReaderWebServices + .checkAccess(manager, user1Ugi, entities, userKey, true); + Assert.assertTrue(entities.size() == 1); + Assert + .assertEquals(user1, entities.iterator().next().getInfo().get(userKey)); + + // incoming ugi is user2Ugi asking for entities + // only user2 entities are allowed to view + entities = createEntities(8, userKey); + TimelineReaderWebServices + .checkAccess(manager, user2Ugi, entities, userKey, true); + Assert.assertTrue(entities.size() == 1); + Assert + .assertEquals(user2, entities.iterator().next().getInfo().get(userKey)); + } + + Set createEntities(int noOfUsers, String userKey) { + Set entities = new LinkedHashSet<>(); + for (int i = 0; i < noOfUsers; i++) { + TimelineEntity e = new TimelineEntity(); + e.setType("user" + i); + e.setId("user" + i); + e.getInfo().put(userKey, "user" + i); + entities.add(e); + } + return entities; + } + +} From a820738e347854aed7cdaa7758d9c0830df624e3 Mon Sep 17 00:00:00 2001 From: Rohith Sharma K S Date: Fri, 29 Jun 2018 10:21:25 -0700 Subject: [PATCH 05/47] YARN-8469. [UI2] URL needs to be trimmed to handle index.html redirection while accessing via knox. Contributed by Sunil Govindan. --- .../src/main/webapp/app/initializers/loader.js | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js index 66428697758..96c8fcfff90 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js @@ -205,5 +205,13 @@ export default { const skipTrailingSlash = function(path) { path = path.replace('ui2/', ''); - return path.replace(/\/$/, ''); + path = path.replace(/\/$/, ''); + console.log('base url:' + path) + if(path.includes("redirect")) { + var to = path.lastIndexOf('/'); + to = to == -1 ? path.length : to + 1; + path = path.substring(0, to); + console.log('base url after redirect:' + path) + } + return path; }; From 100470140d86eede0fa240a9aa93226f274ee4f5 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 29 Jun 2018 13:06:28 -0500 Subject: [PATCH 06/47] YARN-8451. Multiple NM heartbeat thread created when a slow NM resync with RM. Contributed by Botong Huang --- .../yarn/server/nodemanager/NodeManager.java | 66 ++++++++++++------- .../nodemanager/TestNodeManagerResync.java | 56 ++++++++++++++++ 2 files changed, 98 insertions(+), 24 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java index 2748a8fb568..c8234bd2d47 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java @@ -128,6 +128,7 @@ public class NodeManager extends CompositeService // the NM collector service is set only if the timeline service v.2 is enabled private NMCollectorService nmCollectorService; private NodeStatusUpdater nodeStatusUpdater; + private AtomicBoolean resyncingWithRM = new AtomicBoolean(false); private NodeResourceMonitor nodeResourceMonitor; private static CompositeServiceShutdownHook nodeManagerShutdownHook; private NMStateStoreService nmStore = null; @@ -393,7 +394,7 @@ public class NodeManager extends CompositeService addService(del); // NodeManager level dispatcher - this.dispatcher = new AsyncDispatcher("NM Event dispatcher"); + this.dispatcher = createNMDispatcher(); nodeHealthChecker = new NodeHealthCheckerService( @@ -517,31 +518,41 @@ public class NodeManager extends CompositeService } protected void resyncWithRM() { - //we do not want to block dispatcher thread here - new Thread() { - @Override - public void run() { - try { - if (!rmWorkPreservingRestartEnabled) { - LOG.info("Cleaning up running containers on resync"); - containerManager.cleanupContainersOnNMResync(); - // Clear all known collectors for resync. - if (context.getKnownCollectors() != null) { - context.getKnownCollectors().clear(); + // Create a thread for resync because we do not want to block dispatcher + // thread here. Also use locking to make sure only one thread is running at + // a time. + if (this.resyncingWithRM.getAndSet(true)) { + // Some other thread is already created for resyncing, do nothing + } else { + // We have got the lock, create a new thread + new Thread() { + @Override + public void run() { + try { + if (!rmWorkPreservingRestartEnabled) { + LOG.info("Cleaning up running containers on resync"); + containerManager.cleanupContainersOnNMResync(); + // Clear all known collectors for resync. + if (context.getKnownCollectors() != null) { + context.getKnownCollectors().clear(); + } + } else { + LOG.info("Preserving containers on resync"); + // Re-register known timeline collectors. + reregisterCollectors(); } - } else { - LOG.info("Preserving containers on resync"); - // Re-register known timeline collectors. - reregisterCollectors(); + ((NodeStatusUpdaterImpl) nodeStatusUpdater) + .rebootNodeStatusUpdaterAndRegisterWithRM(); + } catch (YarnRuntimeException e) { + LOG.error("Error while rebooting NodeStatusUpdater.", e); + shutDown(NodeManagerStatus.EXCEPTION.getExitCode()); + } finally { + // Release lock + resyncingWithRM.set(false); } - ((NodeStatusUpdaterImpl) nodeStatusUpdater) - .rebootNodeStatusUpdaterAndRegisterWithRM(); - } catch (YarnRuntimeException e) { - LOG.error("Error while rebooting NodeStatusUpdater.", e); - shutDown(NodeManagerStatus.EXCEPTION.getExitCode()); } - } - }.start(); + }.start(); + } } /** @@ -946,7 +957,14 @@ public class NodeManager extends CompositeService ContainerManagerImpl getContainerManager() { return containerManager; } - + + /** + * Unit test friendly. + */ + protected AsyncDispatcher createNMDispatcher() { + return new AsyncDispatcher("NM Event dispatcher"); + } + //For testing Dispatcher getNMDispatcher(){ return dispatcher; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java index cf33775fcf2..b3f4e1bcb86 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java @@ -37,6 +37,7 @@ import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.Path; @@ -64,7 +65,9 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factories.RecordFactory; @@ -107,6 +110,7 @@ public class TestNodeManagerResync { private FileContext localFS; private CyclicBarrier syncBarrier; private CyclicBarrier updateBarrier; + private AtomicInteger resyncThreadCount; private AtomicBoolean assertionFailedInThread = new AtomicBoolean(false); private AtomicBoolean isNMShutdownCalled = new AtomicBoolean(false); private final NodeManagerEvent resyncEvent = @@ -125,6 +129,7 @@ public class TestNodeManagerResync { nmLocalDir.mkdirs(); syncBarrier = new CyclicBarrier(2); updateBarrier = new CyclicBarrier(2); + resyncThreadCount = new AtomicInteger(0); } @After @@ -185,6 +190,41 @@ public class TestNodeManagerResync { } } + @SuppressWarnings("resource") + @Test(timeout = 30000) + public void testNMMultipleResyncEvent() + throws IOException, InterruptedException { + TestNodeManager1 nm = new TestNodeManager1(false); + YarnConfiguration conf = createNMConfig(); + + int resyncEventCount = 4; + try { + nm.init(conf); + nm.start(); + Assert.assertEquals(1, nm.getNMRegistrationCount()); + for (int i = 0; i < resyncEventCount; i++) { + nm.getNMDispatcher().getEventHandler().handle(resyncEvent); + } + + DrainDispatcher dispatcher = (DrainDispatcher) nm.getNMDispatcher(); + dispatcher.await(); + LOG.info("NM dispatcher drained"); + + // Wait for the resync thread to finish + try { + syncBarrier.await(); + } catch (BrokenBarrierException e) { + } + LOG.info("Barrier wait done for the resync thread"); + + // Resync should only happen once + Assert.assertEquals(2, nm.getNMRegistrationCount()); + Assert.assertFalse("NM shutdown called.", isNMShutdownCalled.get()); + } finally { + nm.stop(); + } + } + @SuppressWarnings("resource") @Test(timeout=10000) public void testNMshutdownWhenResyncThrowException() throws IOException, @@ -399,6 +439,11 @@ public class TestNodeManagerResync { existingCid = cId; } + @Override + protected AsyncDispatcher createNMDispatcher() { + return new DrainDispatcher(); + } + @Override protected NodeStatusUpdater createNodeStatusUpdater(Context context, Dispatcher dispatcher, NodeHealthCheckerService healthChecker) { @@ -410,6 +455,14 @@ public class TestNodeManagerResync { return registrationCount; } + @Override + protected void shutDown(int exitCode) { + synchronized (isNMShutdownCalled) { + isNMShutdownCalled.set(true); + isNMShutdownCalled.notify(); + } + } + class TestNodeStatusUpdaterImpl1 extends MockNodeStatusUpdater { public TestNodeStatusUpdaterImpl1(Context context, Dispatcher dispatcher, @@ -428,6 +481,9 @@ public class TestNodeManagerResync { ConcurrentMap containers = getNMContext().getContainers(); + if (resyncThreadCount.incrementAndGet() > 1) { + throw new YarnRuntimeException("Multiple resync thread created!"); + } try { try { if (containersShouldBePreserved) { From d36f6b9e93e4c30d24d0e837cb00bd24ffa8f274 Mon Sep 17 00:00:00 2001 From: Eric E Payne Date: Fri, 29 Jun 2018 18:18:32 +0000 Subject: [PATCH 07/47] HADOOP-15548: Randomize local dirs. Contributed by Jim Brennan. --- .../apache/hadoop/fs/LocalDirAllocator.java | 7 ++- .../hadoop/fs/TestLocalDirAllocator.java | 59 +++++++++++++++++++ 2 files changed, 65 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java index c1e9d21ecc7..1c216f430af 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java @@ -418,7 +418,12 @@ public class LocalDirAllocator { } } } else { - int dirNum = ctx.getAndIncrDirNumLastAccessed(); + // Start linear search with random increment if possible + int randomInc = 1; + if (numDirs > 2) { + randomInc += dirIndexRandomizer.nextInt(numDirs - 1); + } + int dirNum = ctx.getAndIncrDirNumLastAccessed(randomInc); while (numDirsSearched < numDirs) { long capacity = ctx.dirDF[dirNum].getAvailable(); if (capacity > size) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java index 825efe046da..acda898ea13 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java @@ -265,6 +265,65 @@ public class TestLocalDirAllocator { } } + /** + * Five buffer dirs, on read-write disk. + * + * Try to create a whole bunch of files. + * Verify that each successive creation uses a different disk + * than the previous one (for sized requests). + * + * Would ideally check statistical properties of distribution, but + * we don't have the nerve to risk false-positives here. + * + * @throws Exception + */ + @Test (timeout = 30000) + public void testCreateManyFilesRandom() throws Exception { + assumeNotWindows(); + final int numDirs = 5; + final int numTries = 100; + String[] dirs = new String[numDirs]; + for (int d = 0; d < numDirs; ++d) { + dirs[d] = buildBufferDir(ROOT, d); + } + boolean next_dir_not_selected_at_least_once = false; + try { + conf.set(CONTEXT, dirs[0] + "," + dirs[1] + "," + dirs[2] + "," + + dirs[3] + "," + dirs[4]); + Path[] paths = new Path[5]; + for (int d = 0; d < numDirs; ++d) { + paths[d] = new Path(dirs[d]); + assertTrue(localFs.mkdirs(paths[d])); + } + + int inDir=0; + int prevDir = -1; + int[] counts = new int[5]; + for(int i = 0; i < numTries; ++i) { + File result = createTempFile(SMALL_FILE_SIZE); + for (int d = 0; d < numDirs; ++d) { + if (result.getPath().startsWith(paths[d].toUri().getPath())) { + inDir = d; + break; + } + } + // Verify we always select a different dir + assertNotEquals(prevDir, inDir); + // Verify we are not always selecting the next dir - that was the old + // algorithm. + if ((prevDir != -1) && (inDir != ((prevDir + 1) % numDirs))) { + next_dir_not_selected_at_least_once = true; + } + prevDir = inDir; + counts[inDir]++; + result.delete(); + } + } finally { + rmBufferDirs(); + } + assertTrue(next_dir_not_selected_at_least_once); + } + /** Two buffer dirs. The first dir does not exist & is on a read-only disk; * The second dir exists & is RW * getLocalPathForWrite with checkAccess set to false should create a parent From cdb084426bc27a9f902da0b6927a3354a307dd82 Mon Sep 17 00:00:00 2001 From: Giovanni Matteo Fumarola Date: Fri, 29 Jun 2018 11:47:30 -0700 Subject: [PATCH 08/47] YARN-8481. AMRMProxyPolicies should accept heartbeat response from new/unknown subclusters. Contributed by Botong Huang. --- .../policies/amrmproxy/BroadcastAMRMProxyPolicy.java | 11 ----------- .../policies/amrmproxy/RejectAMRMProxyPolicy.java | 4 ---- .../TestBroadcastAMRMProxyFederationPolicy.java | 11 +++-------- 3 files changed, 3 insertions(+), 23 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/BroadcastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/BroadcastAMRMProxyPolicy.java index 679f4d5fa41..7fddb8ea952 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/BroadcastAMRMProxyPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/BroadcastAMRMProxyPolicy.java @@ -19,10 +19,8 @@ package org.apache.hadoop.yarn.server.federation.policies.amrmproxy; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.records.ResourceRequest; @@ -30,7 +28,6 @@ import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext; import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContextValidator; import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException; -import org.apache.hadoop.yarn.server.federation.policies.exceptions.UnknownSubclusterException; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo; @@ -40,8 +37,6 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo; */ public class BroadcastAMRMProxyPolicy extends AbstractAMRMProxyPolicy { - private Set knownClusterIds = new HashSet<>(); - @Override public void reinitialize( FederationPolicyInitializationContext policyContext) @@ -65,7 +60,6 @@ public class BroadcastAMRMProxyPolicy extends AbstractAMRMProxyPolicy { // simply broadcast the resource request to all sub-clusters for (SubClusterId subClusterId : activeSubclusters.keySet()) { answer.put(subClusterId, resourceRequests); - knownClusterIds.add(subClusterId); } return answer; @@ -74,11 +68,6 @@ public class BroadcastAMRMProxyPolicy extends AbstractAMRMProxyPolicy { @Override public void notifyOfResponse(SubClusterId subClusterId, AllocateResponse response) throws YarnException { - if (!knownClusterIds.contains(subClusterId)) { - throw new UnknownSubclusterException( - "The response is received from a subcluster that is unknown to this " - + "policy."); - } // stateless policy does not care about responses } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/RejectAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/RejectAMRMProxyPolicy.java index 3783df645c5..450060671c7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/RejectAMRMProxyPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/RejectAMRMProxyPolicy.java @@ -18,10 +18,8 @@ package org.apache.hadoop.yarn.server.federation.policies.amrmproxy; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.records.ResourceRequest; @@ -38,8 +36,6 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; */ public class RejectAMRMProxyPolicy extends AbstractAMRMProxyPolicy { - private Set knownClusterIds = new HashSet<>(); - @Override public void reinitialize(FederationPolicyInitializationContext policyContext) throws FederationPolicyInitializationException { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestBroadcastAMRMProxyFederationPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestBroadcastAMRMProxyFederationPolicy.java index a21f53dc924..df5da85a915 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestBroadcastAMRMProxyFederationPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestBroadcastAMRMProxyFederationPolicy.java @@ -89,7 +89,7 @@ public class TestBroadcastAMRMProxyFederationPolicy } @Test - public void testNotifyOfResponse() throws Exception { + public void testNotifyOfResponseFromUnknownSubCluster() throws Exception { String[] hosts = new String[] {"host1", "host2" }; List resourceRequests = FederationPoliciesTestUtil .createResourceRequests(hosts, 2 * 1024, 2, 1, 3, null, false); @@ -97,13 +97,8 @@ public class TestBroadcastAMRMProxyFederationPolicy ((FederationAMRMProxyPolicy) getPolicy()) .splitResourceRequests(resourceRequests); - try { - ((FederationAMRMProxyPolicy) getPolicy()).notifyOfResponse( - SubClusterId.newInstance("sc3"), mock(AllocateResponse.class)); - Assert.fail(); - } catch (FederationPolicyException f) { - System.out.println("Expected: " + f.getMessage()); - } + ((FederationAMRMProxyPolicy) getPolicy()).notifyOfResponse( + SubClusterId.newInstance("sc3"), mock(AllocateResponse.class)); ((FederationAMRMProxyPolicy) getPolicy()).notifyOfResponse( SubClusterId.newInstance("sc1"), mock(AllocateResponse.class)); From d40121845e0e1ace6b349180ced53292353dbb7c Mon Sep 17 00:00:00 2001 From: Nanda kumar Date: Sun, 1 Jul 2018 17:11:46 +0530 Subject: [PATCH 09/47] HADOOP-15574: Suppress build error if there are no docs after excluding private annotations. Contributed by Takanobu Asanuma. --- .../tools/ExcludePrivateAnnotationsStandardDoclet.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/ExcludePrivateAnnotationsStandardDoclet.java b/hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/ExcludePrivateAnnotationsStandardDoclet.java index 2176ea55d6f..5c535c8e9e3 100644 --- a/hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/ExcludePrivateAnnotationsStandardDoclet.java +++ b/hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/ExcludePrivateAnnotationsStandardDoclet.java @@ -38,7 +38,11 @@ public class ExcludePrivateAnnotationsStandardDoclet { public static boolean start(RootDoc root) { System.out.println( ExcludePrivateAnnotationsStandardDoclet.class.getSimpleName()); - return Standard.start(RootDocProcessor.process(root)); + RootDoc excludedDoc = RootDocProcessor.process(root); + if (excludedDoc.specifiedPackages().length == 0) { + return true; + } + return Standard.start(excludedDoc); } public static int optionLength(String option) { From 6ba99741086170b83c38d3e7e715d9e8046a1e00 Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Mon, 2 Jul 2018 12:02:19 +0200 Subject: [PATCH 10/47] HDFS-13703. Avoid allocation of CorruptedBlocks hashmap when no corrupted blocks are hit. Contributed by Todd Lipcon. --- .../java/org/apache/hadoop/hdfs/DFSInputStream.java | 2 +- .../java/org/apache/hadoop/hdfs/DFSUtilClient.java | 11 ++++++----- .../apache/hadoop/hdfs/server/datanode/DataNode.java | 2 +- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index 573b860fab1..1bdc50a5e27 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -1423,7 +1423,7 @@ public class DFSInputStream extends FSInputStream Map> corruptedBlockMap = corruptedBlocks.getCorruptionMap(); - if (corruptedBlockMap.isEmpty()) { + if (corruptedBlockMap == null) { return; } List reportList = new ArrayList<>(corruptedBlockMap.size()); diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java index 6c0b106ab21..313b973550c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java @@ -751,14 +751,14 @@ public class DFSUtilClient { public static class CorruptedBlocks { private Map> corruptionMap; - public CorruptedBlocks() { - this.corruptionMap = new HashMap<>(); - } - /** * Indicate a block replica on the specified datanode is corrupted */ public void addCorruptedBlock(ExtendedBlock blk, DatanodeInfo node) { + if (corruptionMap == null) { + corruptionMap = new HashMap<>(); + } + Set dnSet = corruptionMap.get(blk); if (dnSet == null) { dnSet = new HashSet<>(); @@ -770,7 +770,8 @@ public class DFSUtilClient { } /** - * @return the map that contains all the corruption entries. + * @return the map that contains all the corruption entries, or null if + * there were no corrupted entries */ public Map> getCorruptionMap() { return corruptionMap; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index 96b0f36f9e6..4baafb92847 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -1278,7 +1278,7 @@ public class DataNode extends ReconfigurableBase DFSUtilClient.CorruptedBlocks corruptedBlocks) throws IOException { Map> corruptionMap = corruptedBlocks.getCorruptionMap(); - if (!corruptionMap.isEmpty()) { + if (corruptionMap != null) { for (Map.Entry> entry : corruptionMap.entrySet()) { for (DatanodeInfo dnInfo : entry.getValue()) { From 5d748bd056a32f2c6922514cd0c5b31d866a9919 Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Mon, 2 Jul 2018 12:11:06 +0200 Subject: [PATCH 11/47] HDFS-13702. Remove HTrace hooks from DFSClient to reduce CPU usage. Contributed by Todd Lipcon. --- .../org/apache/hadoop/hdfs/DFSClient.java | 19 ---- .../apache/hadoop/hdfs/DFSInputStream.java | 44 ++------- .../hdfs/client/impl/BlockReaderFactory.java | 16 +-- .../hdfs/client/impl/BlockReaderLocal.java | 97 ++++++++----------- .../client/impl/BlockReaderLocalLegacy.java | 44 ++++----- .../hdfs/client/impl/BlockReaderRemote.java | 19 +--- .../erasurecode/StripedBlockReader.java | 2 +- .../hdfs/server/namenode/NamenodeFsck.java | 1 - .../hdfs/client/impl/BlockReaderTestUtil.java | 2 - .../client/impl/TestBlockReaderLocal.java | 2 - .../TestBlockTokenWithDFS.java | 2 - .../datanode/TestDataNodeVolumeFailure.java | 2 - 12 files changed, 70 insertions(+), 180 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 96c45053fc4..85d65124fa0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -3070,25 +3070,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory, return scope; } - /** - * Full detailed tracing for read requests: path, position in the file, - * and length. - * - * @param reqLen requested length - */ - TraceScope newReaderTraceScope(String description, String path, long pos, - int reqLen) { - TraceScope scope = newPathTraceScope(description, path); - scope.addKVAnnotation("pos", Long.toString(pos)); - scope.addKVAnnotation("reqLen", Integer.toString(reqLen)); - return scope; - } - - /** Add the returned length info to the scope. */ - void addRetLenToReaderScope(TraceScope scope, int retLen) { - scope.addKVAnnotation("retLen", Integer.toString(retLen)); - } - /** * Get the erasure coding policy information for the specified path * diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index 1bdc50a5e27..e5640d2ba6a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -85,8 +85,6 @@ import org.apache.hadoop.util.IdentityHashStore; import org.apache.hadoop.util.StopWatch; import org.apache.hadoop.util.StringUtils; import org.apache.htrace.core.SpanId; -import org.apache.htrace.core.TraceScope; -import org.apache.htrace.core.Tracer; import com.google.common.annotations.VisibleForTesting; @@ -641,7 +639,6 @@ public class DFSInputStream extends FSInputStream setClientCacheContext(dfsClient.getClientContext()). setUserGroupInformation(dfsClient.ugi). setConfiguration(dfsClient.getConfiguration()). - setTracer(dfsClient.getTracer()). build(); } @@ -821,31 +818,14 @@ public class DFSInputStream extends FSInputStream } ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf, off, len, readStatistics, dfsClient); - try (TraceScope scope = - dfsClient.newReaderTraceScope("DFSInputStream#byteArrayRead", - src, getPos(), len)) { - int retLen = readWithStrategy(byteArrayReader); - if (retLen < len) { - dfsClient.addRetLenToReaderScope(scope, retLen); - } - return retLen; - } + return readWithStrategy(byteArrayReader); } @Override public synchronized int read(final ByteBuffer buf) throws IOException { ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf, readStatistics, dfsClient); - int reqLen = buf.remaining(); - try (TraceScope scope = - dfsClient.newReaderTraceScope("DFSInputStream#byteBufferRead", - src, getPos(), reqLen)){ - int retLen = readWithStrategy(byteBufferReader); - if (retLen < reqLen) { - dfsClient.addRetLenToReaderScope(scope, retLen); - } - return retLen; - } + return readWithStrategy(byteBufferReader); } private DNAddrPair chooseDataNode(LocatedBlock block, @@ -1026,16 +1006,12 @@ public class DFSInputStream extends FSInputStream final ByteBuffer bb, final CorruptedBlocks corruptedBlocks, final int hedgedReadId) { - final SpanId parentSpanId = Tracer.getCurrentSpanId(); return new Callable() { @Override public ByteBuffer call() throws Exception { DFSClientFaultInjector.get().sleepBeforeHedgedGet(); - try (TraceScope ignored = dfsClient.getTracer(). - newScope("hedgedRead" + hedgedReadId, parentSpanId)) { - actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks); - return bb; - } + actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks); + return bb; } }; } @@ -1336,16 +1312,8 @@ public class DFSInputStream extends FSInputStream if (length == 0) { return 0; } - try (TraceScope scope = dfsClient. - newReaderTraceScope("DFSInputStream#byteArrayPread", - src, position, length)) { - ByteBuffer bb = ByteBuffer.wrap(buffer, offset, length); - int retLen = pread(position, bb); - if (retLen < length) { - dfsClient.addRetLenToReaderScope(scope, retLen); - } - return retLen; - } + ByteBuffer bb = ByteBuffer.wrap(buffer, offset, length); + return pread(position, bb); } private int pread(long position, ByteBuffer buffer) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java index a8c73a42200..1003b957c64 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java @@ -75,7 +75,6 @@ import org.apache.hadoop.util.Time; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import org.apache.htrace.core.Tracer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -189,11 +188,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { */ private Configuration configuration; - /** - * The HTrace tracer to use. - */ - private Tracer tracer; - /** * Information about the domain socket path we should use to connect to the * local peer-- or null if we haven't examined the local domain socket. @@ -298,11 +292,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { return this; } - public BlockReaderFactory setTracer(Tracer tracer) { - this.tracer = tracer; - return this; - } - @VisibleForTesting public static void setFailureInjectorForTesting(FailureInjector injector) { failureInjector = injector; @@ -451,7 +440,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { try { return BlockReaderLocalLegacy.newBlockReader(conf, userGroupInformation, configuration, fileName, block, token, - datanode, startOffset, length, storageType, tracer); + datanode, startOffset, length, storageType); } catch (RemoteException remoteException) { ioe = remoteException.unwrapRemoteException( InvalidToken.class, AccessControlException.class); @@ -509,7 +498,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { setVerifyChecksum(verifyChecksum). setCachingStrategy(cachingStrategy). setStorageType(storageType). - setTracer(tracer). build(); } @@ -860,7 +848,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { return BlockReaderRemote.newBlockReader( fileName, block, token, startOffset, length, verifyChecksum, clientName, peer, datanode, - clientContext.getPeerCache(), cachingStrategy, tracer, + clientContext.getPeerCache(), cachingStrategy, networkDistance); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocal.java index df0f65fc2e2..9c1ef461b04 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocal.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocal.java @@ -35,8 +35,6 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DirectBufferPool; import org.apache.hadoop.util.Timer; -import org.apache.htrace.core.TraceScope; -import org.apache.htrace.core.Tracer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,7 +81,6 @@ class BlockReaderLocal implements BlockReader { private long dataPos; private ExtendedBlock block; private StorageType storageType; - private Tracer tracer; private ShortCircuitConf shortCircuitConf; public Builder(ShortCircuitConf conf) { @@ -131,11 +128,6 @@ class BlockReaderLocal implements BlockReader { return this; } - public Builder setTracer(Tracer tracer) { - this.tracer = tracer; - return this; - } - public BlockReaderLocal build() { Preconditions.checkNotNull(replica); return new BlockReaderLocal(this); @@ -244,11 +236,6 @@ class BlockReaderLocal implements BlockReader { */ private StorageType storageType; - /** - * The Tracer to use. - */ - private final Tracer tracer; - private BlockReaderLocal(Builder builder) { this.replica = builder.replica; this.dataIn = replica.getDataStream().getChannel(); @@ -278,7 +265,6 @@ class BlockReaderLocal implements BlockReader { } this.maxReadaheadLength = maxReadaheadChunks * bytesPerChecksum; this.storageType = builder.storageType; - this.tracer = builder.tracer; if (builder.shortCircuitConf.isScrMetricsEnabled()) { metricsInitializationLock.lock(); @@ -360,52 +346,49 @@ class BlockReaderLocal implements BlockReader { */ private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum) throws IOException { - try (TraceScope ignored = tracer.newScope( - "BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")")) { - int total = 0; - long startDataPos = dataPos; - int startBufPos = buf.position(); - while (buf.hasRemaining()) { - int nRead = blockReaderIoProvider.read(dataIn, buf, dataPos); - if (nRead < 0) { - break; - } - dataPos += nRead; - total += nRead; - } - if (canSkipChecksum) { - freeChecksumBufIfExists(); - return total; - } - if (total > 0) { - try { - buf.limit(buf.position()); - buf.position(startBufPos); - createChecksumBufIfNeeded(); - int checksumsNeeded = (total + bytesPerChecksum - 1) / - bytesPerChecksum; - checksumBuf.clear(); - checksumBuf.limit(checksumsNeeded * checksumSize); - long checksumPos = BlockMetadataHeader.getHeaderSize() - + ((startDataPos / bytesPerChecksum) * checksumSize); - while (checksumBuf.hasRemaining()) { - int nRead = checksumIn.read(checksumBuf, checksumPos); - if (nRead < 0) { - throw new IOException("Got unexpected checksum file EOF at " + - checksumPos + ", block file position " + startDataPos + - " for block " + block + " of file " + filename); - } - checksumPos += nRead; - } - checksumBuf.flip(); - - checksum.verifyChunkedSums(buf, checksumBuf, filename, startDataPos); - } finally { - buf.position(buf.limit()); - } + int total = 0; + long startDataPos = dataPos; + int startBufPos = buf.position(); + while (buf.hasRemaining()) { + int nRead = blockReaderIoProvider.read(dataIn, buf, dataPos); + if (nRead < 0) { + break; } + dataPos += nRead; + total += nRead; + } + if (canSkipChecksum) { + freeChecksumBufIfExists(); return total; } + if (total > 0) { + try { + buf.limit(buf.position()); + buf.position(startBufPos); + createChecksumBufIfNeeded(); + int checksumsNeeded = (total + bytesPerChecksum - 1) / + bytesPerChecksum; + checksumBuf.clear(); + checksumBuf.limit(checksumsNeeded * checksumSize); + long checksumPos = BlockMetadataHeader.getHeaderSize() + + ((startDataPos / bytesPerChecksum) * checksumSize); + while (checksumBuf.hasRemaining()) { + int nRead = checksumIn.read(checksumBuf, checksumPos); + if (nRead < 0) { + throw new IOException("Got unexpected checksum file EOF at " + + checksumPos + ", block file position " + startDataPos + + " for block " + block + " of file " + filename); + } + checksumPos += nRead; + } + checksumBuf.flip(); + + checksum.verifyChunkedSums(buf, checksumBuf, filename, startDataPos); + } finally { + buf.position(buf.limit()); + } + } + return total; } private boolean createNoChecksumContext() { diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocalLegacy.java index 7d20a83b2ec..e1e38c6e943 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocalLegacy.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocalLegacy.java @@ -51,8 +51,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DirectBufferPool; -import org.apache.htrace.core.TraceScope; -import org.apache.htrace.core.Tracer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -184,7 +182,6 @@ class BlockReaderLocalLegacy implements BlockReader { private long startOffset; private final String filename; private long blockId; - private final Tracer tracer; /** * The only way this object can be instantiated. @@ -193,8 +190,8 @@ class BlockReaderLocalLegacy implements BlockReader { UserGroupInformation userGroupInformation, Configuration configuration, String file, ExtendedBlock blk, Token token, DatanodeInfo node, - long startOffset, long length, StorageType storageType, - Tracer tracer) throws IOException { + long startOffset, long length, StorageType storageType) + throws IOException { final ShortCircuitConf scConf = conf.getShortCircuitConf(); LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node .getIpcPort()); @@ -239,11 +236,10 @@ class BlockReaderLocalLegacy implements BlockReader { long firstChunkOffset = startOffset - (startOffset % checksum.getBytesPerChecksum()); localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, - startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn, - tracer); + startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn); } else { localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, - startOffset, dataIn, tracer); + startOffset, dataIn); } } catch (IOException e) { // remove from cache @@ -320,17 +316,17 @@ class BlockReaderLocalLegacy implements BlockReader { } private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile, - ExtendedBlock block, long startOffset, FileInputStream dataIn, - Tracer tracer) throws IOException { + ExtendedBlock block, long startOffset, FileInputStream dataIn) + throws IOException { this(conf, hdfsfile, block, startOffset, DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false, - dataIn, startOffset, null, tracer); + dataIn, startOffset, null); } private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile, ExtendedBlock block, long startOffset, DataChecksum checksum, boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset, - FileInputStream checksumIn, Tracer tracer) throws IOException { + FileInputStream checksumIn) throws IOException { this.filename = hdfsfile; this.checksum = checksum; this.verifyChecksum = verifyChecksum; @@ -369,7 +365,6 @@ class BlockReaderLocalLegacy implements BlockReader { bufferPool.returnBuffer(checksumBuff); } } - this.tracer = tracer; } /** @@ -377,23 +372,20 @@ class BlockReaderLocalLegacy implements BlockReader { */ private int fillBuffer(FileInputStream stream, ByteBuffer buf) throws IOException { - try (TraceScope ignored = tracer. - newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")")) { - int bytesRead = stream.getChannel().read(buf); - if (bytesRead < 0) { + int bytesRead = stream.getChannel().read(buf); + if (bytesRead < 0) { + //EOF + return bytesRead; + } + while (buf.remaining() > 0) { + int n = stream.getChannel().read(buf); + if (n < 0) { //EOF return bytesRead; } - while (buf.remaining() > 0) { - int n = stream.getChannel().read(buf); - if (n < 0) { - //EOF - return bytesRead; - } - bytesRead += n; - } - return bytesRead; + bytesRead += n; } + return bytesRead; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote.java index 5a2ce40a682..caf15e41a0d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote.java @@ -49,11 +49,9 @@ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.shortcircuit.ClientMmap; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; -import org.apache.htrace.core.TraceScope; import com.google.common.annotations.VisibleForTesting; -import org.apache.htrace.core.Tracer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -121,8 +119,6 @@ public class BlockReaderRemote implements BlockReader { private boolean sentStatusCode = false; - private final Tracer tracer; - private final int networkDistance; @VisibleForTesting @@ -139,10 +135,7 @@ public class BlockReaderRemote implements BlockReader { if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) { - try (TraceScope ignored = tracer.newScope( - "BlockReaderRemote2#readNextPacket(" + blockId + ")")) { - readNextPacket(); - } + readNextPacket(); } LOG.trace("Finishing read #{}", randomId); @@ -163,10 +156,7 @@ public class BlockReaderRemote implements BlockReader { public synchronized int read(ByteBuffer buf) throws IOException { if (curDataSlice == null || (curDataSlice.remaining() == 0 && bytesNeededToFinish > 0)) { - try (TraceScope ignored = tracer.newScope( - "BlockReaderRemote2#readNextPacket(" + blockId + ")")) { - readNextPacket(); - } + readNextPacket(); } if (curDataSlice.remaining() == 0) { // we're at EOF now @@ -280,7 +270,6 @@ public class BlockReaderRemote implements BlockReader { long startOffset, long firstChunkOffset, long bytesToRead, Peer peer, DatanodeID datanodeID, PeerCache peerCache, - Tracer tracer, int networkDistance) { // Path is used only for printing block and file information in debug this.peer = peer; @@ -300,7 +289,6 @@ public class BlockReaderRemote implements BlockReader { this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset); bytesPerChecksum = this.checksum.getBytesPerChecksum(); checksumSize = this.checksum.getChecksumSize(); - this.tracer = tracer; this.networkDistance = networkDistance; } @@ -397,7 +385,6 @@ public class BlockReaderRemote implements BlockReader { Peer peer, DatanodeID datanodeID, PeerCache peerCache, CachingStrategy cachingStrategy, - Tracer tracer, int networkDistance) throws IOException { // in and out will be closed when sock is closed (by the caller) final DataOutputStream out = new DataOutputStream(new BufferedOutputStream( @@ -431,7 +418,7 @@ public class BlockReaderRemote implements BlockReader { return new BlockReaderRemote(file, block.getBlockId(), checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID, - peerCache, tracer, networkDistance); + peerCache, networkDistance); } static void checkSuccess( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java index 5e77de54d59..cbef31807ea 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java @@ -128,7 +128,7 @@ class StripedBlockReader { return BlockReaderRemote.newBlockReader( "dummy", block, blockToken, offsetInBlock, block.getNumBytes() - offsetInBlock, true, "", peer, source, - null, stripedReader.getCachingStrategy(), datanode.getTracer(), -1); + null, stripedReader.getCachingStrategy(), -1); } catch (IOException e) { LOG.info("Exception while creating remote block reader, datanode {}", source, e); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java index a82c9fea549..5e7bab5dfa8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java @@ -1032,7 +1032,6 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { setCachingStrategy(CachingStrategy.newDropBehind()). setClientCacheContext(dfs.getClientContext()). setConfiguration(namenode.getConf()). - setTracer(tracer). setRemotePeerFactory(new RemotePeerFactory() { @Override public Peer newConnectedPeer(InetSocketAddress addr, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/BlockReaderTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/BlockReaderTestUtil.java index 57f5cf82f2e..e2f55e53f36 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/BlockReaderTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/BlockReaderTestUtil.java @@ -30,7 +30,6 @@ import java.util.Random; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FsTracer; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.BlockReader; import org.apache.hadoop.hdfs.ClientContext; @@ -206,7 +205,6 @@ public class BlockReaderTestUtil { setCachingStrategy(CachingStrategy.newDefaultStrategy()). setConfiguration(fs.getConf()). setAllowShortCircuitLocalReads(true). - setTracer(FsTracer.get(fs.getConf())). setRemotePeerFactory(new RemotePeerFactory() { @Override public Peer newConnectedPeer(InetSocketAddress addr, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocal.java index b9bb4956963..ace21c00c41 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocal.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocal.java @@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId; -import org.apache.hadoop.fs.FsTracer; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.unix.DomainSocket; @@ -208,7 +207,6 @@ public class TestBlockReaderLocal { setShortCircuitReplica(replica). setCachingStrategy(new CachingStrategy(false, readahead)). setVerifyChecksum(checksum). - setTracer(FsTracer.get(conf)). build(); dataIn = null; metaIn = null; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java index 3fbcd26aed1..b57c4f3a40d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java @@ -33,7 +33,6 @@ import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FsTracer; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.BlockReader; import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory; @@ -167,7 +166,6 @@ public class TestBlockTokenWithDFS { setCachingStrategy(CachingStrategy.newDefaultStrategy()). setClientCacheContext(ClientContext.getFromConf(conf)). setConfiguration(conf). - setTracer(FsTracer.get(conf)). setRemotePeerFactory(new RemotePeerFactory() { @Override public Peer newConnectedPeer(InetSocketAddress addr, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java index c116ce0fa14..16c0cfaabea 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java @@ -40,7 +40,6 @@ import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.FsTracer; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.BlockReader; import org.apache.hadoop.hdfs.ClientContext; @@ -655,7 +654,6 @@ public class TestDataNodeVolumeFailure { setCachingStrategy(CachingStrategy.newDefaultStrategy()). setClientCacheContext(ClientContext.getFromConf(conf)). setConfiguration(conf). - setTracer(FsTracer.get(conf)). setRemotePeerFactory(new RemotePeerFactory() { @Override public Peer newConnectedPeer(InetSocketAddress addr, From f51da9c4d1423c2ac92eb4f40e973264e7e968cc Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Mon, 2 Jul 2018 18:31:21 +0200 Subject: [PATCH 12/47] HADOOP-15554. Improve JIT performance for Configuration parsing. Contributed by Todd Lipcon. --- .../org/apache/hadoop/conf/Configuration.java | 458 +++++++++++------- 1 file changed, 276 insertions(+), 182 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java index b1125e588c0..a78e3119993 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java @@ -41,6 +41,7 @@ import java.io.Writer; import java.lang.ref.WeakReference; import java.net.InetSocketAddress; import java.net.JarURLConnection; +import java.net.MalformedURLException; import java.net.URL; import java.net.URLConnection; import java.util.ArrayList; @@ -2981,187 +2982,11 @@ public class Configuration implements Iterable>, if(returnCachedProperties) { toAddTo = new Properties(); } - DeprecationContext deprecations = deprecationContext.get(); - StringBuilder token = new StringBuilder(); - String confName = null; - String confValue = null; - String confInclude = null; - String confTag = null; - boolean confFinal = false; - boolean fallbackAllowed = false; - boolean fallbackEntered = false; - boolean parseToken = false; - LinkedList confSource = new LinkedList(); - - while (reader.hasNext()) { - switch (reader.next()) { - case XMLStreamConstants.START_ELEMENT: - switch (reader.getLocalName()) { - case "property": - confName = null; - confValue = null; - confFinal = false; - confTag = null; - confSource.clear(); - - // First test for short format configuration - int attrCount = reader.getAttributeCount(); - for (int i = 0; i < attrCount; i++) { - String propertyAttr = reader.getAttributeLocalName(i); - if ("name".equals(propertyAttr)) { - confName = StringInterner.weakIntern( - reader.getAttributeValue(i)); - } else if ("value".equals(propertyAttr)) { - confValue = StringInterner.weakIntern( - reader.getAttributeValue(i)); - } else if ("final".equals(propertyAttr)) { - confFinal = "true".equals(reader.getAttributeValue(i)); - } else if ("source".equals(propertyAttr)) { - confSource.add(StringInterner.weakIntern( - reader.getAttributeValue(i))); - } else if ("tag".equals(propertyAttr)) { - confTag = StringInterner - .weakIntern(reader.getAttributeValue(i)); - } - } - break; - case "name": - case "value": - case "final": - case "source": - case "tag": - parseToken = true; - token.setLength(0); - break; - case "include": - // Determine href for xi:include - confInclude = null; - attrCount = reader.getAttributeCount(); - for (int i = 0; i < attrCount; i++) { - String attrName = reader.getAttributeLocalName(i); - if ("href".equals(attrName)) { - confInclude = reader.getAttributeValue(i); - } - } - if (confInclude == null) { - break; - } - if (isRestricted) { - throw new RuntimeException("Error parsing resource " + wrapper - + ": XInclude is not supported for restricted resources"); - } - // Determine if the included resource is a classpath resource - // otherwise fallback to a file resource - // xi:include are treated as inline and retain current source - URL include = getResource(confInclude); - if (include != null) { - Resource classpathResource = new Resource(include, name, - wrapper.isParserRestricted()); - loadResource(properties, classpathResource, quiet); - } else { - URL url; - try { - url = new URL(confInclude); - url.openConnection().connect(); - } catch (IOException ioe) { - File href = new File(confInclude); - if (!href.isAbsolute()) { - // Included resources are relative to the current resource - File baseFile = new File(name).getParentFile(); - href = new File(baseFile, href.getPath()); - } - if (!href.exists()) { - // Resource errors are non-fatal iff there is 1 xi:fallback - fallbackAllowed = true; - break; - } - url = href.toURI().toURL(); - } - Resource uriResource = new Resource(url, name, - wrapper.isParserRestricted()); - loadResource(properties, uriResource, quiet); - } - break; - case "fallback": - fallbackEntered = true; - break; - case "configuration": - break; - default: - break; - } - break; - - case XMLStreamConstants.CHARACTERS: - if (parseToken) { - char[] text = reader.getTextCharacters(); - token.append(text, reader.getTextStart(), reader.getTextLength()); - } - break; - - case XMLStreamConstants.END_ELEMENT: - switch (reader.getLocalName()) { - case "name": - if (token.length() > 0) { - confName = StringInterner.weakIntern(token.toString().trim()); - } - break; - case "value": - if (token.length() > 0) { - confValue = StringInterner.weakIntern(token.toString()); - } - break; - case "final": - confFinal = "true".equals(token.toString()); - break; - case "source": - confSource.add(StringInterner.weakIntern(token.toString())); - break; - case "tag": - if (token.length() > 0) { - confTag = StringInterner.weakIntern(token.toString()); - } - break; - case "include": - if (fallbackAllowed && !fallbackEntered) { - throw new IOException("Fetch fail on include for '" - + confInclude + "' with no fallback while loading '" - + name + "'"); - } - fallbackAllowed = false; - fallbackEntered = false; - break; - case "property": - if (confName == null || (!fallbackAllowed && fallbackEntered)) { - break; - } - confSource.add(name); - // Read tags and put them in propertyTagsMap - if (confTag != null) { - readTagFromConfig(confTag, confName, confValue, confSource); - } - - DeprecatedKeyInfo keyInfo = - deprecations.getDeprecatedKeyMap().get(confName); - if (keyInfo != null) { - keyInfo.clearAccessed(); - for (String key : keyInfo.newKeys) { - // update new keys with deprecated key's value - loadProperty(toAddTo, name, key, confValue, confFinal, - confSource.toArray(new String[confSource.size()])); - } - } else { - loadProperty(toAddTo, name, confName, confValue, confFinal, - confSource.toArray(new String[confSource.size()])); - } - break; - default: - break; - } - default: - break; - } + List items = new Parser(reader, wrapper, quiet).parse(); + for (ParsedItem item : items) { + loadProperty(toAddTo, item.name, item.key, item.value, + item.isFinal, item.sources); } reader.close(); @@ -3179,6 +3004,275 @@ public class Configuration implements Iterable>, } } + private static class ParsedItem { + String name; + String key; + String value; + boolean isFinal; + String[] sources; + + ParsedItem(String name, String key, String value, + boolean isFinal, String[] sources) { + this.name = name; + this.key = key; + this.value = value; + this.isFinal = isFinal; + this.sources = sources; + } + } + + /** + * Parser to consume SAX stream of XML elements from a Configuration. + */ + private class Parser { + private final XMLStreamReader2 reader; + private final Resource wrapper; + private final String name; + private final String[] nameSingletonArray; + private final boolean isRestricted; + private final boolean quiet; + + DeprecationContext deprecations = deprecationContext.get(); + + private StringBuilder token = new StringBuilder(); + private String confName = null; + private String confValue = null; + private String confInclude = null; + private String confTag = null; + private boolean confFinal = false; + private boolean fallbackAllowed = false; + private boolean fallbackEntered = false; + private boolean parseToken = false; + private List confSource = new ArrayList<>(); + private List results = new ArrayList<>(); + + Parser(XMLStreamReader2 reader, + Resource wrapper, + boolean quiet) { + this.reader = reader; + this.wrapper = wrapper; + this.name = wrapper.getName(); + this.nameSingletonArray = new String[]{ name }; + this.isRestricted = wrapper.isParserRestricted(); + this.quiet = quiet; + + } + + List parse() throws IOException, XMLStreamException { + while (reader.hasNext()) { + parseNext(); + } + return results; + } + + private void handleStartElement() throws MalformedURLException { + switch (reader.getLocalName()) { + case "property": + handleStartProperty(); + break; + + case "name": + case "value": + case "final": + case "source": + case "tag": + parseToken = true; + token.setLength(0); + break; + case "include": + handleInclude(); + break; + case "fallback": + fallbackEntered = true; + break; + case "configuration": + break; + default: + break; + } + } + + private void handleStartProperty() { + confName = null; + confValue = null; + confFinal = false; + confTag = null; + confSource.clear(); + + // First test for short format configuration + int attrCount = reader.getAttributeCount(); + for (int i = 0; i < attrCount; i++) { + String propertyAttr = reader.getAttributeLocalName(i); + if ("name".equals(propertyAttr)) { + confName = StringInterner.weakIntern( + reader.getAttributeValue(i)); + } else if ("value".equals(propertyAttr)) { + confValue = StringInterner.weakIntern( + reader.getAttributeValue(i)); + } else if ("final".equals(propertyAttr)) { + confFinal = "true".equals(reader.getAttributeValue(i)); + } else if ("source".equals(propertyAttr)) { + confSource.add(StringInterner.weakIntern( + reader.getAttributeValue(i))); + } else if ("tag".equals(propertyAttr)) { + confTag = StringInterner + .weakIntern(reader.getAttributeValue(i)); + } + } + } + + private void handleInclude() throws MalformedURLException { + // Determine href for xi:include + confInclude = null; + int attrCount = reader.getAttributeCount(); + for (int i = 0; i < attrCount; i++) { + String attrName = reader.getAttributeLocalName(i); + if ("href".equals(attrName)) { + confInclude = reader.getAttributeValue(i); + } + } + if (confInclude == null) { + return; + } + if (isRestricted) { + throw new RuntimeException("Error parsing resource " + wrapper + + ": XInclude is not supported for restricted resources"); + } + // Determine if the included resource is a classpath resource + // otherwise fallback to a file resource + // xi:include are treated as inline and retain current source + URL include = getResource(confInclude); + if (include != null) { + Resource classpathResource = new Resource(include, name, + wrapper.isParserRestricted()); + // This is only called recursively while the lock is already held + // by this thread, but synchronizing avoids a findbugs warning. + synchronized (Configuration.this) { + loadResource(properties, classpathResource, quiet); + } + } else { + URL url; + try { + url = new URL(confInclude); + url.openConnection().connect(); + } catch (IOException ioe) { + File href = new File(confInclude); + if (!href.isAbsolute()) { + // Included resources are relative to the current resource + File baseFile = new File(name).getParentFile(); + href = new File(baseFile, href.getPath()); + } + if (!href.exists()) { + // Resource errors are non-fatal iff there is 1 xi:fallback + fallbackAllowed = true; + return; + } + url = href.toURI().toURL(); + } + Resource uriResource = new Resource(url, name, + wrapper.isParserRestricted()); + // This is only called recursively while the lock is already held + // by this thread, but synchronizing avoids a findbugs warning. + synchronized (Configuration.this) { + loadResource(properties, uriResource, quiet); + } + } + } + + void handleEndElement() throws IOException { + String tokenStr = token.toString(); + switch (reader.getLocalName()) { + case "name": + if (token.length() > 0) { + confName = StringInterner.weakIntern(tokenStr.trim()); + } + break; + case "value": + if (token.length() > 0) { + confValue = StringInterner.weakIntern(tokenStr); + } + break; + case "final": + confFinal = "true".equals(tokenStr); + break; + case "source": + confSource.add(StringInterner.weakIntern(tokenStr)); + break; + case "tag": + if (token.length() > 0) { + confTag = StringInterner.weakIntern(tokenStr); + } + break; + case "include": + if (fallbackAllowed && !fallbackEntered) { + throw new IOException("Fetch fail on include for '" + + confInclude + "' with no fallback while loading '" + + name + "'"); + } + fallbackAllowed = false; + fallbackEntered = false; + break; + case "property": + handleEndProperty(); + break; + default: + break; + } + } + + void handleEndProperty() { + if (confName == null || (!fallbackAllowed && fallbackEntered)) { + return; + } + String[] confSourceArray; + if (confSource.isEmpty()) { + confSourceArray = nameSingletonArray; + } else { + confSource.add(name); + confSourceArray = confSource.toArray(new String[confSource.size()]); + } + + // Read tags and put them in propertyTagsMap + if (confTag != null) { + readTagFromConfig(confTag, confName, confValue, confSourceArray); + } + + DeprecatedKeyInfo keyInfo = + deprecations.getDeprecatedKeyMap().get(confName); + + if (keyInfo != null) { + keyInfo.clearAccessed(); + for (String key : keyInfo.newKeys) { + // update new keys with deprecated key's value + results.add(new ParsedItem( + name, key, confValue, confFinal, confSourceArray)); + } + } else { + results.add(new ParsedItem(name, confName, confValue, confFinal, + confSourceArray)); + } + } + + void parseNext() throws IOException, XMLStreamException { + switch (reader.next()) { + case XMLStreamConstants.START_ELEMENT: + handleStartElement(); + break; + case XMLStreamConstants.CHARACTERS: + if (parseToken) { + char[] text = reader.getTextCharacters(); + token.append(text, reader.getTextStart(), reader.getTextLength()); + } + break; + case XMLStreamConstants.END_ELEMENT: + handleEndElement(); + break; + default: + break; + } + } + } + /** * Add tags defined in HADOOP_TAGS_SYSTEM, HADOOP_TAGS_CUSTOM. * @param prop @@ -3225,7 +3319,7 @@ public class Configuration implements Iterable>, * @param confSource */ private void readTagFromConfig(String attributeValue, String confName, String - confValue, List confSource) { + confValue, String[] confSource) { for (String tagStr : attributeValue.split(",")) { try { tagStr = tagStr.trim(); @@ -3243,7 +3337,7 @@ public class Configuration implements Iterable>, } catch (Exception ex) { // Log the exception at trace level. LOG.trace("Tag '{}' for property:{} Source:{}", tagStr, confName, - Arrays.toString(confSource.toArray()), ex); + confSource, ex); } } } From fef20a446f7bf9f29e0f0ee690987fb6fc78a031 Mon Sep 17 00:00:00 2001 From: Wei-Chiu Chuang Date: Mon, 2 Jul 2018 09:41:15 -0700 Subject: [PATCH 13/47] HDFS-13635. Incorrect message when block is not found. Contributed by Gabor Bota. --- .../hadoop/hdfs/server/datanode/ReplicaNotFoundException.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java index 90f257f1dd7..946950ce622 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java @@ -37,7 +37,7 @@ public class ReplicaNotFoundException extends IOException { "Cannot recover append/close to a replica that's not FINALIZED and not RBW" + " "; public final static String NON_EXISTENT_REPLICA = - "Cannot append to a non-existent replica "; + "Replica does not exist "; public final static String UNEXPECTED_GS_REPLICA = "Cannot append to a replica with unexpected generation stamp "; From 5cc2541a163591181b80bf2ec42c1e7e7f8929f5 Mon Sep 17 00:00:00 2001 From: Eric Yang Date: Mon, 2 Jul 2018 13:37:51 -0400 Subject: [PATCH 14/47] YARN-8465. Fixed docker container status for node manager restart. Contributed by Shane Kumpf --- .../linux/runtime/DockerLinuxContainerRuntime.java | 4 ++-- .../runtime/ContainerExecutionException.java | 6 ++++++ .../linux/runtime/TestDockerContainerRuntime.java | 10 ++++++++-- 3 files changed, 16 insertions(+), 4 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java index f13ba59b00b..c89d5fb136e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java @@ -1027,7 +1027,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime { handleContainerKill(ctx, env, signal); } } catch (ContainerExecutionException e) { - LOG.warn("Signal docker container failed. Exception: ", e); throw new ContainerExecutionException("Signal docker container failed", e.getExitCode(), e.getOutput(), e.getErrorOutput()); } @@ -1201,7 +1200,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime { if (!new File(procFs + File.separator + pid).exists()) { String msg = "Liveliness check failed for PID: " + pid + ". Container may have already completed."; - throw new ContainerExecutionException(msg); + throw new ContainerExecutionException(msg, + PrivilegedOperation.ResultCode.INVALID_CONTAINER_PID.getValue()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/runtime/ContainerExecutionException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/runtime/ContainerExecutionException.java index 31472777042..735db1f6082 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/runtime/ContainerExecutionException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/runtime/ContainerExecutionException.java @@ -53,6 +53,12 @@ public class ContainerExecutionException extends YarnException { errorOutput = OUTPUT_UNSET; } + public ContainerExecutionException(String message, int exitCode) { + super(message); + this.exitCode = exitCode; + this.output = OUTPUT_UNSET; + this.errorOutput = OUTPUT_UNSET; + } public ContainerExecutionException(String message, int exitCode, String output, String errorOutput) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java index d85c403f1fd..855ec44c2ae 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java @@ -1492,7 +1492,7 @@ public class TestDockerContainerRuntime { runtime.signalContainer(builder.build()); } - @Test(expected = ContainerExecutionException.class) + @Test public void testContainerLivelinessNoFileException() throws Exception { DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime( mockExecutor, mockCGroupsHandler); @@ -1501,7 +1501,13 @@ public class TestDockerContainerRuntime { .setExecutionAttribute(PID, signalPid) .setExecutionAttribute(SIGNAL, ContainerExecutor.Signal.NULL); runtime.initialize(enableMockContainerExecutor(conf), null); - runtime.signalContainer(builder.build()); + try { + runtime.signalContainer(builder.build()); + } catch (ContainerExecutionException e) { + Assert.assertEquals( + PrivilegedOperation.ResultCode.INVALID_CONTAINER_PID.getValue(), + e.getExitCode()); + } } @Test From 1804a31515e541b3371925aa895589919b54d443 Mon Sep 17 00:00:00 2001 From: Inigo Goiri Date: Mon, 2 Jul 2018 10:48:20 -0700 Subject: [PATCH 15/47] HDFS-13536. [PROVIDED Storage] HA for InMemoryAliasMap. Contributed by Virajith Jalaparti. --- .../org/apache/hadoop/hdfs/DFSUtilClient.java | 4 +- .../hdfs/client/HdfsClientConfigKeys.java | 3 + .../ha/ConfiguredFailoverProxyProvider.java | 9 +- ...InMemoryAliasMapFailoverProxyProvider.java | 38 ++ .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 5 +- .../java/org/apache/hadoop/hdfs/DFSUtil.java | 37 +- .../apache/hadoop/hdfs/NameNodeProxies.java | 15 +- ...liasMapProtocolClientSideTranslatorPB.java | 95 ++++- .../aliasmap/InMemoryAliasMapProtocol.java | 5 + .../InMemoryLevelDBAliasMapServer.java | 19 +- .../impl/InMemoryLevelDBAliasMapClient.java | 80 ++-- .../src/main/resources/hdfs-default.xml | 22 +- .../apache/hadoop/hdfs/MiniDFSCluster.java | 13 +- .../apache/hadoop/hdfs/MiniDFSNNTopology.java | 2 +- .../TestInMemoryLevelDBAliasMapClient.java | 7 + .../namenode/ITestProvidedImplementation.java | 371 +++++++++++++++--- 16 files changed, 615 insertions(+), 110 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/InMemoryAliasMapFailoverProxyProvider.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java index 313b973550c..3fac7c8c100 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java @@ -396,7 +396,7 @@ public class DFSUtilClient { * @param keys Set of keys to look for in the order of preference * @return a map(nameserviceId to map(namenodeId to InetSocketAddress)) */ - static Map> getAddresses( + public static Map> getAddresses( Configuration conf, String defaultAddress, String... keys) { Collection nameserviceIds = getNameServiceIds(conf); return getAddressesForNsIds(conf, nameserviceIds, defaultAddress, keys); @@ -426,7 +426,7 @@ public class DFSUtilClient { return ret; } - static Map getAddressesForNameserviceId( + public static Map getAddressesForNameserviceId( Configuration conf, String nsId, String defaultValue, String... keys) { Collection nnIds = getNameNodeIds(conf, nsId); Map ret = Maps.newLinkedHashMap(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java index f2cec314ffe..a8126700d05 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java @@ -184,6 +184,9 @@ public interface HdfsClientConfigKeys { "dfs.namenode.snapshot.capture.openfiles"; boolean DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES_DEFAULT = false; + String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS = + "dfs.provided.aliasmap.inmemory.dnrpc-address"; + /** * These are deprecated config keys to client code. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java index 96722fcfab3..f46532ad972 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java @@ -37,6 +37,8 @@ import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY; + /** * A FailoverProxyProvider implementation which allows one to configure * multiple URIs to connect to during fail-over. A random configured address is @@ -60,6 +62,11 @@ public class ConfiguredFailoverProxyProvider extends public ConfiguredFailoverProxyProvider(Configuration conf, URI uri, Class xface, HAProxyFactory factory) { + this(conf, uri, xface, factory, DFS_NAMENODE_RPC_ADDRESS_KEY); + } + + public ConfiguredFailoverProxyProvider(Configuration conf, URI uri, + Class xface, HAProxyFactory factory, String addressKey) { this.xface = xface; this.conf = new Configuration(conf); int maxRetries = this.conf.getInt( @@ -81,7 +88,7 @@ public class ConfiguredFailoverProxyProvider extends ugi = UserGroupInformation.getCurrentUser(); Map> map = - DFSUtilClient.getHaNnRpcAddresses(conf); + DFSUtilClient.getAddresses(conf, null, addressKey); Map addressesInNN = map.get(uri.getHost()); if (addressesInNN == null || addressesInNN.size() == 0) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/InMemoryAliasMapFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/InMemoryAliasMapFailoverProxyProvider.java new file mode 100644 index 00000000000..6525942341a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/InMemoryAliasMapFailoverProxyProvider.java @@ -0,0 +1,38 @@ +/** + * 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.hdfs.server.namenode.ha; + +import org.apache.hadoop.conf.Configuration; + +import java.net.URI; + +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS; + +/** + * A {@link ConfiguredFailoverProxyProvider} implementation used to connect + * to an InMemoryAliasMap. + */ +public class InMemoryAliasMapFailoverProxyProvider + extends ConfiguredFailoverProxyProvider { + + public InMemoryAliasMapFailoverProxyProvider( + Configuration conf, URI uri, Class xface, HAProxyFactory factory) { + super(conf, uri, xface, factory, + DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index dde7eb79c2f..cc902b00772 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -86,8 +86,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys { HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY; public static final String DFS_NAMENODE_BACKUP_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50105"; public static final String DFS_NAMENODE_BACKUP_SERVICE_RPC_ADDRESS_KEY = "dfs.namenode.backup.dnrpc-address"; - public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS = "dfs.provided.aliasmap.inmemory.dnrpc-address"; + public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS = + HdfsClientConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS; public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT = "0.0.0.0:50200"; + public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_BIND_HOST = "dfs.provided.aliasmap.inmemory.rpc.bind-host"; + public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR = "dfs.provided.aliasmap.inmemory.leveldb.dir"; public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE = "dfs.provided.aliasmap.inmemory.batch-size"; public static final int DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE_DEFAULT = 500; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java index 4c94e380ca4..f7cd32b5585 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java @@ -1130,7 +1130,42 @@ public class DFSUtil { return getSuffixIDs(conf, addressKey, null, nnId, LOCAL_ADDRESS_MATCHER)[0]; } - + + /** + * Determine the {@link InetSocketAddress} to bind to, for any service. + * In case of HA or federation, the address is assumed to specified as + * {@code confKey}.NAMESPACEID.NAMENODEID as appropriate. + * + * @param conf configuration. + * @param confKey configuration key (prefix if HA/federation) used to + * specify the address for the service. + * @param defaultValue default value for the address. + * @param bindHostKey configuration key (prefix if HA/federation) + * specifying host to bind to. + * @return the address to bind to. + */ + public static InetSocketAddress getBindAddress(Configuration conf, + String confKey, String defaultValue, String bindHostKey) { + InetSocketAddress address; + String nsId = DFSUtil.getNamenodeNameServiceId(conf); + String bindHostActualKey; + if (nsId != null) { + String namenodeId = HAUtil.getNameNodeId(conf, nsId); + address = DFSUtilClient.getAddressesForNameserviceId( + conf, nsId, null, confKey).get(namenodeId); + bindHostActualKey = DFSUtil.addKeySuffixes(bindHostKey, nsId, namenodeId); + } else { + address = NetUtils.createSocketAddr(conf.get(confKey, defaultValue)); + bindHostActualKey = bindHostKey; + } + + String bindHost = conf.get(bindHostActualKey); + if (bindHost == null || bindHost.isEmpty()) { + bindHost = address.getHostName(); + } + return new InetSocketAddress(bindHost, address.getPort()); + } + /** * Returns nameservice Id and namenode Id when the local host matches the * configuration parameter {@code addressKey}.. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java index d556c907c49..b63d26b85ae 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java @@ -31,10 +31,13 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo; import org.apache.hadoop.hdfs.protocol.ClientProtocol; +import org.apache.hadoop.hdfs.protocolPB.AliasMapProtocolPB; +import org.apache.hadoop.hdfs.protocolPB.InMemoryAliasMapProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB; import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB; import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB; import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB; +import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol; import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider; import org.apache.hadoop.hdfs.server.namenode.ha.NameNodeHAProxyFactory; import org.apache.hadoop.hdfs.server.protocol.JournalProtocol; @@ -184,6 +187,8 @@ public class NameNodeProxies { conf, ugi); } else if (xface == RefreshCallQueueProtocol.class) { proxy = (T) createNNProxyWithRefreshCallQueueProtocol(nnAddr, conf, ugi); + } else if (xface == InMemoryAliasMapProtocol.class) { + proxy = (T) createNNProxyWithInMemoryAliasMapProtocol(nnAddr, conf, ugi); } else { String message = "Unsupported protocol found when creating the proxy " + "connection to NameNode: " + @@ -194,7 +199,15 @@ public class NameNodeProxies { return new ProxyAndInfo(proxy, dtService, nnAddr); } - + + private static InMemoryAliasMapProtocol createNNProxyWithInMemoryAliasMapProtocol( + InetSocketAddress address, Configuration conf, UserGroupInformation ugi) + throws IOException { + AliasMapProtocolPB proxy = (AliasMapProtocolPB) createNameNodeProxy( + address, conf, ugi, AliasMapProtocolPB.class, 30000); + return new InMemoryAliasMapProtocolClientSideTranslatorPB(proxy); + } + private static JournalProtocol createNNProxyWithJournalProtocol( InetSocketAddress address, Configuration conf, UserGroupInformation ugi) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java index fc23c88c185..2025c16d1c3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java @@ -20,27 +20,38 @@ import com.google.protobuf.ServiceException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.HAUtil; +import org.apache.hadoop.hdfs.NameNodeProxies; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation; import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap; import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol; import org.apache.hadoop.hdfs.server.common.FileRegion; +import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider; +import org.apache.hadoop.hdfs.server.namenode.ha.InMemoryAliasMapFailoverProxyProvider; import org.apache.hadoop.ipc.ProtobufHelper; -import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.net.NetUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; +import java.io.Closeable; import java.io.IOException; -import java.net.InetSocketAddress; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Optional; import java.util.stream.Collectors; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSUtil.addKeySuffixes; +import static org.apache.hadoop.hdfs.DFSUtil.createUri; +import static org.apache.hadoop.hdfs.DFSUtilClient.getNameServiceIds; +import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX; import static org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.*; import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.*; @@ -52,7 +63,7 @@ import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.*; @InterfaceAudience.Private @InterfaceStability.Unstable public class InMemoryAliasMapProtocolClientSideTranslatorPB - implements InMemoryAliasMapProtocol { + implements InMemoryAliasMapProtocol, Closeable { private static final Logger LOG = LoggerFactory @@ -60,22 +71,61 @@ public class InMemoryAliasMapProtocolClientSideTranslatorPB private AliasMapProtocolPB rpcProxy; - public InMemoryAliasMapProtocolClientSideTranslatorPB(Configuration conf) { - String addr = conf.getTrimmed(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS, - DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT); - InetSocketAddress aliasMapAddr = NetUtils.createSocketAddr(addr); + public InMemoryAliasMapProtocolClientSideTranslatorPB( + AliasMapProtocolPB rpcProxy) { + this.rpcProxy = rpcProxy; + } - RPC.setProtocolEngine(conf, AliasMapProtocolPB.class, - ProtobufRpcEngine.class); - LOG.info("Connecting to address: " + addr); - try { - rpcProxy = RPC.getProxy(AliasMapProtocolPB.class, - RPC.getProtocolVersion(AliasMapProtocolPB.class), aliasMapAddr, null, - conf, NetUtils.getDefaultSocketFactory(conf), 0); - } catch (IOException e) { - throw new RuntimeException( - "Error in connecting to " + addr + " Got: " + e); + public static Collection init(Configuration conf) { + Collection aliasMaps = new ArrayList<>(); + // Try to connect to all configured nameservices as it is not known which + // nameservice supports the AliasMap. + for (String nsId : getNameServiceIds(conf)) { + try { + URI namenodeURI = null; + Configuration newConf = new Configuration(conf); + if (HAUtil.isHAEnabled(conf, nsId)) { + // set the failover-proxy provider if HA is enabled. + newConf.setClass( + addKeySuffixes(PROXY_PROVIDER_KEY_PREFIX, nsId), + InMemoryAliasMapFailoverProxyProvider.class, + AbstractNNFailoverProxyProvider.class); + namenodeURI = new URI(HdfsConstants.HDFS_URI_SCHEME + "://" + nsId); + } else { + String key = + addKeySuffixes(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS, nsId); + String addr = conf.get(key); + if (addr != null) { + namenodeURI = createUri(HdfsConstants.HDFS_URI_SCHEME, + NetUtils.createSocketAddr(addr)); + } + } + if (namenodeURI != null) { + aliasMaps.add(NameNodeProxies + .createProxy(newConf, namenodeURI, InMemoryAliasMapProtocol.class) + .getProxy()); + LOG.info("Connected to InMemoryAliasMap at {}", namenodeURI); + } + } catch (IOException | URISyntaxException e) { + LOG.warn("Exception in connecting to InMemoryAliasMap for nameservice " + + "{}: {}", nsId, e); + } } + // if a separate AliasMap is configured using + // DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS, try to connect it. + if (conf.get(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS) != null) { + URI uri = createUri("hdfs", NetUtils.createSocketAddr( + conf.get(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS))); + try { + aliasMaps.add(NameNodeProxies + .createProxy(conf, uri, InMemoryAliasMapProtocol.class).getProxy()); + LOG.info("Connected to InMemoryAliasMap at {}", uri); + } catch (IOException e) { + LOG.warn("Exception in connecting to InMemoryAliasMap at {}: {}", uri, + e); + } + } + return aliasMaps; } @Override @@ -168,7 +218,12 @@ public class InMemoryAliasMapProtocolClientSideTranslatorPB } } - public void stop() { - RPC.stopProxy(rpcProxy); + @Override + public void close() throws IOException { + LOG.info("Stopping rpcProxy in" + + "InMemoryAliasMapProtocolClientSideTranslatorPB"); + if (rpcProxy != null) { + RPC.stopProxy(rpcProxy); + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java index 89f590cf292..c3824e58265 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java @@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation; import org.apache.hadoop.hdfs.server.common.FileRegion; +import org.apache.hadoop.io.retry.Idempotent; import javax.annotation.Nonnull; import java.io.IOException; @@ -69,6 +70,7 @@ public interface InMemoryAliasMapProtocol { * FileRegions and the next marker. * @throws IOException */ + @Idempotent InMemoryAliasMap.IterationResult list(Optional marker) throws IOException; @@ -80,6 +82,7 @@ public interface InMemoryAliasMapProtocol { * @throws IOException */ @Nonnull + @Idempotent Optional read(@Nonnull Block block) throws IOException; @@ -90,6 +93,7 @@ public interface InMemoryAliasMapProtocol { * @param providedStorageLocation * @throws IOException */ + @Idempotent void write(@Nonnull Block block, @Nonnull ProvidedStorageLocation providedStorageLocation) throws IOException; @@ -99,5 +103,6 @@ public interface InMemoryAliasMapProtocol { * @return the block pool id associated with the Namenode running * the in-memory alias map. */ + @Idempotent String getBlockPoolId() throws IOException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java index 4edc9a29130..1d06f132854 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java @@ -25,7 +25,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation; import org.apache.hadoop.hdfs.protocolPB.AliasMapProtocolPB; @@ -34,9 +33,13 @@ import org.apache.hadoop.ipc.RPC; import javax.annotation.Nonnull; import java.io.Closeable; import java.io.IOException; +import java.net.InetSocketAddress; import java.util.Optional; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_BIND_HOST; +import static org.apache.hadoop.hdfs.DFSUtil.getBindAddress; import static org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.*; import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.CheckedFunction2; @@ -79,18 +82,16 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol, AliasMapProtocolService .newReflectiveBlockingService(aliasMapProtocolXlator); - String rpcAddress = - conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS, - DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT); - String[] split = rpcAddress.split(":"); - String bindHost = split[0]; - Integer port = Integer.valueOf(split[1]); + InetSocketAddress rpcAddress = getBindAddress(conf, + DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS, + DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT, + DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_BIND_HOST); aliasMapServer = new RPC.Builder(conf) .setProtocol(AliasMapProtocolPB.class) .setInstance(aliasMapProtocolService) - .setBindAddress(bindHost) - .setPort(port) + .setBindAddress(rpcAddress.getHostName()) + .setPort(rpcAddress.getPort()) .setNumHandlers(1) .setVerbose(true) .build(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java index d3891846cee..fb5ee93c19a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java @@ -24,11 +24,17 @@ import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation; import org.apache.hadoop.hdfs.protocolPB.InMemoryAliasMapProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap; +import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol; import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap; import org.apache.hadoop.hdfs.server.common.FileRegion; +import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.security.UserGroupInformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -44,17 +50,28 @@ import java.util.Optional; public class InMemoryLevelDBAliasMapClient extends BlockAliasMap implements Configurable { + private static final Logger LOG = + LoggerFactory.getLogger(InMemoryLevelDBAliasMapClient.class); private Configuration conf; - private InMemoryAliasMapProtocolClientSideTranslatorPB aliasMap; - private String blockPoolID; + private Collection aliasMaps; @Override public void close() { - aliasMap.stop(); + if (aliasMaps != null) { + for (InMemoryAliasMapProtocol aliasMap : aliasMaps) { + RPC.stopProxy(aliasMap); + } + } } class LevelDbReader extends BlockAliasMap.Reader { + private InMemoryAliasMapProtocol aliasMap; + + LevelDbReader(InMemoryAliasMapProtocol aliasMap) { + this.aliasMap = aliasMap; + } + @Override public Optional resolve(Block block) throws IOException { Optional read = aliasMap.read(block); @@ -114,6 +131,13 @@ public class InMemoryLevelDBAliasMapClient extends BlockAliasMap } class LevelDbWriter extends BlockAliasMap.Writer { + + private InMemoryAliasMapProtocol aliasMap; + + LevelDbWriter(InMemoryAliasMapProtocol aliasMap) { + this.aliasMap = aliasMap; + } + @Override public void store(FileRegion fileRegion) throws IOException { aliasMap.write(fileRegion.getBlock(), @@ -130,40 +154,53 @@ public class InMemoryLevelDBAliasMapClient extends BlockAliasMap throw new UnsupportedOperationException("Unable to start " + "InMemoryLevelDBAliasMapClient as security is enabled"); } + aliasMaps = new ArrayList<>(); } + private InMemoryAliasMapProtocol getAliasMap(String blockPoolID) + throws IOException { + if (blockPoolID == null) { + throw new IOException("Block pool id required to get aliasmap reader"); + } + // if a block pool id has been supplied, and doesn't match the associated + // block pool ids, return null. + for (InMemoryAliasMapProtocol aliasMap : aliasMaps) { + try { + String aliasMapBlockPoolId = aliasMap.getBlockPoolId(); + if (aliasMapBlockPoolId != null && + aliasMapBlockPoolId.equals(blockPoolID)) { + return aliasMap; + } + } catch (IOException e) { + LOG.error("Exception in retrieving block pool id {}", e); + } + } + throw new IOException( + "Unable to retrive InMemoryAliasMap for block pool id " + blockPoolID); + } @Override public Reader getReader(Reader.Options opts, String blockPoolID) throws IOException { - if (this.blockPoolID == null) { - this.blockPoolID = aliasMap.getBlockPoolId(); - } - // if a block pool id has been supplied, and doesn't match the associated - // block pool id, return null. - if (blockPoolID != null && this.blockPoolID != null - && !this.blockPoolID.equals(blockPoolID)) { - return null; - } - return new LevelDbReader(); + InMemoryAliasMapProtocol aliasMap = getAliasMap(blockPoolID); + LOG.info("Loading InMemoryAliasMapReader for block pool id {}", + blockPoolID); + return new LevelDbReader(aliasMap); } @Override public Writer getWriter(Writer.Options opts, String blockPoolID) throws IOException { - if (this.blockPoolID == null) { - this.blockPoolID = aliasMap.getBlockPoolId(); - } - if (blockPoolID != null && !this.blockPoolID.equals(blockPoolID)) { - return null; - } - return new LevelDbWriter(); + InMemoryAliasMapProtocol aliasMap = getAliasMap(blockPoolID); + LOG.info("Loading InMemoryAliasMapWriter for block pool id {}", + blockPoolID); + return new LevelDbWriter(aliasMap); } @Override public void setConf(Configuration conf) { this.conf = conf; - this.aliasMap = new InMemoryAliasMapProtocolClientSideTranslatorPB(conf); + aliasMaps = InMemoryAliasMapProtocolClientSideTranslatorPB.init(conf); } @Override @@ -174,5 +211,4 @@ public class InMemoryLevelDBAliasMapClient extends BlockAliasMap @Override public void refresh() throws IOException { } - } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 146ae6c9c62..6dd2d927962 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -4817,9 +4817,27 @@ dfs.provided.aliasmap.inmemory.dnrpc-address - 0.0.0.0:50200 + - The address where the aliasmap server will be running + The address where the aliasmap server will be running. In the case of + HA/Federation where multiple namenodes exist, and if the Namenode is + configured to run the aliasmap server + (dfs.provided.aliasmap.inmemory.enabled is set to true), + the name service id is added to the name, e.g., + dfs.provided.aliasmap.inmemory.rpc.address.EXAMPLENAMESERVICE. + The value of this property will take the form of host:rpc-port. + + + + + dfs.provided.aliasmap.inmemory.rpc.bind-host + + + The actual address the in-memory aliasmap server will bind to. + If this optional address is set, it overrides the hostname portion of + dfs.provided.aliasmap.inmemory.rpc.address. + This is useful for making the name node listen on all interfaces by + setting it to 0.0.0.0. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java index c2e2a68af2b..a2e59515d8b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java @@ -1187,7 +1187,7 @@ public class MiniDFSCluster implements AutoCloseable { } - private void initNameNodeConf(Configuration conf, String nameserviceId, int nsIndex, String nnId, + protected void initNameNodeConf(Configuration conf, String nameserviceId, int nsIndex, String nnId, boolean manageNameDfsDirs, boolean enableManagedDfsDirsRedundancy, int nnIndex) throws IOException { if (nameserviceId != null) { @@ -1379,6 +1379,17 @@ public class MiniDFSCluster implements AutoCloseable { return null; } + public List getNNIndexes(String nameserviceId) { + int count = 0; + List nnIndexes = new ArrayList<>(); + for (NameNodeInfo nn : namenodes.values()) { + if (nn.getNameserviceId().equals(nameserviceId)) { + nnIndexes.add(count); + } + count++; + } + return nnIndexes; + } /** * wait for the given namenode to get out of safemode. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java index b9786a32a75..c21ff809b36 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java @@ -227,7 +227,7 @@ public class MiniDFSNNTopology { this.nnId = nnId; } - String getNnId() { + public String getNnId() { return nnId; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java index 61a15589783..f0626335bb0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java @@ -32,6 +32,7 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -341,4 +342,10 @@ public class TestInMemoryLevelDBAliasMapClient { assertThat(actualFileRegions).containsExactlyInAnyOrder( expectedFileRegions.toArray(new FileRegion[0])); } + + @Test + public void testServerBindHost() throws Exception { + conf.set(DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY, "0.0.0.0"); + writeRead(); + } } \ No newline at end of file diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java index 7d3ab0ea8f9..e3f4decd59a 100644 --- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java +++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java @@ -24,12 +24,18 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.io.Writer; import java.net.InetSocketAddress; +import java.net.URI; import java.nio.ByteBuffer; import java.nio.channels.Channels; import java.nio.channels.ReadableByteChannel; -import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.Random; import java.util.Set; @@ -44,14 +50,14 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.MiniDFSNNTopology; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; -import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap; -import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; @@ -60,8 +66,10 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.blockmanagement.ProvidedStorageMap; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap; import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.InMemoryLevelDBAliasMapClient; +import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.LevelDBFileRegionAliasMap; import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap; import org.apache.hadoop.hdfs.server.datanode.DataNode; @@ -71,6 +79,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl; import org.apache.hadoop.hdfs.server.protocol.StorageReport; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NodeBase; import org.junit.After; import org.junit.Before; @@ -80,6 +90,12 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_BIND_HOST; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LEVELDB_PATH; +import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI; import static org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.fileNameFromBlockPoolID; import static org.apache.hadoop.net.NodeBase.PATH_SEPARATOR_STR; import static org.junit.Assert.*; @@ -106,6 +122,7 @@ public class ITestProvidedImplementation { private final int baseFileLen = 1024; private long providedDataSize = 0; private final String bpid = "BP-1234-10.1.1.1-1224"; + private static final String clusterID = "CID-PROVIDED"; private Configuration conf; private MiniDFSCluster cluster; @@ -214,36 +231,78 @@ public class ITestProvidedImplementation { StorageType[] storageTypes, StorageType[][] storageTypesPerDatanode, boolean doFormat, String[] racks) throws IOException { + startCluster(nspath, numDatanodes, + storageTypes, storageTypesPerDatanode, + doFormat, racks, null, + new MiniDFSCluster.Builder(conf)); + } + + void startCluster(Path nspath, int numDatanodes, + StorageType[] storageTypes, + StorageType[][] storageTypesPerDatanode, + boolean doFormat, String[] racks, + MiniDFSNNTopology topo, + MiniDFSCluster.Builder builder) throws IOException { conf.set(DFS_NAMENODE_NAME_DIR_KEY, nspath.toString()); + builder.format(doFormat) + .manageNameDfsDirs(doFormat) + .numDataNodes(numDatanodes) + .racks(racks); if (storageTypesPerDatanode != null) { - cluster = new MiniDFSCluster.Builder(conf) - .format(doFormat) - .manageNameDfsDirs(doFormat) - .numDataNodes(numDatanodes) - .storageTypes(storageTypesPerDatanode) - .racks(racks) - .build(); + builder.storageTypes(storageTypesPerDatanode); } else if (storageTypes != null) { - cluster = new MiniDFSCluster.Builder(conf) - .format(doFormat) - .manageNameDfsDirs(doFormat) - .numDataNodes(numDatanodes) - .storagesPerDatanode(storageTypes.length) - .storageTypes(storageTypes) - .racks(racks) - .build(); - } else { - cluster = new MiniDFSCluster.Builder(conf) - .format(doFormat) - .manageNameDfsDirs(doFormat) - .numDataNodes(numDatanodes) - .racks(racks) - .build(); + builder.storagesPerDatanode(storageTypes.length) + .storageTypes(storageTypes); } + if (topo != null) { + builder.nnTopology(topo); + // If HA or Federation is enabled and formatting is set to false, + // copy the FSImage to all Namenode directories. + if ((topo.isHA() || topo.isFederated()) && !doFormat) { + builder.manageNameDfsDirs(true); + builder.enableManagedDfsDirsRedundancy(false); + builder.manageNameDfsSharedDirs(false); + List nnDirs = + getProvidedNamenodeDirs(MiniDFSCluster.getBaseDirectory(), topo); + for (File nnDir : nnDirs) { + MiniDFSCluster.copyNameDirs( + Collections.singletonList(nspath.toUri()), + Collections.singletonList(fileAsURI(nnDir)), + conf); + } + } + } + cluster = builder.build(); cluster.waitActive(); } + private static List getProvidedNamenodeDirs(String baseDir, + MiniDFSNNTopology topo) { + List nnDirs = new ArrayList<>(); + int nsCounter = 0; + for (MiniDFSNNTopology.NSConf nsConf : topo.getNameservices()) { + int nnCounter = nsCounter; + for (MiniDFSNNTopology.NNConf nnConf : nsConf.getNNs()) { + if (providedNameservice.equals(nsConf.getId())) { + // only add the first one + File[] nnFiles = + MiniDFSCluster.getNameNodeDirectory( + baseDir, nsCounter, nnCounter); + if (nnFiles == null || nnFiles.length == 0) { + throw new RuntimeException("Failed to get a location for the" + + "Namenode directory for namespace: " + nsConf.getId() + + " and namenodeId: " + nnConf.getNnId()); + } + nnDirs.add(nnFiles[0]); + } + nnCounter++; + } + nsCounter = nnCounter; + } + return nnDirs; + } + @Test(timeout=20000) public void testLoadImage() throws Exception { final long seed = r.nextLong(); @@ -405,8 +464,8 @@ public class ITestProvidedImplementation { return ret; } - private void verifyFileSystemContents() throws Exception { - FileSystem fs = cluster.getFileSystem(); + private void verifyFileSystemContents(int nnIndex) throws Exception { + FileSystem fs = cluster.getFileSystem(nnIndex); int count = 0; // read NN metadata, verify contents match for (TreePath e : new FSTreeWalk(providedPath, conf)) { @@ -766,41 +825,255 @@ public class ITestProvidedImplementation { } } - - @Test - public void testInMemoryAliasMap() throws Exception { - conf.setClass(ImageWriter.Options.UGI_CLASS, - FsUGIResolver.class, UGIResolver.class); + private File createInMemoryAliasMapImage() throws Exception { + conf.setClass(ImageWriter.Options.UGI_CLASS, FsUGIResolver.class, + UGIResolver.class); conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS, InMemoryLevelDBAliasMapClient.class, BlockAliasMap.class); - conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS, - "localhost:32445"); + conf.set(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS, "localhost:32445"); File tempDirectory = - Files.createTempDirectory("in-memory-alias-map").toFile(); - File leveDBPath = new File(tempDirectory, bpid); - leveDBPath.mkdirs(); - conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR, + new File(new Path(nnDirPath, "in-memory-alias-map").toUri()); + File levelDBDir = new File(tempDirectory, bpid); + levelDBDir.mkdirs(); + conf.set(DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR, tempDirectory.getAbsolutePath()); - conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true); conf.setInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 10); - InMemoryLevelDBAliasMapServer levelDBAliasMapServer = - new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init, bpid); - levelDBAliasMapServer.setConf(conf); - levelDBAliasMapServer.start(); + conf.set(DFS_PROVIDED_ALIASMAP_LEVELDB_PATH, + tempDirectory.getAbsolutePath()); createImage(new FSTreeWalk(providedPath, conf), nnDirPath, - FixedBlockResolver.class, "", - InMemoryLevelDBAliasMapClient.class); - levelDBAliasMapServer.close(); + FixedBlockResolver.class, clusterID, + LevelDBFileRegionAliasMap.class); + return tempDirectory; + } + + @Test + public void testInMemoryAliasMap() throws Exception { + File aliasMapImage = createInMemoryAliasMapImage(); // start cluster with two datanodes, // each with 1 PROVIDED volume and other DISK volume + conf.setBoolean(DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true); + conf.setInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 10); startCluster(nnDirPath, 2, new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null, false); - verifyFileSystemContents(); - FileUtils.deleteDirectory(tempDirectory); + verifyFileSystemContents(0); + FileUtils.deleteDirectory(aliasMapImage); + } + + /** + * Find a free port that hasn't been assigned yet. + * + * @param usedPorts set of ports that have already been assigned. + * @param maxTrials maximum number of random ports to try before failure. + * @return an unassigned port. + */ + private int getUnAssignedPort(Set usedPorts, int maxTrials) { + int count = 0; + while (count < maxTrials) { + int port = NetUtils.getFreeSocketPort(); + if (usedPorts.contains(port)) { + count++; + } else { + return port; + } + } + return -1; + } + + private static String providedNameservice; + + /** + * Extends the {@link MiniDFSCluster.Builder} to create instances of + * {@link MiniDFSClusterBuilderAliasMap}. + */ + private static class MiniDFSClusterBuilderAliasMap + extends MiniDFSCluster.Builder { + + MiniDFSClusterBuilderAliasMap(Configuration conf) { + super(conf); + } + + @Override + public MiniDFSCluster build() throws IOException { + return new MiniDFSClusterAliasMap(this); + } + } + + /** + * Extends {@link MiniDFSCluster} to correctly configure the InMemoryAliasMap. + */ + private static class MiniDFSClusterAliasMap extends MiniDFSCluster { + + private Map> formattedDirsByNamespaceId; + private Set completedNNs; + + MiniDFSClusterAliasMap(MiniDFSCluster.Builder builder) throws IOException { + super(builder); + } + + @Override + protected void initNameNodeConf(Configuration conf, String nameserviceId, + int nsIndex, String nnId, boolean manageNameDfsDirs, + boolean enableManagedDfsDirsRedundancy, int nnIndex) + throws IOException { + + if (formattedDirsByNamespaceId == null) { + formattedDirsByNamespaceId = new HashMap<>(); + completedNNs = new HashSet<>(); + } + + super.initNameNodeConf(conf, nameserviceId, nsIndex, nnId, + manageNameDfsDirs, enableManagedDfsDirsRedundancy, nnIndex); + + if (providedNameservice.equals(nameserviceId)) { + // configure the InMemoryAliasMp. + conf.setBoolean(DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true); + String directory = conf.get(DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR); + if (directory == null) { + throw new IllegalArgumentException("In-memory alias map configured" + + "with the proper location; Set " + + DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR); + } + // get the name of the directory (final component in path) used for map. + // Assume that the aliasmap configured with the same final component + // name in all Namenodes but is located in the path specified by + // DFS_NAMENODE_NAME_DIR_KEY + String dirName = new Path(directory).getName(); + String nnDir = + conf.getTrimmedStringCollection(DFS_NAMENODE_NAME_DIR_KEY) + .iterator().next(); + conf.set(DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR, + new File(new Path(nnDir, dirName).toUri()).getAbsolutePath()); + conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true); + } else { + if (!completedNNs.contains(nnIndex)) { + // format the NN directories for non-provided namespaces + // if the directory for a namespace has been formatted, copy it over. + Collection namespaceDirs = FSNamesystem.getNamespaceDirs(conf); + if (formattedDirsByNamespaceId.containsKey(nameserviceId)) { + copyNameDirs(formattedDirsByNamespaceId.get(nameserviceId), + namespaceDirs, conf); + } else { + for (URI nameDirUri : namespaceDirs) { + File nameDir = new File(nameDirUri); + if (nameDir.exists() && !FileUtil.fullyDelete(nameDir)) { + throw new IOException("Could not fully delete " + nameDir); + } + } + HdfsServerConstants.StartupOption.FORMAT.setClusterId(clusterID); + DFSTestUtil.formatNameNode(conf); + formattedDirsByNamespaceId.put(nameserviceId, namespaceDirs); + } + conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, false); + completedNNs.add(nnIndex); + } + } + } + } + + /** + * Configures the addresseses of the InMemoryAliasMap. + * + * @param topology the MiniDFS topology to use. + * @param providedNameservice the nameservice id that supports provided. + */ + private void configureAliasMapAddresses(MiniDFSNNTopology topology, + String providedNameservice) { + conf.unset(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS); + Set assignedPorts = new HashSet<>(); + for (MiniDFSNNTopology.NSConf nsConf : topology.getNameservices()) { + for (MiniDFSNNTopology.NNConf nnConf : nsConf.getNNs()) { + if (providedNameservice.equals(nsConf.getId())) { + String key = + DFSUtil.addKeySuffixes(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS, + nsConf.getId(), nnConf.getNnId()); + int port = getUnAssignedPort(assignedPorts, 10); + if (port == -1) { + throw new RuntimeException("No free ports available"); + } + assignedPorts.add(port); + conf.set(key, "127.0.0.1:" + port); + + String binHostKey = + DFSUtil.addKeySuffixes( + DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_BIND_HOST, + nsConf.getId(), nnConf.getNnId()); + conf.set(binHostKey, "0.0.0.0"); + } + } + } + } + + /** + * Verify the mounted contents of the Filesystem. + * + * @param topology the topology of the cluster. + * @param providedNameservice the namespace id of the provided namenodes. + * @throws Exception + */ + private void verifyPathsWithHAFailoverIfNecessary(MiniDFSNNTopology topology, + String providedNameservice) throws Exception { + List nnIndexes = cluster.getNNIndexes(providedNameservice); + if (topology.isHA()) { + int nn1 = nnIndexes.get(0); + int nn2 = nnIndexes.get(1); + try { + verifyFileSystemContents(nn1); + fail("Read operation should fail as no Namenode is active"); + } catch (RemoteException e) { + LOG.info("verifyPaths failed!. Expected exception: {}" + e); + } + cluster.transitionToActive(nn1); + LOG.info("Verifying data from NN with index = {}", nn1); + verifyFileSystemContents(nn1); + // transition to the second namenode. + cluster.transitionToStandby(nn1); + cluster.transitionToActive(nn2); + LOG.info("Verifying data from NN with index = {}", nn2); + verifyFileSystemContents(nn2); + + cluster.shutdownNameNodes(); + try { + verifyFileSystemContents(nn2); + fail("Read operation should fail as no Namenode is active"); + } catch (NullPointerException e) { + LOG.info("verifyPaths failed!. Expected exception: {}" + e); + } + } else { + verifyFileSystemContents(nnIndexes.get(0)); + } + } + + @Test + public void testInMemoryAliasMapMultiTopologies() throws Exception { + MiniDFSNNTopology[] topologies = + new MiniDFSNNTopology[] { + MiniDFSNNTopology.simpleHATopology(), + MiniDFSNNTopology.simpleFederatedTopology(3), + MiniDFSNNTopology.simpleHAFederatedTopology(3) + }; + + for (MiniDFSNNTopology topology : topologies) { + LOG.info("Starting test with topology with HA = {}, federation = {}", + topology.isHA(), topology.isFederated()); + setSeed(); + createInMemoryAliasMapImage(); + conf.setBoolean(DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true); + conf.setInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 10); + providedNameservice = topology.getNameservices().get(0).getId(); + // configure the AliasMap addresses + configureAliasMapAddresses(topology, providedNameservice); + startCluster(nnDirPath, 2, + new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, + null, false, null, topology, + new MiniDFSClusterBuilderAliasMap(conf)); + + verifyPathsWithHAFailoverIfNecessary(topology, providedNameservice); + shutdown(); + } } private DatanodeDescriptor getDatanodeDescriptor(DatanodeManager dnm, @@ -919,7 +1192,7 @@ public class ITestProvidedImplementation { startCluster(nnDirPath, racks.length, new StorageType[]{StorageType.PROVIDED, StorageType.DISK}, null, false, racks); - verifyFileSystemContents(); + verifyFileSystemContents(0); setAndUnsetReplication("/" + filePrefix + (numFiles - 1) + fileSuffix); cluster.shutdown(); } From ab2f8343a9810c9ec8b0920215a0897e7f671aba Mon Sep 17 00:00:00 2001 From: Nanda kumar Date: Tue, 3 Jul 2018 00:51:16 +0530 Subject: [PATCH 16/47] HDDS-206. Ozone shell command doesn't respect KSM port set in ozone-site.xml. Contributed by Shashikant Banerjee. --- .../ozone/client/OzoneClientFactory.java | 13 +++++----- .../org/apache/hadoop/ozone/KsmUtils.java | 24 ++++++++++++------- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java index dae94aa956d..3085b0daa68 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java @@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.client; import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.KsmUtils; import org.apache.hadoop.ozone.client.protocol.ClientProtocol; import org.apache.hadoop.ozone.client.rest.RestClient; import org.apache.hadoop.ozone.client.rpc.RpcClient; @@ -37,10 +38,7 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys .OZONE_CLIENT_PROTOCOL; import static org.apache.hadoop.ozone.ksm.KSMConfigKeys .OZONE_KSM_HTTP_ADDRESS_KEY; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys - .OZONE_KSM_HTTP_BIND_PORT_DEFAULT; import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_ADDRESS_KEY; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_PORT_DEFAULT; /** * Factory class to create different types of OzoneClients. @@ -108,8 +106,9 @@ public final class OzoneClientFactory { */ public static OzoneClient getRpcClient(String ksmHost) throws IOException { - return getRpcClient(ksmHost, OZONE_KSM_PORT_DEFAULT, - new OzoneConfiguration()); + Configuration config = new OzoneConfiguration(); + int port = KsmUtils.getKsmRpcPort(config); + return getRpcClient(ksmHost, port, config); } /** @@ -185,7 +184,9 @@ public final class OzoneClientFactory { */ public static OzoneClient getRestClient(String ksmHost) throws IOException { - return getRestClient(ksmHost, OZONE_KSM_HTTP_BIND_PORT_DEFAULT); + Configuration config = new OzoneConfiguration(); + int port = KsmUtils.getKsmRestPort(config); + return getRestClient(ksmHost, port, config); } /** diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/KsmUtils.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/KsmUtils.java index ebada1c6e47..1025963aa87 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/KsmUtils.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/KsmUtils.java @@ -26,6 +26,8 @@ import com.google.common.base.Optional; import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys; import static org.apache.hadoop.hdds.HddsUtils.getPortNumberFromConfigKeys; import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_ADDRESS_KEY; +import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY; +import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_HTTP_BIND_PORT_DEFAULT; import static org.apache.hadoop.ozone.ksm.KSMConfigKeys .OZONE_KSM_BIND_HOST_DEFAULT; import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_PORT_DEFAULT; @@ -49,13 +51,9 @@ public final class KsmUtils { final Optional host = getHostNameFromConfigKeys(conf, OZONE_KSM_ADDRESS_KEY); - // If no port number is specified then we'll just try the defaultBindPort. - final Optional port = getPortNumberFromConfigKeys(conf, - OZONE_KSM_ADDRESS_KEY); - return NetUtils.createSocketAddr( host.or(OZONE_KSM_BIND_HOST_DEFAULT) + ":" + - port.or(OZONE_KSM_PORT_DEFAULT)); + getKsmRpcPort(conf)); } /** @@ -76,12 +74,22 @@ public final class KsmUtils { " details on configuring Ozone."); } + return NetUtils.createSocketAddr( + host.get() + ":" + getKsmRpcPort(conf)); + } + + public static int getKsmRpcPort(Configuration conf) { // If no port number is specified then we'll just try the defaultBindPort. final Optional port = getPortNumberFromConfigKeys(conf, OZONE_KSM_ADDRESS_KEY); - - return NetUtils.createSocketAddr( - host.get() + ":" + port.or(OZONE_KSM_PORT_DEFAULT)); + return port.or(OZONE_KSM_PORT_DEFAULT); } + public static int getKsmRestPort(Configuration conf) { + // If no port number is specified then we'll just try the default + // HTTP BindPort. + final Optional port = + getPortNumberFromConfigKeys(conf, OZONE_KSM_HTTP_ADDRESS_KEY); + return port.or(OZONE_KSM_HTTP_BIND_PORT_DEFAULT); + } } From 53e267fa7232add3c21174382d91b2607aa6becf Mon Sep 17 00:00:00 2001 From: Shane Kumpf Date: Mon, 2 Jul 2018 16:18:32 -0600 Subject: [PATCH 17/47] YARN-8485. Priviledged container app launch is failing intermittently. Contributed by Eric Yang --- .../src/main/native/container-executor/impl/utils/docker-util.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c index ffc349a10fd..d364227a57a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c @@ -1235,7 +1235,7 @@ static int check_privileges(const char *user) { if (ret != 1) { int child_pid = fork(); if (child_pid == 0) { - execl("/bin/sudo", "sudo", "-U", user, "-n", "-l", "docker", NULL); + execl("/usr/bin/sudo", "sudo", "-U", user, "-n", "-l", "docker", NULL); exit(INITIALIZE_USER_FAILED); } else { while ((waitid = waitpid(child_pid, &statval, 0)) != child_pid) { From fa9ef15ecd6dc30fb260e1c342a2b51505d39b6b Mon Sep 17 00:00:00 2001 From: Sunil G Date: Mon, 2 Jul 2018 15:34:37 -0700 Subject: [PATCH 18/47] YARN-8415. TimelineWebServices.getEntity should throw ForbiddenException instead of 404 when ACL checks fail. Contributed by Suma Shivaprasad. --- .../yarn/server/timeline/RollingLevelDBTimelineStore.java | 6 ++++++ .../hadoop/yarn/server/timeline/TimelineDataManager.java | 7 ++++++- .../yarn/server/timeline/webapp/TimelineWebServices.java | 4 ++++ .../server/timeline/webapp/TestTimelineWebServices.java | 2 +- 4 files changed, 17 insertions(+), 2 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDBTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDBTimelineStore.java index 36b5ce84307..255547b18b6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDBTimelineStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDBTimelineStore.java @@ -413,6 +413,9 @@ public class RollingLevelDBTimelineStore extends AbstractService implements EnumSet fields) throws IOException { Long revStartTime = getStartTimeLong(entityId, entityType); if (revStartTime == null) { + if ( LOG.isDebugEnabled()) { + LOG.debug("Could not find start time for {} {} ", entityType, entityId); + } return null; } byte[] prefix = KeyBuilder.newInstance().add(entityType) @@ -421,6 +424,9 @@ public class RollingLevelDBTimelineStore extends AbstractService implements DB db = entitydb.getDBForStartTime(revStartTime); if (db == null) { + if ( LOG.isDebugEnabled()) { + LOG.debug("Could not find db for {} {} ", entityType, entityId); + } return null; } try (DBIterator iterator = db.iterator()) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java index 56b71faf2ed..c5381967e13 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java @@ -219,7 +219,12 @@ public class TimelineDataManager extends AbstractService { // check ACLs if (!timelineACLsManager.checkAccess( callerUGI, ApplicationAccessType.VIEW_APP, entity)) { - entity = null; + final String user = callerUGI != null ? callerUGI.getShortUserName(): + null; + throw new YarnException( + user + " is not allowed to get the timeline entity " + + "{ id: " + entity.getEntityId() + ", type: " + + entity.getEntityType() + " }."); } } return entity; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java index be8e3c599e0..9423e7f71b5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java @@ -162,6 +162,10 @@ public class TimelineWebServices { parseStr(entityId), parseFieldsStr(fields, ","), getUser(req)); + } catch (YarnException e) { + // The user doesn't have the access to override the existing domain. + LOG.info(e.getMessage(), e); + throw new ForbiddenException(e); } catch (IllegalArgumentException e) { throw new BadRequestException(e); } catch (Exception e) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java index ca78cbcbab3..b6d2967d2ac 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java @@ -709,7 +709,7 @@ public class TestTimelineWebServices extends JerseyTestBase { .get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8, response.getType().toString()); - assertResponseStatusCode(Status.NOT_FOUND, response.getStatusInfo()); + assertResponseStatusCode(Status.FORBIDDEN, response.getStatusInfo()); } finally { timelineACLsManager.setAdminACLsManager(oldAdminACLsManager); } From 7296b644f7c44400f03995668573a4cfd2ee552c Mon Sep 17 00:00:00 2001 From: Inigo Goiri Date: Mon, 2 Jul 2018 16:36:25 -0700 Subject: [PATCH 19/47] HDFS-13714. Fix TestNameNodePrunesMissingStorages test failures on Windows. Contributed by Lukas Majercak. --- .../blockmanagement/TestNameNodePrunesMissingStorages.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java index 96d227d1779..05b6d3023b3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java @@ -294,8 +294,9 @@ public class TestNameNodePrunesMissingStorages { in = null; out.close(); out = null; - newVersionFile.renameTo(versionFile); - success = true; + // Delete old version file + success = versionFile.delete(); + success &= newVersionFile.renameTo(versionFile); } finally { if (in != null) { in.close(); From 59a3038bc3d7913dca3de971026bc32cef536a2d Mon Sep 17 00:00:00 2001 From: Bharat Viswanadham Date: Mon, 2 Jul 2018 21:43:18 -0700 Subject: [PATCH 20/47] HDFS-13715:diskbalancer does not work if one of the blockpools are empty on a Federated cluster. Contributed by Bharat Viswanadham --- .../hdfs/server/datanode/DiskBalancer.java | 29 ++++--- .../fsdataset/impl/FsDatasetImpl.java | 8 ++ .../server/diskbalancer/TestDiskBalancer.java | 75 +++++++++++++++++-- 3 files changed, 96 insertions(+), 16 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java index 91c3624024d..53db022dbf5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hdfs.server.datanode; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.commons.codec.digest.DigestUtils; import org.apache.hadoop.classification.InterfaceAudience; @@ -77,7 +78,8 @@ import static java.util.concurrent.TimeUnit.SECONDS; @InterfaceAudience.Private public class DiskBalancer { - private static final Logger LOG = LoggerFactory.getLogger(DiskBalancer + @VisibleForTesting + public static final Logger LOG = LoggerFactory.getLogger(DiskBalancer .class); private final FsDatasetSpi dataset; private final String dataNodeUUID; @@ -902,15 +904,19 @@ public class DiskBalancer { try { ExtendedBlock block = iter.nextBlock(); - // A valid block is a finalized block, we iterate until we get - // finalized blocks - if (!this.dataset.isValidBlock(block)) { - continue; - } + if (block != null) { + // A valid block is a finalized block, we iterate until we get + // finalized blocks + if (!this.dataset.isValidBlock(block)) { + continue; + } - // We don't look for the best, we just do first fit - if (isLessThanNeeded(block.getNumBytes(), item)) { - return block; + // We don't look for the best, we just do first fit + if (isLessThanNeeded(block.getNumBytes(), item)) { + return block; + } + } else { + LOG.info("There are no blocks in the blockPool {}", iter.getBlockPoolId()); } } catch (IOException e) { @@ -1126,6 +1132,11 @@ public class DiskBalancer { Thread.currentThread().interrupt(); item.incErrorCount(); this.setExitFlag(); + } catch (RuntimeException ex) { + // Exiting if any run time exceptions. + LOG.error("Got an unexpected Runtime Exception {}", ex); + item.incErrorCount(); + this.setExitFlag(); } } } finally { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java index 695a421d410..89c278a34c4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java @@ -1927,6 +1927,10 @@ class FsDatasetImpl implements FsDatasetSpi { */ @Override // FsDatasetSpi public boolean isValidBlock(ExtendedBlock b) { + // If block passed is null, we should return false. + if (b == null) { + return false; + } return isValid(b, ReplicaState.FINALIZED); } @@ -1935,6 +1939,10 @@ class FsDatasetImpl implements FsDatasetSpi { */ @Override // {@link FsDatasetSpi} public boolean isValidRbw(final ExtendedBlock b) { + // If block passed is null, we should return false. + if (b == null) { + return false; + } return isValid(b, ReplicaState.RBW); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java index deae6eab7ac..55cc57ed03f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java @@ -154,7 +154,7 @@ public class TestDiskBalancer { NodePlan plan = dataMover.generatePlan(); dataMover.executePlan(plan); dataMover.verifyPlanExectionDone(); - dataMover.verifyAllVolumesHaveData(); + dataMover.verifyAllVolumesHaveData(true); dataMover.verifyTolerance(plan, 0, sourceDiskIndex, 10); } finally { cluster.shutdown(); @@ -209,7 +209,7 @@ public class TestDiskBalancer { NodePlan plan = dataMover.generatePlan(); dataMover.executePlan(plan); dataMover.verifyPlanExectionDone(); - dataMover.verifyAllVolumesHaveData(); + dataMover.verifyAllVolumesHaveData(true); dataMover.verifyTolerance(plan, 0, sourceDiskIndex, 10); } finally { cluster.shutdown(); @@ -217,6 +217,66 @@ public class TestDiskBalancer { } + + @Test + public void testDiskBalancerWithFedClusterWithOneNameServiceEmpty() throws + Exception { + Configuration conf = new HdfsConfiguration(); + conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true); + final int blockCount = 100; + final int blockSize = 1024; + final int diskCount = 2; + final int dataNodeCount = 1; + final int dataNodeIndex = 0; + final int sourceDiskIndex = 0; + final long cap = blockSize * 3L * blockCount; + + conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, blockSize); + + final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf) + .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2)) + .numDataNodes(dataNodeCount) + .storagesPerDatanode(diskCount) + .storageCapacities(new long[] {cap, cap}) + .build(); + cluster.waitActive(); + + DFSTestUtil.setFederatedConfiguration(cluster, conf); + + final String fileName = "/tmp.txt"; + final Path filePath = new Path(fileName); + long fileLen = blockCount * blockSize; + + + //Writing data only to one nameservice. + FileSystem fs = cluster.getFileSystem(0); + TestBalancer.createFile(cluster, filePath, fileLen, (short) 1, + 0); + DFSTestUtil.waitReplication(fs, filePath, (short) 1); + + + GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer + .captureLogs(DiskBalancer.LOG); + + try { + DataMover dataMover = new DataMover(cluster, dataNodeIndex, + sourceDiskIndex, conf, blockSize, blockCount); + dataMover.moveDataToSourceDisk(); + NodePlan plan = dataMover.generatePlan(); + dataMover.executePlan(plan); + dataMover.verifyPlanExectionDone(); + //Because here we have one nameservice empty, don't check + // blockPoolCount. + dataMover.verifyAllVolumesHaveData(false); + } finally { + Assert.assertTrue(logCapturer.getOutput().contains("There are no " + + "blocks in the blockPool")); + cluster.shutdown(); + } + + } + @Test public void testBalanceDataBetweenMultiplePairsOfVolumes() throws Exception { @@ -255,7 +315,7 @@ public class TestDiskBalancer { dataMover.executePlan(plan); dataMover.verifyPlanExectionDone(); - dataMover.verifyAllVolumesHaveData(); + dataMover.verifyAllVolumesHaveData(true); dataMover.verifyTolerance(plan, 0, sourceDiskIndex, 10); } finally { cluster.shutdown(); @@ -296,7 +356,7 @@ public class TestDiskBalancer { dataMover.moveDataToSourceDisk(); NodePlan plan = dataMover.generatePlan(); dataMover.executePlanDuringDiskRemove(plan); - dataMover.verifyAllVolumesHaveData(); + dataMover.verifyAllVolumesHaveData(true); dataMover.verifyTolerance(plan, 0, sourceDiskIndex, 10); } catch (Exception e) { Assert.fail("Unexpected exception: " + e); @@ -651,14 +711,15 @@ public class TestDiskBalancer { /** * Once diskBalancer is run, all volumes mush has some data. */ - public void verifyAllVolumesHaveData() throws IOException { + public void verifyAllVolumesHaveData(boolean checkblockPoolCount) throws + IOException { node = cluster.getDataNodes().get(dataNodeIndex); try (FsDatasetSpi.FsVolumeReferences refs = node.getFSDataset().getFsVolumeReferences()) { for (FsVolumeSpi volume : refs) { - assertTrue(DiskBalancerTestUtil.getBlockCount(volume, true) > 0); + assertTrue(DiskBalancerTestUtil.getBlockCount(volume, checkblockPoolCount) > 0); LOG.info("{} : Block Count : {}", refs, DiskBalancerTestUtil - .getBlockCount(volume, true)); + .getBlockCount(volume, checkblockPoolCount)); } } } From d9ba6f3656e8dc97d2813181e27d12e52dca4328 Mon Sep 17 00:00:00 2001 From: Junping Du Date: Tue, 3 Jul 2018 14:46:44 +0800 Subject: [PATCH 21/47] yarn.resourcemanager.fail-fast is used inconsistently. Contributed by Yuanbo Liu. --- .../conf/capacity-scheduler.xml | 10 ++++++++++ .../scheduler/capacity/CapacityScheduler.java | 6 +++--- .../capacity/CapacitySchedulerConfiguration.java | 10 ++++++++++ .../resourcemanager/TestWorkPreservingRMRestart.java | 2 ++ 4 files changed, 25 insertions(+), 3 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml index 62654cacc48..38526d18dbf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml @@ -207,4 +207,14 @@ + + + yarn.scheduler.capacity.application.fail-fast + false + + Whether RM should fail during recovery if previous applications' + queue is no longer valid. + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index 54bbf24ef77..b59636a4c45 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -786,7 +786,7 @@ public class CapacityScheduler extends if (queue == null) { //During a restart, this indicates a queue was removed, which is //not presently supported - if (!YarnConfiguration.shouldRMFailFast(getConfig())) { + if (!getConfiguration().shouldAppFailFast(getConfig())) { this.rmContext.getDispatcher().getEventHandler().handle( new RMAppEvent(applicationId, RMAppEventType.KILL, "Application killed on recovery as it" @@ -807,7 +807,7 @@ public class CapacityScheduler extends if (!(queue instanceof LeafQueue)) { // During RM restart, this means leaf queue was converted to a parent // queue, which is not supported for running apps. - if (!YarnConfiguration.shouldRMFailFast(getConfig())) { + if (!getConfiguration().shouldAppFailFast(getConfig())) { this.rmContext.getDispatcher().getEventHandler().handle( new RMAppEvent(applicationId, RMAppEventType.KILL, "Application killed on recovery as it was " @@ -866,7 +866,7 @@ public class CapacityScheduler extends return autoCreateLeafQueue(placementContext); } catch (YarnException | IOException e) { if (isRecovery) { - if (!YarnConfiguration.shouldRMFailFast(getConfig())) { + if (!getConfiguration().shouldAppFailFast(getConfig())) { LOG.error("Could not auto-create leaf queue " + queueName + " due to : ", e); this.rmContext.getDispatcher().getEventHandler().handle( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java index f94654e2802..e8de096c6a4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java @@ -249,6 +249,12 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur public static final String SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_PENDING_BACKLOGS = SCHEDULE_ASYNCHRONOUSLY_PREFIX + ".maximum-pending-backlogs"; + @Private + public static final String APP_FAIL_FAST = PREFIX + "application.fail-fast"; + + @Private + public static final boolean DEFAULT_APP_FAIL_FAST = false; + @Private public static final Integer DEFAULT_SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_PENDING_BACKLOGS = 100; @@ -1336,6 +1342,10 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur return getBoolean(LAZY_PREEMPTION_ENABLED, DEFAULT_LAZY_PREEMPTION_ENABLED); } + public boolean shouldAppFailFast(Configuration conf) { + return conf.getBoolean(APP_FAIL_FAST, DEFAULT_APP_FAIL_FAST); + } + private static final String PREEMPTION_CONFIG_PREFIX = "yarn.resourcemanager.monitor.capacity.preemption."; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java index e4c83e32dc7..88c19a1d075 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java @@ -760,6 +760,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase MockMemoryRMStateStore memStore, RMState state) throws Exception { // Restart RM with fail-fast as false. App should be killed. csConf.setBoolean(YarnConfiguration.RM_FAIL_FAST, false); + csConf.setBoolean(CapacitySchedulerConfiguration.APP_FAIL_FAST, false); rm2 = new MockRM(csConf, memStore); rm2.start(); @@ -794,6 +795,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase // Now restart RM with fail-fast as true. QueueException should be thrown. csConf.setBoolean(YarnConfiguration.RM_FAIL_FAST, true); + csConf.setBoolean(CapacitySchedulerConfiguration.APP_FAIL_FAST, true); MockRM rm = new MockRM(csConf, memStore2); try { rm.start(); From 344f324710522ffb27852c1a673c4f7d3d6eac4b Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Tue, 3 Jul 2018 11:07:45 +0200 Subject: [PATCH 22/47] HDFS-13712. BlockReaderRemote.read() logging improvement. Contributed by Gergo Repas. --- .../hadoop/hdfs/client/impl/BlockReaderRemote.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote.java index caf15e41a0d..ea1baed45d0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote.java @@ -129,16 +129,22 @@ public class BlockReaderRemote implements BlockReader { @Override public synchronized int read(byte[] buf, int off, int len) throws IOException { - UUID randomId = (LOG.isTraceEnabled() ? UUID.randomUUID() : null); - LOG.trace("Starting read #{} file {} from datanode {}", - randomId, filename, datanodeID.getHostName()); + boolean logTraceEnabled = LOG.isTraceEnabled(); + UUID randomId = null; + if (logTraceEnabled) { + randomId = UUID.randomUUID(); + LOG.trace("Starting read #{} file {} from datanode {}", + randomId, filename, datanodeID.getHostName()); + } if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) { readNextPacket(); } - LOG.trace("Finishing read #{}", randomId); + if (logTraceEnabled) { + LOG.trace("Finishing read #{}", randomId); + } if (curDataSlice.remaining() == 0) { // we're at EOF now From 51654a3962bcd0482c0e1dd32765e9533e3f9158 Mon Sep 17 00:00:00 2001 From: Sunil G Date: Tue, 3 Jul 2018 09:59:20 -0700 Subject: [PATCH 23/47] YARN-8459. Improve Capacity Scheduler logs to debug invalid states. Contributed by Wangda Tan. --- .../scheduler/capacity/CapacityScheduler.java | 39 ++++++++++++------- .../scheduler/capacity/ParentQueue.java | 36 +++++++++++------ .../allocator/AbstractContainerAllocator.java | 13 ++++--- .../common/fica/FiCaSchedulerApp.java | 5 +++ 4 files changed, 63 insertions(+), 30 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index b59636a4c45..37f56deb119 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -1234,8 +1234,10 @@ public class CapacityScheduler extends updateDemandForQueue.getOrderingPolicy().demandUpdated(application); } - LOG.info("Allocation for application " + applicationAttemptId + " : " + - allocation + " with cluster resource : " + getClusterResource()); + if (LOG.isDebugEnabled()) { + LOG.info("Allocation for application " + applicationAttemptId + " : " + + allocation + " with cluster resource : " + getClusterResource()); + } return allocation; } @@ -1479,10 +1481,17 @@ public class CapacityScheduler extends private CSAssignment allocateContainerOnSingleNode( CandidateNodeSet candidates, FiCaSchedulerNode node, boolean withNodeHeartbeat) { + if (LOG.isDebugEnabled()) { + LOG.debug( + "Trying to schedule on node: " + node.getNodeName() + ", available: " + + node.getUnallocatedResource()); + } + // Backward compatible way to make sure previous behavior which allocation // driven by node heartbeat works. if (getNode(node.getNodeID()) != node) { - LOG.error("Trying to schedule on a removed node, please double check."); + LOG.error("Trying to schedule on a removed node, please double check, " + + "nodeId=" + node.getNodeID()); return null; } @@ -1496,14 +1505,19 @@ public class CapacityScheduler extends FiCaSchedulerApp reservedApplication = getCurrentAttemptForContainer( reservedContainer.getContainerId()); if (reservedApplication == null) { - LOG.error("Trying to schedule for a finished app, please double check."); + LOG.error( + "Trying to schedule for a finished app, please double check. nodeId=" + + node.getNodeID() + " container=" + reservedContainer + .getContainerId()); return null; } // Try to fulfill the reservation - LOG.info( - "Trying to fulfill reservation for application " + reservedApplication - .getApplicationId() + " on node: " + node.getNodeID()); + if (LOG.isDebugEnabled()) { + LOG.debug("Trying to fulfill reservation for application " + + reservedApplication.getApplicationId() + " on node: " + node + .getNodeID()); + } LeafQueue queue = ((LeafQueue) reservedApplication.getQueue()); assignment = queue.assignContainers(getClusterResource(), candidates, @@ -1567,12 +1581,6 @@ public class CapacityScheduler extends return null; } - if (LOG.isDebugEnabled()) { - LOG.debug( - "Trying to schedule on node: " + node.getNodeName() + ", available: " - + node.getUnallocatedResource()); - } - return allocateOrReserveNewContainers(candidates, withNodeHeartbeat); } @@ -2888,6 +2896,11 @@ public class CapacityScheduler extends LOG.info("Failed to accept allocation proposal"); } + if (LOG.isDebugEnabled()) { + LOG.debug("Allocation proposal accepted=" + isSuccess + ", proposal=" + + request); + } + // Update unconfirmed allocated resource. if (updateUnconfirmedAllocatedResource) { app.decUnconfirmedRes(request.getTotalAllocatedResource()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java index bb4823e1dd2..2363b8809e8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java @@ -90,6 +90,8 @@ public class ParentQueue extends AbstractCSQueue { private QueueOrderingPolicy queueOrderingPolicy; + private long lastSkipQueueDebugLoggingTimestamp = -1; + public ParentQueue(CapacitySchedulerContext cs, String queueName, CSQueue parent, CSQueue old) throws IOException { super(cs, queueName, parent, old); @@ -539,9 +541,14 @@ public class ParentQueue extends AbstractCSQueue { if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY && !accessibleToPartition(candidates.getPartition())) { if (LOG.isDebugEnabled()) { - LOG.debug("Skip this queue=" + getQueuePath() - + ", because it is not able to access partition=" + candidates - .getPartition()); + long now = System.currentTimeMillis(); + // Do logging every 1 sec to avoid excessive logging. + if (now - this.lastSkipQueueDebugLoggingTimestamp > 1000) { + LOG.debug("Skip this queue=" + getQueuePath() + + ", because it is not able to access partition=" + candidates + .getPartition()); + this.lastSkipQueueDebugLoggingTimestamp = now; + } } ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, @@ -561,10 +568,15 @@ public class ParentQueue extends AbstractCSQueue { if (!super.hasPendingResourceRequest(candidates.getPartition(), clusterResource, schedulingMode)) { if (LOG.isDebugEnabled()) { - LOG.debug("Skip this queue=" + getQueuePath() - + ", because it doesn't need more resource, schedulingMode=" - + schedulingMode.name() + " node-partition=" + candidates - .getPartition()); + long now = System.currentTimeMillis(); + // Do logging every 1 sec to avoid excessive logging. + if (now - this.lastSkipQueueDebugLoggingTimestamp > 1000) { + LOG.debug("Skip this queue=" + getQueuePath() + + ", because it doesn't need more resource, schedulingMode=" + + schedulingMode.name() + " node-partition=" + candidates + .getPartition()); + this.lastSkipQueueDebugLoggingTimestamp = now; + } } ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, @@ -666,12 +678,12 @@ public class ParentQueue extends AbstractCSQueue { assignment.setIncreasedAllocation( assignedToChild.isIncreasedAllocation()); - LOG.info("assignedContainer" + " queue=" + getQueueName() - + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity=" - + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() - + " cluster=" + clusterResource); - if (LOG.isDebugEnabled()) { + LOG.debug("assignedContainer reserved=" + isReserved + " queue=" + + getQueueName() + " usedCapacity=" + getUsedCapacity() + + " absoluteUsedCapacity=" + getAbsoluteUsedCapacity() + " used=" + + queueUsage.getUsed() + " cluster=" + clusterResource); + LOG.debug( "ParentQ=" + getQueueName() + " assignedSoFarInThisIteration=" + assignment.getResource() + " usedCapacity=" diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java index 36665d66440..bfc129f526f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java @@ -93,11 +93,14 @@ public abstract class AbstractContainerAllocator { assignment.setType(result.getContainerNodeType()); if (result.getAllocationState() == AllocationState.RESERVED) { - // This is a reserved container - LOG.info("Reserved container " + " application=" - + application.getApplicationId() + " resource=" + allocatedResource - + " queue=" + appInfo.getQueueName() - + " cluster=" + clusterResource); + if (LOG.isDebugEnabled()) { + // This is a reserved container + // Since re-reservation could happen again and again for already + // reserved containers. only do this in debug log. + LOG.debug("Reserved container " + " application=" + application + .getApplicationId() + " resource=" + allocatedResource + " queue=" + + appInfo.getQueueName() + " cluster=" + clusterResource); + } assignment.getAssignmentInformation().addReservationDetails( updatedContainer, application.getCSLeafQueue().getQueuePath()); assignment.getAssignmentInformation().incrReservations(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java index 3ec81915706..3b1b82ca717 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java @@ -607,6 +607,11 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { schedulerContainer.getRmContainer(), schedulerContainer.getRmContainer().getContainer(), reReservation); + + LOG.info("Reserved container=" + rmContainer.getContainerId() + + ", on node=" + schedulerContainer.getSchedulerNode() + + " with resource=" + rmContainer + .getAllocatedOrReservedResource()); } } } finally { From c0ef7e7680d882e2182f48f033109678a48742ab Mon Sep 17 00:00:00 2001 From: Anu Engineer Date: Tue, 3 Jul 2018 11:38:14 -0700 Subject: [PATCH 24/47] HDDS-198. Create AuditLogger mechanism to be used by OM, SCM and Datanode. Contributed by Dinesh Chitlangia. --- hadoop-hdds/common/pom.xml | 18 +++ .../hadoop/ozone/audit/AuditAction.java | 30 ++++ .../hadoop/ozone/audit/AuditEventStatus.java | 36 +++++ .../hadoop/ozone/audit/AuditLogger.java | 128 +++++++++++++++++ .../hadoop/ozone/audit/AuditLoggerType.java | 37 +++++ .../hadoop/ozone/audit/AuditMarker.java | 38 +++++ .../apache/hadoop/ozone/audit/Auditable.java | 32 +++++ .../hadoop/ozone/audit/package-info.java | 123 ++++++++++++++++ .../hadoop/ozone/audit/DummyAction.java | 51 +++++++ .../hadoop/ozone/audit/DummyEntity.java | 57 ++++++++ .../ozone/audit/TestOzoneAuditLogger.java | 131 ++++++++++++++++++ .../hadoop/ozone/audit/package-info.java | 23 +++ .../src/test/resources/log4j2.properties | 76 ++++++++++ .../apache/hadoop/ozone/audit/OMAction.java | 51 +++++++ .../hadoop/ozone/audit/package-info.java | 22 +++ 15 files changed, 853 insertions(+) create mode 100644 hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditAction.java create mode 100644 hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditEventStatus.java create mode 100644 hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java create mode 100644 hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java create mode 100644 hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMarker.java create mode 100644 hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/Auditable.java create mode 100644 hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java create mode 100644 hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/DummyAction.java create mode 100644 hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/DummyEntity.java create mode 100644 hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java create mode 100644 hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/package-info.java create mode 100644 hadoop-hdds/common/src/test/resources/log4j2.properties create mode 100644 hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java create mode 100644 hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java diff --git a/hadoop-hdds/common/pom.xml b/hadoop-hdds/common/pom.xml index a8a634c0a2c..406852280b5 100644 --- a/hadoop-hdds/common/pom.xml +++ b/hadoop-hdds/common/pom.xml @@ -31,6 +31,8 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> hdds true + 2.11.0 + 3.4.2 @@ -81,6 +83,22 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> test-jar + + org.apache.logging.log4j + log4j-api + ${log4j2.version} + + + org.apache.logging.log4j + log4j-core + ${log4j2.version} + + + com.lmax + disruptor + ${disruptor.version} + + diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditAction.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditAction.java new file mode 100644 index 00000000000..8c1d6f0c67d --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditAction.java @@ -0,0 +1,30 @@ +/** + * 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.ozone.audit; + +/** + * Interface to define AuditAction. + */ +public interface AuditAction { + /** + * Implementation must override. + * @return String + */ + String getAction(); +} + diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditEventStatus.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditEventStatus.java new file mode 100644 index 00000000000..098ab6b2f7f --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditEventStatus.java @@ -0,0 +1,36 @@ +/** + * 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.ozone.audit; + +/** + * Enum to define AuditEventStatus values. + */ +public enum AuditEventStatus { + SUCCESS("SUCCESS"), + FAILURE("FAILURE"); + + private String status; + + AuditEventStatus(String status){ + this.status = status; + } + + public String getStatus() { + return status; + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java new file mode 100644 index 00000000000..46ffaab9ef5 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java @@ -0,0 +1,128 @@ +/** + * 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.ozone.audit; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Marker; +import org.apache.logging.log4j.message.StructuredDataMessage; +import org.apache.logging.log4j.spi.ExtendedLogger; + +import java.util.Map; + +/** + * Class to define Audit Logger for Ozone. + */ +public class AuditLogger { + + private ExtendedLogger logger; + + private static final String SUCCESS = AuditEventStatus.SUCCESS.getStatus(); + private static final String FAILURE = AuditEventStatus.FAILURE.getStatus(); + private static final String FQCN = AuditLogger.class.getName(); + private static final Marker WRITE_MARKER = AuditMarker.WRITE.getMarker(); + private static final Marker READ_MARKER = AuditMarker.READ.getMarker(); + + /** + * Parametrized Constructor to initialize logger. + * @param type + */ + public AuditLogger(AuditLoggerType type){ + initializeLogger(type); + } + + /** + * Initializes the logger with specific type. + * @param loggerType specified one of the values from enum AuditLoggerType. + */ + private void initializeLogger(AuditLoggerType loggerType){ + this.logger = LogManager.getContext(false).getLogger(loggerType.getType()); + } + + @VisibleForTesting + public ExtendedLogger getLogger() { + return logger; + } + + public void logWriteSuccess(AuditAction type, Map data) { + logWriteSuccess(type, data, Level.INFO); + } + + public void logWriteSuccess(AuditAction type, Map data, Level + level) { + StructuredDataMessage msg = new StructuredDataMessage("", SUCCESS, + type.getAction(), data); + this.logger.logIfEnabled(FQCN, level, WRITE_MARKER, msg, null); + } + + + public void logWriteFailure(AuditAction type, Map data) { + logWriteFailure(type, data, Level.INFO, null); + } + + public void logWriteFailure(AuditAction type, Map data, Level + level) { + logWriteFailure(type, data, level, null); + } + + public void logWriteFailure(AuditAction type, Map data, + Throwable exception) { + logWriteFailure(type, data, Level.INFO, exception); + } + + public void logWriteFailure(AuditAction type, Map data, Level + level, Throwable exception) { + StructuredDataMessage msg = new StructuredDataMessage("", FAILURE, + type.getAction(), data); + this.logger.logIfEnabled(FQCN, level, WRITE_MARKER, msg, exception); + } + + public void logReadSuccess(AuditAction type, Map data) { + logReadSuccess(type, data, Level.INFO); + } + + public void logReadSuccess(AuditAction type, Map data, Level + level) { + StructuredDataMessage msg = new StructuredDataMessage("", SUCCESS, + type.getAction(), data); + this.logger.logIfEnabled(FQCN, level, READ_MARKER, msg, null); + } + + public void logReadFailure(AuditAction type, Map data) { + logReadFailure(type, data, Level.INFO, null); + } + + public void logReadFailure(AuditAction type, Map data, Level + level) { + logReadFailure(type, data, level, null); + } + + public void logReadFailure(AuditAction type, Map data, + Throwable exception) { + logReadFailure(type, data, Level.INFO, exception); + } + + public void logReadFailure(AuditAction type, Map data, Level + level, Throwable exception) { + StructuredDataMessage msg = new StructuredDataMessage("", FAILURE, + type.getAction(), data); + this.logger.logIfEnabled(FQCN, level, READ_MARKER, msg, exception); + } + +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java new file mode 100644 index 00000000000..18241c7712a --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java @@ -0,0 +1,37 @@ +/** + * 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.ozone.audit; + +/** + * Enumeration for defining types of Audit Loggers in Ozone. + */ +public enum AuditLoggerType { + DNLOGGER("DNAudit"), + OMLOGGER("OMAudit"), + SCMLOGGER("SCMAudit"); + + private String type; + + public String getType() { + return type; + } + + AuditLoggerType(String type){ + this.type = type; + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMarker.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMarker.java new file mode 100644 index 00000000000..505b9580715 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMarker.java @@ -0,0 +1,38 @@ +/** + * 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.ozone.audit; + +import org.apache.logging.log4j.Marker; +import org.apache.logging.log4j.MarkerManager; + +/** + * Defines audit marker types. + */ +public enum AuditMarker { + WRITE(MarkerManager.getMarker("WRITE")), + READ(MarkerManager.getMarker("READ")); + + private Marker marker; + + AuditMarker(Marker marker){ + this.marker = marker; + } + + public Marker getMarker(){ + return marker; + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/Auditable.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/Auditable.java new file mode 100644 index 00000000000..d388bca72f1 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/Auditable.java @@ -0,0 +1,32 @@ +/** + * 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.ozone.audit; + +import java.util.Map; + +/** + * Interface to make an entity auditable. + */ +public interface Auditable { + /** + * Must override in implementation. + * @return Map with values to be logged in audit. + */ + Map toAuditMap(); +} + diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java new file mode 100644 index 00000000000..3743fddd4f2 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java @@ -0,0 +1,123 @@ +/** + * 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.ozone.audit; +/** + ****************************************************************************** + * Important + * 1. Any changes to classes in this package can render the logging + * framework broken. + * 2. The logger framework has been designed keeping in mind future + * plans to build a log parser. + * 3. Please exercise great caution when attempting changes in this package. + ****************************************************************************** + * + * + * This package lays the foundation for Audit logging in Ozone. + * AuditLogging in Ozone has been built using log4j2 which brings in new + * features that facilitate turning on/off selective audit events by using + * MarkerFilter, checking for change in logging configuration periodically + * and reloading the changes, use of disruptor framework for improved + * Asynchronous logging. + * + * The log4j2 configurations can be specified in XML, YAML, JSON and + * Properties file. For Ozone, we are using the Properties file due to sheer + * simplicity, readability and ease of modification. + * + * log4j2 configuration file can be passed to startup command with option + * -Dlog4j.configurationFile unlike -Dlog4j.configuration in log4j 1.x + * + ****************************************************************************** + * Understanding the Audit Logging framework in Ozone. + ****************************************************************************** + * **** Auditable *** + * This is an interface to mark an entity as auditable. + * This interface must be implemented by entities requiring audit logging. + * For example - KSMVolumeArgs, KSMBucketArgs. + * The implementing class must override toAuditMap() to return an + * instance of Map where both Key and Value are String. + * + * Key: must not contain any spaces. If the key is multi word then use + * camel case. + * Value: if it is a collection/array, then it must be converted to a comma + * delimited string + * + * *** AuditAction *** + * This is an interface to define the various type of actions to be audited. + * To ensure separation of concern, for each sub-component you must create an + * Enum to implement AuditAction. + * Structure of Enum can be referred from the test class DummyAction. + * + * For starters, we expect following 3 implementations of AuditAction: + * OMAction - to define action types for Ozone Manager + * SCMAction - to define action types for Storage Container manager + * DNAction - to define action types for Datanode + * + * *** AuditEventStatus *** + * Enum to define Audit event status like success and failure. + * This is used in AuditLogger.logXXX() methods. + * + * * *** AuditLogger *** + * This is where the audit logging magic unfolds. + * The class has 2 Markers defined - READ and WRITE. + * These markers are used to tag when logging events. + * + * *** AuditLoggerType *** + * Enum to define the various AuditLoggers in Ozone + * + * *** AuditMarker *** + * Enum to define various Audit Markers used in AuditLogging. + * + * **************************************************************************** + * Usage + * **************************************************************************** + * Using the AuditLogger to log events: + * 1. Get a logger by specifying the appropriate logger type + * Example: ExtendedLogger AUDIT = new AuditLogger(AuditLoggerType.OMLogger) + * + * 2. Log Read/Write and Success/Failure event as needed. + * Example + * AUDIT.logWriteSuccess(AuditAction type, Map data, Level + * level) + * + * If logging is done without specifying Level, then Level implicitly + * defaults to INFO + * AUDIT.logWriteSuccess(AuditAction type, Map data) + * + * See sample invocations in src/test in the following class: + * org.apache.hadoop.ozone.audit.TestOzoneAuditLogger + * + * **************************************************************************** + * Defining new Logger types + * **************************************************************************** + * New Logger type can be added with following steps: + * 1. Update AuditLoggerType to add the new type + * 2. Create new Enum by implementing AuditAction if needed + * 3. Ensure the required entity implements Auditable + * + * **************************************************************************** + * Defining new Marker types + * **************************************************************************** + * New Markers can be configured as follows: + * 1. Define new markers in AuditMarker + * 2. Get the Marker in AuditLogger for use in the log methods, example: + * private static final Marker WRITE_MARKER = AuditMarker.WRITE.getMarker(); + * 3. Define log methods in AuditLogger to use the new Marker type + * 4. Call these new methods from the required classes to audit with these + * new markers + * 5. The marker based filtering can be configured in log4j2 configurations + * Refer log4j2.properties in src/test/resources for a sample. + */ diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/DummyAction.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/DummyAction.java new file mode 100644 index 00000000000..6044c0a2f55 --- /dev/null +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/DummyAction.java @@ -0,0 +1,51 @@ +/** + * 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.ozone.audit; + +/** + * Enum to define Dummy AuditAction Type for test. + */ +public enum DummyAction implements AuditAction { + + CREATE_VOLUME("CREATE_VOLUME"), + CREATE_BUCKET("CREATE_BUCKET"), + CREATE_KEY("CREATE_KEY"), + READ_VOLUME("READ_VOLUME"), + READ_BUCKET("READ_BUCKET"), + READ_KEY("READ_BUCKET"), + UPDATE_VOLUME("UPDATE_VOLUME"), + UPDATE_BUCKET("UPDATE_BUCKET"), + UPDATE_KEY("UPDATE_KEY"), + DELETE_VOLUME("DELETE_VOLUME"), + DELETE_BUCKET("DELETE_BUCKET"), + DELETE_KEY("DELETE_KEY"), + SET_OWNER("SET_OWNER"), + SET_QUOTA("SET_QUOTA"); + + private String action; + + DummyAction(String action) { + this.action = action; + } + + @Override + public String getAction() { + return this.action; + } + +} diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/DummyEntity.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/DummyEntity.java new file mode 100644 index 00000000000..0c2d98fab29 --- /dev/null +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/DummyEntity.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 + *

+ * 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.ozone.audit; + +import java.util.HashMap; +import java.util.Map; + +/** + * DummyEntity that implements Auditable for test purpose. + */ +public class DummyEntity implements Auditable { + + private String key1; + private String key2; + + public DummyEntity(){ + this.key1 = "value1"; + this.key2 = "value2"; + } + public String getKey1() { + return key1; + } + + public void setKey1(String key1) { + this.key1 = key1; + } + + public String getKey2() { + return key2; + } + + public void setKey2(String key2) { + this.key2 = key2; + } + + @Override + public Map toAuditMap() { + Map auditMap = new HashMap<>(); + auditMap.put("key1", this.key1); + auditMap.put("key2", this.key2); + return auditMap; + } +} diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java new file mode 100644 index 00000000000..d3cc9e4ce8a --- /dev/null +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java @@ -0,0 +1,131 @@ +/** + * 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.ozone.audit; + +import org.apache.commons.io.FileUtils; +import org.apache.logging.log4j.Level; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.List; + +import static org.junit.Assert.assertTrue; + +/** + * Test Ozone Audit Logger. + */ +public class TestOzoneAuditLogger { + + private static final Logger LOG = LoggerFactory.getLogger + (TestOzoneAuditLogger.class.getName()); + private static AuditLogger AUDIT = new AuditLogger(AuditLoggerType.OMLOGGER); + public DummyEntity auditableObj = new DummyEntity(); + + @BeforeClass + public static void setUp(){ + System.setProperty("log4j.configurationFile", "log4j2.properties"); + } + + @AfterClass + public static void tearDown() { + File file = new File("audit.log"); + if (FileUtils.deleteQuietly(file)) { + LOG.info(file.getName() + + " has been deleted as all tests have completed."); + } else { + LOG.info("audit.log could not be deleted."); + } + } + + /** + * Ensures WriteSuccess events are logged @ INFO and above. + */ + @Test + public void logInfoWriteSuccess() throws IOException { + AUDIT.logWriteSuccess(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap(), Level.INFO); + String expected = "[INFO ] OMAudit - CREATE_VOLUME [ key1=\"value1\" " + + "key2=\"value2\"] SUCCESS"; + verifyLog(expected); + } + + /** + * Test to verify default log level is INFO + */ + @Test + public void verifyDefaultLogLevel() throws IOException { + AUDIT.logWriteSuccess(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap()); + String expected = "[INFO ] OMAudit - CREATE_VOLUME [ key1=\"value1\" " + + "key2=\"value2\"] SUCCESS"; + verifyLog(expected); + } + + /** + * Test to verify WriteFailure events are logged as ERROR. + */ + @Test + public void logErrorWriteFailure() throws IOException { + AUDIT.logWriteFailure(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap(), Level.ERROR); + String expected = "[ERROR] OMAudit - CREATE_VOLUME [ key1=\"value1\" " + + "key2=\"value2\"] FAILURE"; + verifyLog(expected); + } + + /** + * Test to verify no READ event is logged. + */ + @Test + public void notLogReadEvents() throws IOException { + AUDIT.logReadSuccess(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.INFO); + AUDIT.logReadFailure(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.INFO); + AUDIT.logReadFailure(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.ERROR); + AUDIT.logReadFailure(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.ERROR, + new Exception("test")); + verifyLog(null); + } + + /** + * Test to ensure DEBUG level messages are not logged when INFO is enabled. + */ + @Test + public void notLogDebugEvents() throws IOException { + AUDIT.logWriteSuccess(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap(), Level.DEBUG); + AUDIT.logReadSuccess(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.DEBUG); + verifyLog(null); + } + + public void verifyLog(String expected) throws IOException { + File file = new File("audit.log"); + List lines = FileUtils.readLines(file, (String)null); + if(expected == null){ + // When no log entry is expected, the log file must be empty + assertTrue(lines.size() == 0); + } else { + // When log entry is expected, the log file will contain one line and + // that must be equal to the expected string + assertTrue(expected.equalsIgnoreCase(lines.get(0))); + //empty the file + lines.remove(0); + FileUtils.writeLines(file, lines, false); + } + } +} diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/package-info.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/package-info.java new file mode 100644 index 00000000000..1222ad04e08 --- /dev/null +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/package-info.java @@ -0,0 +1,23 @@ +/** + * 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.ozone.audit; +/** + * Unit tests of Ozone Audit Logger. + * For test purpose, the log4j2 configuration is loaded from file at: + * src/test/resources/log4j2.properties + */ diff --git a/hadoop-hdds/common/src/test/resources/log4j2.properties b/hadoop-hdds/common/src/test/resources/log4j2.properties new file mode 100644 index 00000000000..d60df187eb1 --- /dev/null +++ b/hadoop-hdds/common/src/test/resources/log4j2.properties @@ -0,0 +1,76 @@ +# +# 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. +# +name=PropertiesConfig + +# Checks for config change periodically and reloads +monitorInterval=5 + +filter=read, write +# filter.read.onMatch = DENY avoids logging all READ events +# filter.read.onMatch = ACCEPT permits logging all READ events +# The above two settings ignore the log levels in configuration +# filter.read.onMatch = NEUTRAL permits logging of only those READ events +# which are attempted at log level equal or greater than log level specified +# in the configuration +filter.read.type = MarkerFilter +filter.read.marker = READ +filter.read.onMatch = DENY +filter.read.onMismatch = NEUTRAL + +# filter.write.onMatch = DENY avoids logging all WRITE events +# filter.write.onMatch = ACCEPT permits logging all WRITE events +# The above two settings ignore the log levels in configuration +# filter.write.onMatch = NEUTRAL permits logging of only those WRITE events +# which are attempted at log level equal or greater than log level specified +# in the configuration +filter.write.type = MarkerFilter +filter.write.marker = WRITE +filter.write.onMatch = NEUTRAL +filter.write.onMismatch = NEUTRAL + +# Log Levels are organized from most specific to least: +# OFF (most specific, no logging) +# FATAL (most specific, little data) +# ERROR +# WARN +# INFO +# DEBUG +# TRACE (least specific, a lot of data) +# ALL (least specific, all data) + +appenders = console, audit +appender.console.type = Console +appender.console.name = STDOUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = [%-5level] %c{1} - %msg%n + +appender.audit.type = File +appender.audit.name = AUDITLOG +appender.audit.fileName=audit.log +appender.audit.layout.type=PatternLayout +appender.audit.layout.pattern= [%-5level] %c{1} - %msg%n + +loggers=audit +logger.audit.type=AsyncLogger +logger.audit.name=OMAudit +logger.audit.level = INFO +logger.audit.appenderRefs = audit +logger.audit.appenderRef.file.ref = AUDITLOG + +rootLogger.level = INFO +rootLogger.appenderRefs = stdout +rootLogger.appenderRef.stdout.ref = STDOUT diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java new file mode 100644 index 00000000000..d780ea2c93b --- /dev/null +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java @@ -0,0 +1,51 @@ +/** + * 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.ozone.audit; + +/** + * Enum to define OM Action types for Audit. + */ +public enum OMAction implements AuditAction { + + CREATE_VOLUME("CREATE_VOLUME"), + CREATE_BUCKET("CREATE_BUCKET"), + CREATE_KEY("CREATE_KEY"), + READ_VOLUME("READ_VOLUME"), + READ_BUCKET("READ_BUCKET"), + READ_KEY("READ_BUCKET"), + UPDATE_VOLUME("UPDATE_VOLUME"), + UPDATE_BUCKET("UPDATE_BUCKET"), + UPDATE_KEY("UPDATE_KEY"), + DELETE_VOLUME("DELETE_VOLUME"), + DELETE_BUCKET("DELETE_BUCKET"), + DELETE_KEY("DELETE_KEY"), + SET_OWNER("SET_OWNER"), + SET_QUOTA("SET_QUOTA"); + + private String action; + + OMAction(String action) { + this.action = action; + } + + @Override + public String getAction() { + return this.action; + } + +} diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java new file mode 100644 index 00000000000..0f887909d49 --- /dev/null +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java @@ -0,0 +1,22 @@ +/** + * 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.ozone.audit; +/** + * This package defines OMAction - an implementation of AuditAction + * OMAction defines audit action types for various actions that will be + * audited in OzoneManager. + */ From 93ac01cb59b99b84b4f1ff26c089dcb5ce1b7c89 Mon Sep 17 00:00:00 2001 From: Aaron Fabbri Date: Tue, 3 Jul 2018 13:50:11 -0700 Subject: [PATCH 25/47] HADOOP-15215 s3guard set-capacity command to fail on read/write of 0 (Gabor Bota) --- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 10 ++++++++++ .../s3guard/AbstractS3GuardToolTestBase.java | 18 ++++++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 527697f00f3..19dc32a896e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -439,6 +439,10 @@ public abstract class S3GuardTool extends Configured implements Tool { static class SetCapacity extends S3GuardTool { public static final String NAME = "set-capacity"; public static final String PURPOSE = "Alter metadata store IO capacity"; + public static final String READ_CAP_INVALID = "Read capacity must have " + + "value greater than or equal to 1."; + public static final String WRITE_CAP_INVALID = "Write capacity must have " + + "value greater than or equal to 1."; private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" + "\t" + PURPOSE + "\n\n" + "Common options:\n" + @@ -478,11 +482,17 @@ public abstract class S3GuardTool extends Configured implements Tool { String readCap = getCommandFormat().getOptValue(READ_FLAG); if (StringUtils.isNotEmpty(readCap)) { + Preconditions.checkArgument(Integer.parseInt(readCap) > 0, + READ_CAP_INVALID); + S3GuardTool.println(out, "Read capacity set to %s", readCap); options.put(S3GUARD_DDB_TABLE_CAPACITY_READ_KEY, readCap); } String writeCap = getCommandFormat().getOptValue(WRITE_FLAG); if (StringUtils.isNotEmpty(writeCap)) { + Preconditions.checkArgument(Integer.parseInt(writeCap) > 0, + WRITE_CAP_INVALID); + S3GuardTool.println(out, "Write capacity set to %s", writeCap); options.put(S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY, writeCap); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java index 7d75f5233a4..f591e3258cf 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java @@ -51,6 +51,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.StringUtils; +import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_NAME_KEY; import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_METASTORE_NULL; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_BAD_STATE; @@ -286,6 +287,23 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase { "prune", testPath.toString()); } + @Test + public void testSetCapacityFailFast() throws Exception{ + Configuration conf = getConfiguration(); + conf.set(S3GUARD_DDB_TABLE_NAME_KEY, getFileSystem().getBucket()); + + S3GuardTool.SetCapacity cmdR = new S3GuardTool.SetCapacity(conf); + String[] argsR = new String[]{cmdR.getName(), "-read", "0", "s3a://bucket"}; + intercept(IllegalArgumentException.class, + S3GuardTool.SetCapacity.READ_CAP_INVALID, () -> cmdR.run(argsR)); + + S3GuardTool.SetCapacity cmdW = new S3GuardTool.SetCapacity(conf); + String[] argsW = new String[]{cmdW.getName(), "-write", "0", + "s3a://bucket"}; + intercept(IllegalArgumentException.class, + S3GuardTool.SetCapacity.WRITE_CAP_INVALID, () -> cmdW.run(argsW)); + } + @Test public void testDestroyNoBucket() throws Throwable { intercept(FileNotFoundException.class, From 7ca4f0cefa220c752920822c8d16469ab3b09b37 Mon Sep 17 00:00:00 2001 From: Anu Engineer Date: Tue, 3 Jul 2018 13:30:19 -0700 Subject: [PATCH 26/47] HDDS-175. Refactor ContainerInfo to remove Pipeline object from it. Contributed by Ajay Kumar. --- .../scm/client/ContainerOperationClient.java | 109 ++++++++--- .../hadoop/hdds/scm/client/ScmClient.java | 38 +++- .../common/helpers/ContainerInfo.java | 169 ++++++++++++------ .../common/helpers/ContainerWithPipeline.java | 131 ++++++++++++++ .../StorageContainerLocationProtocol.java | 13 +- ...ocationProtocolClientSideTranslatorPB.java | 26 ++- ...ocationProtocolServerSideTranslatorPB.java | 25 ++- .../StorageContainerLocationProtocol.proto | 15 +- hadoop-hdds/common/src/main/proto/hdds.proto | 9 +- .../hdds/scm/block/BlockManagerImpl.java | 80 +++++---- .../DatanodeDeletedBlockTransactions.java | 11 +- .../container/CloseContainerEventHandler.java | 26 ++- .../hdds/scm/container/ContainerMapping.java | 128 +++++++++++-- .../scm/container/ContainerStateManager.java | 30 +++- .../hadoop/hdds/scm/container/Mapping.java | 26 ++- .../scm/container/closer/ContainerCloser.java | 15 +- .../container/states/ContainerStateMap.java | 7 +- .../hdds/scm/pipelines/PipelineManager.java | 27 ++- .../hdds/scm/pipelines/PipelineSelector.java | 16 ++ .../scm/pipelines/ratis/RatisManagerImpl.java | 1 + .../standalone/StandaloneManagerImpl.java | 1 + .../scm/server/SCMClientProtocolServer.java | 14 +- .../hdds/scm/block/TestDeletedBlockLog.java | 13 +- .../TestCloseContainerEventHandler.java | 31 ++-- .../scm/container/TestContainerMapping.java | 27 +-- .../container/closer/TestContainerCloser.java | 18 +- .../hdds/scm/node/TestContainerPlacement.java | 6 +- .../cli/container/CloseContainerHandler.java | 10 +- .../cli/container/DeleteContainerHandler.java | 9 +- .../cli/container/InfoContainerHandler.java | 11 +- .../client/io/ChunkGroupInputStream.java | 15 +- .../client/io/ChunkGroupOutputStream.java | 9 +- .../ozone/protocolPB/OzonePBHelper.java | 30 ++++ .../container/TestContainerStateManager.java | 161 ++++++++++------- .../hadoop/ozone/TestContainerOperations.java | 11 +- .../ozone/TestStorageContainerManager.java | 6 +- .../TestStorageContainerManagerHelper.java | 10 +- .../ozone/client/rpc/TestOzoneRpcClient.java | 4 +- .../TestCloseContainerByPipeline.java | 21 +-- .../ksm/TestContainerReportWithKeys.java | 2 +- .../ozone/scm/TestAllocateContainer.java | 6 +- .../ozone/scm/TestContainerSmallFile.java | 36 ++-- .../apache/hadoop/ozone/scm/TestSCMCli.java | 135 +++++++------- .../ozone/scm/TestXceiverClientManager.java | 62 ++++--- .../ozone/scm/TestXceiverClientMetrics.java | 14 +- .../genesis/BenchMarkContainerStateMap.java | 16 +- .../apache/hadoop/ozone/scm/cli/SQLCLI.java | 63 +++---- 47 files changed, 1139 insertions(+), 504 deletions(-) create mode 100644 hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerWithPipeline.java create mode 100644 hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OzonePBHelper.java diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java index 07f6cec2f81..b04f8c4d2ce 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hdds.scm.client; import com.google.common.base.Preconditions; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientSpi; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.protocolPB @@ -87,16 +88,17 @@ public class ContainerOperationClient implements ScmClient { * @inheritDoc */ @Override - public ContainerInfo createContainer(String owner) + public ContainerWithPipeline createContainer(String owner) throws IOException { XceiverClientSpi client = null; try { - ContainerInfo container = + ContainerWithPipeline containerWithPipeline = storageContainerLocationClient.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), owner); - Pipeline pipeline = container.getPipeline(); - client = xceiverClientManager.acquireClient(pipeline, container.getContainerID()); + Pipeline pipeline = containerWithPipeline.getPipeline(); + client = xceiverClientManager.acquireClient(pipeline, + containerWithPipeline.getContainerInfo().getContainerID()); // Allocated State means that SCM has allocated this pipeline in its // namespace. The client needs to create the pipeline on the machines @@ -106,8 +108,9 @@ public class ContainerOperationClient implements ScmClient { if (pipeline.getLifeCycleState() == ALLOCATED) { createPipeline(client, pipeline); } - createContainer(client, container.getContainerID()); - return container; + createContainer(client, + containerWithPipeline.getContainerInfo().getContainerID()); + return containerWithPipeline; } finally { if (client != null) { xceiverClientManager.releaseClient(client); @@ -197,17 +200,17 @@ public class ContainerOperationClient implements ScmClient { * @inheritDoc */ @Override - public ContainerInfo createContainer(HddsProtos.ReplicationType type, + public ContainerWithPipeline createContainer(HddsProtos.ReplicationType type, HddsProtos.ReplicationFactor factor, String owner) throws IOException { XceiverClientSpi client = null; try { // allocate container on SCM. - ContainerInfo container = + ContainerWithPipeline containerWithPipeline = storageContainerLocationClient.allocateContainer(type, factor, owner); - Pipeline pipeline = container.getPipeline(); + Pipeline pipeline = containerWithPipeline.getPipeline(); client = xceiverClientManager.acquireClient(pipeline, - container.getContainerID()); + containerWithPipeline.getContainerInfo().getContainerID()); // Allocated State means that SCM has allocated this pipeline in its // namespace. The client needs to create the pipeline on the machines @@ -217,9 +220,10 @@ public class ContainerOperationClient implements ScmClient { } // connect to pipeline leader and allocate container on leader datanode. client = xceiverClientManager.acquireClient(pipeline, - container.getContainerID()); - createContainer(client, container.getContainerID()); - return container; + containerWithPipeline.getContainerInfo().getContainerID()); + createContainer(client, + containerWithPipeline.getContainerInfo().getContainerID()); + return containerWithPipeline; } finally { if (client != null) { xceiverClientManager.releaseClient(client); @@ -256,24 +260,27 @@ public class ContainerOperationClient implements ScmClient { } /** - * Delete the container, this will release any resource it uses. - * @param pipeline - Pipeline that represents the container. - * @param force - True to forcibly delete the container. + * Deletes an existing container. + * + * @param containerId - ID of the container. + * @param pipeline - Pipeline that represents the container. + * @param force - true to forcibly delete the container. * @throws IOException */ @Override - public void deleteContainer(long containerID, Pipeline pipeline, boolean force) - throws IOException { + public void deleteContainer(long containerId, Pipeline pipeline, + boolean force) throws IOException { XceiverClientSpi client = null; try { - client = xceiverClientManager.acquireClient(pipeline, containerID); + client = xceiverClientManager.acquireClient(pipeline, containerId); String traceID = UUID.randomUUID().toString(); - ContainerProtocolCalls.deleteContainer(client, containerID, force, traceID); + ContainerProtocolCalls + .deleteContainer(client, containerId, force, traceID); storageContainerLocationClient - .deleteContainer(containerID); + .deleteContainer(containerId); if (LOG.isDebugEnabled()) { LOG.debug("Deleted container {}, leader: {}, machines: {} ", - containerID, + containerId, pipeline.getLeader(), pipeline.getMachines()); } @@ -284,6 +291,19 @@ public class ContainerOperationClient implements ScmClient { } } + /** + * Delete the container, this will release any resource it uses. + * @param containerID - containerID. + * @param force - True to forcibly delete the container. + * @throws IOException + */ + @Override + public void deleteContainer(long containerID, boolean force) + throws IOException { + ContainerWithPipeline info = getContainerWithPipeline(containerID); + deleteContainer(containerID, info.getPipeline(), force); + } + /** * {@inheritDoc} */ @@ -297,9 +317,9 @@ public class ContainerOperationClient implements ScmClient { /** * Get meta data from an existing container. * - * @param pipeline - pipeline that represents the container. - * @return ContainerInfo - a message of protobuf which has basic info - * of a container. + * @param containerID - ID of the container. + * @param pipeline - Pipeline where the container is located. + * @return ContainerInfo * @throws IOException */ @Override @@ -325,6 +345,19 @@ public class ContainerOperationClient implements ScmClient { } } + /** + * Get meta data from an existing container. + * @param containerID - ID of the container. + * @return ContainerInfo - a message of protobuf which has basic info + * of a container. + * @throws IOException + */ + @Override + public ContainerData readContainer(long containerID) throws IOException { + ContainerWithPipeline info = getContainerWithPipeline(containerID); + return readContainer(containerID, info.getPipeline()); + } + /** * Given an id, return the pipeline associated with the container. * @param containerId - String Container ID @@ -337,6 +370,19 @@ public class ContainerOperationClient implements ScmClient { return storageContainerLocationClient.getContainer(containerId); } + /** + * Gets a container by Name -- Throws if the container does not exist. + * + * @param containerId - Container ID + * @return ContainerWithPipeline + * @throws IOException + */ + @Override + public ContainerWithPipeline getContainerWithPipeline(long containerId) + throws IOException { + return storageContainerLocationClient.getContainerWithPipeline(containerId); + } + /** * Close a container. * @@ -391,6 +437,19 @@ public class ContainerOperationClient implements ScmClient { } } + /** + * Close a container. + * + * @throws IOException + */ + @Override + public void closeContainer(long containerId) + throws IOException { + ContainerWithPipeline info = getContainerWithPipeline(containerId); + Pipeline pipeline = info.getPipeline(); + closeContainer(containerId, pipeline); + } + /** * Get the the current usage information. * @param containerID - ID of the container. diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java index b52819a3001..ecb21735e9c 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.scm.client; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos @@ -45,7 +46,7 @@ public interface ScmClient { * @return ContainerInfo * @throws IOException */ - ContainerInfo createContainer(String owner) throws IOException; + ContainerWithPipeline createContainer(String owner) throws IOException; /** * Gets a container by Name -- Throws if the container does not exist. @@ -55,6 +56,14 @@ public interface ScmClient { */ ContainerInfo getContainer(long containerId) throws IOException; + /** + * Gets a container by Name -- Throws if the container does not exist. + * @param containerId - Container ID + * @return ContainerWithPipeline + * @throws IOException + */ + ContainerWithPipeline getContainerWithPipeline(long containerId) throws IOException; + /** * Close a container. * @@ -64,6 +73,14 @@ public interface ScmClient { */ void closeContainer(long containerId, Pipeline pipeline) throws IOException; + /** + * Close a container. + * + * @param containerId - ID of the container. + * @throws IOException + */ + void closeContainer(long containerId) throws IOException; + /** * Deletes an existing container. * @param containerId - ID of the container. @@ -73,6 +90,14 @@ public interface ScmClient { */ void deleteContainer(long containerId, Pipeline pipeline, boolean force) throws IOException; + /** + * Deletes an existing container. + * @param containerId - ID of the container. + * @param force - true to forcibly delete the container. + * @throws IOException + */ + void deleteContainer(long containerId, boolean force) throws IOException; + /** * Lists a range of containers and get their info. * @@ -95,6 +120,15 @@ public interface ScmClient { ContainerData readContainer(long containerID, Pipeline pipeline) throws IOException; + /** + * Read meta data from an existing container. + * @param containerID - ID of the container. + * @return ContainerInfo + * @throws IOException + */ + ContainerData readContainer(long containerID) + throws IOException; + /** * Gets the container size -- Computed by SCM from Container Reports. * @param containerID - ID of the container. @@ -110,7 +144,7 @@ public interface ScmClient { * @return ContainerInfo * @throws IOException - in case of error. */ - ContainerInfo createContainer(HddsProtos.ReplicationType type, + ContainerWithPipeline createContainer(HddsProtos.ReplicationType type, HddsProtos.ReplicationFactor replicationFactor, String owner) throws IOException; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java index ee05c8768af..9593717ced1 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java @@ -15,34 +15,39 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hdds.scm.container.common.helpers; +import static java.lang.Math.max; + import com.fasterxml.jackson.annotation.JsonAutoDetect; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.PropertyAccessor; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.base.Preconditions; -import org.apache.commons.lang3.builder.EqualsBuilder; -import org.apache.commons.lang3.builder.HashCodeBuilder; -import org.apache.hadoop.hdds.scm.container.ContainerID; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.util.Time; - +import java.io.Externalizable; import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; import java.util.Arrays; import java.util.Comparator; - -import static java.lang.Math.max; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.util.Time; /** * Class wraps ozone container info. */ -public class ContainerInfo - implements Comparator, Comparable { +public class ContainerInfo implements Comparator, + Comparable, Externalizable { private static final ObjectWriter WRITER; + private static final String SERIALIZATION_ERROR_MSG = "Java serialization not" + + " supported. Use protobuf instead."; static { ObjectMapper mapper = new ObjectMapper(); @@ -53,7 +58,9 @@ public class ContainerInfo } private HddsProtos.LifeCycleState state; - private Pipeline pipeline; + private String pipelineName; + private ReplicationFactor replicationFactor; + private ReplicationType replicationType; // Bytes allocated by SCM for clients. private long allocatedBytes; // Actual container usage, updated through heartbeat. @@ -75,15 +82,17 @@ public class ContainerInfo ContainerInfo( long containerID, HddsProtos.LifeCycleState state, - Pipeline pipeline, + String pipelineName, long allocatedBytes, long usedBytes, long numberOfKeys, long stateEnterTime, String owner, - long deleteTransactionId) { + long deleteTransactionId, + ReplicationFactor replicationFactor, + ReplicationType repType) { this.containerID = containerID; - this.pipeline = pipeline; + this.pipelineName = pipelineName; this.allocatedBytes = allocatedBytes; this.usedBytes = usedBytes; this.numberOfKeys = numberOfKeys; @@ -92,6 +101,8 @@ public class ContainerInfo this.stateEnterTime = stateEnterTime; this.owner = owner; this.deleteTransactionId = deleteTransactionId; + this.replicationFactor = replicationFactor; + this.replicationType = repType; } /** @@ -102,16 +113,18 @@ public class ContainerInfo public static ContainerInfo fromProtobuf(HddsProtos.SCMContainerInfo info) { ContainerInfo.Builder builder = new ContainerInfo.Builder(); - builder.setPipeline(Pipeline.getFromProtoBuf(info.getPipeline())); - builder.setAllocatedBytes(info.getAllocatedBytes()); - builder.setUsedBytes(info.getUsedBytes()); - builder.setNumberOfKeys(info.getNumberOfKeys()); - builder.setState(info.getState()); - builder.setStateEnterTime(info.getStateEnterTime()); - builder.setOwner(info.getOwner()); - builder.setContainerID(info.getContainerID()); - builder.setDeleteTransactionId(info.getDeleteTransactionId()); - return builder.build(); + return builder.setPipelineName(info.getPipelineName()) + .setAllocatedBytes(info.getAllocatedBytes()) + .setUsedBytes(info.getUsedBytes()) + .setNumberOfKeys(info.getNumberOfKeys()) + .setState(info.getState()) + .setStateEnterTime(info.getStateEnterTime()) + .setOwner(info.getOwner()) + .setContainerID(info.getContainerID()) + .setDeleteTransactionId(info.getDeleteTransactionId()) + .setReplicationFactor(info.getReplicationFactor()) + .setReplicationType(info.getReplicationType()) + .build(); } public long getContainerID() { @@ -130,8 +143,12 @@ public class ContainerInfo return stateEnterTime; } - public Pipeline getPipeline() { - return pipeline; + public ReplicationFactor getReplicationFactor() { + return replicationFactor; + } + + public String getPipelineName() { + return pipelineName; } public long getAllocatedBytes() { @@ -177,6 +194,10 @@ public class ContainerInfo return lastUsed; } + public ReplicationType getReplicationType() { + return replicationType; + } + public void updateLastUsedTime() { lastUsed = Time.monotonicNow(); } @@ -190,19 +211,17 @@ public class ContainerInfo public HddsProtos.SCMContainerInfo getProtobuf() { HddsProtos.SCMContainerInfo.Builder builder = HddsProtos.SCMContainerInfo.newBuilder(); - builder.setPipeline(getPipeline().getProtobufMessage()); - builder.setAllocatedBytes(getAllocatedBytes()); - builder.setUsedBytes(getUsedBytes()); - builder.setNumberOfKeys(getNumberOfKeys()); - builder.setState(state); - builder.setStateEnterTime(stateEnterTime); - builder.setContainerID(getContainerID()); - builder.setDeleteTransactionId(deleteTransactionId); - - if (getOwner() != null) { - builder.setOwner(getOwner()); - } - return builder.build(); + return builder.setAllocatedBytes(getAllocatedBytes()) + .setContainerID(getContainerID()) + .setUsedBytes(getUsedBytes()) + .setNumberOfKeys(getNumberOfKeys()).setState(getState()) + .setStateEnterTime(getStateEnterTime()).setContainerID(getContainerID()) + .setDeleteTransactionId(getDeleteTransactionId()) + .setPipelineName(getPipelineName()) + .setReplicationFactor(getReplicationFactor()) + .setReplicationType(getReplicationType()) + .setOwner(getOwner()) + .build(); } public String getOwner() { @@ -217,7 +236,7 @@ public class ContainerInfo public String toString() { return "ContainerInfo{" + "state=" + state - + ", pipeline=" + pipeline + + ", pipelineName=" + pipelineName + ", stateEnterTime=" + stateEnterTime + ", owner=" + owner + '}'; @@ -252,9 +271,7 @@ public class ContainerInfo public int hashCode() { return new HashCodeBuilder(11, 811) .append(getContainerID()) - .append(pipeline.getFactor()) - .append(pipeline.getType()) - .append(owner) + .append(getOwner()) .toHashCode(); } @@ -327,12 +344,44 @@ public class ContainerInfo this.data = Arrays.copyOf(data, data.length); } } + + /** + * Throws IOException as default java serialization is not supported. Use + * serialization via protobuf instead. + * + * @param out the stream to write the object to + * @throws IOException Includes any I/O exceptions that may occur + * @serialData Overriding methods should use this tag to describe + * the data layout of this Externalizable object. + * List the sequence of element types and, if possible, + * relate the element to a public/protected field and/or + * method of this Externalizable class. + */ + @Override + public void writeExternal(ObjectOutput out) throws IOException { + throw new IOException(SERIALIZATION_ERROR_MSG); + } + + /** + * Throws IOException as default java serialization is not supported. Use + * serialization via protobuf instead. + * + * @param in the stream to read data from in order to restore the object + * @throws IOException if I/O errors occur + * @throws ClassNotFoundException If the class for an object being + * restored cannot be found. + */ + @Override + public void readExternal(ObjectInput in) + throws IOException, ClassNotFoundException { + throw new IOException(SERIALIZATION_ERROR_MSG); + } + /** * Builder class for ContainerInfo. */ public static class Builder { private HddsProtos.LifeCycleState state; - private Pipeline pipeline; private long allocated; private long used; private long keys; @@ -340,6 +389,25 @@ public class ContainerInfo private String owner; private long containerID; private long deleteTransactionId; + private String pipelineName; + private ReplicationFactor replicationFactor; + private ReplicationType replicationType; + + public Builder setReplicationType( + ReplicationType replicationType) { + this.replicationType = replicationType; + return this; + } + + public Builder setPipelineName(String pipelineName) { + this.pipelineName = pipelineName; + return this; + } + + public Builder setReplicationFactor(ReplicationFactor repFactor) { + this.replicationFactor = repFactor; + return this; + } public Builder setContainerID(long id) { Preconditions.checkState(id >= 0); @@ -352,11 +420,6 @@ public class ContainerInfo return this; } - public Builder setPipeline(Pipeline containerPipeline) { - this.pipeline = containerPipeline; - return this; - } - public Builder setAllocatedBytes(long bytesAllocated) { this.allocated = bytesAllocated; return this; @@ -388,9 +451,9 @@ public class ContainerInfo } public ContainerInfo build() { - return new - ContainerInfo(containerID, state, pipeline, allocated, - used, keys, stateEnterTime, owner, deleteTransactionId); + return new ContainerInfo(containerID, state, pipelineName, allocated, + used, keys, stateEnterTime, owner, deleteTransactionId, + replicationFactor, replicationType); } } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerWithPipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerWithPipeline.java new file mode 100644 index 00000000000..e71d429b988 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerWithPipeline.java @@ -0,0 +1,131 @@ +/* + * 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.hdds.scm.container.common.helpers; + +import java.util.Comparator; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; + +/** + * Class wraps ozone container info. + */ +public class ContainerWithPipeline + implements Comparator, Comparable { + + private final ContainerInfo containerInfo; + private final Pipeline pipeline; + + public ContainerWithPipeline(ContainerInfo containerInfo, Pipeline pipeline) { + this.containerInfo = containerInfo; + this.pipeline = pipeline; + } + + public ContainerInfo getContainerInfo() { + return containerInfo; + } + + public Pipeline getPipeline() { + return pipeline; + } + + public static ContainerWithPipeline fromProtobuf(HddsProtos.ContainerWithPipeline allocatedContainer) { + return new ContainerWithPipeline( + ContainerInfo.fromProtobuf(allocatedContainer.getContainerInfo()), + Pipeline.getFromProtoBuf(allocatedContainer.getPipeline())); + } + + public HddsProtos.ContainerWithPipeline getProtobuf() { + HddsProtos.ContainerWithPipeline.Builder builder = + HddsProtos.ContainerWithPipeline.newBuilder(); + builder.setContainerInfo(getContainerInfo().getProtobuf()) + .setPipeline(getPipeline().getProtobufMessage()); + + return builder.build(); + } + + + @Override + public String toString() { + return containerInfo.toString() + " | " + pipeline.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + ContainerWithPipeline that = (ContainerWithPipeline) o; + + return new EqualsBuilder() + .append(getContainerInfo(), that.getContainerInfo()) + .append(getPipeline(), that.getPipeline()) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(11, 811) + .append(getContainerInfo()) + .append(getPipeline()) + .toHashCode(); + } + + /** + * Compares its two arguments for order. Returns a negative integer, zero, or + * a positive integer as the first argument is less than, equal to, or greater + * than the second.

+ * + * @param o1 the first object to be compared. + * @param o2 the second object to be compared. + * @return a negative integer, zero, or a positive integer as the first + * argument is less than, equal to, or greater than the second. + * @throws NullPointerException if an argument is null and this comparator + * does not permit null arguments + * @throws ClassCastException if the arguments' types prevent them from + * being compared by this comparator. + */ + @Override + public int compare(ContainerWithPipeline o1, ContainerWithPipeline o2) { + return o1.getContainerInfo().compareTo(o2.getContainerInfo()); + } + + /** + * Compares this object with the specified object for order. Returns a + * negative integer, zero, or a positive integer as this object is less than, + * equal to, or greater than the specified object. + * + * @param o the object to be compared. + * @return a negative integer, zero, or a positive integer as this object is + * less than, equal to, or greater than the specified object. + * @throws NullPointerException if the specified object is null + * @throws ClassCastException if the specified object's type prevents it + * from being compared to this object. + */ + @Override + public int compareTo(ContainerWithPipeline o) { + return this.compare(this, o); + } + +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java index e8d85e0084a..b787409248c 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.scm.protocol; import org.apache.hadoop.hdds.scm.ScmInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -38,7 +39,7 @@ public interface StorageContainerLocationProtocol { * set of datanodes that should be used creating this container. * */ - ContainerInfo allocateContainer(HddsProtos.ReplicationType replicationType, + ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType replicationType, HddsProtos.ReplicationFactor factor, String owner) throws IOException; @@ -53,6 +54,16 @@ public interface StorageContainerLocationProtocol { */ ContainerInfo getContainer(long containerID) throws IOException; + /** + * Ask SCM the location of the container. SCM responds with a group of + * nodes where this container and its replicas are located. + * + * @param containerID - ID of the container. + * @return ContainerWithPipeline - the container info with the pipeline. + * @throws IOException + */ + ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException; + /** * Ask SCM a list of containers with a range of container names * and the limit of count. diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java index bba4e172be1..4b03d122aae 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java @@ -20,7 +20,10 @@ import com.google.common.base.Preconditions; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineResponseProto; import org.apache.hadoop.hdds.scm.ScmInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; @@ -95,7 +98,7 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB * @throws IOException */ @Override - public ContainerInfo allocateContainer(HddsProtos.ReplicationType type, + public ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType type, HddsProtos.ReplicationFactor factor, String owner) throws IOException { ContainerRequestProto request = ContainerRequestProto.newBuilder() @@ -114,7 +117,7 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB throw new IOException(response.hasErrorMessage() ? response.getErrorMessage() : "Allocate container failed."); } - return ContainerInfo.fromProtobuf(response.getContainerInfo()); + return ContainerWithPipeline.fromProtobuf(response.getContainerWithPipeline()); } public ContainerInfo getContainer(long containerID) throws IOException { @@ -133,6 +136,25 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB } } + /** + * {@inheritDoc} + */ + public ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException { + Preconditions.checkState(containerID >= 0, + "Container ID cannot be negative"); + GetContainerWithPipelineRequestProto request = GetContainerWithPipelineRequestProto + .newBuilder() + .setContainerID(containerID) + .build(); + try { + GetContainerWithPipelineResponseProto response = + rpcProxy.getContainerWithPipeline(NULL_RPC_CONTROLLER, request); + return ContainerWithPipeline.fromProtobuf(response.getContainerWithPipeline()); + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + /** * {@inheritDoc} */ diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java index 70a0e8a8dac..d66919f3568 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java @@ -21,7 +21,10 @@ package org.apache.hadoop.ozone.protocolPB; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineResponseProto; import org.apache.hadoop.hdds.scm.ScmInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB; @@ -82,10 +85,11 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB public ContainerResponseProto allocateContainer(RpcController unused, ContainerRequestProto request) throws ServiceException { try { - ContainerInfo container = impl.allocateContainer(request.getReplicationType(), - request.getReplicationFactor(), request.getOwner()); + ContainerWithPipeline containerWithPipeline = impl + .allocateContainer(request.getReplicationType(), + request.getReplicationFactor(), request.getOwner()); return ContainerResponseProto.newBuilder() - .setContainerInfo(container.getProtobuf()) + .setContainerWithPipeline(containerWithPipeline.getProtobuf()) .setErrorCode(ContainerResponseProto.Error.success) .build(); @@ -108,6 +112,21 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB } } + @Override + public GetContainerWithPipelineResponseProto getContainerWithPipeline( + RpcController controller, GetContainerWithPipelineRequestProto request) + throws ServiceException { + try { + ContainerWithPipeline container = impl + .getContainerWithPipeline(request.getContainerID()); + return GetContainerWithPipelineResponseProto.newBuilder() + .setContainerWithPipeline(container.getProtobuf()) + .build(); + } catch (IOException e) { + throw new ServiceException(e); + } + } + @Override public SCMListContainerResponseProto listContainer(RpcController controller, SCMListContainerRequestProto request) throws ServiceException { diff --git a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto index 090e6ebde49..143c2aefa0f 100644 --- a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto +++ b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto @@ -52,7 +52,7 @@ message ContainerResponseProto { errorContainerMissing = 3; } required Error errorCode = 1; - required SCMContainerInfo containerInfo = 2; + required ContainerWithPipeline containerWithPipeline = 2; optional string errorMessage = 3; } @@ -64,6 +64,14 @@ message GetContainerResponseProto { required SCMContainerInfo containerInfo = 1; } +message GetContainerWithPipelineRequestProto { + required int64 containerID = 1; +} + +message GetContainerWithPipelineResponseProto { + required ContainerWithPipeline containerWithPipeline = 1; +} + message SCMListContainerRequestProto { required uint32 count = 1; optional uint64 startContainerID = 2; @@ -171,6 +179,11 @@ service StorageContainerLocationProtocolService { */ rpc getContainer(GetContainerRequestProto) returns (GetContainerResponseProto); + /** + * Returns the pipeline for a given container. + */ + rpc getContainerWithPipeline(GetContainerWithPipelineRequestProto) returns (GetContainerWithPipelineResponseProto); + rpc listContainer(SCMListContainerRequestProto) returns (SCMListContainerResponseProto); /** diff --git a/hadoop-hdds/common/src/main/proto/hdds.proto b/hadoop-hdds/common/src/main/proto/hdds.proto index 816efa7c253..1c9ee199262 100644 --- a/hadoop-hdds/common/src/main/proto/hdds.proto +++ b/hadoop-hdds/common/src/main/proto/hdds.proto @@ -132,7 +132,7 @@ enum LifeCycleEvent { message SCMContainerInfo { required int64 containerID = 1; required LifeCycleState state = 2; - required Pipeline pipeline = 3; + optional string pipelineName = 3; // This is not total size of container, but space allocated by SCM for // clients to write blocks required uint64 allocatedBytes = 4; @@ -141,6 +141,13 @@ message SCMContainerInfo { optional int64 stateEnterTime = 7; required string owner = 8; optional int64 deleteTransactionId = 9; + required ReplicationFactor replicationFactor = 10; + required ReplicationType replicationType = 11; +} + +message ContainerWithPipeline { + required SCMContainerInfo containerInfo = 1; + required Pipeline pipeline = 2; } message GetScmInfoRequestProto { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java index 7cfbdabe1a9..953f71eaecd 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java @@ -16,10 +16,12 @@ */ package org.apache.hadoop.hdds.scm.block; +import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.Mapping; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.node.NodeManager; @@ -156,13 +158,13 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { lock.lock(); try { for (int i = 0; i < count; i++) { - ContainerInfo containerInfo = null; + ContainerWithPipeline containerWithPipeline = null; try { // TODO: Fix this later when Ratis is made the Default. - containerInfo = containerManager.allocateContainer(type, factor, + containerWithPipeline = containerManager.allocateContainer(type, factor, owner); - if (containerInfo == null) { + if (containerWithPipeline == null) { LOG.warn("Unable to allocate container."); continue; } @@ -231,30 +233,27 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { can use different kind of policies. */ - ContainerInfo containerInfo; + ContainerWithPipeline containerWithPipeline; // Look for ALLOCATED container that matches all other parameters. - containerInfo = - containerManager - .getStateManager() - .getMatchingContainer( - size, owner, type, factor, HddsProtos.LifeCycleState - .ALLOCATED); - if (containerInfo != null) { - containerManager.updateContainerState(containerInfo.getContainerID(), + containerWithPipeline = containerManager + .getMatchingContainerWithPipeline(size, owner, type, factor, + HddsProtos.LifeCycleState.ALLOCATED); + if (containerWithPipeline != null) { + containerManager.updateContainerState( + containerWithPipeline.getContainerInfo().getContainerID(), HddsProtos.LifeCycleEvent.CREATE); - return newBlock(containerInfo, HddsProtos.LifeCycleState.ALLOCATED); + return newBlock(containerWithPipeline, + HddsProtos.LifeCycleState.ALLOCATED); } // Since we found no allocated containers that match our criteria, let us // look for OPEN containers that match the criteria. - containerInfo = - containerManager - .getStateManager() - .getMatchingContainer(size, owner, type, factor, HddsProtos - .LifeCycleState.OPEN); - if (containerInfo != null) { - return newBlock(containerInfo, HddsProtos.LifeCycleState.OPEN); + containerWithPipeline = containerManager + .getMatchingContainerWithPipeline(size, owner, type, factor, + HddsProtos.LifeCycleState.OPEN); + if (containerWithPipeline != null) { + return newBlock(containerWithPipeline, HddsProtos.LifeCycleState.OPEN); } // We found neither ALLOCATED or OPEN Containers. This generally means @@ -264,16 +263,15 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { preAllocateContainers(containerProvisionBatchSize, type, factor, owner); // Since we just allocated a set of containers this should work - containerInfo = - containerManager - .getStateManager() - .getMatchingContainer( - size, owner, type, factor, HddsProtos.LifeCycleState - .ALLOCATED); - if (containerInfo != null) { - containerManager.updateContainerState(containerInfo.getContainerID(), + containerWithPipeline = containerManager + .getMatchingContainerWithPipeline(size, owner, type, factor, + HddsProtos.LifeCycleState.ALLOCATED); + if (containerWithPipeline != null) { + containerManager.updateContainerState( + containerWithPipeline.getContainerInfo().getContainerID(), HddsProtos.LifeCycleEvent.CREATE); - return newBlock(containerInfo, HddsProtos.LifeCycleState.ALLOCATED); + return newBlock(containerWithPipeline, + HddsProtos.LifeCycleState.ALLOCATED); } // we have tried all strategies we know and but somehow we are not able @@ -290,18 +288,28 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { } } + private String getChannelName(ReplicationType type) { + switch (type) { + case RATIS: + return "RA" + UUID.randomUUID().toString().substring(3); + case STAND_ALONE: + return "SA" + UUID.randomUUID().toString().substring(3); + default: + return "RA" + UUID.randomUUID().toString().substring(3); + } + } + /** * newBlock - returns a new block assigned to a container. * - * @param containerInfo - Container Info. + * @param containerWithPipeline - Container Info. * @param state - Current state of the container. * @return AllocatedBlock */ - private AllocatedBlock newBlock( - ContainerInfo containerInfo, HddsProtos.LifeCycleState state) - throws IOException { - - if (containerInfo.getPipeline().getMachines().size() == 0) { + private AllocatedBlock newBlock(ContainerWithPipeline containerWithPipeline, + HddsProtos.LifeCycleState state) throws IOException { + ContainerInfo containerInfo = containerWithPipeline.getContainerInfo(); + if (containerWithPipeline.getPipeline().getDatanodes().size() == 0) { LOG.error("Pipeline Machine count is zero."); return null; } @@ -317,7 +325,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean { AllocatedBlock.Builder abb = new AllocatedBlock.Builder() .setBlockID(new BlockID(containerID, localID)) - .setPipeline(containerInfo.getPipeline()) + .setPipeline(containerWithPipeline.getPipeline()) .setShouldCreateContainer(createContainer); LOG.trace("New block allocated : {} Container ID: {}", localID, containerID); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java index 32290cc99ba..d71e7b05fe3 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hdds.scm.block; import com.google.common.collect.ArrayListMultimap; import org.apache.hadoop.hdds.scm.container.Mapping; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction; @@ -29,6 +28,7 @@ import java.util.List; import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; +import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; /** * A wrapper class to hold info about datanode and all deleted block @@ -54,21 +54,22 @@ public class DatanodeDeletedBlockTransactions { } public void addTransaction(DeletedBlocksTransaction tx) throws IOException { - ContainerInfo info = null; + Pipeline pipeline = null; try { - info = mappingService.getContainer(tx.getContainerID()); + pipeline = mappingService.getContainerWithPipeline(tx.getContainerID()) + .getPipeline(); } catch (IOException e) { SCMBlockDeletingService.LOG.warn("Got container info error.", e); } - if (info == null) { + if (pipeline == null) { SCMBlockDeletingService.LOG.warn( "Container {} not found, continue to process next", tx.getContainerID()); return; } - for (DatanodeDetails dd : info.getPipeline().getMachines()) { + for (DatanodeDetails dd : pipeline.getMachines()) { UUID dnID = dd.getUuid(); if (transactions.containsKey(dnID)) { List txs = transactions.get(dnID); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java index 16e84a3c5f3..7b24538a6bc 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java @@ -16,9 +16,11 @@ */ package org.apache.hadoop.hdds.scm.container; +import java.io.IOException; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.server.events.EventHandler; import org.apache.hadoop.hdds.server.events.EventPublisher; @@ -54,22 +56,32 @@ public class CloseContainerEventHandler implements EventHandler { LOG.info("Close container Event triggered for container : {}", containerID.getId()); - ContainerStateManager stateManager = containerManager.getStateManager(); - ContainerInfo info = stateManager.getContainer(containerID); - if (info == null) { - LOG.info("Container with id : {} does not exist", containerID.getId()); + ContainerWithPipeline containerWithPipeline = null; + ContainerInfo info; + try { + containerWithPipeline = containerManager.getContainerWithPipeline(containerID.getId()); + info = containerWithPipeline.getContainerInfo(); + if (info == null) { + LOG.info("Failed to update the container state. Container with id : {} " + + "does not exist", containerID.getId()); + return; + } + } catch (IOException e) { + LOG.info("Failed to update the container state. Container with id : {} " + + "does not exist", containerID.getId()); return; } + if (info.getState() == HddsProtos.LifeCycleState.OPEN) { - for (DatanodeDetails datanode : info.getPipeline().getMachines()) { + for (DatanodeDetails datanode : containerWithPipeline.getPipeline().getMachines()) { containerManager.getNodeManager().addDatanodeCommand(datanode.getUuid(), new CloseContainerCommand(containerID.getId(), - info.getPipeline().getType())); + info.getReplicationType())); } try { // Finalize event will make sure the state of the container transitions // from OPEN to CLOSING in containerStateManager. - stateManager + containerManager.getStateManager() .updateContainerState(info, HddsProtos.LifeCycleEvent.FINALIZE); } catch (SCMException ex) { LOG.error("Failed to update the container state for container : {}" diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java index 9fd30f2ad03..e25c5b47405 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java @@ -21,6 +21,10 @@ import com.google.common.base.Preconditions; import com.google.common.primitives.Longs; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.SCMContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.closer.ContainerCloser; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; @@ -166,6 +170,44 @@ public class ContainerMapping implements Mapping { } } + /** + * Returns the ContainerInfo from the container ID. + * + * @param containerID - ID of container. + * @return - ContainerWithPipeline such as creation state and the pipeline. + * @throws IOException + */ + @Override + public ContainerWithPipeline getContainerWithPipeline(long containerID) + throws IOException { + ContainerInfo contInfo; + lock.lock(); + try { + byte[] containerBytes = containerStore.get( + Longs.toByteArray(containerID)); + if (containerBytes == null) { + throw new SCMException( + "Specified key does not exist. key : " + containerID, + SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER); + } + HddsProtos.SCMContainerInfo temp = HddsProtos.SCMContainerInfo.PARSER + .parseFrom(containerBytes); + contInfo = ContainerInfo.fromProtobuf(temp); + Pipeline pipeline = pipelineSelector + .getPipeline(contInfo.getPipelineName(), + contInfo.getReplicationType()); + + if(pipeline == null) { + pipeline = pipelineSelector + .getReplicationPipeline(contInfo.getReplicationType(), + contInfo.getReplicationFactor()); + } + return new ContainerWithPipeline(contInfo, pipeline); + } finally { + lock.unlock(); + } + } + /** * {@inheritDoc} */ @@ -208,13 +250,15 @@ public class ContainerMapping implements Mapping { * @throws IOException - Exception */ @Override - public ContainerInfo allocateContainer( + public ContainerWithPipeline allocateContainer( ReplicationType type, ReplicationFactor replicationFactor, String owner) throws IOException { ContainerInfo containerInfo; + ContainerWithPipeline containerWithPipeline; + if (!nodeManager.isOutOfChillMode()) { throw new SCMException( "Unable to create container while in chill mode", @@ -223,9 +267,9 @@ public class ContainerMapping implements Mapping { lock.lock(); try { - containerInfo = - containerStateManager.allocateContainer( + containerWithPipeline = containerStateManager.allocateContainer( pipelineSelector, type, replicationFactor, owner); + containerInfo = containerWithPipeline.getContainerInfo(); byte[] containerIDBytes = Longs.toByteArray( containerInfo.getContainerID()); @@ -234,7 +278,7 @@ public class ContainerMapping implements Mapping { } finally { lock.unlock(); } - return containerInfo; + return containerWithPipeline; } /** @@ -380,6 +424,35 @@ public class ContainerMapping implements Mapping { return containerStateManager; } + /** + * Return a container matching the attributes specified. + * + * @param size - Space needed in the Container. + * @param owner - Owner of the container - A specific nameservice. + * @param type - Replication Type {StandAlone, Ratis} + * @param factor - Replication Factor {ONE, THREE} + * @param state - State of the Container-- {Open, Allocated etc.} + * @return ContainerInfo, null if there is no match found. + */ + public ContainerWithPipeline getMatchingContainerWithPipeline(final long size, + String owner, ReplicationType type, ReplicationFactor factor, + LifeCycleState state) throws IOException { + ContainerInfo containerInfo = getStateManager() + .getMatchingContainer(size, owner, type, factor, state); + if (containerInfo == null) { + return null; + } + Pipeline pipeline = pipelineSelector + .getPipeline(containerInfo.getPipelineName(), + containerInfo.getReplicationType()); + if (pipeline == null) { + pipelineSelector + .getReplicationPipeline(containerInfo.getReplicationType(), + containerInfo.getReplicationFactor()); + } + return new ContainerWithPipeline(containerInfo, pipeline); + } + /** * Process container report from Datanode. *

@@ -415,7 +488,7 @@ public class ContainerMapping implements Mapping { HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes); HddsProtos.SCMContainerInfo newState = - reconcileState(datanodeState, knownState); + reconcileState(datanodeState, knownState, datanodeDetails); // FIX ME: This can be optimized, we write twice to memory, where a // single write would work well. @@ -425,8 +498,14 @@ public class ContainerMapping implements Mapping { containerStore.put(dbKey, newState.toByteArray()); // If the container is closed, then state is already written to SCM + Pipeline pipeline = pipelineSelector.getPipeline(newState.getPipelineName(), newState.getReplicationType()); + if(pipeline == null) { + pipeline = pipelineSelector + .getReplicationPipeline(newState.getReplicationType(), + newState.getReplicationFactor()); + } // DB.TODO: So can we can write only once to DB. - if (closeContainerIfNeeded(newState)) { + if (closeContainerIfNeeded(newState, pipeline)) { LOG.info("Closing the Container: {}", newState.getContainerID()); } } else { @@ -447,15 +526,22 @@ public class ContainerMapping implements Mapping { * * @param datanodeState - State from the Datanode. * @param knownState - State inside SCM. + * @param dnDetails * @return new SCM State for this container. */ private HddsProtos.SCMContainerInfo reconcileState( StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState, - HddsProtos.SCMContainerInfo knownState) { + SCMContainerInfo knownState, DatanodeDetails dnDetails) { HddsProtos.SCMContainerInfo.Builder builder = HddsProtos.SCMContainerInfo.newBuilder(); - builder.setContainerID(knownState.getContainerID()); - builder.setPipeline(knownState.getPipeline()); + builder.setContainerID(knownState.getContainerID()) + .setPipelineName(knownState.getPipelineName()) + .setReplicationType(knownState.getReplicationType()) + .setReplicationFactor(knownState.getReplicationFactor()); + + // TODO: If current state doesn't have this DN in list of DataNodes with replica + // then add it in list of replicas. + // If used size is greater than allocated size, we will be updating // allocated size with used size. This update is done as a fallback // mechanism in case SCM crashes without properly updating allocated @@ -464,13 +550,13 @@ public class ContainerMapping implements Mapping { long usedSize = datanodeState.getUsed(); long allocated = knownState.getAllocatedBytes() > usedSize ? knownState.getAllocatedBytes() : usedSize; - builder.setAllocatedBytes(allocated); - builder.setUsedBytes(usedSize); - builder.setNumberOfKeys(datanodeState.getKeyCount()); - builder.setState(knownState.getState()); - builder.setStateEnterTime(knownState.getStateEnterTime()); - builder.setContainerID(knownState.getContainerID()); - builder.setDeleteTransactionId(knownState.getDeleteTransactionId()); + builder.setAllocatedBytes(allocated) + .setUsedBytes(usedSize) + .setNumberOfKeys(datanodeState.getKeyCount()) + .setState(knownState.getState()) + .setStateEnterTime(knownState.getStateEnterTime()) + .setContainerID(knownState.getContainerID()) + .setDeleteTransactionId(knownState.getDeleteTransactionId()); if (knownState.getOwner() != null) { builder.setOwner(knownState.getOwner()); } @@ -485,9 +571,11 @@ public class ContainerMapping implements Mapping { * one protobuf in one file and another definition in another file. * * @param newState - This is the state we maintain in SCM. + * @param pipeline * @throws IOException */ - private boolean closeContainerIfNeeded(HddsProtos.SCMContainerInfo newState) + private boolean closeContainerIfNeeded(SCMContainerInfo newState, + Pipeline pipeline) throws IOException { float containerUsedPercentage = 1.0f * newState.getUsedBytes() / this.size; @@ -498,7 +586,7 @@ public class ContainerMapping implements Mapping { // We will call closer till get to the closed state. // That is SCM will make this call repeatedly until we reach the closed // state. - closer.close(newState); + closer.close(newState, pipeline); if (shouldClose(scmInfo)) { // This event moves the Container from Open to Closing State, this is @@ -598,10 +686,12 @@ public class ContainerMapping implements Mapping { .setAllocatedBytes(info.getAllocatedBytes()) .setNumberOfKeys(oldInfo.getNumberOfKeys()) .setOwner(oldInfo.getOwner()) - .setPipeline(oldInfo.getPipeline()) + .setPipelineName(oldInfo.getPipelineName()) .setState(oldInfo.getState()) .setUsedBytes(oldInfo.getUsedBytes()) .setDeleteTransactionId(oldInfo.getDeleteTransactionId()) + .setReplicationFactor(oldInfo.getReplicationFactor()) + .setReplicationType(oldInfo.getReplicationType()) .build(); containerStore.put(dbKey, newInfo.getProtobuf().toByteArray()); } else { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java index 08733bd707d..870ab1d1e5e 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hdds.scm.container; import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.scm.ScmConfigKeys; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.states.ContainerState; @@ -279,10 +280,10 @@ public class ContainerStateManager implements Closeable { * @param selector -- Pipeline selector class. * @param type -- Replication type. * @param replicationFactor - Replication replicationFactor. - * @return Container Info. + * @return ContainerWithPipeline * @throws IOException on Failure. */ - public ContainerInfo allocateContainer(PipelineSelector selector, HddsProtos + public ContainerWithPipeline allocateContainer(PipelineSelector selector, HddsProtos .ReplicationType type, HddsProtos.ReplicationFactor replicationFactor, String owner) throws IOException { @@ -295,7 +296,7 @@ public class ContainerStateManager implements Closeable { ContainerInfo containerInfo = new ContainerInfo.Builder() .setState(HddsProtos.LifeCycleState.ALLOCATED) - .setPipeline(pipeline) + .setPipelineName(pipeline.getPipelineName()) // This is bytes allocated for blocks inside container, not the // container size .setAllocatedBytes(0) @@ -305,11 +306,13 @@ public class ContainerStateManager implements Closeable { .setOwner(owner) .setContainerID(containerCount.incrementAndGet()) .setDeleteTransactionId(0) + .setReplicationFactor(replicationFactor) + .setReplicationType(pipeline.getType()) .build(); Preconditions.checkNotNull(containerInfo); containers.addContainer(containerInfo); LOG.trace("New container allocated: {}", containerInfo); - return containerInfo; + return new ContainerWithPipeline(containerInfo, pipeline); } /** @@ -432,8 +435,8 @@ public class ContainerStateManager implements Closeable { containerInfo.updateLastUsedTime(); ContainerState key = new ContainerState(owner, - containerInfo.getPipeline().getType(), - containerInfo.getPipeline().getFactor()); + containerInfo.getReplicationType(), + containerInfo.getReplicationFactor()); lastUsedMap.put(key, containerInfo.containerID()); return containerInfo; } @@ -457,6 +460,20 @@ public class ContainerStateManager implements Closeable { factor, type); } + /** + * Returns the containerInfo with pipeline for the given container id. + * @param selector -- Pipeline selector class. + * @param containerID id of the container + * @return ContainerInfo containerInfo + * @throws IOException + */ + public ContainerWithPipeline getContainer(PipelineSelector selector, + ContainerID containerID) throws IOException { + ContainerInfo info = containers.getContainerInfo(containerID.getId()); + Pipeline pipeline = selector.getPipeline(info.getPipelineName(), info.getReplicationType()); + return new ContainerWithPipeline(info, pipeline); + } + /** * Returns the containerInfo for the given container id. * @param containerID id of the container @@ -466,6 +483,7 @@ public class ContainerStateManager implements Closeable { public ContainerInfo getContainer(ContainerID containerID) { return containers.getContainerInfo(containerID.getId()); } + @Override public void close() throws IOException { } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java index e77a4b60f25..f52eb05ce01 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java @@ -17,6 +17,10 @@ package org.apache.hadoop.hdds.scm.container; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto @@ -42,6 +46,16 @@ public interface Mapping extends Closeable { */ ContainerInfo getContainer(long containerID) throws IOException; + /** + * Returns the ContainerInfo from the container ID. + * + * @param containerID - ID of container. + * @return - ContainerWithPipeline such as creation state and the pipeline. + * @throws IOException + */ + ContainerWithPipeline getContainerWithPipeline(long containerID) + throws IOException; + /** * Returns containers under certain conditions. * Search container IDs from start ID(exclusive), @@ -65,10 +79,10 @@ public interface Mapping extends Closeable { * * @param replicationFactor - replication factor of the container. * @param owner - * @return - Container Info. + * @return - ContainerWithPipeline. * @throws IOException */ - ContainerInfo allocateContainer(HddsProtos.ReplicationType type, + ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType type, HddsProtos.ReplicationFactor replicationFactor, String owner) throws IOException; @@ -120,4 +134,12 @@ public interface Mapping extends Closeable { * @return NodeManager */ NodeManager getNodeManager(); + + /** + * Returns the ContainerWithPipeline. + * @return NodeManager + */ + public ContainerWithPipeline getMatchingContainerWithPipeline(final long size, + String owner, ReplicationType type, ReplicationFactor factor, + LifeCycleState state) throws IOException; } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java index cbb2ba75c2e..3ca8ba91e65 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java @@ -22,6 +22,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.SCMContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -90,8 +92,10 @@ public class ContainerCloser { * lives. * * @param info - ContainerInfo. + * @param pipeline */ - public void close(HddsProtos.SCMContainerInfo info) { + public void close(SCMContainerInfo info, + Pipeline pipeline) { if (commandIssued.containsKey(info.getContainerID())) { // We check if we issued a close command in last 3 * reportInterval secs. @@ -126,13 +130,10 @@ public class ContainerCloser { // this queue can be emptied by a datanode after a close report is send // to SCM. In that case also, data node will ignore this command. - HddsProtos.Pipeline pipeline = info.getPipeline(); - for (HddsProtos.DatanodeDetailsProto datanodeDetails : - pipeline.getMembersList()) { - nodeManager.addDatanodeCommand( - DatanodeDetails.getFromProtoBuf(datanodeDetails).getUuid(), + for (DatanodeDetails datanodeDetails : pipeline.getMachines()) { + nodeManager.addDatanodeCommand(datanodeDetails.getUuid(), new CloseContainerCommand(info.getContainerID(), - pipeline.getType())); + info.getReplicationType())); } if (!commandIssued.containsKey(info.getContainerID())) { commandIssued.put(info.getContainerID(), diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java index 48c6423d02e..3ada8fe1928 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java @@ -116,7 +116,8 @@ public class ContainerStateMap { public void addContainer(ContainerInfo info) throws SCMException { Preconditions.checkNotNull(info, "Container Info cannot be null"); - Preconditions.checkNotNull(info.getPipeline(), "Pipeline cannot be null"); + Preconditions.checkArgument(info.getReplicationFactor().getNumber() > 0, + "ExpectedReplicaCount should be greater than 0"); try (AutoCloseableLock lock = autoLock.acquire()) { ContainerID id = ContainerID.valueof(info.getContainerID()); @@ -129,8 +130,8 @@ public class ContainerStateMap { lifeCycleStateMap.insert(info.getState(), id); ownerMap.insert(info.getOwner(), id); - factorMap.insert(info.getPipeline().getFactor(), id); - typeMap.insert(info.getPipeline().getType(), id); + factorMap.insert(info.getReplicationFactor(), id); + typeMap.insert(info.getReplicationType(), id); LOG.trace("Created container with {} successfully.", id); } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java index 48affa41129..a1fbce6c9ee 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java @@ -16,6 +16,9 @@ */ package org.apache.hadoop.hdds.scm.pipelines; +import java.util.LinkedList; +import java.util.Map; +import java.util.WeakHashMap; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; @@ -25,7 +28,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.LinkedList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; @@ -36,11 +38,13 @@ public abstract class PipelineManager { private static final Logger LOG = LoggerFactory.getLogger(PipelineManager.class); private final List activePipelines; + private final Map activePipelineMap; private final AtomicInteger pipelineIndex; public PipelineManager() { activePipelines = new LinkedList<>(); pipelineIndex = new AtomicInteger(0); + activePipelineMap = new WeakHashMap<>(); } /** @@ -76,6 +80,7 @@ public abstract class PipelineManager { "replicationType:{} replicationFactor:{}", pipeline.getPipelineName(), replicationType, replicationFactor); activePipelines.add(pipeline); + activePipelineMap.put(pipeline.getPipelineName(), pipeline); } else { pipeline = findOpenPipeline(replicationType, replicationFactor); @@ -94,6 +99,26 @@ public abstract class PipelineManager { } } + /** + * This function to get pipeline with given pipeline name. + * + * @param pipelineName + * @return a Pipeline. + */ + public synchronized final Pipeline getPipeline(String pipelineName) { + Pipeline pipeline = null; + + // 1. Check if pipeline channel already exists + if (activePipelineMap.containsKey(pipelineName)) { + pipeline = activePipelineMap.get(pipelineName); + LOG.debug("Returning pipeline for pipelineName:{}", pipelineName); + return pipeline; + } else { + LOG.debug("Unable to find pipeline for pipelineName:{}", pipelineName); + } + return pipeline; + } + protected int getReplicationCount(ReplicationFactor factor) { switch (factor) { case ONE: diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java index 508ca9bd3b6..3846a8426dd 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm.pipelines; import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.scm.ScmConfigKeys; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.placement.algorithms .ContainerPlacementPolicy; @@ -176,6 +177,21 @@ public class PipelineSelector { getPipeline(replicationFactor, replicationType); } + /** + * This function to return pipeline for given pipeline name and replication + * type. + */ + public Pipeline getPipeline(String pipelineName, + ReplicationType replicationType) throws IOException { + if (pipelineName == null) { + return null; + } + PipelineManager manager = getPipelineManager(replicationType); + Preconditions.checkNotNull(manager, "Found invalid pipeline manager"); + LOG.debug("Getting replication pipeline forReplicationType {} :" + + " pipelineName:{}", replicationType, pipelineName); + return manager.getPipeline(pipelineName); + } /** * Creates a pipeline from a specified set of Nodes. */ diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java index ace87582349..189060e0f6f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm.pipelines.ratis; import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.scm.XceiverClientRatis; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.placement.algorithms .ContainerPlacementPolicy; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java index e76027fb2b4..579a3a260db 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java @@ -17,6 +17,7 @@ package org.apache.hadoop.hdds.scm.pipelines.standalone; import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.container.placement.algorithms .ContainerPlacementPolicy; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java index d73cccd886d..e1d478f404d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hdds.protocol.proto .StorageContainerLocationProtocolProtos; import org.apache.hadoop.hdds.scm.HddsServerUtil; import org.apache.hadoop.hdds.scm.ScmInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; @@ -145,11 +146,12 @@ public class SCMClientProtocolServer implements } @Override - public ContainerInfo allocateContainer(HddsProtos.ReplicationType + public ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType replicationType, HddsProtos.ReplicationFactor factor, String owner) throws IOException { String remoteUser = getRpcRemoteUsername(); getScm().checkAdminAccess(remoteUser); + return scm.getScmContainerManager() .allocateContainer(replicationType, factor, owner); } @@ -162,6 +164,14 @@ public class SCMClientProtocolServer implements .getContainer(containerID); } + @Override + public ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException { + String remoteUser = getRpcRemoteUsername(); + getScm().checkAdminAccess(remoteUser); + return scm.getScmContainerManager() + .getContainerWithPipeline(containerID); + } + @Override public List listContainer(long startContainerID, int count) throws IOException { @@ -248,7 +258,7 @@ public class SCMClientProtocolServer implements throws IOException { // TODO: will be addressed in future patch. // This is needed only for debugging purposes to make sure cluster is - // working correctly. + // working correctly. return null; } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java index d06d568ae09..9255ec701ff 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java @@ -22,6 +22,7 @@ import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.hdds.scm.container.ContainerMapping; import org.apache.hadoop.hdds.scm.container.Mapping; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -362,10 +363,16 @@ public class TestDeletedBlockLog { pipeline.addMember(dd); ContainerInfo.Builder builder = new ContainerInfo.Builder(); - builder.setPipeline(pipeline); + builder.setPipelineName(pipeline.getPipelineName()) + .setReplicationType(pipeline.getType()) + .setReplicationFactor(pipeline.getFactor()); - ContainerInfo conatinerInfo = builder.build(); - Mockito.doReturn(conatinerInfo).when(mappingService) + ContainerInfo containerInfo = builder.build(); + ContainerWithPipeline containerWithPipeline = new ContainerWithPipeline( + containerInfo, pipeline); + Mockito.doReturn(containerInfo).when(mappingService) .getContainer(containerID); + Mockito.doReturn(containerWithPipeline).when(mappingService) + .getContainerWithPipeline(containerID); } } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java index 09ade3ea630..721dbf6b582 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java @@ -21,7 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.container.common.SCMTestUtils; @@ -97,7 +97,7 @@ public class TestCloseContainerEventHandler { new ContainerID(id)); eventQueue.processAll(1000); Assert.assertTrue(logCapturer.getOutput() - .contains("Container with id : " + id + " does not exist")); + .contains("Failed to update the container state")); } @Test @@ -105,11 +105,12 @@ public class TestCloseContainerEventHandler { GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer .captureLogs(CloseContainerEventHandler.LOG); - ContainerInfo info = mapping + ContainerWithPipeline containerWithPipeline = mapping .allocateContainer(HddsProtos.ReplicationType.STAND_ALONE, HddsProtos.ReplicationFactor.ONE, "ozone"); - ContainerID id = new ContainerID(info.getContainerID()); - DatanodeDetails datanode = info.getPipeline().getLeader(); + ContainerID id = new ContainerID( + containerWithPipeline.getContainerInfo().getContainerID()); + DatanodeDetails datanode = containerWithPipeline.getPipeline().getLeader(); int closeCount = nodeManager.getCommandCount(datanode); eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, id); eventQueue.processAll(1000); @@ -125,7 +126,8 @@ public class TestCloseContainerEventHandler { mapping.updateContainerState(id.getId(), CREATE); mapping.updateContainerState(id.getId(), CREATED); eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, - new ContainerID(info.getContainerID())); + new ContainerID( + containerWithPipeline.getContainerInfo().getContainerID())); eventQueue.processAll(1000); Assert.assertEquals(closeCount + 1, nodeManager.getCommandCount(datanode)); Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING, @@ -137,20 +139,23 @@ public class TestCloseContainerEventHandler { GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer .captureLogs(CloseContainerEventHandler.LOG); - ContainerInfo info = mapping + ContainerWithPipeline containerWithPipeline = mapping .allocateContainer(HddsProtos.ReplicationType.RATIS, HddsProtos.ReplicationFactor.THREE, "ozone"); - ContainerID id = new ContainerID(info.getContainerID()); + ContainerID id = new ContainerID( + containerWithPipeline.getContainerInfo().getContainerID()); int[] closeCount = new int[3]; eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, id); eventQueue.processAll(1000); int i = 0; - for (DatanodeDetails details : info.getPipeline().getMachines()) { + for (DatanodeDetails details : containerWithPipeline.getPipeline() + .getMachines()) { closeCount[i] = nodeManager.getCommandCount(details); i++; } i = 0; - for (DatanodeDetails details : info.getPipeline().getMachines()) { + for (DatanodeDetails details : containerWithPipeline.getPipeline() + .getMachines()) { Assert.assertEquals(closeCount[i], nodeManager.getCommandCount(details)); i++; } @@ -161,12 +166,12 @@ public class TestCloseContainerEventHandler { //Execute these state transitions so that we can close the container. mapping.updateContainerState(id.getId(), CREATE); mapping.updateContainerState(id.getId(), CREATED); - eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, - new ContainerID(info.getContainerID())); + eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, id); eventQueue.processAll(1000); i = 0; // Make sure close is queued for each datanode on the pipeline - for (DatanodeDetails details : info.getPipeline().getMachines()) { + for (DatanodeDetails details : containerWithPipeline.getPipeline() + .getMachines()) { Assert.assertEquals(closeCount[i] + 1, nodeManager.getCommandCount(details)); Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING, diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java index eefb639d59e..42ab126d4d7 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java @@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -103,7 +104,7 @@ public class TestContainerMapping { @Test public void testallocateContainer() throws Exception { - ContainerInfo containerInfo = mapping.allocateContainer( + ContainerWithPipeline containerInfo = mapping.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), containerOwner); @@ -120,7 +121,7 @@ public class TestContainerMapping { */ Set pipelineList = new TreeSet<>(); for (int x = 0; x < 30; x++) { - ContainerInfo containerInfo = mapping.allocateContainer( + ContainerWithPipeline containerInfo = mapping.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), containerOwner); @@ -135,14 +136,13 @@ public class TestContainerMapping { @Test public void testGetContainer() throws IOException { - ContainerInfo containerInfo = mapping.allocateContainer( + ContainerWithPipeline containerInfo = mapping.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), containerOwner); Pipeline pipeline = containerInfo.getPipeline(); Assert.assertNotNull(pipeline); - Pipeline newPipeline = mapping.getContainer( - containerInfo.getContainerID()).getPipeline(); + Pipeline newPipeline = containerInfo.getPipeline(); Assert.assertEquals(pipeline.getLeader().getUuid(), newPipeline.getLeader().getUuid()); } @@ -165,12 +165,12 @@ public class TestContainerMapping { public void testContainerCreationLeaseTimeout() throws IOException, InterruptedException { nodeManager.setChillmode(false); - ContainerInfo containerInfo = mapping.allocateContainer( + ContainerWithPipeline containerInfo = mapping.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), containerOwner); - mapping.updateContainerState(containerInfo.getContainerID(), - HddsProtos.LifeCycleEvent.CREATE); + mapping.updateContainerState(containerInfo.getContainerInfo() + .getContainerID(), HddsProtos.LifeCycleEvent.CREATE); Thread.sleep(TIMEOUT + 1000); NavigableSet deleteContainers = mapping.getStateManager() @@ -179,12 +179,14 @@ public class TestContainerMapping { xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.DELETING); - Assert.assertTrue(deleteContainers.contains(containerInfo.containerID())); + Assert.assertTrue(deleteContainers + .contains(containerInfo.getContainerInfo().containerID())); thrown.expect(IOException.class); thrown.expectMessage("Lease Exception"); - mapping.updateContainerState(containerInfo.getContainerID(), - HddsProtos.LifeCycleEvent.CREATED); + mapping + .updateContainerState(containerInfo.getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.CREATED); } @Test @@ -294,10 +296,11 @@ public class TestContainerMapping { private ContainerInfo createContainer() throws IOException { nodeManager.setChillmode(false); - ContainerInfo containerInfo = mapping.allocateContainer( + ContainerWithPipeline containerWithPipeline = mapping.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), containerOwner); + ContainerInfo containerInfo = containerWithPipeline.getContainerInfo(); mapping.updateContainerState(containerInfo.getContainerID(), HddsProtos.LifeCycleEvent.CREATE); mapping.updateContainerState(containerInfo.getContainerID(), diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java index 0d7848f7987..74238a77084 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.container.ContainerMapping; import org.apache.hadoop.hdds.scm.container.MockNodeManager; import org.apache.hadoop.hdds.scm.container.TestContainerMapping; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -91,9 +92,10 @@ public class TestContainerCloser { @Test public void testClose() throws IOException { - ContainerInfo info = mapping.allocateContainer( + ContainerWithPipeline containerWithPipeline = mapping.allocateContainer( HddsProtos.ReplicationType.STAND_ALONE, HddsProtos.ReplicationFactor.ONE, "ozone"); + ContainerInfo info = containerWithPipeline.getContainerInfo(); //Execute these state transitions so that we can close the container. mapping.updateContainerState(info.getContainerID(), CREATE); @@ -101,7 +103,7 @@ public class TestContainerCloser { long currentCount = mapping.getCloser().getCloseCount(); long runCount = mapping.getCloser().getThreadRunCount(); - DatanodeDetails datanode = info.getPipeline().getLeader(); + DatanodeDetails datanode = containerWithPipeline.getPipeline().getLeader(); // Send a container report with used set to 1 GB. This should not close. sendContainerReport(info, 1 * GIGABYTE); @@ -138,9 +140,10 @@ public class TestContainerCloser { configuration.setTimeDuration(OZONE_CONTAINER_REPORT_INTERVAL, 1, TimeUnit.SECONDS); - ContainerInfo info = mapping.allocateContainer( + ContainerWithPipeline containerWithPipeline = mapping.allocateContainer( HddsProtos.ReplicationType.STAND_ALONE, HddsProtos.ReplicationFactor.ONE, "ozone"); + ContainerInfo info = containerWithPipeline.getContainerInfo(); //Execute these state transitions so that we can close the container. mapping.updateContainerState(info.getContainerID(), CREATE); @@ -148,10 +151,10 @@ public class TestContainerCloser { long currentCount = mapping.getCloser().getCloseCount(); long runCount = mapping.getCloser().getThreadRunCount(); + DatanodeDetails datanodeDetails = containerWithPipeline.getPipeline() + .getLeader(); - DatanodeDetails datanodeDetails = info.getPipeline().getLeader(); - - // Send this command twice and assert we have only one command in the queue. + // Send this command twice and assert we have only one command in queue. sendContainerReport(info, 5 * GIGABYTE); sendContainerReport(info, 5 * GIGABYTE); @@ -183,9 +186,10 @@ public class TestContainerCloser { long runCount = mapping.getCloser().getThreadRunCount(); for (int x = 0; x < ContainerCloser.getCleanupWaterMark() + 10; x++) { - ContainerInfo info = mapping.allocateContainer( + ContainerWithPipeline containerWithPipeline = mapping.allocateContainer( HddsProtos.ReplicationType.STAND_ALONE, HddsProtos.ReplicationFactor.ONE, "ozone"); + ContainerInfo info = containerWithPipeline.getContainerInfo(); mapping.updateContainerState(info.getContainerID(), CREATE); mapping.updateContainerState(info.getContainerID(), CREATED); sendContainerReport(info, 5 * GIGABYTE); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java index 5ad28f62656..98b0a28bf25 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java @@ -25,7 +25,7 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.container.ContainerMapping; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.placement.algorithms .ContainerPlacementPolicy; import org.apache.hadoop.hdds.scm.container.placement.algorithms @@ -151,11 +151,11 @@ public class TestContainerPlacement { assertTrue(nodeManager.isOutOfChillMode()); - ContainerInfo containerInfo = containerManager.allocateContainer( + ContainerWithPipeline containerWithPipeline = containerManager.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), "OZONE"); assertEquals(xceiverClientManager.getFactor().getNumber(), - containerInfo.getPipeline().getMachines().size()); + containerWithPipeline.getPipeline().getMachines().size()); } finally { IOUtils.closeQuietly(containerManager); IOUtils.closeQuietly(nodeManager); diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java index 4f3b14385cc..e2267daba54 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java @@ -24,9 +24,9 @@ import org.apache.commons.cli.Options; import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler; import org.apache.hadoop.hdds.scm.cli.SCMCLI; import org.apache.hadoop.hdds.scm.client.ScmClient; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import java.io.IOException; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; /** * The handler of close container command. @@ -51,15 +51,15 @@ public class CloseContainerHandler extends OzoneCommandHandler { } String containerID = cmd.getOptionValue(OPT_CONTAINER_ID); - ContainerInfo container = getScmClient(). - getContainer(Long.parseLong(containerID)); + ContainerWithPipeline container = getScmClient(). + getContainerWithPipeline(Long.parseLong(containerID)); if (container == null) { throw new IOException("Cannot close an non-exist container " + containerID); } logOut("Closing container : %s.", containerID); - getScmClient().closeContainer(container.getContainerID(), - container.getPipeline()); + getScmClient() + .closeContainer(container.getContainerInfo().getContainerID()); logOut("Container closed."); } diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java index 20a6d9eb270..1b266653f03 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java @@ -25,9 +25,9 @@ import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler; import org.apache.hadoop.hdds.scm.client.ScmClient; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import java.io.IOException; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH; import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP; @@ -60,7 +60,7 @@ public class DeleteContainerHandler extends OzoneCommandHandler { String containerID = cmd.getOptionValue(OPT_CONTAINER_ID); - ContainerInfo container = getScmClient().getContainer( + ContainerWithPipeline container = getScmClient().getContainerWithPipeline( Long.parseLong(containerID)); if (container == null) { throw new IOException("Cannot delete an non-exist container " @@ -68,8 +68,9 @@ public class DeleteContainerHandler extends OzoneCommandHandler { } logOut("Deleting container : %s.", containerID); - getScmClient().deleteContainer(container.getContainerID(), - container.getPipeline(), cmd.hasOption(OPT_FORCE)); + getScmClient() + .deleteContainer(container.getContainerInfo().getContainerID(), + container.getPipeline(), cmd.hasOption(OPT_FORCE)); logOut("Container %s deleted.", containerID); } diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java index 6027becd26b..3716aceaa4d 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java @@ -24,7 +24,6 @@ import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler; import org.apache.hadoop.hdds.scm.client.ScmClient; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos .ContainerData; @@ -33,6 +32,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import java.io.IOException; import java.util.stream.Collectors; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH; import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP; @@ -68,13 +68,12 @@ public class InfoContainerHandler extends OzoneCommandHandler { } } String containerID = cmd.getOptionValue(OPT_CONTAINER_ID); - ContainerInfo container = getScmClient(). - getContainer(Long.parseLong(containerID)); + ContainerWithPipeline container = getScmClient(). + getContainerWithPipeline(Long.parseLong(containerID)); Preconditions.checkNotNull(container, "Container cannot be null"); - ContainerData containerData = - getScmClient().readContainer(container.getContainerID(), - container.getPipeline()); + ContainerData containerData = getScmClient().readContainer(container + .getContainerInfo().getContainerID(), container.getPipeline()); // Print container report info. logOut("Container id: %s", containerID); diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java index e1a29182b0d..edd85aabeec 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java @@ -21,8 +21,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.Seekable; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; import org.apache.hadoop.hdds.scm.XceiverClientManager; @@ -271,17 +271,17 @@ public class ChunkGroupInputStream extends InputStream implements Seekable { KsmKeyLocationInfo ksmKeyLocationInfo = keyLocationInfos.get(i); BlockID blockID = ksmKeyLocationInfo.getBlockID(); long containerID = blockID.getContainerID(); - ContainerInfo container = - storageContainerLocationClient.getContainer(containerID); - XceiverClientSpi xceiverClient = - xceiverClientManager.acquireClient(container.getPipeline(), containerID); + ContainerWithPipeline containerWithPipeline = + storageContainerLocationClient.getContainerWithPipeline(containerID); + XceiverClientSpi xceiverClient = xceiverClientManager + .acquireClient(containerWithPipeline.getPipeline(), containerID); boolean success = false; containerKey = ksmKeyLocationInfo.getLocalID(); try { LOG.debug("get key accessing {} {}", containerID, containerKey); groupInputStream.streamOffset[i] = length; - ContainerProtos.KeyData containerKeyData = OzoneContainerTranslation + ContainerProtos.KeyData containerKeyData = OzoneContainerTranslation .containerKeyDataForRead(blockID); ContainerProtos.GetKeyResponseProto response = ContainerProtocolCalls .getKey(xceiverClient, containerKeyData, requestId); @@ -292,7 +292,8 @@ public class ChunkGroupInputStream extends InputStream implements Seekable { } success = true; ChunkInputStream inputStream = new ChunkInputStream( - ksmKeyLocationInfo.getBlockID(), xceiverClientManager, xceiverClient, + ksmKeyLocationInfo.getBlockID(), xceiverClientManager, + xceiverClient, chunks, requestId); groupInputStream.addStream(inputStream, ksmKeyLocationInfo.getLength()); diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java index c6e56b3b9b9..d1a3b46b81c 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; @@ -163,10 +164,12 @@ public class ChunkGroupOutputStream extends OutputStream { private void checkKeyLocationInfo(KsmKeyLocationInfo subKeyInfo) throws IOException { - ContainerInfo container = scmClient.getContainer( - subKeyInfo.getContainerID()); + ContainerWithPipeline containerWithPipeline = scmClient + .getContainerWithPipeline(subKeyInfo.getContainerID()); + ContainerInfo container = containerWithPipeline.getContainerInfo(); + XceiverClientSpi xceiverClient = - xceiverClientManager.acquireClient(container.getPipeline(), + xceiverClientManager.acquireClient(containerWithPipeline.getPipeline(), container.getContainerID()); // create container if needed if (subKeyInfo.getShouldCreateContainer()) { diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OzonePBHelper.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OzonePBHelper.java new file mode 100644 index 00000000000..8361bac0d06 --- /dev/null +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OzonePBHelper.java @@ -0,0 +1,30 @@ +/** + * 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.ozone.protocolPB; + +/** + * Helper class for converting protobuf objects. + */ +public final class OzonePBHelper { + + private OzonePBHelper() { + /** Hidden constructor */ + } + + +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java index bedd5c453b4..bb8565072cf 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.scm.container; import com.google.common.primitives.Longs; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -30,7 +31,6 @@ import org.junit.Before; import org.junit.Test; import java.io.IOException; -import java.nio.charset.Charset; import java.util.ArrayList; import java.util.List; import java.util.NavigableSet; @@ -71,31 +71,35 @@ public class TestContainerStateManager { @Test public void testAllocateContainer() throws IOException { // Allocate a container and verify the container info - ContainerInfo container1 = scm.getClientProtocolServer().allocateContainer( - xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerOwner); + ContainerWithPipeline container1 = scm.getClientProtocolServer() + .allocateContainer( + xceiverClientManager.getType(), + xceiverClientManager.getFactor(), containerOwner); ContainerInfo info = containerStateManager .getMatchingContainer(OzoneConsts.GB * 3, containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.ALLOCATED); - Assert.assertEquals(container1.getContainerID(), info.getContainerID()); + Assert.assertEquals(container1.getContainerInfo().getContainerID(), + info.getContainerID()); Assert.assertEquals(OzoneConsts.GB * 3, info.getAllocatedBytes()); Assert.assertEquals(containerOwner, info.getOwner()); Assert.assertEquals(xceiverClientManager.getType(), - info.getPipeline().getType()); + info.getReplicationType()); Assert.assertEquals(xceiverClientManager.getFactor(), - info.getPipeline().getFactor()); + info.getReplicationFactor()); Assert.assertEquals(HddsProtos.LifeCycleState.ALLOCATED, info.getState()); // Check there are two containers in ALLOCATED state after allocation - ContainerInfo container2 = scm.getClientProtocolServer().allocateContainer( - xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerOwner); + ContainerWithPipeline container2 = scm.getClientProtocolServer() + .allocateContainer( + xceiverClientManager.getType(), + xceiverClientManager.getFactor(), containerOwner); int numContainers = containerStateManager .getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.ALLOCATED).size(); - Assert.assertNotEquals(container1.getContainerID(), container2.getContainerID()); + Assert.assertNotEquals(container1.getContainerInfo().getContainerID(), + container2.getContainerInfo().getContainerID()); Assert.assertEquals(2, numContainers); } @@ -105,14 +109,15 @@ public class TestContainerStateManager { List containers = new ArrayList<>(); for (int i = 0; i < 10; i++) { - ContainerInfo container = scm.getClientProtocolServer().allocateContainer( - xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerOwner); - containers.add(container); + ContainerWithPipeline container = scm.getClientProtocolServer() + .allocateContainer( + xceiverClientManager.getType(), + xceiverClientManager.getFactor(), containerOwner); + containers.add(container.getContainerInfo()); if (i >= 5) { - scm.getScmContainerManager() - .updateContainerState(container.getContainerID(), - HddsProtos.LifeCycleEvent.CREATE); + scm.getScmContainerManager().updateContainerState(container + .getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.CREATE); } } @@ -134,34 +139,40 @@ public class TestContainerStateManager { @Test public void testGetMatchingContainer() throws IOException { - ContainerInfo container1 = scm.getClientProtocolServer(). + ContainerWithPipeline container1 = scm.getClientProtocolServer(). allocateContainer(xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerOwner); - scmContainerMapping.updateContainerState(container1.getContainerID(), - HddsProtos.LifeCycleEvent.CREATE); - scmContainerMapping.updateContainerState(container1.getContainerID(), - HddsProtos.LifeCycleEvent.CREATED); + xceiverClientManager.getFactor(), containerOwner); + scmContainerMapping + .updateContainerState(container1.getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.CREATE); + scmContainerMapping + .updateContainerState(container1.getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.CREATED); - ContainerInfo container2 = scm.getClientProtocolServer(). + ContainerWithPipeline container2 = scm.getClientProtocolServer(). allocateContainer(xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerOwner); + xceiverClientManager.getFactor(), containerOwner); ContainerInfo info = containerStateManager .getMatchingContainer(OzoneConsts.GB * 3, containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.OPEN); - Assert.assertEquals(container1.getContainerID(), info.getContainerID()); + Assert.assertEquals(container1.getContainerInfo().getContainerID(), + info.getContainerID()); info = containerStateManager .getMatchingContainer(OzoneConsts.GB * 3, containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.ALLOCATED); - Assert.assertEquals(container2.getContainerID(), info.getContainerID()); + Assert.assertEquals(container2.getContainerInfo().getContainerID(), + info.getContainerID()); - scmContainerMapping.updateContainerState(container2.getContainerID(), - HddsProtos.LifeCycleEvent.CREATE); - scmContainerMapping.updateContainerState(container2.getContainerID(), - HddsProtos.LifeCycleEvent.CREATED); + scmContainerMapping + .updateContainerState(container2.getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.CREATE); + scmContainerMapping + .updateContainerState(container2.getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.CREATED); // space has already been allocated in container1, now container 2 should // be chosen. @@ -169,7 +180,8 @@ public class TestContainerStateManager { .getMatchingContainer(OzoneConsts.GB * 3, containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.OPEN); - Assert.assertEquals(container2.getContainerID(), info.getContainerID()); + Assert.assertEquals(container2.getContainerInfo().getContainerID(), + info.getContainerID()); } @Test @@ -183,30 +195,33 @@ public class TestContainerStateManager { // Allocate container1 and update its state from ALLOCATED -> CREATING -> // OPEN -> CLOSING -> CLOSED -> DELETING -> DELETED - ContainerInfo container1 = scm.getClientProtocolServer().allocateContainer( - xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerOwner); + ContainerWithPipeline container1 = scm.getClientProtocolServer() + .allocateContainer( + xceiverClientManager.getType(), + xceiverClientManager.getFactor(), containerOwner); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.ALLOCATED).size(); Assert.assertEquals(1, containers); - scmContainerMapping.updateContainerState(container1.getContainerID(), - HddsProtos.LifeCycleEvent.CREATE); + scmContainerMapping + .updateContainerState(container1.getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.CREATE); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.CREATING).size(); Assert.assertEquals(1, containers); - scmContainerMapping.updateContainerState(container1.getContainerID(), - HddsProtos.LifeCycleEvent.CREATED); + scmContainerMapping + .updateContainerState(container1.getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.CREATED); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.OPEN).size(); Assert.assertEquals(1, containers); scmContainerMapping - .updateContainerState(container1.getContainerID(), + .updateContainerState(container1.getContainerInfo().getContainerID(), HddsProtos.LifeCycleEvent.FINALIZE); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), @@ -214,7 +229,7 @@ public class TestContainerStateManager { Assert.assertEquals(1, containers); scmContainerMapping - .updateContainerState(container1.getContainerID(), + .updateContainerState(container1.getContainerInfo().getContainerID(), HddsProtos.LifeCycleEvent.CLOSE); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), @@ -222,7 +237,7 @@ public class TestContainerStateManager { Assert.assertEquals(1, containers); scmContainerMapping - .updateContainerState(container1.getContainerID(), + .updateContainerState(container1.getContainerInfo().getContainerID(), HddsProtos.LifeCycleEvent.DELETE); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), @@ -230,7 +245,7 @@ public class TestContainerStateManager { Assert.assertEquals(1, containers); scmContainerMapping - .updateContainerState(container1.getContainerID(), + .updateContainerState(container1.getContainerInfo().getContainerID(), HddsProtos.LifeCycleEvent.CLEANUP); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), @@ -239,13 +254,15 @@ public class TestContainerStateManager { // Allocate container1 and update its state from ALLOCATED -> CREATING -> // DELETING - ContainerInfo container2 = scm.getClientProtocolServer().allocateContainer( - xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerOwner); - scmContainerMapping.updateContainerState(container2.getContainerID(), - HddsProtos.LifeCycleEvent.CREATE); + ContainerWithPipeline container2 = scm.getClientProtocolServer() + .allocateContainer( + xceiverClientManager.getType(), + xceiverClientManager.getFactor(), containerOwner); scmContainerMapping - .updateContainerState(container2.getContainerID(), + .updateContainerState(container2.getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.CREATE); + scmContainerMapping + .updateContainerState(container2.getContainerInfo().getContainerID(), HddsProtos.LifeCycleEvent.TIMEOUT); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), @@ -254,17 +271,21 @@ public class TestContainerStateManager { // Allocate container1 and update its state from ALLOCATED -> CREATING -> // OPEN -> CLOSING -> CLOSED - ContainerInfo container3 = scm.getClientProtocolServer().allocateContainer( - xceiverClientManager.getType(), - xceiverClientManager.getFactor(), containerOwner); - scmContainerMapping.updateContainerState(container3.getContainerID(), - HddsProtos.LifeCycleEvent.CREATE); - scmContainerMapping.updateContainerState(container3.getContainerID(), - HddsProtos.LifeCycleEvent.CREATED); - scmContainerMapping.updateContainerState(container3.getContainerID(), - HddsProtos.LifeCycleEvent.FINALIZE); + ContainerWithPipeline container3 = scm.getClientProtocolServer() + .allocateContainer( + xceiverClientManager.getType(), + xceiverClientManager.getFactor(), containerOwner); scmContainerMapping - .updateContainerState(container3.getContainerID(), + .updateContainerState(container3.getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.CREATE); + scmContainerMapping + .updateContainerState(container3.getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.CREATED); + scmContainerMapping + .updateContainerState(container3.getContainerInfo().getContainerID(), + HddsProtos.LifeCycleEvent.FINALIZE); + scmContainerMapping + .updateContainerState(container3.getContainerInfo().getContainerID(), HddsProtos.LifeCycleEvent.CLOSE); containers = containerStateManager.getMatchingContainerIDs(containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), @@ -274,12 +295,14 @@ public class TestContainerStateManager { @Test public void testUpdatingAllocatedBytes() throws Exception { - ContainerInfo container1 = scm.getClientProtocolServer().allocateContainer( - xceiverClientManager.getType(), + ContainerWithPipeline container1 = scm.getClientProtocolServer() + .allocateContainer(xceiverClientManager.getType(), xceiverClientManager.getFactor(), containerOwner); - scmContainerMapping.updateContainerState(container1.getContainerID(), + scmContainerMapping.updateContainerState(container1 + .getContainerInfo().getContainerID(), HddsProtos.LifeCycleEvent.CREATE); - scmContainerMapping.updateContainerState(container1.getContainerID(), + scmContainerMapping.updateContainerState(container1 + .getContainerInfo().getContainerID(), HddsProtos.LifeCycleEvent.CREATED); Random ran = new Random(); @@ -292,18 +315,18 @@ public class TestContainerStateManager { .getMatchingContainer(size, containerOwner, xceiverClientManager.getType(), xceiverClientManager.getFactor(), HddsProtos.LifeCycleState.OPEN); - Assert.assertEquals(container1.getContainerID(), info.getContainerID()); + Assert.assertEquals(container1.getContainerInfo().getContainerID(), + info.getContainerID()); ContainerMapping containerMapping = - (ContainerMapping)scmContainerMapping; + (ContainerMapping) scmContainerMapping; // manually trigger a flush, this will persist the allocated bytes value // to disk containerMapping.flushContainerInfo(); // the persisted value should always be equal to allocated size. - byte[] containerBytes = - containerMapping.getContainerStore().get( - Longs.toByteArray(container1.getContainerID())); + byte[] containerBytes = containerMapping.getContainerStore().get( + Longs.toByteArray(container1.getContainerInfo().getContainerID())); HddsProtos.SCMContainerInfo infoProto = HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes); ContainerInfo currentInfo = ContainerInfo.fromProtobuf(infoProto); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java index d4c9d4f7131..129cf0488e3 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java @@ -18,7 +18,7 @@ package org.apache.hadoop.ozone; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -28,7 +28,6 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.client.ContainerOperationClient; import org.apache.hadoop.hdds.scm.client.ScmClient; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB; import org.junit.AfterClass; @@ -78,12 +77,12 @@ public class TestContainerOperations { */ @Test public void testCreate() throws Exception { - ContainerInfo container = storageClient.createContainer(HddsProtos + ContainerWithPipeline container = storageClient.createContainer(HddsProtos .ReplicationType.STAND_ALONE, HddsProtos.ReplicationFactor .ONE, "OZONE"); - assertEquals(container.getContainerID(), - storageClient.getContainer(container.getContainerID()). - getContainerID()); + assertEquals(container.getContainerInfo().getContainerID(), storageClient + .getContainer(container.getContainerInfo().getContainerID()) + .getContainerID()); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java index 0c1d8f2ca7f..d07097ca8b9 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java @@ -22,7 +22,7 @@ import java.io.IOException; import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer; import org.apache.hadoop.hdds.scm.server.SCMStorage; import org.apache.hadoop.hdds.scm.node.NodeManager; @@ -131,7 +131,7 @@ public class TestStorageContainerManager { } try { - ContainerInfo container2 = mockClientServer + ContainerWithPipeline container2 = mockClientServer .allocateContainer(xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, "OZONE"); if (expectPermissionDenied) { @@ -144,7 +144,7 @@ public class TestStorageContainerManager { } try { - ContainerInfo container3 = mockClientServer + ContainerWithPipeline container3 = mockClientServer .allocateContainer(xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, "OZONE"); if (expectPermissionDenied) { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java index c937980f517..4c2a904ef04 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java @@ -23,7 +23,7 @@ import com.google.common.primitives.Longs; import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StorageType; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -158,9 +158,11 @@ public class TestStorageContainerManagerHelper { private MetadataStore getContainerMetadata(Long containerID) throws IOException { - ContainerInfo container = cluster.getStorageContainerManager() - .getClientProtocolServer().getContainer(containerID); - DatanodeDetails leadDN = container.getPipeline().getLeader(); + ContainerWithPipeline containerWithPipeline = cluster + .getStorageContainerManager().getClientProtocolServer() + .getContainerWithPipeline(containerID); + + DatanodeDetails leadDN = containerWithPipeline.getPipeline().getLeader(); OzoneContainer containerServer = getContainerServerByDatanodeUuid(leadDN.getUuidString()); ContainerData containerData = containerServer.getContainerManager() diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java index cafe5db0d07..214382e5f65 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java @@ -390,8 +390,8 @@ public class TestOzoneRpcClient { keyInfo.getLatestVersionLocations().getLocationList()) { ContainerInfo container = storageContainerLocationClient.getContainer(info.getContainerID()); - if ((container.getPipeline().getFactor() != replicationFactor) || - (container.getPipeline().getType() != replicationType)) { + if (!container.getReplicationFactor().equals(replicationFactor) || ( + container.getReplicationType() != replicationType)) { return false; } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java index 265c82bdeb9..3e514e7906a 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java @@ -23,8 +23,6 @@ import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.scm.container.ContainerID; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.MiniOzoneCluster; @@ -35,7 +33,6 @@ import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; -import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; @@ -112,9 +109,9 @@ public class TestCloseContainerByPipeline { .get(0).getBlocksLatestVersionOnly().get(0); long containerID = ksmKeyLocationInfo.getContainerID(); - List datanodes = - cluster.getStorageContainerManager().getContainerInfo(containerID) - .getPipeline().getMachines(); + List datanodes = cluster.getStorageContainerManager() + .getScmContainerManager().getContainerWithPipeline(containerID) + .getPipeline().getMachines(); Assert.assertTrue(datanodes.size() == 1); DatanodeDetails datanodeDetails = datanodes.get(0); @@ -167,9 +164,9 @@ public class TestCloseContainerByPipeline { .get(0).getBlocksLatestVersionOnly().get(0); long containerID = ksmKeyLocationInfo.getContainerID(); - List datanodes = - cluster.getStorageContainerManager().getContainerInfo(containerID) - .getPipeline().getMachines(); + List datanodes = cluster.getStorageContainerManager() + .getScmContainerManager().getContainerWithPipeline(containerID) + .getPipeline().getMachines(); Assert.assertTrue(datanodes.size() == 1); DatanodeDetails datanodeDetails = datanodes.get(0); @@ -220,9 +217,9 @@ public class TestCloseContainerByPipeline { .get(0).getBlocksLatestVersionOnly().get(0); long containerID = ksmKeyLocationInfo.getContainerID(); - List datanodes = - cluster.getStorageContainerManager().getContainerInfo(containerID) - .getPipeline().getMachines(); + List datanodes = cluster.getStorageContainerManager() + .getScmContainerManager().getContainerWithPipeline(containerID) + .getPipeline().getMachines(); Assert.assertTrue(datanodes.size() == 3); GenericTestUtils.LogCapturer logCapturer = diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java index bafba32008f..1cc7ff8f225 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java @@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.client.ReplicationFactor; import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; @@ -32,7 +33,6 @@ import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.junit.AfterClass; import org.junit.BeforeClass; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java index b1e9d26f105..144c562a668 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java @@ -17,14 +17,12 @@ */ package org.apache.hadoop.ozone.scm; -import org.apache.commons.lang3.RandomStringUtils; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -68,7 +66,7 @@ public class TestAllocateContainer { @Test public void testAllocate() throws Exception { - ContainerInfo container = storageContainerLocationClient.allocateContainer( + ContainerWithPipeline container = storageContainerLocationClient.allocateContainer( xceiverClientManager.getType(), xceiverClientManager.getFactor(), containerOwner); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java index ce1fe4615ca..42bb9369861 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java @@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.scm; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -81,17 +81,18 @@ public class TestContainerSmallFile { @Test public void testAllocateWrite() throws Exception { String traceID = UUID.randomUUID().toString(); - ContainerInfo container = + ContainerWithPipeline container = storageContainerLocationClient.allocateContainer( xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, containerOwner); - XceiverClientSpi client = xceiverClientManager.acquireClient( - container.getPipeline(), container.getContainerID()); + XceiverClientSpi client = xceiverClientManager + .acquireClient(container.getPipeline(), + container.getContainerInfo().getContainerID()); ContainerProtocolCalls.createContainer(client, - container.getContainerID(), traceID); + container.getContainerInfo().getContainerID(), traceID); BlockID blockID = ContainerTestHelper.getTestBlockID( - container.getContainerID()); + container.getContainerInfo().getContainerID()); ContainerProtocolCalls.writeSmallFile(client, blockID, "data123".getBytes(), traceID); ContainerProtos.GetSmallFileResponseProto response = @@ -104,20 +105,21 @@ public class TestContainerSmallFile { @Test public void testInvalidKeyRead() throws Exception { String traceID = UUID.randomUUID().toString(); - ContainerInfo container = + ContainerWithPipeline container = storageContainerLocationClient.allocateContainer( xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, containerOwner); - XceiverClientSpi client = xceiverClientManager.acquireClient( - container.getPipeline(), container.getContainerID()); + XceiverClientSpi client = xceiverClientManager + .acquireClient(container.getPipeline(), + container.getContainerInfo().getContainerID()); ContainerProtocolCalls.createContainer(client, - container.getContainerID(), traceID); + container.getContainerInfo().getContainerID(), traceID); thrown.expect(StorageContainerException.class); thrown.expectMessage("Unable to find the key"); BlockID blockID = ContainerTestHelper.getTestBlockID( - container.getContainerID()); + container.getContainerInfo().getContainerID()); // Try to read a Key Container Name ContainerProtos.GetSmallFileResponseProto response = ContainerProtocolCalls.readSmallFile(client, blockID, traceID); @@ -128,20 +130,20 @@ public class TestContainerSmallFile { public void testInvalidContainerRead() throws Exception { String traceID = UUID.randomUUID().toString(); long nonExistContainerID = 8888L; - ContainerInfo container = + ContainerWithPipeline container = storageContainerLocationClient.allocateContainer( xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, containerOwner); - XceiverClientSpi client = xceiverClientManager. - acquireClient(container.getPipeline(), container.getContainerID()); + XceiverClientSpi client = xceiverClientManager + .acquireClient(container.getPipeline(), + container.getContainerInfo().getContainerID()); ContainerProtocolCalls.createContainer(client, - container.getContainerID(), traceID); + container.getContainerInfo().getContainerID(), traceID); BlockID blockID = ContainerTestHelper.getTestBlockID( - container.getContainerID()); + container.getContainerInfo().getContainerID()); ContainerProtocolCalls.writeSmallFile(client, blockID, "data123".getBytes(), traceID); - thrown.expect(StorageContainerException.class); thrown.expectMessage("Unable to find the container"); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java index 732221a4d58..a6bb586f53a 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java @@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.scm; import com.google.common.primitives.Longs; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.MiniOzoneCluster; @@ -136,7 +137,7 @@ public class TestSCMCli { private boolean containerExist(long containerID) { try { ContainerInfo container = scm.getClientProtocolServer() - .getContainer(containerID); + .getContainerWithPipeline(containerID).getContainerInfo(); return container != null && containerID == container.getContainerID(); } catch (IOException e) { @@ -157,31 +158,34 @@ public class TestSCMCli { // 1. Test to delete a non-empty container. // **************************************** // Create an non-empty container - ContainerInfo container = containerOperationClient + ContainerWithPipeline container = containerOperationClient .createContainer(xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, containerOwner); ContainerData cdata = ContainerData .getFromProtBuf(containerOperationClient.readContainer( - container.getContainerID(), container.getPipeline()), conf); - KeyUtils.getDB(cdata, conf).put(Longs.toByteArray(container.getContainerID()), - "someKey".getBytes()); - Assert.assertTrue(containerExist(container.getContainerID())); + container.getContainerInfo().getContainerID()), conf); + KeyUtils.getDB(cdata, conf) + .put(Longs.toByteArray(container.getContainerInfo().getContainerID()), + "someKey".getBytes()); + Assert.assertTrue( + containerExist(container.getContainerInfo().getContainerID())); // Gracefully delete a container should fail because it is open. - delCmd = new String[] {"-container", "-delete", "-c", - Long.toString(container.getContainerID())}; + delCmd = new String[]{"-container", "-delete", "-c", + Long.toString(container.getContainerInfo().getContainerID())}; testErr = new ByteArrayOutputStream(); ByteArrayOutputStream out = new ByteArrayOutputStream(); exitCode = runCommandAndGetOutput(delCmd, out, testErr); assertEquals(EXECUTION_ERROR, exitCode); assertTrue(testErr.toString() .contains("Deleting an open container is not allowed.")); - Assert.assertTrue(containerExist(container.getContainerID())); + Assert.assertTrue( + containerExist(container.getContainerInfo().getContainerID())); // Close the container containerOperationClient.closeContainer( - container.getContainerID(), container.getPipeline()); + container.getContainerInfo().getContainerID()); // Gracefully delete a container should fail because it is not empty. testErr = new ByteArrayOutputStream(); @@ -189,45 +193,49 @@ public class TestSCMCli { assertEquals(EXECUTION_ERROR, exitCode2); assertTrue(testErr.toString() .contains("Container cannot be deleted because it is not empty.")); - Assert.assertTrue(containerExist(container.getContainerID())); + Assert.assertTrue( + containerExist(container.getContainerInfo().getContainerID())); // Try force delete again. - delCmd = new String[] {"-container", "-delete", "-c", - Long.toString(container.getContainerID()), "-f"}; + delCmd = new String[]{"-container", "-delete", "-c", + Long.toString(container.getContainerInfo().getContainerID()), "-f"}; exitCode = runCommandAndGetOutput(delCmd, out, null); assertEquals("Expected success, found:", ResultCode.SUCCESS, exitCode); - assertFalse(containerExist(container.getContainerID())); + assertFalse(containerExist(container.getContainerInfo().getContainerID())); // **************************************** // 2. Test to delete an empty container. // **************************************** // Create an empty container - ContainerInfo emptyContainer = containerOperationClient + ContainerWithPipeline emptyContainer = containerOperationClient .createContainer(xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, containerOwner); - containerOperationClient.closeContainer(emptyContainer.getContainerID(), - container.getPipeline()); - Assert.assertTrue(containerExist(emptyContainer.getContainerID())); + containerOperationClient + .closeContainer(emptyContainer.getContainerInfo().getContainerID()); + Assert.assertTrue( + containerExist(emptyContainer.getContainerInfo().getContainerID())); // Successfully delete an empty container. - delCmd = new String[] {"-container", "-delete", "-c", - Long.toString(emptyContainer.getContainerID())}; + delCmd = new String[]{"-container", "-delete", "-c", + Long.toString(emptyContainer.getContainerInfo().getContainerID())}; exitCode = runCommandAndGetOutput(delCmd, out, null); assertEquals(ResultCode.SUCCESS, exitCode); - assertFalse(containerExist(emptyContainer.getContainerID())); + assertFalse( + containerExist(emptyContainer.getContainerInfo().getContainerID())); // After the container is deleted, // another container can now be recreated. - ContainerInfo newContainer = containerOperationClient. + ContainerWithPipeline newContainer = containerOperationClient. createContainer(xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, containerOwner); - Assert.assertTrue(containerExist(newContainer.getContainerID())); + Assert.assertTrue( + containerExist(newContainer.getContainerInfo().getContainerID())); // **************************************** // 3. Test to delete a non-exist container. // **************************************** - long nonExistContainerID = ContainerTestHelper.getTestContainerID(); - delCmd = new String[] {"-container", "-delete", "-c", + long nonExistContainerID = ContainerTestHelper.getTestContainerID(); + delCmd = new String[]{"-container", "-delete", "-c", Long.toString(nonExistContainerID)}; testErr = new ByteArrayOutputStream(); exitCode = runCommandAndGetOutput(delCmd, out, testErr); @@ -250,45 +258,33 @@ public class TestSCMCli { "LeaderID: %s\n" + "Datanodes: [%s]\n"; - String formatStrWithHash = - "Container id: %s\n" + - "Container State: %s\n" + - "Container Hash: %s\n" + - "Container DB Path: %s\n" + - "Container Path: %s\n" + - "Container Metadata: {%s}\n" + - "LeaderID: %s\n" + - "Datanodes: [%s]\n"; - // Test a non-exist container String containerID = Long.toString(ContainerTestHelper.getTestContainerID()); - String[] info = { "-container", "-info", containerID }; + String[] info = {"-container", "-info", containerID}; int exitCode = runCommandAndGetOutput(info, null, null); assertEquals("Expected Execution Error, Did not find that.", EXECUTION_ERROR, exitCode); // Create an empty container. - ContainerInfo container = containerOperationClient + ContainerWithPipeline container = containerOperationClient .createContainer(xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, containerOwner); - ContainerData data = ContainerData - .getFromProtBuf(containerOperationClient. - readContainer(container.getContainerID(), - container.getPipeline()), conf); + ContainerData data = ContainerData.getFromProtBuf(containerOperationClient + .readContainer(container.getContainerInfo().getContainerID()), conf); - info = new String[] { "-container", "-info", "-c", - Long.toString(container.getContainerID()) }; + info = new String[]{"-container", "-info", "-c", + Long.toString(container.getContainerInfo().getContainerID())}; ByteArrayOutputStream out = new ByteArrayOutputStream(); exitCode = runCommandAndGetOutput(info, out, null); assertEquals("Expected Success, did not find it.", ResultCode.SUCCESS, - exitCode); + exitCode); String openStatus = data.isOpen() ? "OPEN" : "CLOSED"; - String expected = - String.format(formatStr, container.getContainerID(), openStatus, - data.getDBPath(), data.getContainerPath(), "", - datanodeDetails.getHostName(), datanodeDetails.getHostName()); + String expected = String.format(formatStr, container.getContainerInfo() + .getContainerID(), openStatus, data.getDBPath(), + data.getContainerPath(), "", datanodeDetails.getHostName(), + datanodeDetails.getHostName()); assertEquals(expected, out.toString()); out.reset(); @@ -299,40 +295,39 @@ public class TestSCMCli { HddsProtos.ReplicationFactor.ONE, containerOwner); data = ContainerData .getFromProtBuf(containerOperationClient.readContainer( - container.getContainerID(), container.getPipeline()), conf); + container.getContainerInfo().getContainerID()), conf); KeyUtils.getDB(data, conf) .put(containerID.getBytes(), "someKey".getBytes()); - info = new String[] { "-container", "-info", "-c", - Long.toString(container.getContainerID()) }; + info = new String[]{"-container", "-info", "-c", + Long.toString(container.getContainerInfo().getContainerID())}; exitCode = runCommandAndGetOutput(info, out, null); assertEquals(ResultCode.SUCCESS, exitCode); openStatus = data.isOpen() ? "OPEN" : "CLOSED"; - expected = String.format(formatStr, container.getContainerID(), openStatus, - data.getDBPath(), data.getContainerPath(), "", - datanodeDetails.getHostName(), datanodeDetails.getHostName()); + expected = String.format(formatStr, container.getContainerInfo(). + getContainerID(), openStatus, data.getDBPath(), + data.getContainerPath(), "", datanodeDetails.getHostName(), + datanodeDetails.getHostName()); assertEquals(expected, out.toString()); out.reset(); - // Close last container and test info again. - containerOperationClient.closeContainer( - container.getContainerID(), container.getPipeline()); + containerOperationClient + .closeContainer(container.getContainerInfo().getContainerID()); - info = new String[] { "-container", "-info", "-c", - Long.toString(container.getContainerID()) }; + info = new String[]{"-container", "-info", "-c", + Long.toString(container.getContainerInfo().getContainerID())}; exitCode = runCommandAndGetOutput(info, out, null); assertEquals(ResultCode.SUCCESS, exitCode); - data = ContainerData - .getFromProtBuf(containerOperationClient.readContainer( - container.getContainerID(), container.getPipeline()), conf); + data = ContainerData.getFromProtBuf(containerOperationClient + .readContainer(container.getContainerInfo().getContainerID()), conf); openStatus = data.isOpen() ? "OPEN" : "CLOSED"; expected = String - .format(formatStr, container.getContainerID(), openStatus, - data.getDBPath(), data.getContainerPath(), "", + .format(formatStr, container.getContainerInfo().getContainerID(), + openStatus, data.getDBPath(), data.getContainerPath(), "", datanodeDetails.getHostName(), datanodeDetails.getHostName()); assertEquals(expected, out.toString()); } @@ -360,10 +355,10 @@ public class TestSCMCli { // Create 20 containers for testing. List containers = new ArrayList<>(); for (int index = 0; index < 20; index++) { - ContainerInfo container = containerOperationClient.createContainer( + ContainerWithPipeline container = containerOperationClient.createContainer( xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, containerOwner); - containers.add(container); + containers.add(container.getContainerInfo()); } ByteArrayOutputStream out = new ByteArrayOutputStream(); @@ -417,11 +412,11 @@ public class TestSCMCli { @Test public void testCloseContainer() throws Exception { - long containerID = containerOperationClient - .createContainer(xceiverClientManager.getType(), - HddsProtos.ReplicationFactor.ONE, containerOwner).getContainerID(); + long containerID = containerOperationClient.createContainer( + xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE, + containerOwner).getContainerInfo().getContainerID(); ContainerInfo container = scm.getClientProtocolServer() - .getContainer(containerID); + .getContainerWithPipeline(containerID).getContainerInfo(); assertNotNull(container); assertEquals(containerID, container.getContainerID()); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientManager.java index 56f3c7a0fa0..a75264e4c56 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientManager.java @@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.scm; import com.google.common.cache.Cache; import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.hdds.scm.ScmConfigKeys; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -98,22 +98,25 @@ public class TestXceiverClientManager { shouldUseGrpc); XceiverClientManager clientManager = new XceiverClientManager(conf); - ContainerInfo container1 = storageContainerLocationClient + ContainerWithPipeline container1 = storageContainerLocationClient .allocateContainer(clientManager.getType(), clientManager.getFactor(), containerOwner); - XceiverClientSpi client1 = clientManager.acquireClient(container1.getPipeline(), - container1.getContainerID()); + XceiverClientSpi client1 = clientManager + .acquireClient(container1.getPipeline(), + container1.getContainerInfo().getContainerID()); Assert.assertEquals(1, client1.getRefcount()); - ContainerInfo container2 = storageContainerLocationClient + ContainerWithPipeline container2 = storageContainerLocationClient .allocateContainer(clientManager.getType(), clientManager.getFactor(), containerOwner); - XceiverClientSpi client2 = clientManager.acquireClient(container2.getPipeline(), - container2.getContainerID()); + XceiverClientSpi client2 = clientManager + .acquireClient(container2.getPipeline(), + container2.getContainerInfo().getContainerID()); Assert.assertEquals(1, client2.getRefcount()); - XceiverClientSpi client3 = clientManager.acquireClient(container1.getPipeline(), - container1.getContainerID()); + XceiverClientSpi client3 = clientManager + .acquireClient(container1.getPipeline(), + container1.getContainerInfo().getContainerID()); Assert.assertEquals(2, client3.getRefcount()); Assert.assertEquals(2, client1.getRefcount()); Assert.assertEquals(client1, client3); @@ -132,32 +135,35 @@ public class TestXceiverClientManager { Cache cache = clientManager.getClientCache(); - ContainerInfo container1 = + ContainerWithPipeline container1 = storageContainerLocationClient.allocateContainer( clientManager.getType(), HddsProtos.ReplicationFactor.ONE, containerOwner); - XceiverClientSpi client1 = clientManager.acquireClient(container1.getPipeline(), - container1.getContainerID()); + XceiverClientSpi client1 = clientManager + .acquireClient(container1.getPipeline(), + container1.getContainerInfo().getContainerID()); Assert.assertEquals(1, client1.getRefcount()); Assert.assertEquals(container1.getPipeline(), client1.getPipeline()); - ContainerInfo container2 = + ContainerWithPipeline container2 = storageContainerLocationClient.allocateContainer( clientManager.getType(), HddsProtos.ReplicationFactor.ONE, containerOwner); - XceiverClientSpi client2 = clientManager.acquireClient(container2.getPipeline(), - container2.getContainerID()); + XceiverClientSpi client2 = clientManager + .acquireClient(container2.getPipeline(), + container2.getContainerInfo().getContainerID()); Assert.assertEquals(1, client2.getRefcount()); Assert.assertNotEquals(client1, client2); // least recent container (i.e containerName1) is evicted - XceiverClientSpi nonExistent1 = cache.getIfPresent(container1.getContainerID()); + XceiverClientSpi nonExistent1 = cache + .getIfPresent(container1.getContainerInfo().getContainerID()); Assert.assertEquals(null, nonExistent1); // However container call should succeed because of refcount on the client. String traceID1 = "trace" + RandomStringUtils.randomNumeric(4); ContainerProtocolCalls.createContainer(client1, - container1.getContainerID(), traceID1); + container1.getContainerInfo().getContainerID(), traceID1); // After releasing the client, this connection should be closed // and any container operations should fail @@ -166,7 +172,7 @@ public class TestXceiverClientManager { String expectedMessage = "This channel is not connected."; try { ContainerProtocolCalls.createContainer(client1, - container1.getContainerID(), traceID1); + container1.getContainerInfo().getContainerID(), traceID1); Assert.fail("Create container should throw exception on closed" + "client"); } catch (Exception e) { @@ -186,28 +192,30 @@ public class TestXceiverClientManager { Cache cache = clientManager.getClientCache(); - ContainerInfo container1 = + ContainerWithPipeline container1 = storageContainerLocationClient.allocateContainer( clientManager.getType(), clientManager.getFactor(), containerOwner); - XceiverClientSpi client1 = clientManager.acquireClient(container1.getPipeline(), - container1.getContainerID()); + XceiverClientSpi client1 = clientManager + .acquireClient(container1.getPipeline(), + container1.getContainerInfo().getContainerID()); Assert.assertEquals(1, client1.getRefcount()); clientManager.releaseClient(client1); Assert.assertEquals(0, client1.getRefcount()); - ContainerInfo container2 = storageContainerLocationClient + ContainerWithPipeline container2 = storageContainerLocationClient .allocateContainer(clientManager.getType(), clientManager.getFactor(), containerOwner); - XceiverClientSpi client2 = clientManager.acquireClient(container2.getPipeline(), - container2.getContainerID()); + XceiverClientSpi client2 = clientManager + .acquireClient(container2.getPipeline(), + container2.getContainerInfo().getContainerID()); Assert.assertEquals(1, client2.getRefcount()); Assert.assertNotEquals(client1, client2); - // now client 1 should be evicted - XceiverClientSpi nonExistent = cache.getIfPresent(container1.getContainerID()); + XceiverClientSpi nonExistent = cache + .getIfPresent(container1.getContainerInfo().getContainerID()); Assert.assertEquals(null, nonExistent); // Any container operation should now fail @@ -215,7 +223,7 @@ public class TestXceiverClientManager { String expectedMessage = "This channel is not connected."; try { ContainerProtocolCalls.createContainer(client1, - container1.getContainerID(), traceID2); + container1.getContainerInfo().getContainerID(), traceID2); Assert.fail("Create container should throw exception on closed" + "client"); } catch (Exception e) { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientMetrics.java index d6f5d32f3a6..a9781b1d442 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientMetrics.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestXceiverClientMetrics.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos .ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos .ContainerCommandResponseProto; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -79,14 +79,16 @@ public class TestXceiverClientMetrics { OzoneConfiguration conf = new OzoneConfiguration(); XceiverClientManager clientManager = new XceiverClientManager(conf); - ContainerInfo container = storageContainerLocationClient + ContainerWithPipeline container = storageContainerLocationClient .allocateContainer(clientManager.getType(), clientManager.getFactor(), containerOwner); - XceiverClientSpi client = clientManager.acquireClient( - container.getPipeline(), container.getContainerID()); + XceiverClientSpi client = clientManager + .acquireClient(container.getPipeline(), + container.getContainerInfo().getContainerID()); ContainerCommandRequestProto request = ContainerTestHelper - .getCreateContainerRequest(container.getContainerID(), + .getCreateContainerRequest( + container.getContainerInfo().getContainerID(), container.getPipeline()); client.sendCommand(request); @@ -112,7 +114,7 @@ public class TestXceiverClientMetrics { // use async interface for testing pending metrics for (int i = 0; i < numRequest; i++) { BlockID blockID = ContainerTestHelper. - getTestBlockID(container.getContainerID()); + getTestBlockID(container.getContainerInfo().getContainerID()); ContainerProtos.ContainerCommandRequestProto smallFileRequest; smallFileRequest = ContainerTestHelper.getWriteSmallFileRequest( diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkContainerStateMap.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkContainerStateMap.java index 375450ca09e..c344bbe1f6e 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkContainerStateMap.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkContainerStateMap.java @@ -60,7 +60,9 @@ public class BenchMarkContainerStateMap { try { ContainerInfo containerInfo = new ContainerInfo.Builder() .setState(CLOSED) - .setPipeline(pipeline) + .setPipelineName(pipeline.getPipelineName()) + .setReplicationType(pipeline.getType()) + .setReplicationFactor(pipeline.getFactor()) // This is bytes allocated for blocks inside container, not the // container size .setAllocatedBytes(0) @@ -81,7 +83,9 @@ public class BenchMarkContainerStateMap { try { ContainerInfo containerInfo = new ContainerInfo.Builder() .setState(OPEN) - .setPipeline(pipeline) + .setPipelineName(pipeline.getPipelineName()) + .setReplicationType(pipeline.getType()) + .setReplicationFactor(pipeline.getFactor()) // This is bytes allocated for blocks inside container, not the // container size .setAllocatedBytes(0) @@ -101,7 +105,9 @@ public class BenchMarkContainerStateMap { try { ContainerInfo containerInfo = new ContainerInfo.Builder() .setState(OPEN) - .setPipeline(pipeline) + .setPipelineName(pipeline.getPipelineName()) + .setReplicationType(pipeline.getType()) + .setReplicationFactor(pipeline.getFactor()) // This is bytes allocated for blocks inside container, not the // container size .setAllocatedBytes(0) @@ -166,7 +172,9 @@ public class BenchMarkContainerStateMap { int cid = state.containerID.incrementAndGet(); ContainerInfo containerInfo = new ContainerInfo.Builder() .setState(CLOSED) - .setPipeline(pipeline) + .setPipelineName(pipeline.getPipelineName()) + .setReplicationType(pipeline.getType()) + .setReplicationFactor(pipeline.getFactor()) // This is bytes allocated for blocks inside container, not the // container size .setAllocatedBytes(0) diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java index edc0d7b597b..26776c5c35b 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java @@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.scm.cli; import com.google.common.base.Preconditions; import com.google.common.primitives.Longs; +import com.google.protobuf.ByteString; import org.apache.commons.cli.BasicParser; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; @@ -37,7 +38,6 @@ import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyI import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.VolumeInfo; import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.VolumeList; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.Pipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; @@ -86,12 +86,12 @@ public class SQLCLI extends Configured implements Tool { private static final String CREATE_CONTAINER_INFO = "CREATE TABLE containerInfo (" + "containerID LONG PRIMARY KEY NOT NULL, " + - "leaderUUID TEXT NOT NULL)"; - private static final String CREATE_CONTAINER_MEMBERS = - "CREATE TABLE containerMembers (" + - "containerName TEXT NOT NULL, " + - "datanodeUUID TEXT NOT NULL," + - "PRIMARY KEY(containerName, datanodeUUID));"; + "replicationType TEXT NOT NULL," + + "replicationFactor TEXT NOT NULL," + + "usedBytes LONG NOT NULL," + + "allocatedBytes LONG NOT NULL," + + "owner TEXT," + + "numberOfKeys LONG)"; private static final String CREATE_DATANODE_INFO = "CREATE TABLE datanodeInfo (" + "hostName TEXT NOT NULL, " + @@ -99,8 +99,10 @@ public class SQLCLI extends Configured implements Tool { "ipAddress TEXT, " + "containerPort INTEGER NOT NULL);"; private static final String INSERT_CONTAINER_INFO = - "INSERT INTO containerInfo (containerID, leaderUUID) " + - "VALUES (\"%d\", \"%s\")"; + "INSERT INTO containerInfo (containerID, replicationType, " + + "replicationFactor, usedBytes, allocatedBytes, owner, " + + "numberOfKeys) VALUES (\"%d\", \"%s\", \"%s\", \"%d\", \"%d\", " + + "\"%s\", \"%d\")"; private static final String INSERT_DATANODE_INFO = "INSERT INTO datanodeInfo (hostname, datanodeUUid, ipAddress, " + "containerPort) " + @@ -469,10 +471,7 @@ public class SQLCLI extends Configured implements Tool { .setConf(conf).setDbFile(dbFile).build(); Connection conn = connectDB(outPath.toString())) { executeSQL(conn, CREATE_CONTAINER_INFO); - executeSQL(conn, CREATE_CONTAINER_MEMBERS); - executeSQL(conn, CREATE_DATANODE_INFO); - HashSet uuidChecked = new HashSet<>(); dbStore.iterate(null, (key, value) -> { long containerID = Longs.fromByteArray(key); ContainerInfo containerInfo = null; @@ -481,8 +480,7 @@ public class SQLCLI extends Configured implements Tool { Preconditions.checkNotNull(containerInfo); try { //TODO: include container state to sqllite schema - insertContainerDB(conn, containerID, - containerInfo.getPipeline().getProtobufMessage(), uuidChecked); + insertContainerDB(conn, containerInfo, containerID); return true; } catch (SQLException e) { throw new IOException(e); @@ -494,38 +492,23 @@ public class SQLCLI extends Configured implements Tool { /** * Insert into the sqlite DB of container.db. * @param conn the connection to the sqlite DB. - * @param containerID the id of the container. - * @param pipeline the actual container pipeline object. - * @param uuidChecked the uuid that has been already inserted. + * @param containerInfo + * @param containerID * @throws SQLException throws exception. */ - private void insertContainerDB(Connection conn, long containerID, - Pipeline pipeline, Set uuidChecked) throws SQLException { + private void insertContainerDB(Connection conn, ContainerInfo containerInfo, + long containerID) throws SQLException { LOG.info("Insert to sql container db, for container {}", containerID); String insertContainerInfo = String.format( INSERT_CONTAINER_INFO, containerID, - pipeline.getLeaderID()); - executeSQL(conn, insertContainerInfo); + containerInfo.getReplicationType(), + containerInfo.getReplicationFactor(), + containerInfo.getUsedBytes(), + containerInfo.getAllocatedBytes(), + containerInfo.getOwner(), + containerInfo.getNumberOfKeys()); - for (HddsProtos.DatanodeDetailsProto dd : - pipeline.getMembersList()) { - String uuid = dd.getUuid(); - if (!uuidChecked.contains(uuid)) { - // we may also not use this checked set, but catch exception instead - // but this seems a bit cleaner. - String ipAddr = dd.getIpAddress(); - String hostName = dd.getHostName(); - int containerPort = DatanodeDetails.getFromProtoBuf(dd) - .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue(); - String insertMachineInfo = String.format( - INSERT_DATANODE_INFO, hostName, uuid, ipAddr, containerPort); - executeSQL(conn, insertMachineInfo); - uuidChecked.add(uuid); - } - String insertContainerMembers = String.format( - INSERT_CONTAINER_MEMBERS, containerID, uuid); - executeSQL(conn, insertContainerMembers); - } + executeSQL(conn, insertContainerInfo); LOG.info("Insertion completed."); } From 3b637155a47d2aa93284969a96208347a647083d Mon Sep 17 00:00:00 2001 From: Yiqun Lin Date: Wed, 4 Jul 2018 15:03:24 +0800 Subject: [PATCH 27/47] HDFS-13528. RBF: If a directory exceeds quota limit then quota usage is not refreshed for other mount entries. Contributed by Dibyendu Karmakar. --- .../hdfs/server/federation/router/Quota.java | 2 +- .../router/RouterQuotaUpdateService.java | 43 +++- .../federation/router/TestRouterQuota.java | 212 +++++++++++++++++- 3 files changed, 243 insertions(+), 14 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java index dbb6ffa0fdd..413a4e1d099 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java @@ -199,7 +199,7 @@ public class Quota { if (manager != null) { Set childrenPaths = manager.getPaths(path); for (String childPath : childrenPaths) { - locations.addAll(rpcServer.getLocationsForPath(childPath, true)); + locations.addAll(rpcServer.getLocationsForPath(childPath, true, false)); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java index 9fc93c15500..506e2ee2c8c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java @@ -27,6 +27,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.QuotaUsage; import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.server.federation.store.MountTableStore; import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest; import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse; @@ -83,13 +84,40 @@ public class RouterQuotaUpdateService extends PeriodicService { RouterQuotaUsage oldQuota = entry.getQuota(); long nsQuota = oldQuota.getQuota(); long ssQuota = oldQuota.getSpaceQuota(); - // Call RouterRpcServer#getQuotaUsage for getting current quota usage. - QuotaUsage currentQuotaUsage = this.rpcServer.getQuotaModule() - .getQuotaUsage(src); + + QuotaUsage currentQuotaUsage = null; + + // Check whether destination path exists in filesystem. If destination + // is not present, reset the usage. For other mount entry get current + // quota usage + HdfsFileStatus ret = this.rpcServer.getFileInfo(src); + if (ret == null) { + currentQuotaUsage = new RouterQuotaUsage.Builder() + .fileAndDirectoryCount(0) + .quota(nsQuota) + .spaceConsumed(0) + .spaceQuota(ssQuota).build(); + } else { + // Call RouterRpcServer#getQuotaUsage for getting current quota usage. + // If any exception occurs catch it and proceed with other entries. + try { + currentQuotaUsage = this.rpcServer.getQuotaModule() + .getQuotaUsage(src); + } catch (IOException ioe) { + LOG.error("Unable to get quota usage for " + src, ioe); + continue; + } + } + // If quota is not set in some subclusters under federation path, // set quota for this path. if (currentQuotaUsage.getQuota() == HdfsConstants.QUOTA_DONT_SET) { - this.rpcServer.setQuota(src, nsQuota, ssQuota, null); + try { + this.rpcServer.setQuota(src, nsQuota, ssQuota, null); + } catch (IOException ioe) { + LOG.error("Unable to set quota at remote location for " + + src, ioe); + } } RouterQuotaUsage newQuota = generateNewQuota(oldQuota, @@ -221,7 +249,12 @@ public class RouterQuotaUpdateService extends PeriodicService { for (MountTable entry : updateMountTables) { UpdateMountTableEntryRequest updateRequest = UpdateMountTableEntryRequest .newInstance(entry); - getMountTableStore().updateMountTableEntry(updateRequest); + try { + getMountTableStore().updateMountTableEntry(updateRequest); + } catch (IOException e) { + LOG.error("Quota update error for mount entry " + + entry.getSourcePath(), e); + } } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuota.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuota.java index c331c6bdb2c..431b394796b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuota.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuota.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.federation.router; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; import java.io.IOException; import java.util.Collections; @@ -410,8 +411,7 @@ public class TestRouterQuota { updateService.periodicInvoke(); // verify initial quota value - List results = getMountTable(path); - MountTable updatedMountTable = !results.isEmpty() ? results.get(0) : null; + MountTable updatedMountTable = getMountTable(path); RouterQuotaUsage quota = updatedMountTable.getQuota(); assertEquals(nsQuota, quota.getQuota()); assertEquals(ssQuota, quota.getSpaceQuota()); @@ -426,8 +426,7 @@ public class TestRouterQuota { appendData(path + "/file", routerClient, BLOCK_SIZE); updateService.periodicInvoke(); - results = getMountTable(path); - updatedMountTable = !results.isEmpty() ? results.get(0) : null; + updatedMountTable = getMountTable(path); quota = updatedMountTable.getQuota(); // verify if quota has been updated in state store @@ -443,17 +442,18 @@ public class TestRouterQuota { * @return If it was successfully got. * @throws IOException Problems getting entries. */ - private List getMountTable(String path) throws IOException { + private MountTable getMountTable(String path) throws IOException { // Reload the Router cache resolver.loadCache(true); RouterClient client = routerContext.getAdminClient(); MountTableManager mountTableManager = client.getMountTableManager(); GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest .newInstance(path); - GetMountTableEntriesResponse removeResponse = mountTableManager + GetMountTableEntriesResponse response = mountTableManager .getMountTableEntries(getRequest); + List results = response.getEntries(); - return removeResponse.getEntries(); + return !results.isEmpty() ? results.get(0) : null; } @Test @@ -493,4 +493,200 @@ public class TestRouterQuota { assertEquals(updateNsQuota, realQuota.getQuota()); assertEquals(updateSsQuota, realQuota.getSpaceQuota()); } -} + + @Test + public void testQuotaRefreshAfterQuotaExceed() throws Exception { + long nsQuota = 3; + long ssQuota = 100; + final FileSystem nnFs1 = nnContext1.getFileSystem(); + final FileSystem nnFs2 = nnContext2.getFileSystem(); + + // Add two mount tables: + // /setquota1 --> ns0---testdir11 + // /setquota2 --> ns1---testdir12 + nnFs1.mkdirs(new Path("/testdir11")); + nnFs2.mkdirs(new Path("/testdir12")); + MountTable mountTable1 = MountTable.newInstance("/setquota1", + Collections.singletonMap("ns0", "/testdir11")); + mountTable1 + .setQuota(new RouterQuotaUsage.Builder().quota(nsQuota) + .spaceQuota(ssQuota).build()); + addMountTable(mountTable1); + + MountTable mountTable2 = MountTable.newInstance("/setquota2", + Collections.singletonMap("ns1", "/testdir12")); + mountTable2 + .setQuota(new RouterQuotaUsage.Builder().quota(nsQuota) + .spaceQuota(ssQuota).build()); + addMountTable(mountTable2); + + final FileSystem routerFs = routerContext.getFileSystem(); + // Create directory to make directory count equals to nsQuota + routerFs.mkdirs(new Path("/setquota1/" + UUID.randomUUID())); + routerFs.mkdirs(new Path("/setquota1/" + UUID.randomUUID())); + + // create one more directory to exceed the nsQuota + routerFs.mkdirs(new Path("/setquota1/" + UUID.randomUUID())); + + RouterQuotaUpdateService updateService = routerContext.getRouter() + .getQuotaCacheUpdateService(); + // Call RouterQuotaUpdateService#periodicInvoke to update quota cache + updateService.periodicInvoke(); + // Reload the Router cache + resolver.loadCache(true); + + RouterQuotaManager quotaManager = + routerContext.getRouter().getQuotaManager(); + ClientProtocol client1 = nnContext1.getClient().getNamenode(); + ClientProtocol client2 = nnContext2.getClient().getNamenode(); + QuotaUsage quota1 = client1.getQuotaUsage("/testdir11"); + QuotaUsage quota2 = client2.getQuotaUsage("/testdir12"); + QuotaUsage cacheQuota1 = quotaManager.getQuotaUsage("/setquota1"); + QuotaUsage cacheQuota2 = quotaManager.getQuotaUsage("/setquota2"); + + // Verify quota usage + assertEquals(4, quota1.getFileAndDirectoryCount()); + assertEquals(4, cacheQuota1.getFileAndDirectoryCount()); + assertEquals(1, quota2.getFileAndDirectoryCount()); + assertEquals(1, cacheQuota2.getFileAndDirectoryCount()); + + try { + // create new directory to trigger NSQuotaExceededException + routerFs.mkdirs(new Path("/testdir11/" + UUID.randomUUID())); + fail("Mkdir should be failed under dir /testdir11."); + } catch (NSQuotaExceededException ignored) { + } + + // Create directory under the other mount point + routerFs.mkdirs(new Path("/setquota2/" + UUID.randomUUID())); + routerFs.mkdirs(new Path("/setquota2/" + UUID.randomUUID())); + + // Call RouterQuotaUpdateService#periodicInvoke to update quota cache + updateService.periodicInvoke(); + + quota1 = client1.getQuotaUsage("/testdir11"); + cacheQuota1 = quotaManager.getQuotaUsage("/setquota1"); + quota2 = client2.getQuotaUsage("/testdir12"); + cacheQuota2 = quotaManager.getQuotaUsage("/setquota2"); + + // Verify whether quota usage cache is update by periodicInvoke(). + assertEquals(4, quota1.getFileAndDirectoryCount()); + assertEquals(4, cacheQuota1.getFileAndDirectoryCount()); + assertEquals(3, quota2.getFileAndDirectoryCount()); + assertEquals(3, cacheQuota2.getFileAndDirectoryCount()); + } + + /** + * Verify whether mount table and quota usage cache is updated properly. + * {@link RouterQuotaUpdateService#periodicInvoke()} should be able to update + * the cache and the mount table even if the destination directory for some + * mount entry is not present in the filesystem. + */ + @Test + public void testQuotaRefreshWhenDestinationNotPresent() throws Exception { + long nsQuota = 5; + long ssQuota = 3*BLOCK_SIZE; + final FileSystem nnFs = nnContext1.getFileSystem(); + + // Add three mount tables: + // /setdir1 --> ns0---testdir13 + // /setdir2 --> ns0---testdir14 + // Create destination directory + nnFs.mkdirs(new Path("/testdir13")); + nnFs.mkdirs(new Path("/testdir14")); + + MountTable mountTable = MountTable.newInstance("/setdir1", + Collections.singletonMap("ns0", "/testdir13")); + mountTable + .setQuota(new RouterQuotaUsage.Builder().quota(nsQuota) + .spaceQuota(ssQuota).build()); + addMountTable(mountTable); + + mountTable = MountTable.newInstance("/setdir2", + Collections.singletonMap("ns0", "/testdir14")); + mountTable + .setQuota(new RouterQuotaUsage.Builder().quota(nsQuota) + .spaceQuota(ssQuota).build()); + addMountTable(mountTable); + + final DFSClient routerClient = routerContext.getClient(); + // Create file + routerClient.create("/setdir1/file1", true).close(); + routerClient.create("/setdir2/file2", true).close(); + // append data to the file + appendData("/setdir1/file1", routerClient, BLOCK_SIZE); + appendData("/setdir2/file2", routerClient, BLOCK_SIZE); + + RouterQuotaUpdateService updateService = + routerContext.getRouter().getQuotaCacheUpdateService(); + // Update quota cache + updateService.periodicInvoke(); + // Reload the Router cache + resolver.loadCache(true); + + ClientProtocol client1 = nnContext1.getClient().getNamenode(); + RouterQuotaManager quotaManager = + routerContext.getRouter().getQuotaManager(); + QuotaUsage quota1 = client1.getQuotaUsage("/testdir13"); + QuotaUsage quota2 = client1.getQuotaUsage("/testdir14"); + QuotaUsage cacheQuota1 = quotaManager.getQuotaUsage("/setdir1"); + QuotaUsage cacheQuota2 = quotaManager.getQuotaUsage("/setdir2"); + + // Get quota details in mount table + MountTable updatedMountTable = getMountTable("/setdir1"); + RouterQuotaUsage mountQuota1 = updatedMountTable.getQuota(); + updatedMountTable = getMountTable("/setdir2"); + RouterQuotaUsage mountQuota2 = updatedMountTable.getQuota(); + + // Verify quota usage + assertEquals(2, quota1.getFileAndDirectoryCount()); + assertEquals(2, cacheQuota1.getFileAndDirectoryCount()); + assertEquals(2, mountQuota1.getFileAndDirectoryCount()); + assertEquals(2, quota2.getFileAndDirectoryCount()); + assertEquals(2, cacheQuota2.getFileAndDirectoryCount()); + assertEquals(2, mountQuota2.getFileAndDirectoryCount()); + assertEquals(BLOCK_SIZE, quota1.getSpaceConsumed()); + assertEquals(BLOCK_SIZE, cacheQuota1.getSpaceConsumed()); + assertEquals(BLOCK_SIZE, mountQuota1.getSpaceConsumed()); + assertEquals(BLOCK_SIZE, quota2.getSpaceConsumed()); + assertEquals(BLOCK_SIZE, cacheQuota2.getSpaceConsumed()); + assertEquals(BLOCK_SIZE, mountQuota2.getSpaceConsumed()); + + FileSystem routerFs = routerContext.getFileSystem(); + // Remove destination directory for the mount entry + routerFs.delete(new Path("/setdir1"), true); + + // Create file + routerClient.create("/setdir2/file3", true).close(); + // append data to the file + appendData("/setdir2/file3", routerClient, BLOCK_SIZE); + int updatedSpace = BLOCK_SIZE + BLOCK_SIZE; + + // Update quota cache + updateService.periodicInvoke(); + + quota2 = client1.getQuotaUsage("/testdir14"); + cacheQuota1 = quotaManager.getQuotaUsage("/setdir1"); + cacheQuota2 = quotaManager.getQuotaUsage("/setdir2"); + + // Get quota details in mount table + updatedMountTable = getMountTable("/setdir1"); + mountQuota1 = updatedMountTable.getQuota(); + updatedMountTable = getMountTable("/setdir2"); + mountQuota2 = updatedMountTable.getQuota(); + + // If destination is not present the quota usage should be reset to 0 + assertEquals(0, cacheQuota1.getFileAndDirectoryCount()); + assertEquals(0, mountQuota1.getFileAndDirectoryCount()); + assertEquals(0, cacheQuota1.getSpaceConsumed()); + assertEquals(0, mountQuota1.getSpaceConsumed()); + + // Verify current quota usage for other mount entries + assertEquals(3, quota2.getFileAndDirectoryCount()); + assertEquals(3, cacheQuota2.getFileAndDirectoryCount()); + assertEquals(3, mountQuota2.getFileAndDirectoryCount()); + assertEquals(updatedSpace, quota2.getSpaceConsumed()); + assertEquals(updatedSpace, cacheQuota2.getSpaceConsumed()); + assertEquals(updatedSpace, mountQuota2.getSpaceConsumed()); + } +} \ No newline at end of file From 71df8c27c9a0e326232d3baf16414a63b5ea5a4b Mon Sep 17 00:00:00 2001 From: Nanda kumar Date: Thu, 5 Jul 2018 02:11:10 +0530 Subject: [PATCH 28/47] HDDS-212. Introduce NodeStateManager to manage the state of Datanodes in SCM. Contributed by Nanda kumar. --- .../scm/client/ContainerOperationClient.java | 8 +- .../hadoop/hdds/protocol/DatanodeDetails.java | 13 +- .../apache/hadoop/hdds/scm/ScmConfigKeys.java | 4 - .../hadoop/hdds/scm/client/ScmClient.java | 5 +- .../StorageContainerLocationProtocol.java | 5 +- ...ocationProtocolClientSideTranslatorPB.java | 8 +- ...ocationProtocolServerSideTranslatorPB.java | 8 +- .../StorageContainerLocationProtocol.proto | 19 +- hadoop-hdds/common/src/main/proto/hdds.proto | 13 +- .../src/main/resources/ozone-default.xml | 11 - .../hadoop/hdds/scm/HddsServerUtil.java | 11 - .../StorageContainerNodeProtocol.java | 4 +- .../hadoop/hdds/scm/node/DatanodeInfo.java | 109 ++++ .../hdds/scm/node/HeartbeatQueueItem.java | 98 --- .../hadoop/hdds/scm/node/NodeManager.java | 16 +- .../hdds/scm/node/NodeStateManager.java | 575 ++++++++++++++++++ .../hadoop/hdds/scm/node/SCMNodeManager.java | 506 ++------------- .../states/NodeAlreadyExistsException.java | 45 ++ .../hdds/scm/node/states/NodeException.java | 44 ++ .../node/states/NodeNotFoundException.java | 49 ++ .../hdds/scm/node/states/NodeStateMap.java | 281 +++++++++ .../scm/server/SCMClientProtocolServer.java | 60 +- .../SCMDatanodeHeartbeatDispatcher.java | 2 +- .../scm/server/SCMDatanodeProtocolServer.java | 2 +- .../hdds/scm/container/MockNodeManager.java | 58 +- .../hdds/scm/node/TestContainerPlacement.java | 10 +- .../hadoop/hdds/scm/node/TestNodeManager.java | 176 ++---- .../testutils/ReplicationNodeManagerMock.java | 37 +- .../ozone/TestStorageContainerManager.java | 4 +- .../hadoop/ozone/scm/node/TestQueryNode.java | 19 +- .../hadoop/ozone/ksm/KeySpaceManager.java | 6 +- 31 files changed, 1288 insertions(+), 918 deletions(-) create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeInfo.java delete mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HeartbeatQueueItem.java create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeAlreadyExistsException.java create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeException.java create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeNotFoundException.java create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java index b04f8c4d2ce..e7bdaf0f682 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java @@ -37,7 +37,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.EnumSet; import java.util.List; import java.util.UUID; @@ -234,14 +233,14 @@ public class ContainerOperationClient implements ScmClient { /** * Returns a set of Nodes that meet a query criteria. * - * @param nodeStatuses - A set of criteria that we want the node to have. + * @param nodeStatuses - Criteria that we want the node to have. * @param queryScope - Query scope - Cluster or pool. * @param poolName - if it is pool, a pool name is required. * @return A set of nodes that meet the requested criteria. * @throws IOException */ @Override - public HddsProtos.NodePool queryNode(EnumSet + public List queryNode(HddsProtos.NodeState nodeStatuses, HddsProtos.QueryScope queryScope, String poolName) throws IOException { return storageContainerLocationClient.queryNode(nodeStatuses, queryScope, @@ -458,7 +457,8 @@ public class ContainerOperationClient implements ScmClient { */ @Override public long getContainerSize(long containerID) throws IOException { - // TODO : Fix this, it currently returns the capacity but not the current usage. + // TODO : Fix this, it currently returns the capacity + // but not the current usage. long size = getContainerSizeB(); if (size == -1) { throw new IOException("Container size unknown!"); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java index c373e22510a..bae22a22cae 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java @@ -35,7 +35,7 @@ import java.util.UUID; */ @InterfaceAudience.Private @InterfaceStability.Evolving -public final class DatanodeDetails implements Comparable { +public class DatanodeDetails implements Comparable { /** * DataNode's unique identifier in the cluster. @@ -63,6 +63,13 @@ public final class DatanodeDetails implements Comparable { this.ports = ports; } + protected DatanodeDetails(DatanodeDetails datanodeDetails) { + this.uuid = datanodeDetails.uuid; + this.ipAddress = datanodeDetails.ipAddress; + this.hostName = datanodeDetails.hostName; + this.ports = datanodeDetails.ports; + } + /** * Returns the DataNode UUID. * @@ -238,7 +245,7 @@ public final class DatanodeDetails implements Comparable { /** * Builder class for building DatanodeDetails. */ - public static class Builder { + public static final class Builder { private String id; private String ipAddress; private String hostName; @@ -324,7 +331,7 @@ public final class DatanodeDetails implements Comparable { /** * Container to hold DataNode Port details. */ - public static class Port { + public static final class Port { /** * Ports that are supported in DataNode. diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java index df6fbf0c751..ad326dcb7f4 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java @@ -165,10 +165,6 @@ public final class ScmConfigKeys { public static final String OZONE_SCM_DEADNODE_INTERVAL_DEFAULT = "10m"; - public static final String OZONE_SCM_MAX_HB_COUNT_TO_PROCESS = - "ozone.scm.max.hb.count.to.process"; - public static final int OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT = 5000; - public static final String OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL = "ozone.scm.heartbeat.thread.interval"; public static final String OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_DEFAULT = diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java index ecb21735e9c..7955179a17f 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import java.io.IOException; -import java.util.EnumSet; import java.util.List; /** @@ -150,13 +149,13 @@ public interface ScmClient { /** * Returns a set of Nodes that meet a query criteria. - * @param nodeStatuses - A set of criteria that we want the node to have. + * @param nodeStatuses - Criteria that we want the node to have. * @param queryScope - Query scope - Cluster or pool. * @param poolName - if it is pool, a pool name is required. * @return A set of nodes that meet the requested criteria. * @throws IOException */ - HddsProtos.NodePool queryNode(EnumSet nodeStatuses, + List queryNode(HddsProtos.NodeState nodeStatuses, HddsProtos.QueryScope queryScope, String poolName) throws IOException; /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java index b787409248c..581fbd0f06a 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hdds.protocol.proto .StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto; import java.io.IOException; -import java.util.EnumSet; import java.util.List; /** @@ -94,10 +93,10 @@ public interface StorageContainerLocationProtocol { /** * Queries a list of Node Statuses. - * @param nodeStatuses + * @param state * @return List of Datanodes. */ - HddsProtos.NodePool queryNode(EnumSet nodeStatuses, + List queryNode(HddsProtos.NodeState state, HddsProtos.QueryScope queryScope, String poolName) throws IOException; /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java index 4b03d122aae..ac12ea29685 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java @@ -59,7 +59,6 @@ import org.apache.hadoop.ipc.RPC; import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; -import java.util.EnumSet; import java.util.List; /** @@ -215,20 +214,19 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB * @return List of Datanodes. */ @Override - public HddsProtos.NodePool queryNode(EnumSet + public List queryNode(HddsProtos.NodeState nodeStatuses, HddsProtos.QueryScope queryScope, String poolName) throws IOException { // TODO : We support only cluster wide query right now. So ignoring checking // queryScope and poolName Preconditions.checkNotNull(nodeStatuses); - Preconditions.checkState(nodeStatuses.size() > 0); NodeQueryRequestProto request = NodeQueryRequestProto.newBuilder() - .addAllQuery(nodeStatuses) + .setState(nodeStatuses) .setScope(queryScope).setPoolName(poolName).build(); try { NodeQueryResponseProto response = rpcProxy.queryNode(NULL_RPC_CONTROLLER, request); - return response.getDatanodes(); + return response.getDatanodesList(); } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java index d66919f3568..9175ebf9b8d 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java @@ -57,7 +57,6 @@ import org.apache.hadoop.hdds.protocol.proto .StorageContainerLocationProtocolProtos.SCMListContainerResponseProto; import java.io.IOException; -import java.util.EnumSet; import java.util.List; /** @@ -171,13 +170,12 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB StorageContainerLocationProtocolProtos.NodeQueryRequestProto request) throws ServiceException { try { - EnumSet nodeStateEnumSet = EnumSet.copyOf(request - .getQueryList()); - HddsProtos.NodePool datanodes = impl.queryNode(nodeStateEnumSet, + HddsProtos.NodeState nodeState = request.getState(); + List datanodes = impl.queryNode(nodeState, request.getScope(), request.getPoolName()); return StorageContainerLocationProtocolProtos .NodeQueryResponseProto.newBuilder() - .setDatanodes(datanodes) + .addAllDatanodes(datanodes) .build(); } catch (Exception e) { throw new ServiceException(e); diff --git a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto index 143c2aefa0f..68cc35fbb02 100644 --- a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto +++ b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto @@ -118,26 +118,13 @@ message ObjectStageChangeResponseProto { match the NodeState that we are requesting. */ message NodeQueryRequestProto { - - - // Repeated, So we can specify more than one status type. - // These NodeState types are additive for now, in the sense that - // if you specify HEALTHY and FREE_NODE members -- - // Then you get all healthy node which are not raft members. - // - // if you specify all healthy and dead nodes, you will get nothing - // back. Server is not going to dictate what combinations make sense, - // it is entirely up to the caller. - // TODO: Support operators like OR and NOT. Currently it is always an - // implied AND. - - repeated NodeState query = 1; + required NodeState state = 1; required QueryScope scope = 2; optional string poolName = 3; // if scope is pool, then pool name is needed. } message NodeQueryResponseProto { - required NodePool datanodes = 1; + repeated Node datanodes = 1; } /** @@ -194,7 +181,7 @@ service StorageContainerLocationProtocolService { /** * Returns a set of Nodes that meet a criteria. */ - rpc queryNode(NodeQueryRequestProto) returns (NodeQueryResponseProto); + rpc queryNode(NodeQueryRequestProto) returns (NodeQueryResponseProto); /** * Notify from client when begin or finish container or pipeline operations on datanodes. diff --git a/hadoop-hdds/common/src/main/proto/hdds.proto b/hadoop-hdds/common/src/main/proto/hdds.proto index 1c9ee199262..b9def2a3ecb 100644 --- a/hadoop-hdds/common/src/main/proto/hdds.proto +++ b/hadoop-hdds/common/src/main/proto/hdds.proto @@ -69,14 +69,11 @@ enum NodeType { * and getNodeCount. */ enum NodeState { - HEALTHY = 1; - STALE = 2; - DEAD = 3; - DECOMMISSIONING = 4; - DECOMMISSIONED = 5; - RAFT_MEMBER = 6; - FREE_NODE = 7; // Not a member in raft. - INVALID = 8; + HEALTHY = 1; + STALE = 2; + DEAD = 3; + DECOMMISSIONING = 4; + DECOMMISSIONED = 5; } enum QueryScope { diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 25365c8d9d8..568d26712a7 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -773,17 +773,6 @@ The keytab file for Kerberos authentication in SCM. - - ozone.scm.max.hb.count.to.process - 5000 - OZONE, MANAGEMENT, PERFORMANCE - - The maximum number of heartbeat to process per loop of the - heartbeat process thread. Please see - ozone.scm.heartbeat.thread.interval - for more info. - - ozone.scm.names diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java index c734d9b03f6..cc7adbf8fd3 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java @@ -258,17 +258,6 @@ public final class HddsServerUtil { return deadNodeIntervalMs; } - /** - * Returns the maximum number of heartbeat to process per loop of the process - * thread. - * @param conf Configuration - * @return - int -- Number of HBs to process - */ - public static int getMaxHBToProcessPerLoop(Configuration conf) { - return conf.getInt(ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS, - ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT); - } - /** * Timeout value for the RPC from Datanode to SCM, primarily used for * Heartbeats and container reports. diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerNodeProtocol.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerNodeProtocol.java index 790f58acf25..c9ef43f9c12 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerNodeProtocol.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerNodeProtocol.java @@ -59,10 +59,8 @@ public interface StorageContainerNodeProtocol { /** * Send heartbeat to indicate the datanode is alive and doing well. * @param datanodeDetails - Datanode ID. - * @param nodeReport - node report. * @return SCMheartbeat response list */ - List sendHeartbeat(DatanodeDetails datanodeDetails, - NodeReportProto nodeReport); + List processHeartbeat(DatanodeDetails datanodeDetails); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeInfo.java new file mode 100644 index 00000000000..51465ee95d8 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeInfo.java @@ -0,0 +1,109 @@ +/** + * 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.hdds.scm.node; + +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.proto + .StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.util.Time; + +import java.util.List; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * This class extends the primary identifier of a Datanode with ephemeral + * state, eg last reported time, usage information etc. + */ +public class DatanodeInfo extends DatanodeDetails { + + private final ReadWriteLock lock; + + private volatile long lastHeartbeatTime; + private long lastStatsUpdatedTime; + + // If required we can dissect StorageReportProto and store the raw data + private List storageReports; + + /** + * Constructs DatanodeInfo from DatanodeDetails. + * + * @param datanodeDetails Details about the datanode + */ + public DatanodeInfo(DatanodeDetails datanodeDetails) { + super(datanodeDetails); + lock = new ReentrantReadWriteLock(); + lastHeartbeatTime = Time.monotonicNow(); + } + + /** + * Updates the last heartbeat time with current time. + */ + public void updateLastHeartbeatTime() { + try { + lock.writeLock().lock(); + lastHeartbeatTime = Time.monotonicNow(); + } finally { + lock.writeLock().unlock(); + } + } + + /** + * Returns the last heartbeat time. + * + * @return last heartbeat time. + */ + public long getLastHeartbeatTime() { + try { + lock.readLock().lock(); + return lastHeartbeatTime; + } finally { + lock.readLock().unlock(); + } + } + + /** + * Updates the datanode storage reports. + * + * @param reports list of storage report + */ + public void updateStorageReports(List reports) { + try { + lock.writeLock().lock(); + lastStatsUpdatedTime = Time.monotonicNow(); + storageReports = reports; + } finally { + lock.writeLock().unlock(); + } + } + + /** + * Returns the storage reports associated with this datanode. + * + * @return list of storage report + */ + public List getStorageReports() { + try { + lock.readLock().lock(); + return storageReports; + } finally { + lock.readLock().unlock(); + } + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HeartbeatQueueItem.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HeartbeatQueueItem.java deleted file mode 100644 index 04658bdf537..00000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HeartbeatQueueItem.java +++ /dev/null @@ -1,98 +0,0 @@ -/** - * 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.hdds.scm.node; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.NodeReportProto; - -import static org.apache.hadoop.util.Time.monotonicNow; - -/** - * This class represents the item in SCM heartbeat queue. - */ -public class HeartbeatQueueItem { - private DatanodeDetails datanodeDetails; - private long recvTimestamp; - private NodeReportProto nodeReport; - - /** - * - * @param datanodeDetails - datanode ID of the heartbeat. - * @param recvTimestamp - heartbeat receive timestamp. - * @param nodeReport - node report associated with the heartbeat if any. - */ - HeartbeatQueueItem(DatanodeDetails datanodeDetails, long recvTimestamp, - NodeReportProto nodeReport) { - this.datanodeDetails = datanodeDetails; - this.recvTimestamp = recvTimestamp; - this.nodeReport = nodeReport; - } - - /** - * @return datanode ID. - */ - public DatanodeDetails getDatanodeDetails() { - return datanodeDetails; - } - - /** - * @return node report. - */ - public NodeReportProto getNodeReport() { - return nodeReport; - } - - /** - * @return heartbeat receive timestamp. - */ - public long getRecvTimestamp() { - return recvTimestamp; - } - - /** - * Builder for HeartbeatQueueItem. - */ - public static class Builder { - private DatanodeDetails datanodeDetails; - private NodeReportProto nodeReport; - private long recvTimestamp = monotonicNow(); - - public Builder setDatanodeDetails(DatanodeDetails dnDetails) { - this.datanodeDetails = dnDetails; - return this; - } - - public Builder setNodeReport(NodeReportProto report) { - this.nodeReport = report; - return this; - } - - @VisibleForTesting - public Builder setRecvTimestamp(long recvTime) { - this.recvTimestamp = recvTime; - return this; - } - - public HeartbeatQueueItem build() { - return new HeartbeatQueueItem(datanodeDetails, recvTimestamp, nodeReport); - } - } -} \ No newline at end of file diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java index 72d7e946cc0..c13c37c136c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java @@ -17,10 +17,9 @@ */ package org.apache.hadoop.hdds.scm.node; -import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric; import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat; -import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; +import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol; @@ -54,14 +53,14 @@ import java.util.UUID; * list, by calling removeNode. We will throw away this nodes info soon. */ public interface NodeManager extends StorageContainerNodeProtocol, - NodeManagerMXBean, Closeable, Runnable { + NodeManagerMXBean, Closeable { /** * Removes a data node from the management of this Node Manager. * * @param node - DataNode. - * @throws UnregisteredNodeException + * @throws NodeNotFoundException */ - void removeNode(DatanodeDetails node) throws UnregisteredNodeException; + void removeNode(DatanodeDetails node) throws NodeNotFoundException; /** * Gets all Live Datanodes that is currently communicating with SCM. @@ -123,13 +122,6 @@ public interface NodeManager extends StorageContainerNodeProtocol, */ SCMNodeMetric getNodeStat(DatanodeDetails datanodeDetails); - /** - * Wait for the heartbeat is processed by NodeManager. - * @return true if heartbeat has been processed. - */ - @VisibleForTesting - boolean waitForHeartbeatProcessed(); - /** * Returns the node state of a specific node. * @param datanodeDetails DatanodeDetails diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java new file mode 100644 index 00000000000..5543c04039c --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java @@ -0,0 +1,575 @@ +/** + * 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.hdds.scm.node; + +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; +import org.apache.hadoop.hdds.scm.HddsServerUtil; +import org.apache.hadoop.hdds.scm.node.states.NodeAlreadyExistsException; +import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; +import org.apache.hadoop.hdds.scm.node.states.NodeStateMap; +import org.apache.hadoop.ozone.common.statemachine + .InvalidStateTransitionException; +import org.apache.hadoop.ozone.common.statemachine.StateMachine; +import org.apache.hadoop.util.Time; +import org.apache.hadoop.util.concurrent.HadoopExecutors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; + +import static org.apache.hadoop.hdds.scm.ScmConfigKeys + .OZONE_SCM_DEADNODE_INTERVAL; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys + .OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys + .OZONE_SCM_STALENODE_INTERVAL; + +/** + * NodeStateManager maintains the state of all the datanodes in the cluster. All + * the node state change should happen only via NodeStateManager. It also + * runs a heartbeat thread which periodically updates the node state. + *

+ * The getNode(byState) functions make copy of node maps and then creates a list + * based on that. It should be assumed that these get functions always report + * *stale* information. For example, getting the deadNodeCount followed by + * getNodes(DEAD) could very well produce totally different count. Also + * getNodeCount(HEALTHY) + getNodeCount(DEAD) + getNodeCode(STALE), is not + * guaranteed to add up to the total nodes that we know off. Please treat all + * get functions in this file as a snap-shot of information that is inconsistent + * as soon as you read it. + */ +public class NodeStateManager implements Runnable, Closeable { + + /** + * Node's life cycle events. + */ + private enum NodeLifeCycleEvent { + TIMEOUT, RESTORE, RESURRECT, DECOMMISSION, DECOMMISSIONED + } + + private static final Logger LOG = LoggerFactory + .getLogger(NodeStateManager.class); + + /** + * StateMachine for node lifecycle. + */ + private final StateMachine stateMachine; + /** + * This is the map which maintains the current state of all datanodes. + */ + private final NodeStateMap nodeStateMap; + /** + * ExecutorService used for scheduling heartbeat processing thread. + */ + private final ScheduledExecutorService executorService; + /** + * The frequency in which we have run the heartbeat processing thread. + */ + private final long heartbeatCheckerIntervalMs; + /** + * The timeout value which will be used for marking a datanode as stale. + */ + private final long staleNodeIntervalMs; + /** + * The timeout value which will be used for marking a datanode as dead. + */ + private final long deadNodeIntervalMs; + + /** + * Constructs a NodeStateManager instance with the given configuration. + * + * @param conf Configuration + */ + public NodeStateManager(Configuration conf) { + nodeStateMap = new NodeStateMap(); + Set finalStates = new HashSet<>(); + finalStates.add(NodeState.DECOMMISSIONED); + this.stateMachine = new StateMachine<>(NodeState.HEALTHY, finalStates); + initializeStateMachine(); + heartbeatCheckerIntervalMs = HddsServerUtil + .getScmheartbeatCheckerInterval(conf); + staleNodeIntervalMs = HddsServerUtil.getStaleNodeInterval(conf); + deadNodeIntervalMs = HddsServerUtil.getDeadNodeInterval(conf); + Preconditions.checkState(heartbeatCheckerIntervalMs > 0, + OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL + " should be greater than 0."); + Preconditions.checkState(staleNodeIntervalMs < deadNodeIntervalMs, + OZONE_SCM_STALENODE_INTERVAL + " should be less than" + + OZONE_SCM_DEADNODE_INTERVAL); + executorService = HadoopExecutors.newScheduledThreadPool(1, + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("SCM Heartbeat Processing Thread - %d").build()); + executorService.schedule(this, heartbeatCheckerIntervalMs, + TimeUnit.MILLISECONDS); + } + + /* + * + * Node and State Transition Mapping: + * + * State: HEALTHY -------------------> STALE + * Event: TIMEOUT + * + * State: STALE -------------------> DEAD + * Event: TIMEOUT + * + * State: STALE -------------------> HEALTHY + * Event: RESTORE + * + * State: DEAD -------------------> HEALTHY + * Event: RESURRECT + * + * State: HEALTHY -------------------> DECOMMISSIONING + * Event: DECOMMISSION + * + * State: STALE -------------------> DECOMMISSIONING + * Event: DECOMMISSION + * + * State: DEAD -------------------> DECOMMISSIONING + * Event: DECOMMISSION + * + * State: DECOMMISSIONING -------------------> DECOMMISSIONED + * Event: DECOMMISSIONED + * + * Node State Flow + * + * +--------------------------------------------------------+ + * | (RESURRECT) | + * | +--------------------------+ | + * | | (RESTORE) | | + * | | | | + * V V | | + * [HEALTHY]------------------->[STALE]------------------->[DEAD] + * | (TIMEOUT) | (TIMEOUT) | + * | | | + * | | | + * | | | + * | | | + * | (DECOMMISSION) | (DECOMMISSION) | (DECOMMISSION) + * | V | + * +------------------->[DECOMMISSIONING]<----------------+ + * | + * | (DECOMMISSIONED) + * | + * V + * [DECOMMISSIONED] + * + */ + + /** + * Initializes the lifecycle of node state machine. + */ + private void initializeStateMachine() { + stateMachine.addTransition( + NodeState.HEALTHY, NodeState.STALE, NodeLifeCycleEvent.TIMEOUT); + stateMachine.addTransition( + NodeState.STALE, NodeState.DEAD, NodeLifeCycleEvent.TIMEOUT); + stateMachine.addTransition( + NodeState.STALE, NodeState.HEALTHY, NodeLifeCycleEvent.RESTORE); + stateMachine.addTransition( + NodeState.DEAD, NodeState.HEALTHY, NodeLifeCycleEvent.RESURRECT); + stateMachine.addTransition( + NodeState.HEALTHY, NodeState.DECOMMISSIONING, + NodeLifeCycleEvent.DECOMMISSION); + stateMachine.addTransition( + NodeState.STALE, NodeState.DECOMMISSIONING, + NodeLifeCycleEvent.DECOMMISSION); + stateMachine.addTransition( + NodeState.DEAD, NodeState.DECOMMISSIONING, + NodeLifeCycleEvent.DECOMMISSION); + stateMachine.addTransition( + NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONED, + NodeLifeCycleEvent.DECOMMISSIONED); + + } + + /** + * Adds a new node to the state manager. + * + * @param datanodeDetails DatanodeDetails + * + * @throws NodeAlreadyExistsException if the node is already present + */ + public void addNode(DatanodeDetails datanodeDetails) + throws NodeAlreadyExistsException { + nodeStateMap.addNode(datanodeDetails, stateMachine.getInitialState()); + } + + /** + * Get information about the node. + * + * @param datanodeDetails DatanodeDetails + * + * @return DatanodeInfo + * + * @throws NodeNotFoundException if the node is not present + */ + public DatanodeInfo getNode(DatanodeDetails datanodeDetails) + throws NodeNotFoundException { + return nodeStateMap.getNodeInfo(datanodeDetails.getUuid()); + } + + /** + * Updates the last heartbeat time of the node. + * + * @throws NodeNotFoundException if the node is not present + */ + public void updateLastHeartbeatTime(DatanodeDetails datanodeDetails) + throws NodeNotFoundException { + nodeStateMap.getNodeInfo(datanodeDetails.getUuid()) + .updateLastHeartbeatTime(); + } + + /** + * Returns the current state of the node. + * + * @param datanodeDetails DatanodeDetails + * + * @return NodeState + * + * @throws NodeNotFoundException if the node is not present + */ + public NodeState getNodeState(DatanodeDetails datanodeDetails) + throws NodeNotFoundException { + return nodeStateMap.getNodeState(datanodeDetails.getUuid()); + } + + /** + * Returns all the node which are in healthy state. + * + * @return list of healthy nodes + */ + public List getHealthyNodes() { + return getNodes(NodeState.HEALTHY); + } + + /** + * Returns all the node which are in stale state. + * + * @return list of stale nodes + */ + public List getStaleNodes() { + return getNodes(NodeState.STALE); + } + + /** + * Returns all the node which are in dead state. + * + * @return list of dead nodes + */ + public List getDeadNodes() { + return getNodes(NodeState.DEAD); + } + + /** + * Returns all the node which are in the specified state. + * + * @param state NodeState + * + * @return list of nodes + */ + public List getNodes(NodeState state) { + List nodes = new LinkedList<>(); + nodeStateMap.getNodes(state).forEach( + uuid -> { + try { + nodes.add(nodeStateMap.getNodeDetails(uuid)); + } catch (NodeNotFoundException e) { + // This should not happen unless someone else other than + // NodeStateManager is directly modifying NodeStateMap and removed + // the node entry after we got the list of UUIDs. + LOG.error("Inconsistent NodeStateMap! " + nodeStateMap); + } + }); + return nodes; + } + + /** + * Returns all the nodes which have registered to NodeStateManager. + * + * @return all the managed nodes + */ + public List getAllNodes() { + List nodes = new LinkedList<>(); + nodeStateMap.getAllNodes().forEach( + uuid -> { + try { + nodes.add(nodeStateMap.getNodeDetails(uuid)); + } catch (NodeNotFoundException e) { + // This should not happen unless someone else other than + // NodeStateManager is directly modifying NodeStateMap and removed + // the node entry after we got the list of UUIDs. + LOG.error("Inconsistent NodeStateMap! " + nodeStateMap); + } + }); + return nodes; + } + + /** + * Returns the count of healthy nodes. + * + * @return healthy node count + */ + public int getHealthyNodeCount() { + return getNodeCount(NodeState.HEALTHY); + } + + /** + * Returns the count of stale nodes. + * + * @return stale node count + */ + public int getStaleNodeCount() { + return getNodeCount(NodeState.STALE); + } + + /** + * Returns the count of dead nodes. + * + * @return dead node count + */ + public int getDeadNodeCount() { + return getNodeCount(NodeState.DEAD); + } + + /** + * Returns the count of nodes in specified state. + * + * @param state NodeState + * + * @return node count + */ + public int getNodeCount(NodeState state) { + return nodeStateMap.getNodeCount(state); + } + + /** + * Returns the count of all nodes managed by NodeStateManager. + * + * @return node count + */ + public int getTotalNodeCount() { + return nodeStateMap.getTotalNodeCount(); + } + + /** + * Removes a node from NodeStateManager. + * + * @param datanodeDetails DatanodeDetails + * + * @throws NodeNotFoundException if the node is not present + */ + public void removeNode(DatanodeDetails datanodeDetails) + throws NodeNotFoundException { + nodeStateMap.removeNode(datanodeDetails.getUuid()); + } + + /** + * Move Stale or Dead node to healthy if we got a heartbeat from them. + * Move healthy nodes to stale nodes if it is needed. + * Move Stales node to dead if needed. + * + * @see Thread#run() + */ + @Override + public void run() { + + /* + * + * staleNodeDeadline healthyNodeDeadline + * | | + * Dead | Stale | Healthy + * Node | Node | Node + * Window | Window | Window + * ----------------+----------------------------------+-------------------> + * >>-->> time-line >>-->> + * + * Here is the logic of computing the health of a node. +     * +     * 1. We get the current time and look back that the time +     *  when we got a heartbeat from a node. +     *  +     * 2. If the last heartbeat was within the window of healthy node we mark +     *  it as healthy. +     *  +     * 3. If the last HB Time stamp is longer and falls within the window of +     *  Stale Node time, we will mark it as Stale. +     *  +     * 4. If the last HB time is older than the Stale Window, then the node is +     * marked as dead. + * + * The Processing starts from current time and looks backwards in time. + */ + long processingStartTime = Time.monotonicNow(); + // After this time node is considered to be stale. + long healthyNodeDeadline = processingStartTime - staleNodeIntervalMs; + // After this time node is considered to be dead. + long staleNodeDeadline = processingStartTime - deadNodeIntervalMs; + + Predicate healthyNodeCondition = + (lastHbTime) -> lastHbTime >= healthyNodeDeadline; + // staleNodeCondition is superset of stale and dead node + Predicate staleNodeCondition = + (lastHbTime) -> lastHbTime < healthyNodeDeadline; + Predicate deadNodeCondition = + (lastHbTime) -> lastHbTime < staleNodeDeadline; + try { + for (NodeState state : NodeState.values()) { + List nodes = nodeStateMap.getNodes(state); + for (UUID id : nodes) { + DatanodeInfo node = nodeStateMap.getNodeInfo(id); + switch (state) { + case HEALTHY: + // Move the node to STALE if the last heartbeat time is less than + // configured stale-node interval. + updateNodeState(node, staleNodeCondition, state, + NodeLifeCycleEvent.TIMEOUT); + break; + case STALE: + // Move the node to DEAD if the last heartbeat time is less than + // configured dead-node interval. + updateNodeState(node, deadNodeCondition, state, + NodeLifeCycleEvent.TIMEOUT); + // Restore the node if we have received heartbeat before configured + // stale-node interval. + updateNodeState(node, healthyNodeCondition, state, + NodeLifeCycleEvent.RESTORE); + break; + case DEAD: + // Resurrect the node if we have received heartbeat before + // configured stale-node interval. + updateNodeState(node, healthyNodeCondition, state, + NodeLifeCycleEvent.RESURRECT); + break; + // We don't do anything for DECOMMISSIONING and DECOMMISSIONED in + // heartbeat processing. + case DECOMMISSIONING: + case DECOMMISSIONED: + default: + } + } + } + } catch (NodeNotFoundException e) { + // This should not happen unless someone else other than + // NodeStateManager is directly modifying NodeStateMap and removed + // the node entry after we got the list of UUIDs. + LOG.error("Inconsistent NodeStateMap! " + nodeStateMap); + } + long processingEndTime = Time.monotonicNow(); + //If we have taken too much time for HB processing, log that information. + if ((processingEndTime - processingStartTime) > + heartbeatCheckerIntervalMs) { + LOG.error("Total time spend processing datanode HB's is greater than " + + "configured values for datanode heartbeats. Please adjust the" + + " heartbeat configs. Time Spend on HB processing: {} seconds " + + "Datanode heartbeat Interval: {} seconds.", + TimeUnit.MILLISECONDS + .toSeconds(processingEndTime - processingStartTime), + heartbeatCheckerIntervalMs); + } + + // we purposefully make this non-deterministic. Instead of using a + // scheduleAtFixedFrequency we will just go to sleep + // and wake up at the next rendezvous point, which is currentTime + + // heartbeatCheckerIntervalMs. This leads to the issue that we are now + // heart beating not at a fixed cadence, but clock tick + time taken to + // work. + // + // This time taken to work can skew the heartbeat processor thread. + // The reason why we don't care is because of the following reasons. + // + // 1. checkerInterval is general many magnitudes faster than datanode HB + // frequency. + // + // 2. if we have too much nodes, the SCM would be doing only HB + // processing, this could lead to SCM's CPU starvation. With this + // approach we always guarantee that HB thread sleeps for a little while. + // + // 3. It is possible that we will never finish processing the HB's in the + // thread. But that means we have a mis-configured system. We will warn + // the users by logging that information. + // + // 4. And the most important reason, heartbeats are not blocked even if + // this thread does not run, they will go into the processing queue. + + if (!Thread.currentThread().isInterrupted() && + !executorService.isShutdown()) { + executorService.schedule(this, heartbeatCheckerIntervalMs, + TimeUnit.MILLISECONDS); + } else { + LOG.info("Current Thread is interrupted, shutting down HB processing " + + "thread for Node Manager."); + } + + } + + /** + * Updates the node state if the condition satisfies. + * + * @param node DatanodeInfo + * @param condition condition to check + * @param state current state of node + * @param lifeCycleEvent NodeLifeCycleEvent to be applied if condition + * matches + * + * @throws NodeNotFoundException if the node is not present + */ + private void updateNodeState(DatanodeInfo node, Predicate condition, + NodeState state, NodeLifeCycleEvent lifeCycleEvent) + throws NodeNotFoundException { + try { + if (condition.test(node.getLastHeartbeatTime())) { + NodeState newState = stateMachine.getNextState(state, lifeCycleEvent); + nodeStateMap.updateNodeState(node.getUuid(), state, newState); + } + } catch (InvalidStateTransitionException e) { + LOG.warn("Invalid state transition of node {}." + + " Current state: {}, life cycle event: {}", + node, state, lifeCycleEvent); + } + } + + @Override + public void close() { + executorService.shutdown(); + try { + if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) { + executorService.shutdownNow(); + } + + if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) { + LOG.error("Unable to shutdown NodeStateManager properly."); + } + } catch (InterruptedException e) { + executorService.shutdownNow(); + Thread.currentThread().interrupt(); + } + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index adca8eae0c6..15ac3f283e9 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -19,8 +19,8 @@ package org.apache.hadoop.hdds.scm.node; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.hadoop.hdds.scm.HddsServerUtil; +import org.apache.hadoop.hdds.scm.node.states.NodeAlreadyExistsException; +import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.scm.VersionInfo; import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric; @@ -29,7 +29,6 @@ import org.apache.hadoop.hdds.server.events.Event; import org.apache.hadoop.hdds.server.events.EventHandler; import org.apache.hadoop.hdds.server.events.EventPublisher; import org.apache.hadoop.hdds.server.events.TypedEvent; -import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; @@ -50,8 +49,6 @@ import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode; import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand; import org.apache.hadoop.ozone.protocol.commands.ReregisterCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand; -import org.apache.hadoop.util.Time; -import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,39 +60,15 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Queue; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; - -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD; -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState - .HEALTHY; -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState - .INVALID; -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE; -import static org.apache.hadoop.util.Time.monotonicNow; /** * Maintains information about the Datanodes on SCM side. *

* Heartbeats under SCM is very simple compared to HDFS heartbeatManager. *

- * Here we maintain 3 maps, and we propagate a node from healthyNodesMap to - * staleNodesMap to deadNodesMap. This moving of a node from one map to another - * is controlled by 4 configuration variables. These variables define how many - * heartbeats must go missing for the node to move from one map to another. - *

- * Each heartbeat that SCMNodeManager receives is put into heartbeatQueue. The - * worker thread wakes up and grabs that heartbeat from the queue. The worker - * thread will lookup the healthynodes map and set the timestamp if the entry - * is there. if not it will look up stale and deadnodes map. - *

* The getNode(byState) functions make copy of node maps and then creates a list * based on that. It should be assumed that these get functions always report * *stale* information. For example, getting the deadNodeCount followed by @@ -113,33 +86,18 @@ public class SCMNodeManager static final Logger LOG = LoggerFactory.getLogger(SCMNodeManager.class); - /** - * Key = NodeID, value = timestamp. - */ - private final ConcurrentHashMap healthyNodes; - private final ConcurrentHashMap staleNodes; - private final ConcurrentHashMap deadNodes; - private final Queue heartbeatQueue; - private final ConcurrentHashMap nodes; + + private final NodeStateManager nodeStateManager; // Individual live node stats + // TODO: NodeStat should be moved to NodeStatemanager (NodeStateMap) private final ConcurrentHashMap nodeStats; + // Should we maintain aggregated stats? If this is not frequently used, we + // can always calculate it from nodeStats whenever required. // Aggregated node stats private SCMNodeStat scmStat; - // TODO: expose nodeStats and scmStat as metrics - private final AtomicInteger healthyNodeCount; - private final AtomicInteger staleNodeCount; - private final AtomicInteger deadNodeCount; - private final AtomicInteger totalNodes; - private long staleNodeIntervalMs; - private final long deadNodeIntervalMs; - private final long heartbeatCheckerIntervalMs; - private final long datanodeHBIntervalSeconds; - private final ScheduledExecutorService executorService; - private long lastHBcheckStart; - private long lastHBcheckFinished = 0; - private long lastHBProcessedCount; + // Should we create ChillModeManager and extract all the chill mode logic + // to a new class? private int chillModeNodeCount; - private final int maxHBToProcessPerLoop; private final String clusterID; private final VersionInfo version; /** @@ -168,47 +126,19 @@ public class SCMNodeManager */ public SCMNodeManager(OzoneConfiguration conf, String clusterID, StorageContainerManager scmManager) throws IOException { - heartbeatQueue = new ConcurrentLinkedQueue<>(); - healthyNodes = new ConcurrentHashMap<>(); - deadNodes = new ConcurrentHashMap<>(); - staleNodes = new ConcurrentHashMap<>(); - nodes = new ConcurrentHashMap<>(); - nodeStats = new ConcurrentHashMap<>(); - scmStat = new SCMNodeStat(); - - healthyNodeCount = new AtomicInteger(0); - staleNodeCount = new AtomicInteger(0); - deadNodeCount = new AtomicInteger(0); - totalNodes = new AtomicInteger(0); + this.nodeStateManager = new NodeStateManager(conf); + this.nodeStats = new ConcurrentHashMap<>(); + this.scmStat = new SCMNodeStat(); this.clusterID = clusterID; this.version = VersionInfo.getLatestVersion(); - commandQueue = new CommandQueue(); - + this.commandQueue = new CommandQueue(); // TODO: Support this value as a Percentage of known machines. - chillModeNodeCount = 1; - - staleNodeIntervalMs = HddsServerUtil.getStaleNodeInterval(conf); - deadNodeIntervalMs = HddsServerUtil.getDeadNodeInterval(conf); - heartbeatCheckerIntervalMs = - HddsServerUtil.getScmheartbeatCheckerInterval(conf); - datanodeHBIntervalSeconds = HddsServerUtil.getScmHeartbeatInterval(conf); - maxHBToProcessPerLoop = HddsServerUtil.getMaxHBToProcessPerLoop(conf); - - executorService = HadoopExecutors.newScheduledThreadPool(1, - new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat("SCM Heartbeat Processing Thread - %d").build()); - - LOG.info("Entering startup chill mode."); + this.chillModeNodeCount = 1; this.inStartupChillMode = new AtomicBoolean(true); this.inManualChillMode = new AtomicBoolean(false); - - Preconditions.checkState(heartbeatCheckerIntervalMs > 0); - executorService.schedule(this, heartbeatCheckerIntervalMs, - TimeUnit.MILLISECONDS); - - registerMXBean(); - this.scmManager = scmManager; + LOG.info("Entering startup chill mode."); + registerMXBean(); } private void registerMXBean() { @@ -227,12 +157,11 @@ public class SCMNodeManager * Removes a data node from the management of this Node Manager. * * @param node - DataNode. - * @throws UnregisteredNodeException + * @throws NodeNotFoundException */ @Override - public void removeNode(DatanodeDetails node) { - // TODO : Fix me when adding the SCM CLI. - + public void removeNode(DatanodeDetails node) throws NodeNotFoundException { + nodeStateManager.removeNode(node); } /** @@ -244,31 +173,8 @@ public class SCMNodeManager * @return List of Datanodes that are known to SCM in the requested state. */ @Override - public List getNodes(NodeState nodestate) - throws IllegalArgumentException { - Map set; - switch (nodestate) { - case HEALTHY: - synchronized (this) { - set = Collections.unmodifiableMap(new HashMap<>(healthyNodes)); - } - break; - case STALE: - synchronized (this) { - set = Collections.unmodifiableMap(new HashMap<>(staleNodes)); - } - break; - case DEAD: - synchronized (this) { - set = Collections.unmodifiableMap(new HashMap<>(deadNodes)); - } - break; - default: - throw new IllegalArgumentException("Unknown node state requested."); - } - - return set.entrySet().stream().map(entry -> nodes.get(entry.getKey())) - .collect(Collectors.toList()); + public List getNodes(NodeState nodestate) { + return nodeStateManager.getNodes(nodestate); } /** @@ -278,12 +184,7 @@ public class SCMNodeManager */ @Override public List getAllNodes() { - Map set; - synchronized (this) { - set = Collections.unmodifiableMap(new HashMap<>(nodes)); - } - return set.entrySet().stream().map(entry -> nodes.get(entry.getKey())) - .collect(Collectors.toList()); + return nodeStateManager.getAllNodes(); } /** @@ -315,14 +216,16 @@ public class SCMNodeManager if (inStartupChillMode.get()) { return "Still in chill mode, waiting on nodes to report in." + String.format(" %d nodes reported, minimal %d nodes required.", - totalNodes.get(), getMinimumChillModeNodes()); + nodeStateManager.getTotalNodeCount(), getMinimumChillModeNodes()); } if (inManualChillMode.get()) { return "Out of startup chill mode, but in manual chill mode." + - String.format(" %d nodes have reported in.", totalNodes.get()); + String.format(" %d nodes have reported in.", + nodeStateManager.getTotalNodeCount()); } return "Out of chill mode." + - String.format(" %d nodes have reported in.", totalNodes.get()); + String.format(" %d nodes have reported in.", + nodeStateManager.getTotalNodeCount()); } /** @@ -376,33 +279,7 @@ public class SCMNodeManager */ @Override public int getNodeCount(NodeState nodestate) { - switch (nodestate) { - case HEALTHY: - return healthyNodeCount.get(); - case STALE: - return staleNodeCount.get(); - case DEAD: - return deadNodeCount.get(); - case INVALID: - // This is unknown due to the fact that some nodes can be in - // transit between the other states. Returning a count for that is not - // possible. The fact that we have such state is to deal with the fact - // that this information might not be consistent always. - return 0; - default: - return 0; - } - } - - /** - * Used for testing. - * - * @return true if the HB check is done. - */ - @VisibleForTesting - @Override - public boolean waitForHeartbeatProcessed() { - return lastHBcheckFinished != 0; + return nodeStateManager.getNodeCount(nodestate); } /** @@ -413,236 +290,14 @@ public class SCMNodeManager */ @Override public NodeState getNodeState(DatanodeDetails datanodeDetails) { - // There is a subtle race condition here, hence we also support - // the NODEState.UNKNOWN. It is possible that just before we check the - // healthyNodes, we have removed the node from the healthy list but stil - // not added it to Stale Nodes list. - // We can fix that by adding the node to stale list before we remove, but - // then the node is in 2 states to avoid this race condition. Instead we - // just deal with the possibilty of getting a state called unknown. - - UUID id = datanodeDetails.getUuid(); - if(healthyNodes.containsKey(id)) { - return HEALTHY; - } - - if(staleNodes.containsKey(id)) { - return STALE; - } - - if(deadNodes.containsKey(id)) { - return DEAD; - } - - return INVALID; - } - - /** - * This is the real worker thread that processes the HB queue. We do the - * following things in this thread. - *

- * Process the Heartbeats that are in the HB Queue. Move Stale or Dead node to - * healthy if we got a heartbeat from them. Move Stales Node to dead node - * table if it is needed. Move healthy nodes to stale nodes if it is needed. - *

- * if it is a new node, we call register node and add it to the list of nodes. - * This will be replaced when we support registration of a node in SCM. - * - * @see Thread#run() - */ - @Override - public void run() { - lastHBcheckStart = monotonicNow(); - lastHBProcessedCount = 0; - - // Process the whole queue. - while (!heartbeatQueue.isEmpty() && - (lastHBProcessedCount < maxHBToProcessPerLoop)) { - HeartbeatQueueItem hbItem = heartbeatQueue.poll(); - synchronized (this) { - handleHeartbeat(hbItem); - } - // we are shutting down or something give up processing the rest of - // HBs. This will terminate the HB processing thread. - if (Thread.currentThread().isInterrupted()) { - LOG.info("Current Thread is isInterrupted, shutting down HB " + - "processing thread for Node Manager."); - return; - } - } - - if (lastHBProcessedCount >= maxHBToProcessPerLoop) { - LOG.error("SCM is being flooded by heartbeats. Not able to keep up with" + - " the heartbeat counts. Processed {} heartbeats. Breaking out of" + - " loop. Leaving rest to be processed later. ", lastHBProcessedCount); - } - - // Iterate over the Stale nodes and decide if we need to move any node to - // dead State. - long currentTime = monotonicNow(); - for (Map.Entry entry : staleNodes.entrySet()) { - if (currentTime - entry.getValue() > deadNodeIntervalMs) { - synchronized (this) { - moveStaleNodeToDead(entry); - } - } - } - - // Iterate over the healthy nodes and decide if we need to move any node to - // Stale State. - currentTime = monotonicNow(); - for (Map.Entry entry : healthyNodes.entrySet()) { - if (currentTime - entry.getValue() > staleNodeIntervalMs) { - synchronized (this) { - moveHealthyNodeToStale(entry); - } - } - } - lastHBcheckFinished = monotonicNow(); - - monitorHBProcessingTime(); - - // we purposefully make this non-deterministic. Instead of using a - // scheduleAtFixedFrequency we will just go to sleep - // and wake up at the next rendezvous point, which is currentTime + - // heartbeatCheckerIntervalMs. This leads to the issue that we are now - // heart beating not at a fixed cadence, but clock tick + time taken to - // work. - // - // This time taken to work can skew the heartbeat processor thread. - // The reason why we don't care is because of the following reasons. - // - // 1. checkerInterval is general many magnitudes faster than datanode HB - // frequency. - // - // 2. if we have too much nodes, the SCM would be doing only HB - // processing, this could lead to SCM's CPU starvation. With this - // approach we always guarantee that HB thread sleeps for a little while. - // - // 3. It is possible that we will never finish processing the HB's in the - // thread. But that means we have a mis-configured system. We will warn - // the users by logging that information. - // - // 4. And the most important reason, heartbeats are not blocked even if - // this thread does not run, they will go into the processing queue. - - if (!Thread.currentThread().isInterrupted() && - !executorService.isShutdown()) { - executorService.schedule(this, heartbeatCheckerIntervalMs, TimeUnit - .MILLISECONDS); - } else { - LOG.info("Current Thread is interrupted, shutting down HB processing " + - "thread for Node Manager."); + try { + return nodeStateManager.getNodeState(datanodeDetails); + } catch (NodeNotFoundException e) { + // TODO: should we throw NodeNotFoundException? + return null; } } - /** - * If we have taken too much time for HB processing, log that information. - */ - private void monitorHBProcessingTime() { - if (TimeUnit.MILLISECONDS.toSeconds(lastHBcheckFinished - - lastHBcheckStart) > datanodeHBIntervalSeconds) { - LOG.error("Total time spend processing datanode HB's is greater than " + - "configured values for datanode heartbeats. Please adjust the" + - " heartbeat configs. Time Spend on HB processing: {} seconds " + - "Datanode heartbeat Interval: {} seconds , heartbeats " + - "processed: {}", - TimeUnit.MILLISECONDS - .toSeconds(lastHBcheckFinished - lastHBcheckStart), - datanodeHBIntervalSeconds, lastHBProcessedCount); - } - } - - /** - * Moves a Healthy node to a Stale node state. - * - * @param entry - Map Entry - */ - private void moveHealthyNodeToStale(Map.Entry entry) { - LOG.trace("Moving healthy node to stale: {}", entry.getKey()); - healthyNodes.remove(entry.getKey()); - healthyNodeCount.decrementAndGet(); - staleNodes.put(entry.getKey(), entry.getValue()); - staleNodeCount.incrementAndGet(); - - if (scmManager != null) { - // remove stale node's container report - scmManager.removeContainerReport(entry.getKey().toString()); - } - } - - /** - * Moves a Stale node to a dead node state. - * - * @param entry - Map Entry - */ - private void moveStaleNodeToDead(Map.Entry entry) { - LOG.trace("Moving stale node to dead: {}", entry.getKey()); - staleNodes.remove(entry.getKey()); - staleNodeCount.decrementAndGet(); - deadNodes.put(entry.getKey(), entry.getValue()); - deadNodeCount.incrementAndGet(); - - // Update SCM node stats - SCMNodeStat deadNodeStat = nodeStats.get(entry.getKey()); - scmStat.subtract(deadNodeStat); - nodeStats.remove(entry.getKey()); - } - - /** - * Handles a single heartbeat from a datanode. - * - * @param hbItem - heartbeat item from a datanode. - */ - private void handleHeartbeat(HeartbeatQueueItem hbItem) { - lastHBProcessedCount++; - - DatanodeDetails datanodeDetails = hbItem.getDatanodeDetails(); - UUID datanodeUuid = datanodeDetails.getUuid(); - NodeReportProto nodeReport = hbItem.getNodeReport(); - long recvTimestamp = hbItem.getRecvTimestamp(); - long processTimestamp = Time.monotonicNow(); - if (LOG.isTraceEnabled()) { - //TODO: add average queue time of heartbeat request as metrics - LOG.trace("Processing Heartbeat from datanode {}: queueing time {}", - datanodeUuid, processTimestamp - recvTimestamp); - } - - // If this node is already in the list of known and healthy nodes - // just set the last timestamp and return. - if (healthyNodes.containsKey(datanodeUuid)) { - healthyNodes.put(datanodeUuid, processTimestamp); - updateNodeStat(datanodeUuid, nodeReport); - return; - } - - // A stale node has heartbeat us we need to remove the node from stale - // list and move to healthy list. - if (staleNodes.containsKey(datanodeUuid)) { - staleNodes.remove(datanodeUuid); - healthyNodes.put(datanodeUuid, processTimestamp); - healthyNodeCount.incrementAndGet(); - staleNodeCount.decrementAndGet(); - updateNodeStat(datanodeUuid, nodeReport); - return; - } - - // A dead node has heartbeat us, we need to remove that node from dead - // node list and move it to the healthy list. - if (deadNodes.containsKey(datanodeUuid)) { - deadNodes.remove(datanodeUuid); - healthyNodes.put(datanodeUuid, processTimestamp); - deadNodeCount.decrementAndGet(); - healthyNodeCount.incrementAndGet(); - updateNodeStat(datanodeUuid, nodeReport); - return; - } - - LOG.warn("SCM receive heartbeat from unregistered datanode {}", - datanodeUuid); - this.commandQueue.addCommand(datanodeUuid, - new ReregisterCommand()); - } private void updateNodeStat(UUID dnId, NodeReportProto nodeReport) { SCMNodeStat stat = nodeStats.get(dnId); @@ -679,24 +334,6 @@ public class SCMNodeManager @Override public void close() throws IOException { unregisterMXBean(); - executorService.shutdown(); - try { - if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) { - executorService.shutdownNow(); - } - - if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) { - LOG.error("Unable to shutdown NodeManager properly."); - } - } catch (InterruptedException e) { - executorService.shutdownNow(); - Thread.currentThread().interrupt(); - } - } - - @VisibleForTesting - long getLastHBProcessedCount() { - return lastHBProcessedCount; } /** @@ -739,27 +376,22 @@ public class SCMNodeManager datanodeDetails.setHostName(hostname); datanodeDetails.setIpAddress(ip); } - RegisteredCommand responseCommand = verifyDatanodeUUID(datanodeDetails); - if (responseCommand != null) { - return responseCommand; - } UUID dnId = datanodeDetails.getUuid(); - nodes.put(dnId, datanodeDetails); - totalNodes.incrementAndGet(); - healthyNodes.put(dnId, monotonicNow()); - healthyNodeCount.incrementAndGet(); - nodeStats.put(dnId, new SCMNodeStat()); - - if(inStartupChillMode.get() && - totalNodes.get() >= getMinimumChillModeNodes()) { - inStartupChillMode.getAndSet(false); - LOG.info("Leaving startup chill mode."); + try { + nodeStateManager.addNode(datanodeDetails); + nodeStats.put(dnId, new SCMNodeStat()); + if(inStartupChillMode.get() && + nodeStateManager.getTotalNodeCount() >= getMinimumChillModeNodes()) { + inStartupChillMode.getAndSet(false); + LOG.info("Leaving startup chill mode."); + } + // Updating Node Report, as registration is successful + updateNodeStat(datanodeDetails.getUuid(), nodeReport); + LOG.info("Data node with ID: {} Registered.", datanodeDetails.getUuid()); + } catch (NodeAlreadyExistsException e) { + LOG.trace("Datanode is already registered. Datanode: {}", + datanodeDetails.toString()); } - - // Updating Node Report, as registration is successful - updateNodeStat(datanodeDetails.getUuid(), nodeReport); - LOG.info("Data node with ID: {} Registered.", - datanodeDetails.getUuid()); RegisteredCommand.Builder builder = RegisteredCommand.newBuilder().setErrorCode(ErrorCode.success) .setDatanodeUUID(datanodeDetails.getUuidString()) @@ -770,46 +402,25 @@ public class SCMNodeManager return builder.build(); } - /** - * Verifies the datanode does not have a valid UUID already. - * - * @param datanodeDetails - Datanode Details. - * @return SCMCommand - */ - private RegisteredCommand verifyDatanodeUUID( - DatanodeDetails datanodeDetails) { - if (datanodeDetails.getUuid() != null && - nodes.containsKey(datanodeDetails.getUuid())) { - LOG.trace("Datanode is already registered. Datanode: {}", - datanodeDetails.toString()); - return RegisteredCommand.newBuilder() - .setErrorCode(ErrorCode.success) - .setClusterID(this.clusterID) - .setDatanodeUUID(datanodeDetails.getUuidString()) - .build(); - } - return null; - } - /** * Send heartbeat to indicate the datanode is alive and doing well. * * @param datanodeDetails - DatanodeDetailsProto. - * @param nodeReport - node report. * @return SCMheartbeat response. * @throws IOException */ @Override - public List sendHeartbeat( - DatanodeDetails datanodeDetails, NodeReportProto nodeReport) { - + public List processHeartbeat(DatanodeDetails datanodeDetails) { Preconditions.checkNotNull(datanodeDetails, "Heartbeat is missing " + "DatanodeDetails."); - heartbeatQueue.add( - new HeartbeatQueueItem.Builder() - .setDatanodeDetails(datanodeDetails) - .setNodeReport(nodeReport) - .build()); + try { + nodeStateManager.updateLastHeartbeatTime(datanodeDetails); + } catch (NodeNotFoundException e) { + LOG.warn("SCM receive heartbeat from unregistered datanode {}", + datanodeDetails); + commandQueue.addCommand(datanodeDetails.getUuid(), + new ReregisterCommand()); + } return commandQueue.getCommand(datanodeDetails.getUuid()); } @@ -855,11 +466,6 @@ public class SCMNodeManager this.commandQueue.addCommand(dnId, command); } - @VisibleForTesting - public void setStaleNodeIntervalMs(long interval) { - this.staleNodeIntervalMs = interval; - } - @Override public void onMessage(CommandForDatanode commandForDatanode, EventPublisher publisher) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeAlreadyExistsException.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeAlreadyExistsException.java new file mode 100644 index 00000000000..aa5c382f426 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeAlreadyExistsException.java @@ -0,0 +1,45 @@ +/** + * 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.hdds.scm.node.states; + +/** + * This exception represents that there is already a node added to NodeStateMap + * with same UUID. + */ +public class NodeAlreadyExistsException extends NodeException { + + /** + * Constructs an {@code NodeAlreadyExistsException} with {@code null} + * as its error detail message. + */ + public NodeAlreadyExistsException() { + super(); + } + + /** + * Constructs an {@code NodeAlreadyExistsException} with the specified + * detail message. + * + * @param message + * The detail message (which is saved for later retrieval + * by the {@link #getMessage()} method) + */ + public NodeAlreadyExistsException(String message) { + super(message); + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeException.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeException.java new file mode 100644 index 00000000000..c67b55d9531 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeException.java @@ -0,0 +1,44 @@ +/** + * 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.hdds.scm.node.states; + +/** + * This exception represents all node related exceptions in NodeStateMap. + */ +public class NodeException extends Exception { + + /** + * Constructs an {@code NodeException} with {@code null} + * as its error detail message. + */ + public NodeException() { + super(); + } + + /** + * Constructs an {@code NodeException} with the specified + * detail message. + * + * @param message + * The detail message (which is saved for later retrieval + * by the {@link #getMessage()} method) + */ + public NodeException(String message) { + super(message); + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeNotFoundException.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeNotFoundException.java new file mode 100644 index 00000000000..52a352e854b --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeNotFoundException.java @@ -0,0 +1,49 @@ +/** + * 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.hdds.scm.node.states; + +import java.io.IOException; + +/** + * This exception represents that the node that is being accessed does not + * exist in NodeStateMap. + */ +public class NodeNotFoundException extends NodeException { + + + /** + * Constructs an {@code NodeNotFoundException} with {@code null} + * as its error detail message. + */ + public NodeNotFoundException() { + super(); + } + + /** + * Constructs an {@code NodeNotFoundException} with the specified + * detail message. + * + * @param message + * The detail message (which is saved for later retrieval + * by the {@link #getMessage()} method) + */ + public NodeNotFoundException(String message) { + super(message); + } + +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java new file mode 100644 index 00000000000..dd91866d1e7 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java @@ -0,0 +1,281 @@ +/** + * 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.hdds.scm.node.states; + +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; +import org.apache.hadoop.hdds.scm.node.DatanodeInfo; + +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * Maintains the state of datanodes in SCM. This class should only be used by + * NodeStateManager to maintain the state. If anyone wants to change the + * state of a node they should call NodeStateManager, do not directly use + * this class. + */ +public class NodeStateMap { + + /** + * Node id to node info map. + */ + private final ConcurrentHashMap nodeMap; + /** + * Represents the current state of node. + */ + private final ConcurrentHashMap> stateMap; + private final ReadWriteLock lock; + + /** + * Creates a new instance of NodeStateMap with no nodes. + */ + public NodeStateMap() { + lock = new ReentrantReadWriteLock(); + nodeMap = new ConcurrentHashMap<>(); + stateMap = new ConcurrentHashMap<>(); + initStateMap(); + } + + /** + * Initializes the state map with available states. + */ + private void initStateMap() { + for (NodeState state : NodeState.values()) { + stateMap.put(state, new HashSet<>()); + } + } + + /** + * Adds a node to NodeStateMap. + * + * @param datanodeDetails DatanodeDetails + * @param nodeState initial NodeState + * + * @throws NodeAlreadyExistsException if the node already exist + */ + public void addNode(DatanodeDetails datanodeDetails, NodeState nodeState) + throws NodeAlreadyExistsException { + lock.writeLock().lock(); + try { + UUID id = datanodeDetails.getUuid(); + if (nodeMap.containsKey(id)) { + throw new NodeAlreadyExistsException("Node UUID: " + id); + } + nodeMap.put(id, new DatanodeInfo(datanodeDetails)); + stateMap.get(nodeState).add(id); + } finally { + lock.writeLock().unlock(); + } + } + + /** + * Updates the node state. + * + * @param nodeId Node Id + * @param currentState current state + * @param newState new state + * + * @throws NodeNotFoundException if the node is not present + */ + public void updateNodeState(UUID nodeId, NodeState currentState, + NodeState newState)throws NodeNotFoundException { + lock.writeLock().lock(); + try { + if (stateMap.get(currentState).remove(nodeId)) { + stateMap.get(newState).add(nodeId); + } else { + throw new NodeNotFoundException("Node UUID: " + nodeId + + ", not found in state: " + currentState); + } + } finally { + lock.writeLock().unlock(); + } + } + + /** + * Returns DatanodeDetails for the given node id. + * + * @param uuid Node Id + * + * @return DatanodeDetails of the node + * + * @throws NodeNotFoundException if the node is not present + */ + public DatanodeDetails getNodeDetails(UUID uuid) + throws NodeNotFoundException { + return getNodeInfo(uuid); + } + + /** + * Returns DatanodeInfo for the given node id. + * + * @param uuid Node Id + * + * @return DatanodeInfo of the node + * + * @throws NodeNotFoundException if the node is not present + */ + public DatanodeInfo getNodeInfo(UUID uuid) throws NodeNotFoundException { + lock.readLock().lock(); + try { + if (nodeMap.containsKey(uuid)) { + return nodeMap.get(uuid); + } + throw new NodeNotFoundException("Node UUID: " + uuid); + } finally { + lock.readLock().unlock(); + } + } + + + /** + * Returns the list of node ids which are in the specified state. + * + * @param state NodeState + * + * @return list of node ids + */ + public List getNodes(NodeState state) { + lock.readLock().lock(); + try { + return new LinkedList<>(stateMap.get(state)); + } finally { + lock.readLock().unlock(); + } + } + + /** + * Returns the list of all the node ids. + * + * @return list of all the node ids + */ + public List getAllNodes() { + lock.readLock().lock(); + try { + return new LinkedList<>(nodeMap.keySet()); + } finally { + lock.readLock().unlock(); + } + } + + /** + * Returns the count of nodes in the specified state. + * + * @param state NodeState + * + * @return Number of nodes in the specified state + */ + public int getNodeCount(NodeState state) { + lock.readLock().lock(); + try { + return stateMap.get(state).size(); + } finally { + lock.readLock().unlock(); + } + } + + /** + * Returns the total node count. + * + * @return node count + */ + public int getTotalNodeCount() { + lock.readLock().lock(); + try { + return nodeMap.size(); + } finally { + lock.readLock().unlock(); + } + } + + /** + * Returns the current state of the node. + * + * @param uuid node id + * + * @return NodeState + * + * @throws NodeNotFoundException if the node is not found + */ + public NodeState getNodeState(UUID uuid) throws NodeNotFoundException { + lock.readLock().lock(); + try { + for (Map.Entry> entry : stateMap.entrySet()) { + if (entry.getValue().contains(uuid)) { + return entry.getKey(); + } + } + throw new NodeNotFoundException("Node UUID: " + uuid); + } finally { + lock.readLock().unlock(); + } + } + + /** + * Removes the node from NodeStateMap. + * + * @param uuid node id + * + * @throws NodeNotFoundException if the node is not found + */ + public void removeNode(UUID uuid) throws NodeNotFoundException { + lock.writeLock().lock(); + try { + if (nodeMap.containsKey(uuid)) { + for (Map.Entry> entry : stateMap.entrySet()) { + if(entry.getValue().remove(uuid)) { + break; + } + nodeMap.remove(uuid); + } + throw new NodeNotFoundException("Node UUID: " + uuid); + } + } finally { + lock.writeLock().unlock(); + } + } + + /** + * Since we don't hold a global lock while constructing this string, + * the result might be inconsistent. If someone has changed the state of node + * while we are constructing the string, the result will be inconsistent. + * This should only be used for logging. We should not parse this string and + * use it for any critical calculations. + * + * @return current state of NodeStateMap + */ + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("Total number of nodes: ").append(getTotalNodeCount()); + for (NodeState state : NodeState.values()) { + builder.append("Number of nodes in ").append(state).append(" state: ") + .append(getNodeCount(state)); + } + return builder.toString(); + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java index e1d478f404d..aefcf1b6ec1 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java @@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.net.InetSocketAddress; -import java.util.EnumSet; +import java.util.ArrayList; import java.util.LinkedList; import java.util.List; import java.util.Set; @@ -188,27 +188,21 @@ public class SCMClientProtocolServer implements } @Override - public HddsProtos.NodePool queryNode(EnumSet - nodeStatuses, HddsProtos.QueryScope queryScope, String poolName) throws + public List queryNode(HddsProtos.NodeState state, + HddsProtos.QueryScope queryScope, String poolName) throws IOException { if (queryScope == HddsProtos.QueryScope.POOL) { throw new IllegalArgumentException("Not Supported yet"); } - List datanodes = queryNode(nodeStatuses); - HddsProtos.NodePool.Builder poolBuilder = HddsProtos.NodePool.newBuilder(); + List result = new ArrayList<>(); + queryNode(state).forEach(node -> result.add(HddsProtos.Node.newBuilder() + .setNodeID(node.getProtoBufMessage()) + .addNodeStates(state) + .build())); - for (DatanodeDetails datanode : datanodes) { - HddsProtos.Node node = - HddsProtos.Node.newBuilder() - .setNodeID(datanode.getProtoBufMessage()) - .addAllNodeStates(nodeStatuses) - .build(); - poolBuilder.addNodes(node); - } - - return poolBuilder.build(); + return result; } @@ -282,35 +276,12 @@ public class SCMClientProtocolServer implements * operation between the * operators. * - * @param nodeStatuses - A set of NodeStates. + * @param state - NodeStates. * @return List of Datanodes. */ - public List queryNode(EnumSet - nodeStatuses) { - Preconditions.checkNotNull(nodeStatuses, "Node Query set cannot be null"); - Preconditions.checkState(nodeStatuses.size() > 0, "No valid arguments " + - "in the query set"); - List resultList = new LinkedList<>(); - Set currentSet = new TreeSet<>(); - - for (HddsProtos.NodeState nodeState : nodeStatuses) { - Set nextSet = queryNodeState(nodeState); - if ((nextSet == null) || (nextSet.size() == 0)) { - // Right now we only support AND operation. So intersect with - // any empty set is null. - return resultList; - } - // First time we have to add all the elements, next time we have to - // do an intersection operation on the set. - if (currentSet.size() == 0) { - currentSet.addAll(nextSet); - } else { - currentSet.retainAll(nextSet); - } - } - - resultList.addAll(currentSet); - return resultList; + public List queryNode(HddsProtos.NodeState state) { + Preconditions.checkNotNull(state, "Node Query set cannot be null"); + return new LinkedList<>(queryNodeState(state)); } @VisibleForTesting @@ -325,11 +296,6 @@ public class SCMClientProtocolServer implements * @return Set of Datanodes that match the NodeState. */ private Set queryNodeState(HddsProtos.NodeState nodeState) { - if (nodeState == HddsProtos.NodeState.RAFT_MEMBER || nodeState == - HddsProtos.NodeState - .FREE_NODE) { - throw new IllegalStateException("Not implemented yet"); - } Set returnSet = new TreeSet<>(); List tmp = scm.getScmNodeManager().getNodes(nodeState); if ((tmp != null) && (tmp.size() > 0)) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java index 36f10a93dc0..f221584bddf 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java @@ -61,7 +61,7 @@ public final class SCMDatanodeHeartbeatDispatcher { public void dispatch(SCMHeartbeatRequestProto heartbeat) { DatanodeDetails datanodeDetails = DatanodeDetails.getFromProtoBuf(heartbeat.getDatanodeDetails()); - + // should we dispatch heartbeat through eventPublisher? if (heartbeat.hasNodeReport()) { eventPublisher.fireEvent(NODE_REPORT, new NodeReportFromDatanode(datanodeDetails, diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java index 56b07190a54..aef5b03ce22 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java @@ -223,7 +223,7 @@ public class SCMDatanodeProtocolServer implements .getFromProtoBuf(heartbeat.getDatanodeDetails()); NodeReportProto nodeReport = heartbeat.getNodeReport(); List commands = - scm.getScmNodeManager().sendHeartbeat(datanodeDetails, nodeReport); + scm.getScmNodeManager().processHeartbeat(datanodeDetails); List cmdResponses = new LinkedList<>(); for (SCMCommand cmd : commands) { cmdResponses.add(getCommandResponse(cmd)); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java index 80b5d6e182c..33579928798 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java @@ -19,13 +19,11 @@ package org.apache.hadoop.hdds.scm.container; import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric; import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat; import org.apache.hadoop.hdds.scm.node.NodeManager; -import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; +import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.NodeReportProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.StorageReportProto; import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; import org.apache.hadoop.ozone.OzoneConsts; @@ -33,7 +31,6 @@ import org.apache.hadoop.ozone.protocol.VersionResponse; import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand; import org.assertj.core.util.Preconditions; -import org.mockito.Mockito; import java.io.IOException; import java.util.HashMap; @@ -130,11 +127,11 @@ public class MockNodeManager implements NodeManager { * Removes a data node from the management of this Node Manager. * * @param node - DataNode. - * @throws UnregisteredNodeException + * @throws NodeNotFoundException */ @Override public void removeNode(DatanodeDetails node) - throws UnregisteredNodeException { + throws NodeNotFoundException { } @@ -272,16 +269,6 @@ public class MockNodeManager implements NodeManager { return new SCMNodeMetric(nodeMetricMap.get(datanodeDetails.getUuid())); } - /** - * Used for testing. - * - * @return true if the HB check is done. - */ - @Override - public boolean waitForHeartbeatProcessed() { - return false; - } - /** * Returns the node state of a specific node. * @@ -334,21 +321,6 @@ public class MockNodeManager implements NodeManager { } - /** - * When an object implementing interface Runnable is used to - * create a thread, starting the thread causes the object's run - * method to be called in that separately executing thread. - *

- * The general contract of the method run is that it may take any - * action whatsoever. - * - * @see Thread#run() - */ - @Override - public void run() { - - } - /** * Gets the version info from SCM. * @@ -379,32 +351,10 @@ public class MockNodeManager implements NodeManager { * Send heartbeat to indicate the datanode is alive and doing well. * * @param datanodeDetails - Datanode ID. - * @param nodeReport - node report. * @return SCMheartbeat response list */ @Override - public List sendHeartbeat(DatanodeDetails datanodeDetails, - NodeReportProto nodeReport) { - if ((datanodeDetails != null) && (nodeReport != null) && (nodeReport - .getStorageReportCount() > 0)) { - SCMNodeStat stat = this.nodeMetricMap.get(datanodeDetails.getUuid()); - - long totalCapacity = 0L; - long totalRemaining = 0L; - long totalScmUsed = 0L; - List storageReports = nodeReport - .getStorageReportList(); - for (StorageReportProto report : storageReports) { - totalCapacity += report.getCapacity(); - totalRemaining += report.getRemaining(); - totalScmUsed += report.getScmUsed(); - } - aggregateStat.subtract(stat); - stat.set(totalCapacity, totalScmUsed, totalRemaining); - aggregateStat.add(stat); - nodeMetricMap.put(datanodeDetails.getUuid(), stat); - - } + public List processHeartbeat(DatanodeDetails datanodeDetails) { return null; } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java index 98b0a28bf25..c6ea2af1216 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java @@ -36,8 +36,8 @@ import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.StorageReportProto; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; -import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.PathUtils; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -109,6 +109,7 @@ public class TestContainerPlacement { * @throws TimeoutException */ @Test + @Ignore public void testContainerPlacementCapacity() throws IOException, InterruptedException, TimeoutException { OzoneConfiguration conf = getConf(); @@ -135,12 +136,11 @@ public class TestContainerPlacement { String path = testDir.getAbsolutePath() + "/" + id; List reports = TestUtils .createStorageReport(capacity, used, remaining, path, null, id, 1); - nodeManager.sendHeartbeat(datanodeDetails, - TestUtils.createNodeReport(reports)); + nodeManager.processHeartbeat(datanodeDetails); } - GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(), - 100, 4 * 1000); + //TODO: wait for heartbeat to be processed + Thread.sleep(4 * 1000); assertEquals(nodeCount, nodeManager.getNodeCount(HEALTHY)); assertEquals(capacity * nodeCount, (long) nodeManager.getStats().getCapacity().get()); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java index 824a1351942..0a4e33d444c 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java @@ -41,6 +41,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -61,8 +62,6 @@ import static org.apache.hadoop.hdds.scm.ScmConfigKeys .OZONE_SCM_HEARTBEAT_INTERVAL; import static org.apache.hadoop.hdds.scm.ScmConfigKeys .OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL; -import static org.apache.hadoop.hdds.scm.ScmConfigKeys - .OZONE_SCM_MAX_HB_COUNT_TO_PROCESS; import static org.apache.hadoop.hdds.scm.ScmConfigKeys .OZONE_SCM_STALENODE_INTERVAL; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD; @@ -148,14 +147,11 @@ public class TestNodeManager { for (int x = 0; x < nodeManager.getMinimumChillModeNodes(); x++) { DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails( nodeManager); - nodeManager.sendHeartbeat(datanodeDetails, - null); + nodeManager.processHeartbeat(datanodeDetails); } - // Wait for 4 seconds max. - GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(), - 100, 4 * 1000); - + //TODO: wait for heartbeat to be processed + Thread.sleep(4 * 1000); assertTrue("Heartbeat thread should have picked up the" + "scheduled heartbeats and transitioned out of chill mode.", nodeManager.isOutOfChillMode()); @@ -174,8 +170,8 @@ public class TestNodeManager { InterruptedException, TimeoutException { try (SCMNodeManager nodeManager = createNodeManager(getConf())) { - GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(), - 100, 4 * 1000); + //TODO: wait for heartbeat to be processed + Thread.sleep(4 * 1000); assertFalse("No heartbeats, Node manager should have been in" + " chill mode.", nodeManager.isOutOfChillMode()); } @@ -195,10 +191,9 @@ public class TestNodeManager { // Need 100 nodes to come out of chill mode, only one node is sending HB. nodeManager.setMinimumChillModeNodes(100); - nodeManager.sendHeartbeat(TestUtils.getDatanodeDetails(nodeManager), - null); - GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(), - 100, 4 * 1000); + nodeManager.processHeartbeat(TestUtils.getDatanodeDetails(nodeManager)); + //TODO: wait for heartbeat to be processed + Thread.sleep(4 * 1000); assertFalse("Not enough heartbeat, Node manager should have" + "been in chillmode.", nodeManager.isOutOfChillMode()); } @@ -223,12 +218,11 @@ public class TestNodeManager { // Send 10 heartbeat from same node, and assert we never leave chill mode. for (int x = 0; x < 10; x++) { - nodeManager.sendHeartbeat(datanodeDetails, - null); + nodeManager.processHeartbeat(datanodeDetails); } - GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(), - 100, 4 * 1000); + //TODO: wait for heartbeat to be processed + Thread.sleep(4 * 1000); assertFalse("Not enough nodes have send heartbeat to node" + "manager.", nodeManager.isOutOfChillMode()); } @@ -254,14 +248,12 @@ public class TestNodeManager { nodeManager.close(); // These should never be processed. - nodeManager.sendHeartbeat(datanodeDetails, - null); + nodeManager.processHeartbeat(datanodeDetails); // Let us just wait for 2 seconds to prove that HBs are not processed. Thread.sleep(2 * 1000); - assertEquals("Assert new HBs were never processed", 0, - nodeManager.getLastHBProcessedCount()); + //TODO: add assertion } /** @@ -283,8 +275,7 @@ public class TestNodeManager { try (SCMNodeManager nodemanager = createNodeManager(conf)) { nodemanager.register(datanodeDetails, TestUtils.createNodeReport(reports)); - List command = nodemanager.sendHeartbeat( - datanodeDetails, null); + List command = nodemanager.processHeartbeat(datanodeDetails); Assert.assertTrue(nodemanager.getAllNodes().contains(datanodeDetails)); Assert.assertTrue("On regular HB calls, SCM responses a " + "datanode with an empty command list", command.isEmpty()); @@ -302,8 +293,7 @@ public class TestNodeManager { GenericTestUtils.waitFor(new Supplier() { @Override public Boolean get() { List command = - nodemanager.sendHeartbeat(datanodeDetails, - null); + nodemanager.processHeartbeat(datanodeDetails); return command.size() == 1 && command.get(0).getType() .equals(SCMCommandProto.Type.reregisterCommand); } @@ -334,11 +324,10 @@ public class TestNodeManager { for (int x = 0; x < count; x++) { DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails( nodeManager); - nodeManager.sendHeartbeat(datanodeDetails, - null); + nodeManager.processHeartbeat(datanodeDetails); } - GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(), - 100, 4 * 1000); + //TODO: wait for heartbeat to be processed + Thread.sleep(4 * 1000); assertEquals(count, nodeManager.getNodeCount(HEALTHY)); } } @@ -426,19 +415,18 @@ public class TestNodeManager { DatanodeDetails staleNode = TestUtils.getDatanodeDetails(nodeManager); // Heartbeat once - nodeManager.sendHeartbeat(staleNode, - null); + nodeManager.processHeartbeat(staleNode); // Heartbeat all other nodes. for (DatanodeDetails dn : nodeList) { - nodeManager.sendHeartbeat(dn, null); + nodeManager.processHeartbeat(dn); } // Wait for 2 seconds .. and heartbeat good nodes again. Thread.sleep(2 * 1000); for (DatanodeDetails dn : nodeList) { - nodeManager.sendHeartbeat(dn, null); + nodeManager.processHeartbeat(dn); } // Wait for 2 seconds, wait a total of 4 seconds to make sure that the @@ -455,7 +443,7 @@ public class TestNodeManager { // heartbeat good nodes again. for (DatanodeDetails dn : nodeList) { - nodeManager.sendHeartbeat(dn, null); + nodeManager.processHeartbeat(dn); } // 6 seconds is the dead window for this test , so we wait a total of @@ -491,7 +479,7 @@ public class TestNodeManager { public void testScmCheckForErrorOnNullDatanodeDetails() throws IOException, InterruptedException, TimeoutException { try (SCMNodeManager nodeManager = createNodeManager(getConf())) { - nodeManager.sendHeartbeat(null, null); + nodeManager.processHeartbeat(null); } catch (NullPointerException npe) { GenericTestUtils.assertExceptionContains("Heartbeat is missing " + "DatanodeDetails.", npe); @@ -568,12 +556,9 @@ public class TestNodeManager { TestUtils.getDatanodeDetails(nodeManager); DatanodeDetails deadNode = TestUtils.getDatanodeDetails(nodeManager); - nodeManager.sendHeartbeat( - healthyNode, null); - nodeManager.sendHeartbeat( - staleNode, null); - nodeManager.sendHeartbeat( - deadNode, null); + nodeManager.processHeartbeat(healthyNode); + nodeManager.processHeartbeat(staleNode); + nodeManager.processHeartbeat(deadNode); // Sleep so that heartbeat processing thread gets to run. Thread.sleep(500); @@ -599,16 +584,12 @@ public class TestNodeManager { * the 3 second windows. */ - nodeManager.sendHeartbeat( - healthyNode, null); - nodeManager.sendHeartbeat( - staleNode, null); - nodeManager.sendHeartbeat( - deadNode, null); + nodeManager.processHeartbeat(healthyNode); + nodeManager.processHeartbeat(staleNode); + nodeManager.processHeartbeat(deadNode); Thread.sleep(1500); - nodeManager.sendHeartbeat( - healthyNode, null); + nodeManager.processHeartbeat(healthyNode); Thread.sleep(2 * 1000); assertEquals(1, nodeManager.getNodeCount(HEALTHY)); @@ -628,13 +609,10 @@ public class TestNodeManager { * staleNode to move to stale state and deadNode to move to dead state. */ - nodeManager.sendHeartbeat( - healthyNode, null); - nodeManager.sendHeartbeat( - staleNode, null); + nodeManager.processHeartbeat(healthyNode); + nodeManager.processHeartbeat(staleNode); Thread.sleep(1500); - nodeManager.sendHeartbeat( - healthyNode, null); + nodeManager.processHeartbeat(healthyNode); Thread.sleep(2 * 1000); // 3.5 seconds have elapsed for stale node, so it moves into Stale. @@ -667,12 +645,9 @@ public class TestNodeManager { * Cluster State : let us heartbeat all the nodes and verify that we get * back all the nodes in healthy state. */ - nodeManager.sendHeartbeat( - healthyNode, null); - nodeManager.sendHeartbeat( - staleNode, null); - nodeManager.sendHeartbeat( - deadNode, null); + nodeManager.processHeartbeat(healthyNode); + nodeManager.processHeartbeat(staleNode); + nodeManager.processHeartbeat(deadNode); Thread.sleep(500); //Assert all nodes are healthy. assertEquals(3, nodeManager.getAllNodes().size()); @@ -693,7 +668,7 @@ public class TestNodeManager { int sleepDuration) throws InterruptedException { while (!Thread.currentThread().isInterrupted()) { for (DatanodeDetails dn : list) { - manager.sendHeartbeat(dn, null); + manager.processHeartbeat(dn); } Thread.sleep(sleepDuration); } @@ -747,7 +722,6 @@ public class TestNodeManager { conf.setTimeDuration(OZONE_SCM_HEARTBEAT_INTERVAL, 1, SECONDS); conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, SECONDS); conf.setTimeDuration(OZONE_SCM_DEADNODE_INTERVAL, 6, SECONDS); - conf.setInt(OZONE_SCM_MAX_HB_COUNT_TO_PROCESS, 7000); try (SCMNodeManager nodeManager = createNodeManager(conf)) { @@ -779,7 +753,7 @@ public class TestNodeManager { // No Thread just one time HBs the node manager, so that these will be // marked as dead nodes eventually. for (DatanodeDetails dn : deadNodeList) { - nodeManager.sendHeartbeat(dn, null); + nodeManager.processHeartbeat(dn); } @@ -883,54 +857,6 @@ public class TestNodeManager { } } - /** - * Asserts that SCM backs off from HB processing instead of going into an - * infinite loop if SCM is flooded with too many heartbeats. This many not be - * the best thing to do, but SCM tries to protect itself and logs an error - * saying that it is getting flooded with heartbeats. In real world this can - * lead to many nodes becoming stale or dead due to the fact that SCM is not - * able to keep up with heartbeat processing. This test just verifies that SCM - * will log that information. - * @throws TimeoutException - */ - @Test - public void testScmLogsHeartbeatFlooding() throws IOException, - InterruptedException, TimeoutException { - final int healthyCount = 3000; - - // Make the HB process thread run slower. - OzoneConfiguration conf = getConf(); - conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, 500, - TimeUnit.MILLISECONDS); - conf.setTimeDuration(OZONE_SCM_HEARTBEAT_INTERVAL, 1, SECONDS); - conf.setInt(OZONE_SCM_MAX_HB_COUNT_TO_PROCESS, 500); - - try (SCMNodeManager nodeManager = createNodeManager(conf)) { - List healthyList = createNodeSet(nodeManager, - healthyCount); - GenericTestUtils.LogCapturer logCapturer = - GenericTestUtils.LogCapturer.captureLogs(SCMNodeManager.LOG); - Runnable healthyNodeTask = () -> { - try { - // No wait in the HB sending loop. - heartbeatNodeSet(nodeManager, healthyList, 0); - } catch (InterruptedException ignored) { - } - }; - Thread thread1 = new Thread(healthyNodeTask); - thread1.setDaemon(true); - thread1.start(); - - GenericTestUtils.waitFor(() -> logCapturer.getOutput() - .contains("SCM is being " - + "flooded by heartbeats. Not able to keep up" - + " with the heartbeat counts."), - 500, 20 * 1000); - - thread1.interrupt(); - logCapturer.stopCapturing(); - } - } @Test public void testScmEnterAndExitChillMode() throws IOException, @@ -943,8 +869,7 @@ public class TestNodeManager { nodeManager.setMinimumChillModeNodes(10); DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails( nodeManager); - nodeManager.sendHeartbeat( - datanodeDetails, null); + nodeManager.processHeartbeat(datanodeDetails); String status = nodeManager.getChillModeStatus(); Assert.assertThat(status, containsString("Still in chill " + "mode, waiting on nodes to report in.")); @@ -971,7 +896,7 @@ public class TestNodeManager { // Assert that node manager force enter cannot be overridden by nodes HBs. for (int x = 0; x < 20; x++) { DatanodeDetails datanode = TestUtils.getDatanodeDetails(nodeManager); - nodeManager.sendHeartbeat(datanode, null); + nodeManager.processHeartbeat(datanode); } Thread.sleep(500); @@ -995,6 +920,8 @@ public class TestNodeManager { * @throws TimeoutException */ @Test + @Ignore + // TODO: Enable this after we implement NodeReportEvent handler. public void testScmStatsFromNodeReport() throws IOException, InterruptedException, TimeoutException { OzoneConfiguration conf = getConf(); @@ -1015,11 +942,10 @@ public class TestNodeManager { List reports = TestUtils .createStorageReport(capacity, used, free, storagePath, null, dnId, 1); - nodeManager.sendHeartbeat(datanodeDetails, - TestUtils.createNodeReport(reports)); + nodeManager.processHeartbeat(datanodeDetails); } - GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(), - 100, 4 * 1000); + //TODO: wait for heartbeat to be processed + Thread.sleep(4 * 1000); assertEquals(nodeCount, nodeManager.getNodeCount(HEALTHY)); assertEquals(capacity * nodeCount, (long) nodeManager.getStats() .getCapacity().get()); @@ -1038,6 +964,8 @@ public class TestNodeManager { * @throws TimeoutException */ @Test + @Ignore + // TODO: Enable this after we implement NodeReportEvent handler. public void testScmNodeReportUpdate() throws IOException, InterruptedException, TimeoutException { OzoneConfiguration conf = getConf(); @@ -1065,8 +993,7 @@ public class TestNodeManager { .createStorageReport(capacity, scmUsed, remaining, storagePath, null, dnId, 1); - nodeManager.sendHeartbeat(datanodeDetails, - TestUtils.createNodeReport(reports)); + nodeManager.processHeartbeat(datanodeDetails); Thread.sleep(100); } @@ -1146,8 +1073,7 @@ public class TestNodeManager { List reports = TestUtils .createStorageReport(capacity, expectedScmUsed, expectedRemaining, storagePath, null, dnId, 1); - nodeManager.sendHeartbeat(datanodeDetails, - TestUtils.createNodeReport(reports)); + nodeManager.processHeartbeat(datanodeDetails); // Wait up to 5 seconds so that the dead node becomes healthy // Verify usage info should be updated. @@ -1195,7 +1121,7 @@ public class TestNodeManager { eq.processAll(1000L); List command = - nodemanager.sendHeartbeat(datanodeDetails, null); + nodemanager.processHeartbeat(datanodeDetails); Assert.assertEquals(1, command.size()); Assert .assertEquals(command.get(0).getClass(), CloseContainerCommand.class); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java index 1a4dcd7ad20..e15e0fcfc4a 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java @@ -21,7 +21,7 @@ import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric; import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat; import org.apache.hadoop.hdds.scm.node.CommandQueue; import org.apache.hadoop.hdds.scm.node.NodeManager; -import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; +import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; import org.apache.hadoop.hdds.protocol.proto @@ -31,7 +31,6 @@ import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.ozone.protocol.VersionResponse; import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand; -import org.mockito.Mockito; import java.io.IOException; import java.util.List; @@ -90,11 +89,11 @@ public class ReplicationNodeManagerMock implements NodeManager { * Removes a data node from the management of this Node Manager. * * @param node - DataNode. - * @throws UnregisteredNodeException + * @throws NodeNotFoundException */ @Override public void removeNode(DatanodeDetails node) - throws UnregisteredNodeException { + throws NodeNotFoundException { nodeStateMap.remove(node); } @@ -201,16 +200,6 @@ public class ReplicationNodeManagerMock implements NodeManager { } - /** - * Wait for the heartbeat is processed by NodeManager. - * - * @return true if heartbeat has been processed. - */ - @Override - public boolean waitForHeartbeatProcessed() { - return false; - } - /** * Returns the node state of a specific node. * @@ -240,22 +229,6 @@ public class ReplicationNodeManagerMock implements NodeManager { } - /** - * When an object implementing interface Runnable is used - * to create a thread, starting the thread causes the object's - * run method to be called in that separately executing - * thread. - *

- * The general contract of the method run is that it may - * take any action whatsoever. - * - * @see Thread#run() - */ - @Override - public void run() { - - } - /** * Gets the version info from SCM. * @@ -285,12 +258,10 @@ public class ReplicationNodeManagerMock implements NodeManager { * Send heartbeat to indicate the datanode is alive and doing well. * * @param dd - Datanode Details. - * @param nodeReport - node report. * @return SCMheartbeat response list */ @Override - public List sendHeartbeat(DatanodeDetails dd, - NodeReportProto nodeReport) { + public List processHeartbeat(DatanodeDetails dd) { return null; } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java index d07097ca8b9..dd1a8de8903 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java @@ -303,8 +303,8 @@ public class TestStorageContainerManager { GenericTestUtils.waitFor(() -> { NodeManager nodeManager = cluster.getStorageContainerManager() .getScmNodeManager(); - List commands = nodeManager.sendHeartbeat( - nodeManager.getNodes(NodeState.HEALTHY).get(0), null); + List commands = nodeManager.processHeartbeat( + nodeManager.getNodes(NodeState.HEALTHY).get(0)); if (commands != null) { for (SCMCommand cmd : commands) { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestQueryNode.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestQueryNode.java index b999c9276a9..22528e42fae 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestQueryNode.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestQueryNode.java @@ -26,7 +26,7 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import java.util.EnumSet; +import java.util.List; import java.util.concurrent.TimeUnit; import static java.util.concurrent.TimeUnit.SECONDS; @@ -83,11 +83,10 @@ public class TestQueryNode { @Test public void testHealthyNodesCount() throws Exception { - HddsProtos.NodePool pool = scmClient.queryNode( - EnumSet.of(HEALTHY), + List nodes = scmClient.queryNode(HEALTHY, HddsProtos.QueryScope.CLUSTER, ""); assertEquals("Expected live nodes", numOfDatanodes, - pool.getNodesCount()); + nodes.size()); } @Test(timeout = 10 * 1000L) @@ -99,8 +98,8 @@ public class TestQueryNode { cluster.getStorageContainerManager().getNodeCount(STALE) == 2, 100, 4 * 1000); - int nodeCount = scmClient.queryNode(EnumSet.of(STALE), - HddsProtos.QueryScope.CLUSTER, "").getNodesCount(); + int nodeCount = scmClient.queryNode(STALE, + HddsProtos.QueryScope.CLUSTER, "").size(); assertEquals("Mismatch of expected nodes count", 2, nodeCount); GenericTestUtils.waitFor(() -> @@ -108,13 +107,13 @@ public class TestQueryNode { 100, 4 * 1000); // Assert that we don't find any stale nodes. - nodeCount = scmClient.queryNode(EnumSet.of(STALE), - HddsProtos.QueryScope.CLUSTER, "").getNodesCount(); + nodeCount = scmClient.queryNode(STALE, + HddsProtos.QueryScope.CLUSTER, "").size(); assertEquals("Mismatch of expected nodes count", 0, nodeCount); // Assert that we find the expected number of dead nodes. - nodeCount = scmClient.queryNode(EnumSet.of(DEAD), - HddsProtos.QueryScope.CLUSTER, "").getNodesCount(); + nodeCount = scmClient.queryNode(DEAD, + HddsProtos.QueryScope.CLUSTER, "").size(); assertEquals("Mismatch of expected nodes count", 2, nodeCount); } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java index dc8fc913c66..5fa313bbd35 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java @@ -78,7 +78,6 @@ import java.io.IOException; import java.io.PrintStream; import java.net.InetSocketAddress; import java.util.ArrayList; -import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -884,9 +883,8 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl .setValue(scmAddr.getPort()).build()); services.add(scmServiceInfoBuilder.build()); - List nodes = scmContainerClient.queryNode( - EnumSet.of(HEALTHY), HddsProtos.QueryScope.CLUSTER, "") - .getNodesList(); + List nodes = scmContainerClient.queryNode(HEALTHY, + HddsProtos.QueryScope.CLUSTER, ""); for (HddsProtos.Node node : nodes) { HddsProtos.DatanodeDetailsProto datanode = node.getNodeID(); From 17262470246232d0f0651d627a4961e55b1efe6a Mon Sep 17 00:00:00 2001 From: Haibo Chen Date: Thu, 5 Jul 2018 10:42:39 -0700 Subject: [PATCH 29/47] YARN-7556. Fair scheduler configuration should allow resource types in the minResources and maxResources properties. (Daniel Templeton and Szilard Nemeth via Haibo Chen) --- .../dev-support/findbugs-exclude.xml | 17 +- .../hadoop/yarn/api/records/Resource.java | 13 ++ .../api/records/impl/LightWeightResource.java | 23 ++- .../scheduler/fair/ConfigurableResource.java | 69 ++++++- .../fair/FairSchedulerConfiguration.java | 174 +++++++++++++++--- .../allocation/AllocationFileQueueParser.java | 2 +- .../fair/TestFairSchedulerConfiguration.java | 149 +++++++++++---- .../src/site/markdown/FairScheduler.md | 6 +- 8 files changed, 384 insertions(+), 69 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml index 58413618df0..5cc81e57ec6 100644 --- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml @@ -65,11 +65,6 @@ - - - - - @@ -118,6 +113,18 @@ + + + + + + + + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java index 71a6b54352b..173d4c9e777 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java @@ -30,6 +30,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; import org.apache.hadoop.yarn.api.records.impl.LightWeightResource; @@ -75,6 +76,18 @@ public abstract class Resource implements Comparable { @Private public static final int VCORES_INDEX = 1; + /** + * Return a new {@link Resource} instance with all resource values + * initialized to {@code value}. + * @param value the value to use for all resources + * @return a new {@link Resource} instance + */ + @Private + @Unstable + public static Resource newInstance(long value) { + return new LightWeightResource(value); + } + @Public @Stable public static Resource newInstance(int memory, int vCores) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java index a6e6432976f..77f77f312ce 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java @@ -18,9 +18,8 @@ package org.apache.hadoop.yarn.api.records.impl; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.util.resource.ResourceUtils; @@ -58,13 +57,29 @@ import static org.apache.hadoop.yarn.api.records.ResourceInformation.*; * * @see Resource */ -@InterfaceAudience.Private +@Private @Unstable public class LightWeightResource extends Resource { private ResourceInformation memoryResInfo; private ResourceInformation vcoresResInfo; + /** + * Create a new {@link LightWeightResource} instance with all resource values + * initialized to {@code value}. + * @param value the value to use for all resources + */ + public LightWeightResource(long value) { + ResourceInformation[] types = ResourceUtils.getResourceTypesArray(); + initResourceInformations(value, value, types.length); + + for (int i = 2; i < types.length; i++) { + resources[i] = new ResourceInformation(); + ResourceInformation.copy(types[i], resources[i]); + resources[i].setValue(value); + } + } + public LightWeightResource(long memory, int vcores) { int numberOfKnownResourceTypes = ResourceUtils .getNumberOfKnownResourceTypes(); @@ -91,7 +106,7 @@ public class LightWeightResource extends Resource { } } - private void initResourceInformations(long memory, int vcores, + private void initResourceInformations(long memory, long vcores, int numberOfKnownResourceTypes) { this.memoryResInfo = newDefaultInformation(MEMORY_URI, MEMORY_MB.getUnits(), memory); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java index ecdd0111a6b..0c3b0ddc6b6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java @@ -18,9 +18,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; +import java.util.Arrays; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; /** * A {@code ConfigurableResource} object represents an entity that is used to @@ -33,29 +37,53 @@ public class ConfigurableResource { private final Resource resource; private final double[] percentages; - public ConfigurableResource(double[] percentages) { + ConfigurableResource() { + this(getOneHundredPercentArray()); + } + + ConfigurableResource(double[] percentages) { this.percentages = percentages.clone(); this.resource = null; } + ConfigurableResource(long value) { + this(Resource.newInstance(value)); + } + public ConfigurableResource(Resource resource) { this.percentages = null; this.resource = resource; } + private static double[] getOneHundredPercentArray() { + double[] resourcePercentages = + new double[ResourceUtils.getNumberOfKnownResourceTypes()]; + Arrays.fill(resourcePercentages, 1.0); + + return resourcePercentages; + } + /** * Get resource by multiplying the cluster resource and the percentage of * each resource respectively. Return the absolute resource if either * {@code percentages} or {@code clusterResource} is null. * * @param clusterResource the cluster resource - * @return resource + * @return resource the resulting resource */ public Resource getResource(Resource clusterResource) { if (percentages != null && clusterResource != null) { long memory = (long) (clusterResource.getMemorySize() * percentages[0]); int vcore = (int) (clusterResource.getVirtualCores() * percentages[1]); - return Resource.newInstance(memory, vcore); + Resource res = Resource.newInstance(memory, vcore); + ResourceInformation[] clusterInfo = clusterResource.getResources(); + + for (int i = 2; i < clusterInfo.length; i++) { + res.setResourceValue(i, + (long)(clusterInfo[i].getValue() * percentages[i])); + } + + return res; } else { return resource; } @@ -69,4 +97,39 @@ public class ConfigurableResource { public Resource getResource() { return resource; } + + /** + * Set the value of the wrapped resource if this object isn't setup to use + * percentages. If this object is set to use percentages, this method has + * no effect. + * + * @param name the name of the resource + * @param value the value + */ + void setValue(String name, long value) { + if (resource != null) { + resource.setResourceValue(name, value); + } + } + + /** + * Set the percentage of the resource if this object is setup to use + * percentages. If this object is set to use percentages, this method has + * no effect. + * + * @param name the name of the resource + * @param value the percentage + */ + void setPercentage(String name, double value) { + if (percentages != null) { + Integer index = ResourceUtils.getResourceTypeIndex().get(name); + + if (index != null) { + percentages[index] = value; + } else { + throw new ResourceNotFoundException("The requested resource, \"" + + name + "\", could not be found."); + } + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java index b50e4bbe317..8c4932bfe67 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java @@ -33,6 +33,7 @@ import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.UnitsConversionUtil; import org.apache.hadoop.yarn.util.resource.ResourceUtils; @@ -213,6 +214,9 @@ public class FairSchedulerConfiguration extends Configuration { CONF_PREFIX + "reservable-nodes"; public static final float RESERVABLE_NODES_DEFAULT = 0.05f; + private static final String INVALID_RESOURCE_DEFINITION_PREFIX = + "Error reading resource config--invalid resource definition: "; + public FairSchedulerConfiguration() { super(); } @@ -407,54 +411,167 @@ public class FairSchedulerConfiguration extends Configuration { } /** - * Parses a resource config value of a form like "1024", "1024 mb", - * or "1024 mb, 3 vcores". If no units are given, megabytes are assumed. - * - * @throws AllocationConfigurationException + * Parses a resource config value in one of three forms: + *

    + *
  1. Percentage: "50%" or "40% memory, 60% cpu"
  2. + *
  3. New style resources: "vcores=10, memory-mb=1024" + * or "vcores=60%, memory-mb=40%"
  4. + *
  5. Old style resources: "1024 mb, 10 vcores"
  6. + *
+ * In new style resources, any resource that is not specified will be + * set to {@link Long#MAX_VALUE} or 100%, as appropriate. Also, in the new + * style resources, units are not allowed. Units are assumed from the resource + * manager's settings for the resources when the value isn't a percentage. + * + * @param value the resource definition to parse + * @return a {@link ConfigurableResource} that represents the parsed value + * @throws AllocationConfigurationException if the raw value is not a valid + * resource definition */ - public static ConfigurableResource parseResourceConfigValue(String val) + public static ConfigurableResource parseResourceConfigValue(String value) throws AllocationConfigurationException { + return parseResourceConfigValue(value, Long.MAX_VALUE); + } + + /** + * Parses a resource config value in one of three forms: + *
    + *
  1. Percentage: "50%" or "40% memory, 60% cpu"
  2. + *
  3. New style resources: "vcores=10, memory-mb=1024" + * or "vcores=60%, memory-mb=40%"
  4. + *
  5. Old style resources: "1024 mb, 10 vcores"
  6. + *
+ * In new style resources, any resource that is not specified will be + * set to {@code missing} or 0%, as appropriate. Also, in the new style + * resources, units are not allowed. Units are assumed from the resource + * manager's settings for the resources when the value isn't a percentage. + * + * The {@code missing} parameter is only used in the case of new style + * resources without percentages. With new style resources with percentages, + * any missing resources will be assumed to be 100% because percentages are + * only used with maximum resource limits. + * + * @param value the resource definition to parse + * @param missing the value to use for any unspecified resources + * @return a {@link ConfigurableResource} that represents the parsed value + * @throws AllocationConfigurationException if the raw value is not a valid + * resource definition + */ + public static ConfigurableResource parseResourceConfigValue(String value, + long missing) throws AllocationConfigurationException { ConfigurableResource configurableResource; + + if (value.trim().isEmpty()) { + throw new AllocationConfigurationException("Error reading resource " + + "config--the resource string is empty."); + } + try { - val = StringUtils.toLowerCase(val); - if (val.contains("%")) { - configurableResource = new ConfigurableResource( - getResourcePercentage(val)); + if (value.contains("=")) { + configurableResource = parseNewStyleResource(value, missing); + } else if (value.contains("%")) { + configurableResource = parseOldStyleResourceAsPercentage(value); } else { - int memory = findResource(val, "mb"); - int vcores = findResource(val, "vcores"); - configurableResource = new ConfigurableResource( - BuilderUtils.newResource(memory, vcores)); + configurableResource = parseOldStyleResource(value); } - } catch (AllocationConfigurationException ex) { - throw ex; - } catch (Exception ex) { + } catch (RuntimeException ex) { throw new AllocationConfigurationException( "Error reading resource config", ex); } + return configurableResource; } + private static ConfigurableResource parseNewStyleResource(String value, + long missing) throws AllocationConfigurationException { + + final ConfigurableResource configurableResource; + boolean asPercent = value.contains("%"); + if (asPercent) { + configurableResource = new ConfigurableResource(); + } else { + configurableResource = new ConfigurableResource(missing); + } + + String[] resources = value.split(","); + for (String resource : resources) { + String[] parts = resource.split("="); + + if (parts.length != 2) { + throw createConfigException(value, + "Every resource must be of the form: name=value."); + } + + String resourceName = parts[0].trim(); + String resourceValue = parts[1].trim(); + try { + if (asPercent) { + configurableResource.setPercentage(resourceName, + findPercentage(resourceValue, "")); + } else { + configurableResource.setValue(resourceName, + Long.parseLong(resourceValue)); + } + } catch (ResourceNotFoundException ex) { + throw createConfigException(value, "The " + + "resource name, \"" + resourceName + "\" was not " + + "recognized. Please check the value of " + + YarnConfiguration.RESOURCE_TYPES + " in the Resource " + + "Manager's configuration files.", ex); + } catch (NumberFormatException ex) { + // This only comes from Long.parseLong() + throw createConfigException(value, "The " + + "resource values must all be integers. \"" + resourceValue + + "\" is not an integer.", ex); + } catch (AllocationConfigurationException ex) { + // This only comes from findPercentage() + throw createConfigException(value, "The " + + "resource values must all be percentages. \"" + + resourceValue + "\" is either not a number or does not " + + "include the '%' symbol.", ex); + } + } + return configurableResource; + } + + private static ConfigurableResource parseOldStyleResourceAsPercentage( + String value) throws AllocationConfigurationException { + return new ConfigurableResource( + getResourcePercentage(StringUtils.toLowerCase(value))); + } + + private static ConfigurableResource parseOldStyleResource(String value) + throws AllocationConfigurationException { + final String lCaseValue = StringUtils.toLowerCase(value); + int memory = findResource(lCaseValue, "mb"); + int vcores = findResource(lCaseValue, "vcores"); + + return new ConfigurableResource( + BuilderUtils.newResource(memory, vcores)); + } + private static double[] getResourcePercentage( String val) throws AllocationConfigurationException { int numberOfKnownResourceTypes = ResourceUtils .getNumberOfKnownResourceTypes(); double[] resourcePercentage = new double[numberOfKnownResourceTypes]; String[] strings = val.split(","); + if (strings.length == 1) { double percentage = findPercentage(strings[0], ""); for (int i = 0; i < numberOfKnownResourceTypes; i++) { - resourcePercentage[i] = percentage/100; + resourcePercentage[i] = percentage; } } else { - resourcePercentage[0] = findPercentage(val, "memory")/100; - resourcePercentage[1] = findPercentage(val, "cpu")/100; + resourcePercentage[0] = findPercentage(val, "memory"); + resourcePercentage[1] = findPercentage(val, "cpu"); } + return resourcePercentage; } private static double findPercentage(String val, String units) - throws AllocationConfigurationException { + throws AllocationConfigurationException { final Pattern pattern = Pattern.compile("((\\d+)(\\.\\d*)?)\\s*%\\s*" + units); Matcher matcher = pattern.matcher(val); @@ -467,7 +584,22 @@ public class FairSchedulerConfiguration extends Configuration { units); } } - return Double.parseDouble(matcher.group(1)); + return Double.parseDouble(matcher.group(1)) / 100.0; + } + + private static AllocationConfigurationException createConfigException( + String value, String message) { + return createConfigException(value, message, null); + } + + private static AllocationConfigurationException createConfigException( + String value, String message, Throwable t) { + String msg = INVALID_RESOURCE_DEFINITION_PREFIX + value + ". " + message; + if (t != null) { + return new AllocationConfigurationException(msg, t); + } else { + return new AllocationConfigurationException(msg); + } } public long getUpdateInterval() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java index d5a436ed0a9..441c34a1aa1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java @@ -134,7 +134,7 @@ public class AllocationFileQueueParser { if (MIN_RESOURCES.equals(field.getTagName())) { String text = getTrimmedTextData(field); ConfigurableResource val = - FairSchedulerConfiguration.parseResourceConfigValue(text); + FairSchedulerConfiguration.parseResourceConfigValue(text, 0L); builder.minQueueResources(queueName, val.getResource()); } else if (MAX_RESOURCES.equals(field.getTagName())) { String text = getTrimmedTextData(field); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java index 481645bb494..76a5af50960 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java @@ -102,60 +102,145 @@ public class TestFairSchedulerConfiguration { @Test public void testParseResourceConfigValue() throws Exception { - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("2 vcores, 1024 mb").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("1024 mb, 2 vcores").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("2vcores,1024mb").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("1024mb,2vcores").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("1024 mb, 2 vcores").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("1024 Mb, 2 vCores").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue(" 1024 mb, 2 vcores ").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue(" 1024.3 mb, 2.35 vcores ").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue(" 1024. mb, 2. vcores ").getResource()); + Resource expected = BuilderUtils.newResource(5 * 1024, 2); + Resource clusterResource = BuilderUtils.newResource(10 * 1024, 4); - Resource clusterResource = BuilderUtils.newResource(2048, 4); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, + parseResourceConfigValue("2 vcores, 5120 mb").getResource()); + assertEquals(expected, + parseResourceConfigValue("5120 mb, 2 vcores").getResource()); + assertEquals(expected, + parseResourceConfigValue("2vcores,5120mb").getResource()); + assertEquals(expected, + parseResourceConfigValue("5120mb,2vcores").getResource()); + assertEquals(expected, + parseResourceConfigValue("5120mb mb, 2 vcores").getResource()); + assertEquals(expected, + parseResourceConfigValue("5120 Mb, 2 vCores").getResource()); + assertEquals(expected, + parseResourceConfigValue(" 5120 mb, 2 vcores ").getResource()); + assertEquals(expected, + parseResourceConfigValue(" 5120.3 mb, 2.35 vcores ").getResource()); + assertEquals(expected, + parseResourceConfigValue(" 5120. mb, 2. vcores ").getResource()); + + assertEquals(expected, parseResourceConfigValue("50% memory, 50% cpu"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, parseResourceConfigValue("50% Memory, 50% CpU"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("50%").getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 4), + assertEquals(BuilderUtils.newResource(5 * 1024, 4), parseResourceConfigValue("50% memory, 100% cpu"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 4), + assertEquals(BuilderUtils.newResource(5 * 1024, 4), parseResourceConfigValue(" 100% cpu, 50% memory"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 0), + assertEquals(BuilderUtils.newResource(5 * 1024, 0), parseResourceConfigValue("50% memory, 0% cpu"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, parseResourceConfigValue("50 % memory, 50 % cpu"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, parseResourceConfigValue("50%memory,50%cpu"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, parseResourceConfigValue(" 50 % memory, 50 % cpu "). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, parseResourceConfigValue("50.% memory, 50.% cpu"). getResource(clusterResource)); - - clusterResource = BuilderUtils.newResource(1024 * 10, 4); assertEquals(BuilderUtils.newResource((int)(1024 * 10 * 0.109), 2), parseResourceConfigValue("10.9% memory, 50.6% cpu"). getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("50%").getResource(clusterResource)); + + Configuration conf = new Configuration(); + + conf.set(YarnConfiguration.RESOURCE_TYPES, "test1"); + ResourceUtils.resetResourceTypes(conf); + + clusterResource = BuilderUtils.newResource(10 * 1024, 4); + expected = BuilderUtils.newResource(5 * 1024, 2); + expected.setResourceValue("test1", Long.MAX_VALUE); + + assertEquals(expected, + parseResourceConfigValue("vcores=2, memory-mb=5120").getResource()); + assertEquals(expected, + parseResourceConfigValue("memory-mb=5120, vcores=2").getResource()); + assertEquals(expected, + parseResourceConfigValue("vcores=2,memory-mb=5120").getResource()); + assertEquals(expected, parseResourceConfigValue(" vcores = 2 , " + + "memory-mb = 5120 ").getResource()); + + expected.setResourceValue("test1", 0L); + + assertEquals(expected, + parseResourceConfigValue("vcores=2, memory-mb=5120", 0L).getResource()); + assertEquals(expected, + parseResourceConfigValue("memory-mb=5120, vcores=2", 0L).getResource()); + assertEquals(expected, + parseResourceConfigValue("vcores=2,memory-mb=5120", 0L).getResource()); + assertEquals(expected, + parseResourceConfigValue(" vcores = 2 , memory-mb = 5120 ", + 0L).getResource()); + + clusterResource.setResourceValue("test1", 8L); + expected.setResourceValue("test1", 4L); + + assertEquals(expected, + parseResourceConfigValue("50%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("vcores=2, memory-mb=5120, " + + "test1=4").getResource()); + assertEquals(expected, + parseResourceConfigValue("test1=4, vcores=2, " + + "memory-mb=5120").getResource()); + assertEquals(expected, + parseResourceConfigValue("memory-mb=5120, test1=4, " + + "vcores=2").getResource()); + assertEquals(expected, + parseResourceConfigValue("vcores=2,memory-mb=5120," + + "test1=4").getResource()); + assertEquals(expected, + parseResourceConfigValue(" vcores = 2 , memory-mb = 5120 , " + + "test1 = 4 ").getResource()); + + expected = BuilderUtils.newResource(4 * 1024, 3); + expected.setResourceValue("test1", 8L); + + assertEquals(expected, + parseResourceConfigValue("vcores=75%, " + + "memory-mb=40%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("memory-mb=40%, " + + "vcores=75%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("vcores=75%," + + "memory-mb=40%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue(" vcores = 75 % , " + + "memory-mb = 40 % ").getResource(clusterResource)); + + expected.setResourceValue("test1", 4L); + + assertEquals(expected, + parseResourceConfigValue("vcores=75%, memory-mb=40%, " + + "test1=50%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("test1=50%, vcores=75%, " + + "memory-mb=40%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("memory-mb=40%, test1=50%, " + + "vcores=75%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("vcores=75%,memory-mb=40%," + + "test1=50%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue(" vcores = 75 % , memory-mb = 40 % , " + + "test1 = 50 % ").getResource(clusterResource)); } @Test(expected = AllocationConfigurationException.class) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md index 269f5b40f49..b5bcbf5c8e9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md @@ -86,11 +86,11 @@ The allocation file must be in XML format. The format contains five types of ele * **Queue elements**: which represent queues. Queue elements can take an optional attribute 'type', which when set to 'parent' makes it a parent queue. This is useful when we want to create a parent queue without configuring any leaf queues. Each queue element may contain the following properties: - * **minResources**: minimum resources the queue is entitled to, in the form "X mb, Y vcores". For the single-resource fairness policy, the vcores value is ignored. If a queue's minimum share is not satisfied, it will be offered available resources before any other queue under the same parent. Under the single-resource fairness policy, a queue is considered unsatisfied if its memory usage is below its minimum memory share. Under dominant resource fairness, a queue is considered unsatisfied if its usage for its dominant resource with respect to the cluster capacity is below its minimum share for that resource. If multiple queues are unsatisfied in this situation, resources go to the queue with the smallest ratio between relevant resource usage and minimum. Note that it is possible that a queue that is below its minimum may not immediately get up to its minimum when it submits an application, because already-running jobs may be using those resources. + * **minResources**: minimum resources the queue is entitled to, in the form of "X mb, Y vcores" or "vcores=X, memory-mb=Y". The latter form is required when specifying resources other than memory and CPU. For the single-resource fairness policy, the vcores value is ignored. If a queue's minimum share is not satisfied, it will be offered available resources before any other queue under the same parent. Under the single-resource fairness policy, a queue is considered unsatisfied if its memory usage is below its minimum memory share. Under dominant resource fairness, a queue is considered unsatisfied if its usage for its dominant resource with respect to the cluster capacity is below its minimum share for that resource. If multiple queues are unsatisfied in this situation, resources go to the queue with the smallest ratio between relevant resource usage and its minimum. Note that it is possible for a queue that is below its minimum to not immediately get up to its minimum when an application is submitted to the queue, because already-running jobs may be using those resources. - * **maxResources**: maximum resources a queue is allocated, expressed either in absolute values (X mb, Y vcores) or as a percentage of the cluster resources (X% memory, Y% cpu). A queue will not be assigned a container that would put its aggregate usage over this limit. + * **maxResources**: maximum resources a queue will allocated, expressed in the form of "X%", "X% cpu, Y% memory", "X mb, Y vcores", or "vcores=X, memory-mb=Y". The last form is required when specifying resources other than memory and CPU. In the last form, X and Y can either be a percentage or an integer resource value without units. In the latter case the units will be inferred from the default units configured for that resource. A queue will not be assigned a container that would put its aggregate usage over this limit. - * **maxChildResources**: maximum resources an ad hoc child queue is allocated, expressed either in absolute values (X mb, Y vcores) or as a percentage of the cluster resources (X% memory, Y% cpu). An ad hoc child queue will not be assigned a container that would put its aggregate usage over this limit. + * **maxChildResources**: maximum resources an ad hoc child queue will allocated, expressed in the form of "X%", "X% cpu, Y% memory", "X mb, Y vcores", or "vcores=X, memory-mb=Y". The last form is required when specifying resources other than memory and CPU. In the last form, X and Y can either be a percentage or an integer resource value without units. In the latter case the units will be inferred from the default units configured for that resource. An ad hoc child queue will not be assigned a container that would put its aggregate usage over this limit. * **maxRunningApps**: limit the number of apps from the queue to run at once From 99febe7fd50c31c0f5dd40fa7f376f2c1f64f8c3 Mon Sep 17 00:00:00 2001 From: Robert Kanter Date: Thu, 5 Jul 2018 10:54:19 -0700 Subject: [PATCH 30/47] YARN-7451. Add missing tests to verify the presence of custom resources of RM apps and scheduler webservice endpoints (snemeth via rkanter) --- .../resourcemanager/webapp/dao/AppInfo.java | 2 +- .../webapp/dao/SchedulerInfo.java | 8 +- .../fair/TestFairSchedulerConfiguration.java | 9 +- .../webapp/TestRMWebServices.java | 31 +- .../webapp/TestRMWebServicesApps.java | 14 +- ...tRMWebServicesAppsCustomResourceTypes.java | 242 ++++++++++++++++ .../TestRMWebServicesCapacitySched.java | 30 +- ...estRMWebServicesConfigurationMutation.java | 5 + .../TestRMWebServicesFairScheduler.java | 95 +++--- .../TestRMWebServicesSchedulerActivities.java | 2 +- ...tomResourceTypesConfigurationProvider.java | 138 +++++++++ .../FairSchedulerJsonVerifications.java | 139 +++++++++ .../FairSchedulerXmlVerifications.java | 153 ++++++++++ ...vicesFairSchedulerCustomResourceTypes.java | 271 ++++++++++++++++++ .../helper/AppInfoJsonVerifications.java | 123 ++++++++ .../helper/AppInfoXmlVerifications.java | 132 +++++++++ .../webapp/helper/BufferedClientResponse.java | 57 ++++ .../JsonCustomResourceTypeTestcase.java | 77 +++++ .../ResourceRequestsJsonVerifications.java | 252 ++++++++++++++++ .../ResourceRequestsXmlVerifications.java | 215 ++++++++++++++ .../helper/XmlCustomResourceTypeTestCase.java | 112 ++++++++ 21 files changed, 2020 insertions(+), 87 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java index d47f13d9586..9d82bc78c76 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java @@ -479,7 +479,7 @@ public class AppInfo { public int getNumNonAMContainersPreempted() { return numNonAMContainerPreempted; } - + public int getNumAMContainersPreempted() { return numAMContainerPreempted; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java index 81491b14ce1..163f707253a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java @@ -41,8 +41,9 @@ public class SchedulerInfo { protected EnumSet schedulingResourceTypes; protected int maximumClusterPriority; + // JAXB needs this public SchedulerInfo() { - } // JAXB needs this + } public SchedulerInfo(final ResourceManager rm) { ResourceScheduler rs = rm.getResourceScheduler(); @@ -74,7 +75,10 @@ public class SchedulerInfo { } public String getSchedulerResourceTypes() { - return Arrays.toString(minAllocResource.getResource().getResources()); + if (minAllocResource != null) { + return Arrays.toString(minAllocResource.getResource().getResources()); + } + return null; } public int getMaxClusterLevelAppPriority() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java index 76a5af50960..70f83ab3095 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java @@ -48,6 +48,9 @@ import org.apache.log4j.spi.LoggingEvent; import org.junit.Assert; import org.junit.Test; +/** + * Tests fair scheduler configuration. + */ public class TestFairSchedulerConfiguration { private static final String A_CUSTOM_RESOURCE = "a-custom-resource"; @@ -242,12 +245,12 @@ public class TestFairSchedulerConfiguration { parseResourceConfigValue(" vcores = 75 % , memory-mb = 40 % , " + "test1 = 50 % ").getResource(clusterResource)); } - + @Test(expected = AllocationConfigurationException.class) public void testNoUnits() throws Exception { parseResourceConfigValue("1024"); } - + @Test(expected = AllocationConfigurationException.class) public void testOnlyMemory() throws Exception { parseResourceConfigValue("1024mb"); @@ -257,7 +260,7 @@ public class TestFairSchedulerConfiguration { public void testOnlyCPU() throws Exception { parseResourceConfigValue("1024vcores"); } - + @Test(expected = AllocationConfigurationException.class) public void testGibberish() throws Exception { parseResourceConfigValue("1o24vc0res"); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java index 0702d652a02..3902889a4a1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java @@ -53,11 +53,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService; -import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics; -import org.apache.hadoop.yarn.server.resourcemanager.MockRM; -import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; -import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.*; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; @@ -76,11 +72,12 @@ import org.apache.hadoop.yarn.webapp.JerseyTestBase; import org.apache.hadoop.yarn.webapp.WebServicesTestUtils; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; -import org.eclipse.jetty.server.Response; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -96,6 +93,8 @@ import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; import com.sun.jersey.test.framework.WebAppDescriptor; public class TestRMWebServices extends JerseyTestBase { + private static final Logger LOG = + LoggerFactory.getLogger(TestRMWebServices.class); private static MockRM rm; @@ -472,19 +471,19 @@ public class TestRMWebServices extends JerseyTestBase { QueueMetrics metrics = rs.getRootQueueMetrics(); ClusterMetrics clusterMetrics = ClusterMetrics.getMetrics(); - long totalMBExpect = + long totalMBExpect = metrics.getAvailableMB() + metrics.getAllocatedMB(); - long totalVirtualCoresExpect = + long totalVirtualCoresExpect = metrics.getAvailableVirtualCores() + metrics.getAllocatedVirtualCores(); - assertEquals("appsSubmitted doesn't match", + assertEquals("appsSubmitted doesn't match", metrics.getAppsSubmitted(), submittedApps); - assertEquals("appsCompleted doesn't match", + assertEquals("appsCompleted doesn't match", metrics.getAppsCompleted(), completedApps); assertEquals("reservedMB doesn't match", metrics.getReservedMB(), reservedMB); - assertEquals("availableMB doesn't match", + assertEquals("availableMB doesn't match", metrics.getAvailableMB(), availableMB); - assertEquals("allocatedMB doesn't match", + assertEquals("allocatedMB doesn't match", metrics.getAllocatedMB(), allocMB); assertEquals("reservedVirtualCores doesn't match", metrics.getReservedVirtualCores(), reservedVirtualCores); @@ -597,11 +596,13 @@ public class TestRMWebServices extends JerseyTestBase { public void verifyClusterSchedulerFifo(JSONObject json) throws JSONException, Exception { - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals("incorrect number of elements in: " + json, 1, json.length()); JSONObject info = json.getJSONObject("scheduler"); - assertEquals("incorrect number of elements", 1, info.length()); + assertEquals("incorrect number of elements in: " + info, 1, info.length()); info = info.getJSONObject("schedulerInfo"); - assertEquals("incorrect number of elements", 11, info.length()); + + LOG.debug("schedulerInfo: {}", info); + assertEquals("incorrect number of elements in: " + info, 11, info.length()); verifyClusterSchedulerFifoGeneric(info.getString("type"), info.getString("qstate"), (float) info.getDouble("capacity"), diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java index 6c6f400a623..15f94e1bb00 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java @@ -79,7 +79,7 @@ import com.sun.jersey.test.framework.WebAppDescriptor; public class TestRMWebServicesApps extends JerseyTestBase { private static MockRM rm; - + private static final int CONTAINER_MB = 1024; private static class WebServletModule extends ServletModule { @@ -324,7 +324,7 @@ public class TestRMWebServicesApps extends JerseyTestBase { assertEquals("incorrect number of elements", 1, apps.length()); array = apps.getJSONArray("app"); assertEquals("incorrect number of elements", 2, array.length()); - assertTrue("both app states of ACCEPTED and KILLED are not present", + assertTrue("both app states of ACCEPTED and KILLED are not present", (array.getJSONObject(0).getString("state").equals("ACCEPTED") && array.getJSONObject(1).getString("state").equals("KILLED")) || (array.getJSONObject(0).getString("state").equals("KILLED") && @@ -375,12 +375,12 @@ public class TestRMWebServicesApps extends JerseyTestBase { assertEquals("incorrect number of elements", 1, apps.length()); array = apps.getJSONArray("app"); assertEquals("incorrect number of elements", 2, array.length()); - assertTrue("both app states of ACCEPTED and KILLED are not present", + assertTrue("both app states of ACCEPTED and KILLED are not present", (array.getJSONObject(0).getString("state").equals("ACCEPTED") && array.getJSONObject(1).getString("state").equals("KILLED")) || (array.getJSONObject(0).getString("state").equals("KILLED") && array.getJSONObject(1).getString("state").equals("ACCEPTED"))); - + rm.stop(); } @@ -511,7 +511,8 @@ public class TestRMWebServicesApps extends JerseyTestBase { WebResource r = resource(); ClientResponse response = r.path("ws").path("v1").path("cluster") - .path("apps").queryParam("finalStatus", FinalApplicationStatus.UNDEFINED.toString()) + .path("apps").queryParam("finalStatus", + FinalApplicationStatus.UNDEFINED.toString()) .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); @@ -1804,7 +1805,8 @@ public class TestRMWebServicesApps extends JerseyTestBase { int numAttempt = 1; while (true) { // fail the AM by sending CONTAINER_FINISHED event without registering. - amNodeManager.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE); + amNodeManager.nodeHeartbeat(am.getApplicationAttemptId(), 1, + ContainerState.COMPLETE); rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FAILED); if (numAttempt == maxAppAttempts) { rm.waitForState(app1.getApplicationId(), RMAppState.FAILED); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java new file mode 100644 index 00000000000..83e00567eca --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java @@ -0,0 +1,242 @@ +/** + * 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.yarn.server.resourcemanager.webapp; + +import com.google.inject.Guice; +import com.google.inject.servlet.ServletModule; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; +import com.sun.jersey.test.framework.WebAppDescriptor; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.MockAM; +import org.apache.hadoop.yarn.server.resourcemanager.MockNM; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.fairscheduler.CustomResourceTypesConfigurationProvider; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.AppInfoJsonVerifications; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.AppInfoXmlVerifications; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.BufferedClientResponse; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.JsonCustomResourceTypeTestcase; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.ResourceRequestsJsonVerifications; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.ResourceRequestsXmlVerifications; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.XmlCustomResourceTypeTestCase; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; +import org.apache.hadoop.yarn.webapp.GuiceServletConfig; +import org.apache.hadoop.yarn.webapp.JerseyTestBase; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; +import org.junit.Before; +import org.junit.Test; +import org.w3c.dom.Element; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; + +import javax.ws.rs.core.MediaType; +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +/** + * This test verifies that custom resource types are correctly serialized to XML + * and JSON when HTTP GET request is sent to the resource: ws/v1/cluster/apps. + */ +public class TestRMWebServicesAppsCustomResourceTypes extends JerseyTestBase { + + private static MockRM rm; + private static final int CONTAINER_MB = 1024; + + private static class WebServletModule extends ServletModule { + @Override + protected void configureServlets() { + bind(JAXBContextResolver.class); + bind(RMWebServices.class); + bind(GenericExceptionHandler.class); + Configuration conf = new Configuration(); + conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); + conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class, + ResourceScheduler.class); + initResourceTypes(conf); + rm = new MockRM(conf); + bind(ResourceManager.class).toInstance(rm); + serve("/*").with(GuiceContainer.class); + } + + private void initResourceTypes(Configuration conf) { + conf.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS, + CustomResourceTypesConfigurationProvider.class.getName()); + ResourceUtils.resetResourceTypes(conf); + } + } + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + createInjectorForWebServletModule(); + } + + private void createInjectorForWebServletModule() { + GuiceServletConfig + .setInjector(Guice.createInjector(new WebServletModule())); + } + + public TestRMWebServicesAppsCustomResourceTypes() { + super(new WebAppDescriptor.Builder( + "org.apache.hadoop.yarn.server.resourcemanager.webapp") + .contextListenerClass(GuiceServletConfig.class) + .filterClass(com.google.inject.servlet.GuiceFilter.class) + .contextPath("jersey-guice-filter").servletPath("/").build()); + } + + @Test + public void testRunningAppXml() throws Exception { + rm.start(); + MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048); + RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, amNodeManager); + am1.allocate("*", 2048, 1, new ArrayList<>()); + amNodeManager.nodeHeartbeat(true); + + WebResource r = resource(); + WebResource path = r.path("ws").path("v1").path("cluster").path("apps"); + ClientResponse response = + path.accept(MediaType.APPLICATION_XML).get(ClientResponse.class); + + XmlCustomResourceTypeTestCase testCase = + new XmlCustomResourceTypeTestCase(path, + new BufferedClientResponse(response)); + testCase.verify(document -> { + NodeList apps = document.getElementsByTagName("apps"); + assertEquals("incorrect number of apps elements", 1, apps.getLength()); + + NodeList appArray = ((Element)(apps.item(0))) + .getElementsByTagName("app"); + assertEquals("incorrect number of app elements", 1, appArray.getLength()); + + verifyAppsXML(appArray, app1); + }); + + rm.stop(); + } + + @Test + public void testRunningAppJson() throws Exception { + rm.start(); + MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048); + RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, amNodeManager); + am1.allocate("*", 2048, 1, new ArrayList<>()); + amNodeManager.nodeHeartbeat(true); + + WebResource r = resource(); + WebResource path = r.path("ws").path("v1").path("cluster").path("apps"); + ClientResponse response = + path.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + + JsonCustomResourceTypeTestcase testCase = + new JsonCustomResourceTypeTestcase(path, + new BufferedClientResponse(response)); + testCase.verify(json -> { + try { + assertEquals("incorrect number of apps elements", 1, json.length()); + JSONObject apps = json.getJSONObject("apps"); + assertEquals("incorrect number of app elements", 1, apps.length()); + JSONArray array = apps.getJSONArray("app"); + assertEquals("incorrect count of app", 1, array.length()); + + verifyAppInfoJson(array.getJSONObject(0), app1); + } catch (JSONException e) { + throw new RuntimeException(e); + } + }); + + rm.stop(); + } + + private void verifyAppsXML(NodeList appArray, RMApp app) { + for (int i = 0; i < appArray.getLength(); i++) { + Element element = (Element) appArray.item(i); + AppInfoXmlVerifications.verify(element, app); + + NodeList resourceRequests = + element.getElementsByTagName("resourceRequests"); + assertEquals(1, resourceRequests.getLength()); + Node resourceRequest = resourceRequests.item(0); + ResourceRequest rr = + ((AbstractYarnScheduler) rm.getRMContext().getScheduler()) + .getApplicationAttempt( + app.getCurrentAppAttempt().getAppAttemptId()) + .getAppSchedulingInfo().getAllResourceRequests().get(0); + ResourceRequestsXmlVerifications.verifyWithCustomResourceTypes( + (Element) resourceRequest, rr, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + } + + private void verifyAppInfoJson(JSONObject info, RMApp app) throws + JSONException { + int expectedNumberOfElements = getExpectedNumberOfElements(app); + + assertEquals("incorrect number of elements", expectedNumberOfElements, + info.length()); + + AppInfoJsonVerifications.verify(info, app); + + JSONArray resourceRequests = info.getJSONArray("resourceRequests"); + JSONObject requestInfo = resourceRequests.getJSONObject(0); + ResourceRequest rr = + ((AbstractYarnScheduler) rm.getRMContext().getScheduler()) + .getApplicationAttempt(app.getCurrentAppAttempt().getAppAttemptId()) + .getAppSchedulingInfo().getAllResourceRequests().get(0); + + ResourceRequestsJsonVerifications.verifyWithCustomResourceTypes( + requestInfo, rr, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + + private int getExpectedNumberOfElements(RMApp app) { + int expectedNumberOfElements = 40 + 2; // 2 -> resourceRequests + if (app.getApplicationSubmissionContext() + .getNodeLabelExpression() != null) { + expectedNumberOfElements++; + } + + if (app.getAMResourceRequests().get(0).getNodeLabelExpression() != null) { + expectedNumberOfElements++; + } + + if (AppInfo + .getAmRPCAddressFromRMAppAttempt(app.getCurrentAppAttempt()) != null) { + expectedNumberOfElements++; + } + return expectedNumberOfElements; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java index e37f76fa25f..46d0a6614fa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java @@ -146,7 +146,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { config.setUserLimitFactor(B2, 100.0f); config.setCapacity(B3, 0.5f); config.setUserLimitFactor(B3, 100.0f); - + config.setQueues(A1, new String[] {"a1a", "a1b"}); final String A1A = A1 + ".a1a"; config.setCapacity(A1A, 85); @@ -254,7 +254,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { } } - public void verifySubQueueXML(Element qElem, String q, + public void verifySubQueueXML(Element qElem, String q, float parentAbsCapacity, float parentAbsMaxCapacity) throws Exception { NodeList children = qElem.getChildNodes(); @@ -317,30 +317,34 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { private void verifyClusterScheduler(JSONObject json) throws JSONException, Exception { - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals("incorrect number of elements in: " + json, 1, json.length()); JSONObject info = json.getJSONObject("scheduler"); - assertEquals("incorrect number of elements", 1, info.length()); + assertEquals("incorrect number of elements in: " + info, 1, info.length()); info = info.getJSONObject("schedulerInfo"); - assertEquals("incorrect number of elements", 8, info.length()); + assertEquals("incorrect number of elements in: " + info, 8, info.length()); verifyClusterSchedulerGeneric(info.getString("type"), (float) info.getDouble("usedCapacity"), (float) info.getDouble("capacity"), (float) info.getDouble("maxCapacity"), info.getString("queueName")); JSONObject health = info.getJSONObject("health"); assertNotNull(health); - assertEquals("incorrect number of elements", 3, health.length()); + assertEquals("incorrect number of elements in: " + health, 3, + health.length()); JSONArray operationsInfo = health.getJSONArray("operationsInfo"); - assertEquals("incorrect number of elements", 4, operationsInfo.length()); + assertEquals("incorrect number of elements in: " + health, 4, + operationsInfo.length()); JSONArray lastRunDetails = health.getJSONArray("lastRunDetails"); - assertEquals("incorrect number of elements", 3, lastRunDetails.length()); + assertEquals("incorrect number of elements in: " + health, 3, + lastRunDetails.length()); JSONArray arr = info.getJSONObject("queues").getJSONArray("queue"); - assertEquals("incorrect number of elements", 2, arr.length()); + assertEquals("incorrect number of elements in: " + arr, 2, arr.length()); // test subqueues for (int i = 0; i < arr.length(); i++) { JSONObject obj = arr.getJSONObject(i); - String q = CapacitySchedulerConfiguration.ROOT + "." + obj.getString("queueName"); + String q = CapacitySchedulerConfiguration.ROOT + "." + + obj.getString("queueName"); verifySubQueue(obj, q, 100, 100); } } @@ -355,7 +359,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { assertTrue("queueName doesn't match", "root".matches(queueName)); } - private void verifySubQueue(JSONObject info, String q, + private void verifySubQueue(JSONObject info, String q, float parentAbsCapacity, float parentAbsMaxCapacity) throws JSONException, Exception { int numExpectedElements = 20; @@ -464,7 +468,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { csConf.getUserLimitFactor(q), info.userLimitFactor, 1e-3f); } - //Return a child Node of node with the tagname or null if none exists + //Return a child Node of node with the tagname or null if none exists private Node getChildNodeByName(Node node, String tagname) { NodeList nodeList = node.getChildNodes(); for (int i=0; i < nodeList.getLength(); ++i) { @@ -514,7 +518,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { for (int j=0; j + * 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. @@ -16,13 +16,14 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.server.resourcemanager.webapp; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -import javax.ws.rs.core.MediaType; +package org.apache.hadoop.yarn.server.resourcemanager.webapp.fairscheduler; +import com.google.inject.Guice; +import com.google.inject.servlet.ServletModule; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; +import com.sun.jersey.test.framework.WebAppDescriptor; import org.apache.hadoop.http.JettyUtils; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.MockRM; @@ -30,6 +31,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueueManager; + +import org.apache.hadoop.yarn.server.resourcemanager.webapp.JAXBContextResolver; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServices; import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; import org.apache.hadoop.yarn.webapp.GuiceServletConfig; import org.apache.hadoop.yarn.webapp.JerseyTestBase; @@ -38,18 +42,18 @@ import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; import org.junit.Before; import org.junit.Test; +import javax.ws.rs.core.MediaType; -import com.google.inject.Guice; -import com.google.inject.servlet.ServletModule; -import com.sun.jersey.api.client.ClientResponse; -import com.sun.jersey.api.client.WebResource; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import com.sun.jersey.test.framework.WebAppDescriptor; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +/** + * Tests RM Webservices fair scheduler resources. + */ public class TestRMWebServicesFairScheduler extends JerseyTestBase { private static MockRM rm; private static YarnConfiguration conf; - + private static class WebServletModule extends ServletModule { @Override protected void configureServlets() { @@ -58,7 +62,7 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase { bind(GenericExceptionHandler.class); conf = new YarnConfiguration(); conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class, - ResourceScheduler.class); + ResourceScheduler.class); rm = new MockRM(conf); bind(ResourceManager.class).toInstance(rm); serve("/*").with(GuiceContainer.class); @@ -66,32 +70,32 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase { } static { - GuiceServletConfig.setInjector( - Guice.createInjector(new WebServletModule())); + GuiceServletConfig + .setInjector(Guice.createInjector(new WebServletModule())); } @Before @Override public void setUp() throws Exception { super.setUp(); - GuiceServletConfig.setInjector( - Guice.createInjector(new WebServletModule())); + GuiceServletConfig + .setInjector(Guice.createInjector(new WebServletModule())); } public TestRMWebServicesFairScheduler() { super(new WebAppDescriptor.Builder( "org.apache.hadoop.yarn.server.resourcemanager.webapp") - .contextListenerClass(GuiceServletConfig.class) - .filterClass(com.google.inject.servlet.GuiceFilter.class) - .contextPath("jersey-guice-filter").servletPath("/").build()); + .contextListenerClass(GuiceServletConfig.class) + .filterClass(com.google.inject.servlet.GuiceFilter.class) + .contextPath("jersey-guice-filter").servletPath("/").build()); } - + @Test - public void testClusterScheduler() throws JSONException, Exception { + public void testClusterScheduler() throws JSONException { WebResource r = resource(); - ClientResponse response = r.path("ws").path("v1").path("cluster") - .path("scheduler").accept(MediaType.APPLICATION_JSON) - .get(ClientResponse.class); + ClientResponse response = + r.path("ws").path("v1").path("cluster").path("scheduler") + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); JSONObject json = response.getEntity(JSONObject.class); @@ -99,52 +103,51 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase { } @Test - public void testClusterSchedulerSlash() throws JSONException, Exception { + public void testClusterSchedulerSlash() throws JSONException { WebResource r = resource(); - ClientResponse response = r.path("ws").path("v1").path("cluster") - .path("scheduler/").accept(MediaType.APPLICATION_JSON) - .get(ClientResponse.class); + ClientResponse response = + r.path("ws").path("v1").path("cluster").path("scheduler/") + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); JSONObject json = response.getEntity(JSONObject.class); verifyClusterScheduler(json); } - + @Test - public void testClusterSchedulerWithSubQueues() throws JSONException, - Exception { - FairScheduler scheduler = (FairScheduler)rm.getResourceScheduler(); + public void testClusterSchedulerWithSubQueues() + throws JSONException { + FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); QueueManager queueManager = scheduler.getQueueManager(); // create LeafQueue queueManager.getLeafQueue("root.q.subqueue1", true); queueManager.getLeafQueue("root.q.subqueue2", true); WebResource r = resource(); - ClientResponse response = r.path("ws").path("v1").path("cluster") - .path("scheduler").accept(MediaType.APPLICATION_JSON) - .get(ClientResponse.class); + ClientResponse response = + r.path("ws").path("v1").path("cluster").path("scheduler") + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); JSONObject json = response.getEntity(JSONObject.class); JSONArray subQueueInfo = json.getJSONObject("scheduler") .getJSONObject("schedulerInfo").getJSONObject("rootQueue") - .getJSONObject("childQueues").getJSONArray("queue") - .getJSONObject(1).getJSONObject("childQueues").getJSONArray("queue"); + .getJSONObject("childQueues").getJSONArray("queue").getJSONObject(1) + .getJSONObject("childQueues").getJSONArray("queue"); // subQueueInfo is consist of subqueue1 and subqueue2 info assertEquals(2, subQueueInfo.length()); // Verify 'childQueues' field is omitted from FairSchedulerLeafQueueInfo. try { subQueueInfo.getJSONObject(1).getJSONObject("childQueues"); - fail("FairSchedulerQueueInfo should omit field 'childQueues'" + - "if child queue is empty."); + fail("FairSchedulerQueueInfo should omit field 'childQueues'" + + "if child queue is empty."); } catch (JSONException je) { assertEquals("JSONObject[\"childQueues\"] not found.", je.getMessage()); } } - private void verifyClusterScheduler(JSONObject json) throws JSONException, - Exception { + private void verifyClusterScheduler(JSONObject json) throws JSONException { assertEquals("incorrect number of elements", 1, json.length()); JSONObject info = json.getJSONObject("scheduler"); assertEquals("incorrect number of elements", 1, info.length()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java index 1e61186c3ab..40cf483cd3a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java @@ -457,7 +457,7 @@ public class TestRMWebServicesSchedulerActivities if (object.getClass() == JSONObject.class) { assertEquals("Number of allocations is wrong", 1, realValue); } else if (object.getClass() == JSONArray.class) { - assertEquals("Number of allocations is wrong", + assertEquals("Number of allocations is wrong in: " + object, ((JSONArray) object).length(), realValue); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java new file mode 100644 index 00000000000..bb1fce05a46 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java @@ -0,0 +1,138 @@ +/* + * 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.yarn.server.resourcemanager.webapp.fairscheduler; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.LocalConfigurationProvider; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static java.util.stream.Collectors.toList; + +/** + * This class can generate an XML configuration file of custom resource types. + * See createInitialResourceTypes for the default values. All custom resource + * type is prefixed with CUSTOM_RESOURCE_PREFIX. Please use the + * getConfigurationInputStream method to get an InputStream of the XML. If you + * want to have different number of resources in your tests, please see usages + * of this class in this test class: + * {@link TestRMWebServicesFairSchedulerCustomResourceTypes} + * + */ +public class CustomResourceTypesConfigurationProvider + extends LocalConfigurationProvider { + + private static class CustomResourceTypes { + private int count; + private String xml; + + CustomResourceTypes(String xml, int count) { + this.xml = xml; + this.count = count; + } + + public int getCount() { + return count; + } + + public String getXml() { + return xml; + } + } + + private static final String CUSTOM_RESOURCE_PREFIX = "customResource-"; + + private static CustomResourceTypes customResourceTypes = + createInitialResourceTypes(); + + private static CustomResourceTypes createInitialResourceTypes() { + return createCustomResourceTypes(2); + } + + private static CustomResourceTypes createCustomResourceTypes(int count) { + List resourceTypeNames = generateResourceTypeNames(count); + + List resourceUnitXmlElements = IntStream.range(0, count) + .boxed() + .map(i -> getResourceUnitsXml(resourceTypeNames.get(i))) + .collect(toList()); + + StringBuilder sb = new StringBuilder("\n"); + sb.append(getResourceTypesXml(resourceTypeNames)); + + for (String resourceUnitXml : resourceUnitXmlElements) { + sb.append(resourceUnitXml); + + } + sb.append(""); + + return new CustomResourceTypes(sb.toString(), count); + } + + private static List generateResourceTypeNames(int count) { + return IntStream.range(0, count) + .boxed() + .map(i -> CUSTOM_RESOURCE_PREFIX + i) + .collect(toList()); + } + + private static String getResourceUnitsXml(String resource) { + return "\n" + "yarn.resource-types." + resource + + ".units\n" + "k\n" + "\n"; + } + + private static String getResourceTypesXml(List resources) { + final String resourceTypes = makeCommaSeparatedString(resources); + + return "\n" + "yarn.resource-types\n" + "" + + resourceTypes + "\n" + "\n"; + } + + private static String makeCommaSeparatedString(List resources) { + return resources.stream().collect(Collectors.joining(",")); + } + + @Override + public InputStream getConfigurationInputStream(Configuration bootstrapConf, + String name) throws YarnException, IOException { + if (YarnConfiguration.RESOURCE_TYPES_CONFIGURATION_FILE.equals(name)) { + return new ByteArrayInputStream( + customResourceTypes.getXml().getBytes()); + } else { + return super.getConfigurationInputStream(bootstrapConf, name); + } + } + + public static void reset() { + customResourceTypes = createInitialResourceTypes(); + } + + public static void setNumberOfResourceTypes(int count) { + customResourceTypes = createCustomResourceTypes(count); + } + + public static List getCustomResourceTypes() { + return generateResourceTypeNames(customResourceTypes.getCount()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java new file mode 100644 index 00000000000..924411a4de0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java @@ -0,0 +1,139 @@ +/* + * 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.yarn.server.resourcemanager.webapp.fairscheduler; + +import com.google.common.collect.Sets; +import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; + +import java.util.List; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * This test helper class is primarily used by + * {@link TestRMWebServicesFairSchedulerCustomResourceTypes}. + */ +public class FairSchedulerJsonVerifications { + + private static final Set RESOURCE_FIELDS = + Sets.newHashSet("minResources", "amUsedResources", "amMaxResources", + "fairResources", "clusterResources", "reservedResources", + "maxResources", "usedResources", "steadyFairResources", + "demandResources"); + private final Set customResourceTypes; + + FairSchedulerJsonVerifications(List customResourceTypes) { + this.customResourceTypes = Sets.newHashSet(customResourceTypes); + } + + public void verify(JSONObject jsonObject) { + try { + verifyResourcesContainDefaultResourceTypes(jsonObject, RESOURCE_FIELDS); + verifyResourcesContainCustomResourceTypes(jsonObject, RESOURCE_FIELDS); + } catch (JSONException e) { + throw new RuntimeException(e); + } + } + + private void verifyResourcesContainDefaultResourceTypes(JSONObject queue, + Set resourceCategories) throws JSONException { + for (String resourceCategory : resourceCategories) { + boolean hasResourceCategory = queue.has(resourceCategory); + assertTrue("Queue " + queue + " does not have resource category key: " + + resourceCategory, hasResourceCategory); + verifyResourceContainsDefaultResourceTypes( + queue.getJSONObject(resourceCategory)); + } + } + + private void verifyResourceContainsDefaultResourceTypes( + JSONObject jsonObject) { + Object memory = jsonObject.opt("memory"); + Object vCores = jsonObject.opt("vCores"); + + assertNotNull("Key 'memory' not found in: " + jsonObject, memory); + assertNotNull("Key 'vCores' not found in: " + jsonObject, vCores); + } + + private void verifyResourcesContainCustomResourceTypes(JSONObject queue, + Set resourceCategories) throws JSONException { + for (String resourceCategory : resourceCategories) { + assertTrue("Queue " + queue + " does not have resource category key: " + + resourceCategory, queue.has(resourceCategory)); + verifyResourceContainsAllCustomResourceTypes( + queue.getJSONObject(resourceCategory)); + } + } + + private void verifyResourceContainsAllCustomResourceTypes( + JSONObject resourceCategory) throws JSONException { + assertTrue("resourceCategory does not have resourceInformations: " + + resourceCategory, resourceCategory.has("resourceInformations")); + + JSONObject resourceInformations = + resourceCategory.getJSONObject("resourceInformations"); + assertTrue( + "resourceInformations does not have resourceInformation object: " + + resourceInformations, + resourceInformations.has("resourceInformation")); + JSONArray customResources = + resourceInformations.getJSONArray("resourceInformation"); + + // customResources will include vcores / memory as well + assertEquals( + "Different number of custom resource types found than expected", + customResourceTypes.size(), customResources.length() - 2); + + for (int i = 0; i < customResources.length(); i++) { + JSONObject customResource = customResources.getJSONObject(i); + assertTrue("Resource type does not have name field: " + customResource, + customResource.has("name")); + assertTrue("Resource type does not have name resourceType field: " + + customResource, customResource.has("resourceType")); + assertTrue( + "Resource type does not have name units field: " + customResource, + customResource.has("units")); + assertTrue( + "Resource type does not have name value field: " + customResource, + customResource.has("value")); + + String name = customResource.getString("name"); + String unit = customResource.getString("units"); + String resourceType = customResource.getString("resourceType"); + Long value = customResource.getLong("value"); + + if (ResourceInformation.MEMORY_URI.equals(name) + || ResourceInformation.VCORES_URI.equals(name)) { + continue; + } + + assertTrue("Custom resource type " + name + " not found", + customResourceTypes.contains(name)); + assertEquals("k", unit); + assertEquals(ResourceTypes.COUNTABLE, + ResourceTypes.valueOf(resourceType)); + assertNotNull("Custom resource value " + value + " is null!", value); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java new file mode 100644 index 00000000000..63ae7b74f1c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java @@ -0,0 +1,153 @@ +/* + * 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.yarn.server.resourcemanager.webapp.fairscheduler; + + +import com.google.common.collect.Sets; +import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; + +import java.util.List; +import java.util.Set; + +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.XmlCustomResourceTypeTestCase.toXml; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlLong; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * This test helper class is primarily used by + * {@link TestRMWebServicesFairSchedulerCustomResourceTypes}. + */ +public class FairSchedulerXmlVerifications { + + private static final Set RESOURCE_FIELDS = Sets.newHashSet( + "minResources", "amUsedResources", "amMaxResources", "fairResources", + "clusterResources", "reservedResources", "maxResources", "usedResources", + "steadyFairResources", "demandResources"); + private final Set customResourceTypes; + + FairSchedulerXmlVerifications(List customResourceTypes) { + this.customResourceTypes = Sets.newHashSet(customResourceTypes); + } + + public void verify(Element element) { + verifyResourcesContainDefaultResourceTypes(element, RESOURCE_FIELDS); + verifyResourcesContainCustomResourceTypes(element, RESOURCE_FIELDS); + } + + private void verifyResourcesContainDefaultResourceTypes(Element queue, + Set resourceCategories) { + for (String resourceCategory : resourceCategories) { + boolean hasResourceCategory = hasChild(queue, resourceCategory); + assertTrue("Queue " + queue + " does not have resource category key: " + + resourceCategory, hasResourceCategory); + verifyResourceContainsDefaultResourceTypes( + (Element) queue.getElementsByTagName(resourceCategory).item(0)); + } + } + + private void verifyResourceContainsDefaultResourceTypes( + Element element) { + Object memory = opt(element, "memory"); + Object vCores = opt(element, "vCores"); + + assertNotNull("Key 'memory' not found in: " + element, memory); + assertNotNull("Key 'vCores' not found in: " + element, vCores); + } + + private void verifyResourcesContainCustomResourceTypes(Element queue, + Set resourceCategories) { + for (String resourceCategory : resourceCategories) { + assertTrue("Queue " + queue + " does not have key for resourceCategory: " + + resourceCategory, hasChild(queue, resourceCategory)); + verifyResourceContainsCustomResourceTypes( + (Element) queue.getElementsByTagName(resourceCategory).item(0)); + } + } + + private void verifyResourceContainsCustomResourceTypes( + Element resourceCategory) { + assertEquals( + toXml(resourceCategory) + + " should have only one resourceInformations child!", + 1, resourceCategory.getElementsByTagName("resourceInformations") + .getLength()); + Element resourceInformations = (Element) resourceCategory + .getElementsByTagName("resourceInformations").item(0); + + NodeList customResources = + resourceInformations.getElementsByTagName("resourceInformation"); + + // customResources will include vcores / memory as well + assertEquals( + "Different number of custom resource types found than expected", + customResourceTypes.size(), customResources.getLength() - 2); + + for (int i = 0; i < customResources.getLength(); i++) { + Element customResource = (Element) customResources.item(i); + String name = getXmlString(customResource, "name"); + String unit = getXmlString(customResource, "units"); + String resourceType = getXmlString(customResource, "resourceType"); + Long value = getXmlLong(customResource, "value"); + + if (ResourceInformation.MEMORY_URI.equals(name) + || ResourceInformation.VCORES_URI.equals(name)) { + continue; + } + + assertTrue("Custom resource type " + name + " not found", + customResourceTypes.contains(name)); + assertEquals("k", unit); + assertEquals(ResourceTypes.COUNTABLE, + ResourceTypes.valueOf(resourceType)); + assertNotNull("Resource value should not be null for resource type " + + resourceType + ", listing xml contents: " + toXml(customResource), + value); + } + } + + private Object opt(Node node, String child) { + NodeList nodes = getElementsByTagNameInternal(node, child); + if (nodes.getLength() > 0) { + return nodes.item(0); + } + + return null; + } + + private boolean hasChild(Node node, String child) { + return getElementsByTagNameInternal(node, child).getLength() > 0; + } + + private NodeList getElementsByTagNameInternal(Node node, String child) { + if (node instanceof Element) { + return ((Element) node).getElementsByTagName(child); + } else if (node instanceof Document) { + return ((Document) node).getElementsByTagName(child); + } else { + throw new IllegalStateException("Unknown type of wrappedObject: " + node + + ", type: " + node.getClass()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java new file mode 100644 index 00000000000..de4d5a10d1f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java @@ -0,0 +1,271 @@ +/* + * 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.yarn.server.resourcemanager.webapp.fairscheduler; + +import com.google.inject.Guice; +import com.google.inject.servlet.ServletModule; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; +import com.sun.jersey.test.framework.WebAppDescriptor; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueueManager; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.JAXBContextResolver; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServices; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.*; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; +import org.apache.hadoop.yarn.webapp.GuiceServletConfig; +import org.apache.hadoop.yarn.webapp.JerseyTestBase; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.w3c.dom.Element; +import javax.ws.rs.core.MediaType; +import java.lang.reflect.Method; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; + +/** + * This class is to test response representations of queue resources, + * explicitly setting custom resource types. with the help of + * {@link CustomResourceTypesConfigurationProvider} + */ +public class TestRMWebServicesFairSchedulerCustomResourceTypes + extends JerseyTestBase { + private static MockRM rm; + private static YarnConfiguration conf; + + private static class WebServletModule extends ServletModule { + @Override + protected void configureServlets() { + bind(JAXBContextResolver.class); + bind(RMWebServices.class); + bind(GenericExceptionHandler.class); + conf = new YarnConfiguration(); + conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class, + ResourceScheduler.class); + initResourceTypes(conf); + rm = new MockRM(conf); + bind(ResourceManager.class).toInstance(rm); + serve("/*").with(GuiceContainer.class); + } + + private void initResourceTypes(YarnConfiguration conf) { + conf.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS, + CustomResourceTypesConfigurationProvider.class.getName()); + ResourceUtils.resetResourceTypes(conf); + } + } + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + createInjectorForWebServletModule(); + } + + @After + public void tearDown() { + ResourceUtils.resetResourceTypes(new Configuration()); + } + + private void createInjectorForWebServletModule() { + GuiceServletConfig + .setInjector(Guice.createInjector(new WebServletModule())); + } + + @After + public void teardown() { + CustomResourceTypesConfigurationProvider.reset(); + } + + public TestRMWebServicesFairSchedulerCustomResourceTypes() { + super(new WebAppDescriptor.Builder( + "org.apache.hadoop.yarn.server.resourcemanager.webapp") + .contextListenerClass(GuiceServletConfig.class) + .filterClass(com.google.inject.servlet.GuiceFilter.class) + .contextPath("jersey-guice-filter").servletPath("/").build()); + } + + @Test + public void testClusterSchedulerWithCustomResourceTypesJson() { + FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); + QueueManager queueManager = scheduler.getQueueManager(); + // create LeafQueues + queueManager.getLeafQueue("root.q.subqueue1", true); + queueManager.getLeafQueue("root.q.subqueue2", true); + + FSLeafQueue subqueue1 = + queueManager.getLeafQueue("root.q.subqueue1", false); + incrementUsedResourcesOnQueue(subqueue1, 33L); + + WebResource path = + resource().path("ws").path("v1").path("cluster").path("scheduler"); + ClientResponse response = + path.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + + verifyJsonResponse(path, response, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + + @Test + public void testClusterSchedulerWithCustomResourceTypesXml() { + FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); + QueueManager queueManager = scheduler.getQueueManager(); + // create LeafQueues + queueManager.getLeafQueue("root.q.subqueue1", true); + queueManager.getLeafQueue("root.q.subqueue2", true); + + FSLeafQueue subqueue1 = + queueManager.getLeafQueue("root.q.subqueue1", false); + incrementUsedResourcesOnQueue(subqueue1, 33L); + + WebResource path = + resource().path("ws").path("v1").path("cluster").path("scheduler"); + ClientResponse response = + path.accept(MediaType.APPLICATION_XML).get(ClientResponse.class); + + verifyXmlResponse(path, response, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + + @Test + public void testClusterSchedulerWithElevenCustomResourceTypesXml() { + CustomResourceTypesConfigurationProvider.setNumberOfResourceTypes(11); + createInjectorForWebServletModule(); + + FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); + QueueManager queueManager = scheduler.getQueueManager(); + // create LeafQueues + queueManager.getLeafQueue("root.q.subqueue1", true); + queueManager.getLeafQueue("root.q.subqueue2", true); + + FSLeafQueue subqueue1 = + queueManager.getLeafQueue("root.q.subqueue1", false); + incrementUsedResourcesOnQueue(subqueue1, 33L); + + WebResource path = + resource().path("ws").path("v1").path("cluster").path("scheduler"); + ClientResponse response = + path.accept(MediaType.APPLICATION_XML).get(ClientResponse.class); + + verifyXmlResponse(path, response, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + + @Test + public void testClusterSchedulerElevenWithCustomResourceTypesJson() { + CustomResourceTypesConfigurationProvider.setNumberOfResourceTypes(11); + createInjectorForWebServletModule(); + + FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); + QueueManager queueManager = scheduler.getQueueManager(); + // create LeafQueues + queueManager.getLeafQueue("root.q.subqueue1", true); + queueManager.getLeafQueue("root.q.subqueue2", true); + + FSLeafQueue subqueue1 = + queueManager.getLeafQueue("root.q.subqueue1", false); + incrementUsedResourcesOnQueue(subqueue1, 33L); + + WebResource path = + resource().path("ws").path("v1").path("cluster").path("scheduler"); + ClientResponse response = + path.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + + verifyJsonResponse(path, response, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + + private void verifyJsonResponse(WebResource path, ClientResponse response, + List customResourceTypes) { + JsonCustomResourceTypeTestcase testCase = + new JsonCustomResourceTypeTestcase(path, + new BufferedClientResponse(response)); + testCase.verify(json -> { + try { + JSONArray queues = json.getJSONObject("scheduler") + .getJSONObject("schedulerInfo").getJSONObject("rootQueue") + .getJSONObject("childQueues").getJSONArray("queue"); + + // childQueueInfo consists of subqueue1 and subqueue2 info + assertEquals(2, queues.length()); + JSONObject firstChildQueue = queues.getJSONObject(0); + new FairSchedulerJsonVerifications(customResourceTypes) + .verify(firstChildQueue); + } catch (JSONException e) { + throw new RuntimeException(e); + } + }); + } + + private void verifyXmlResponse(WebResource path, ClientResponse response, + List customResourceTypes) { + XmlCustomResourceTypeTestCase testCase = new XmlCustomResourceTypeTestCase( + path, new BufferedClientResponse(response)); + + testCase.verify(xml -> { + Element scheduler = + (Element) xml.getElementsByTagName("scheduler").item(0); + Element schedulerInfo = + (Element) scheduler.getElementsByTagName("schedulerInfo").item(0); + Element rootQueue = + (Element) schedulerInfo.getElementsByTagName("rootQueue").item(0); + + Element childQueues = + (Element) rootQueue.getElementsByTagName("childQueues").item(0); + Element queue = + (Element) childQueues.getElementsByTagName("queue").item(0); + new FairSchedulerXmlVerifications(customResourceTypes).verify(queue); + }); + } + + private void incrementUsedResourcesOnQueue(final FSLeafQueue queue, + final long value) { + try { + Method incUsedResourceMethod = queue.getClass().getSuperclass() + .getDeclaredMethod("incUsedResource", Resource.class); + incUsedResourceMethod.setAccessible(true); + + Map customResources = + CustomResourceTypesConfigurationProvider.getCustomResourceTypes() + .stream() + .collect(Collectors.toMap(Function.identity(), v -> value)); + + incUsedResourceMethod.invoke(queue, + Resource.newInstance(20, 30, customResources)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java new file mode 100644 index 00000000000..4ab1443ce54 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java @@ -0,0 +1,123 @@ +/* + * 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.yarn.server.resourcemanager.webapp.helper; + +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.checkStringEqual; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.checkStringMatch; +import static org.junit.Assert.*; + +/** + * Contains all value verifications that are needed to verify {@link AppInfo} + * JSON objects. + */ +public final class AppInfoJsonVerifications { + + private AppInfoJsonVerifications() { + //utility class + } + + /** + * Tests whether {@link AppInfo} representation object contains the required + * values as per defined in the specified app parameter. + * @param app an RMApp instance that contains the required values + * to test against. + */ + public static void verify(JSONObject info, RMApp app) throws JSONException { + checkStringMatch("id", app.getApplicationId().toString(), + info.getString("id")); + checkStringMatch("user", app.getUser(), info.getString("user")); + checkStringMatch("name", app.getName(), info.getString("name")); + checkStringMatch("applicationType", app.getApplicationType(), + info.getString("applicationType")); + checkStringMatch("queue", app.getQueue(), info.getString("queue")); + assertEquals("priority doesn't match", 0, info.getInt("priority")); + checkStringMatch("state", app.getState().toString(), + info.getString("state")); + checkStringMatch("finalStatus", app.getFinalApplicationStatus().toString(), + info.getString("finalStatus")); + assertEquals("progress doesn't match", 0, + (float) info.getDouble("progress"), 0.0); + if ("UNASSIGNED".equals(info.getString("trackingUI"))) { + checkStringMatch("trackingUI", "UNASSIGNED", + info.getString("trackingUI")); + } + checkStringEqual("diagnostics", app.getDiagnostics().toString(), + info.getString("diagnostics")); + assertEquals("clusterId doesn't match", + ResourceManager.getClusterTimeStamp(), info.getLong("clusterId")); + assertEquals("startedTime doesn't match", app.getStartTime(), + info.getLong("startedTime")); + assertEquals("finishedTime doesn't match", app.getFinishTime(), + info.getLong("finishedTime")); + assertTrue("elapsed time not greater than 0", + info.getLong("elapsedTime") > 0); + checkStringMatch("amHostHttpAddress", + app.getCurrentAppAttempt().getMasterContainer().getNodeHttpAddress(), + info.getString("amHostHttpAddress")); + assertTrue("amContainerLogs doesn't match", + info.getString("amContainerLogs").startsWith("http://")); + assertTrue("amContainerLogs doesn't contain user info", + info.getString("amContainerLogs").endsWith("/" + app.getUser())); + assertEquals("allocatedMB doesn't match", 1024, info.getInt("allocatedMB")); + assertEquals("allocatedVCores doesn't match", 1, + info.getInt("allocatedVCores")); + assertEquals("queueUsagePerc doesn't match", 50.0f, + (float) info.getDouble("queueUsagePercentage"), 0.01f); + assertEquals("clusterUsagePerc doesn't match", 50.0f, + (float) info.getDouble("clusterUsagePercentage"), 0.01f); + assertEquals("numContainers doesn't match", 1, + info.getInt("runningContainers")); + assertNotNull("preemptedResourceSecondsMap should not be null", + info.getJSONObject("preemptedResourceSecondsMap")); + assertEquals("preemptedResourceMB doesn't match", + app.getRMAppMetrics().getResourcePreempted().getMemorySize(), + info.getInt("preemptedResourceMB")); + assertEquals("preemptedResourceVCores doesn't match", + app.getRMAppMetrics().getResourcePreempted().getVirtualCores(), + info.getInt("preemptedResourceVCores")); + assertEquals("numNonAMContainerPreempted doesn't match", + app.getRMAppMetrics().getNumNonAMContainersPreempted(), + info.getInt("numNonAMContainerPreempted")); + assertEquals("numAMContainerPreempted doesn't match", + app.getRMAppMetrics().getNumAMContainersPreempted(), + info.getInt("numAMContainerPreempted")); + assertEquals("Log aggregation Status doesn't match", + app.getLogAggregationStatusForAppReport().toString(), + info.getString("logAggregationStatus")); + assertEquals("unmanagedApplication doesn't match", + app.getApplicationSubmissionContext().getUnmanagedAM(), + info.getBoolean("unmanagedApplication")); + + if (app.getApplicationSubmissionContext() + .getNodeLabelExpression() != null) { + assertEquals("appNodeLabelExpression doesn't match", + app.getApplicationSubmissionContext().getNodeLabelExpression(), + info.getString("appNodeLabelExpression")); + } + assertEquals("amNodeLabelExpression doesn't match", + app.getAMResourceRequests().get(0).getNodeLabelExpression(), + info.getString("amNodeLabelExpression")); + assertEquals("amRPCAddress", + AppInfo.getAmRPCAddressFromRMAppAttempt(app.getCurrentAppAttempt()), + info.getString("amRPCAddress")); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java new file mode 100644 index 00000000000..7c5b6dbdeb2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java @@ -0,0 +1,132 @@ +/* + * 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.yarn.server.resourcemanager.webapp.helper; + +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo; +import org.apache.hadoop.yarn.webapp.WebServicesTestUtils; +import org.w3c.dom.Element; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.checkStringMatch; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlBoolean; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlFloat; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlInt; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlLong; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Contains all value verifications that are needed to verify {@link AppInfo} + * XML documents. + */ +public final class AppInfoXmlVerifications { + + private AppInfoXmlVerifications() { + //utility class + } + + /** + * Tests whether {@link AppInfo} representation object contains the required + * values as per defined in the specified app parameter. + * @param info + * @param app an RMApp instance that contains the required values + */ + public static void verify(Element info, RMApp app) { + checkStringMatch("id", app.getApplicationId() + .toString(), getXmlString(info, "id")); + checkStringMatch("user", app.getUser(), + getXmlString(info, "user")); + checkStringMatch("name", app.getName(), + getXmlString(info, "name")); + checkStringMatch("applicationType", + app.getApplicationType(), getXmlString(info, "applicationType")); + checkStringMatch("queue", app.getQueue(), + getXmlString(info, "queue")); + assertEquals("priority doesn't match", 0, getXmlInt(info, "priority")); + checkStringMatch("state", app.getState().toString(), + getXmlString(info, "state")); + checkStringMatch("finalStatus", app + .getFinalApplicationStatus().toString(), + getXmlString(info, "finalStatus")); + assertEquals("progress doesn't match", 0, getXmlFloat(info, "progress"), + 0.0); + if ("UNASSIGNED".equals(getXmlString(info, "trackingUI"))) { + checkStringMatch("trackingUI", "UNASSIGNED", + getXmlString(info, "trackingUI")); + } + WebServicesTestUtils.checkStringEqual("diagnostics", + app.getDiagnostics().toString(), getXmlString(info, "diagnostics")); + assertEquals("clusterId doesn't match", + ResourceManager.getClusterTimeStamp(), + getXmlLong(info, "clusterId")); + assertEquals("startedTime doesn't match", app.getStartTime(), + getXmlLong(info, "startedTime")); + assertEquals("finishedTime doesn't match", app.getFinishTime(), + getXmlLong(info, "finishedTime")); + assertTrue("elapsed time not greater than 0", + getXmlLong(info, "elapsedTime") > 0); + checkStringMatch("amHostHttpAddress", app + .getCurrentAppAttempt().getMasterContainer() + .getNodeHttpAddress(), + getXmlString(info, "amHostHttpAddress")); + assertTrue("amContainerLogs doesn't match", + getXmlString(info, "amContainerLogs").startsWith("http://")); + assertTrue("amContainerLogs doesn't contain user info", + getXmlString(info, "amContainerLogs").endsWith("/" + app.getUser())); + assertEquals("allocatedMB doesn't match", 1024, + getXmlInt(info, "allocatedMB")); + assertEquals("allocatedVCores doesn't match", 1, + getXmlInt(info, "allocatedVCores")); + assertEquals("queueUsagePerc doesn't match", 50.0f, + getXmlFloat(info, "queueUsagePercentage"), 0.01f); + assertEquals("clusterUsagePerc doesn't match", 50.0f, + getXmlFloat(info, "clusterUsagePercentage"), 0.01f); + assertEquals("numContainers doesn't match", 1, + getXmlInt(info, "runningContainers")); + assertNotNull("preemptedResourceSecondsMap should not be null", + info.getElementsByTagName("preemptedResourceSecondsMap")); + assertEquals("preemptedResourceMB doesn't match", app + .getRMAppMetrics().getResourcePreempted().getMemorySize(), + getXmlInt(info, "preemptedResourceMB")); + assertEquals("preemptedResourceVCores doesn't match", app + .getRMAppMetrics().getResourcePreempted().getVirtualCores(), + getXmlInt(info, "preemptedResourceVCores")); + assertEquals("numNonAMContainerPreempted doesn't match", app + .getRMAppMetrics().getNumNonAMContainersPreempted(), + getXmlInt(info, "numNonAMContainerPreempted")); + assertEquals("numAMContainerPreempted doesn't match", app + .getRMAppMetrics().getNumAMContainersPreempted(), + getXmlInt(info, "numAMContainerPreempted")); + assertEquals("Log aggregation Status doesn't match", app + .getLogAggregationStatusForAppReport().toString(), + getXmlString(info, "logAggregationStatus")); + assertEquals("unmanagedApplication doesn't match", app + .getApplicationSubmissionContext().getUnmanagedAM(), + getXmlBoolean(info, "unmanagedApplication")); + assertEquals("unmanagedApplication doesn't match", + app.getApplicationSubmissionContext().getNodeLabelExpression(), + getXmlString(info, "appNodeLabelExpression")); + assertEquals("unmanagedApplication doesn't match", + app.getAMResourceRequests().get(0).getNodeLabelExpression(), + getXmlString(info, "amNodeLabelExpression")); + assertEquals("amRPCAddress", + AppInfo.getAmRPCAddressFromRMAppAttempt(app.getCurrentAppAttempt()), + getXmlString(info, "amRPCAddress")); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.java new file mode 100644 index 00000000000..a8990ca695a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.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 + * + * 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.yarn.server.resourcemanager.webapp.helper; + + +import com.sun.jersey.api.client.ClientHandlerException; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.UniformInterfaceException; + +import javax.ws.rs.core.MediaType; +import java.io.IOException; + +/** + * This class is merely a wrapper for {@link ClientResponse}. Given that the + * entity input stream of {@link ClientResponse} can be read only once by + * default and for some tests it is convenient to read the input stream many + * times, this class hides the details of how to do that and prevents + * unnecessary code duplication in tests. + */ +public class BufferedClientResponse { + private ClientResponse response; + + public BufferedClientResponse(ClientResponse response) { + response.bufferEntity(); + this.response = response; + } + + public T getEntity(Class clazz) + throws ClientHandlerException, UniformInterfaceException { + try { + response.getEntityInputStream().reset(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return response.getEntity(clazz); + } + + public MediaType getType() { + return response.getType(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java new file mode 100644 index 00000000000..9d6a111d7ac --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java @@ -0,0 +1,77 @@ +/** + * 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.yarn.server.resourcemanager.webapp.helper; + +import com.sun.jersey.api.client.WebResource; +import org.apache.hadoop.http.JettyUtils; +import org.codehaus.jettison.json.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.ws.rs.core.MediaType; + +import java.util.function.Consumer; + +import static org.junit.Assert.*; + +/** + * This class hides the implementation details of how to verify the structure of + * JSON responses. Tests should only provide the path of the + * {@link WebResource}, the response from the resource and + * the verifier Consumer to + * {@link JsonCustomResourceTypeTestcase#verify(Consumer)}. An instance of + * {@link JSONObject} will be passed to that consumer to be able to + * verify the response. + */ +public class JsonCustomResourceTypeTestcase { + private static final Logger LOG = + LoggerFactory.getLogger(JsonCustomResourceTypeTestcase.class); + + private final WebResource path; + private final BufferedClientResponse response; + private final JSONObject parsedResponse; + + public JsonCustomResourceTypeTestcase(WebResource path, + BufferedClientResponse response) { + this.path = path; + this.response = response; + this.parsedResponse = response.getEntity(JSONObject.class); + } + + public void verify(Consumer verifier) { + assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, + response.getType().toString()); + + logResponse(); + + String responseStr = response.getEntity(String.class); + if (responseStr == null || responseStr.isEmpty()) { + throw new IllegalStateException("Response is null or empty!"); + } + verifier.accept(parsedResponse); + } + + private void logResponse() { + String responseStr = response.getEntity(String.class); + LOG.info("Raw response from service URL {}: {}", path.toString(), + responseStr); + LOG.info("Parsed response from service URL {}: {}", path.toString(), + parsedResponse); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java new file mode 100644 index 00000000000..6e58a89692b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java @@ -0,0 +1,252 @@ +/* + * 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.yarn.server.resourcemanager.webapp.helper; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; + +import java.util.List; +import java.util.Map; + +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Performs value verifications on + * {@link org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceRequestInfo} + * objects against the values of {@link ResourceRequest}. With the help of the + * {@link Builder}, users can also make verifications of the custom resource + * types and its values. + */ +public class ResourceRequestsJsonVerifications { + private final ResourceRequest resourceRequest; + private final JSONObject requestInfo; + private final Map customResourceTypes; + private final List expectedCustomResourceTypes; + + ResourceRequestsJsonVerifications(Builder builder) { + this.resourceRequest = builder.resourceRequest; + this.requestInfo = builder.requestInfo; + this.customResourceTypes = builder.customResourceTypes; + this.expectedCustomResourceTypes = builder.expectedCustomResourceTypes; + } + + public static void verify(JSONObject requestInfo, ResourceRequest rr) + throws JSONException { + createDefaultBuilder(requestInfo, rr).build().verify(); + } + + public static void verifyWithCustomResourceTypes(JSONObject requestInfo, + ResourceRequest resourceRequest, List expectedResourceTypes) + throws JSONException { + + createDefaultBuilder(requestInfo, resourceRequest) + .withExpectedCustomResourceTypes(expectedResourceTypes) + .withCustomResourceTypes( + extractActualCustomResourceTypes(requestInfo, expectedResourceTypes)) + .build().verify(); + } + + private static Builder createDefaultBuilder(JSONObject requestInfo, + ResourceRequest resourceRequest) { + return new ResourceRequestsJsonVerifications.Builder() + .withRequest(resourceRequest) + .withRequestInfoJson(requestInfo); + } + + private static Map extractActualCustomResourceTypes( + JSONObject requestInfo, List expectedResourceTypes) + throws JSONException { + JSONObject capability = requestInfo.getJSONObject("capability"); + Map resourceAndValue = + extractCustomResorceTypeValues(capability, expectedResourceTypes); + Map.Entry resourceEntry = + resourceAndValue.entrySet().iterator().next(); + + assertTrue( + "Found resource type: " + resourceEntry.getKey() + + " is not in expected resource types: " + expectedResourceTypes, + expectedResourceTypes.contains(resourceEntry.getKey())); + + return resourceAndValue; + } + + private static Map extractCustomResorceTypeValues( + JSONObject capability, List expectedResourceTypes) + throws JSONException { + assertTrue( + "resourceCategory does not have resourceInformations: " + capability, + capability.has("resourceInformations")); + + JSONObject resourceInformations = + capability.getJSONObject("resourceInformations"); + assertTrue( + "resourceInformations does not have resourceInformation object: " + + resourceInformations, + resourceInformations.has("resourceInformation")); + JSONArray customResources = + resourceInformations.getJSONArray("resourceInformation"); + + // customResources will include vcores / memory as well + assertEquals( + "Different number of custom resource types found than expected", + expectedResourceTypes.size(), customResources.length() - 2); + + Map resourceValues = Maps.newHashMap(); + for (int i = 0; i < customResources.length(); i++) { + JSONObject customResource = customResources.getJSONObject(i); + assertTrue("Resource type does not have name field: " + customResource, + customResource.has("name")); + assertTrue("Resource type does not have name resourceType field: " + + customResource, customResource.has("resourceType")); + assertTrue( + "Resource type does not have name units field: " + customResource, + customResource.has("units")); + assertTrue( + "Resource type does not have name value field: " + customResource, + customResource.has("value")); + + String name = customResource.getString("name"); + String unit = customResource.getString("units"); + String resourceType = customResource.getString("resourceType"); + Long value = customResource.getLong("value"); + + if (ResourceInformation.MEMORY_URI.equals(name) + || ResourceInformation.VCORES_URI.equals(name)) { + continue; + } + + assertTrue("Custom resource type " + name + " not found", + expectedResourceTypes.contains(name)); + assertEquals("k", unit); + assertEquals(ResourceTypes.COUNTABLE, + ResourceTypes.valueOf(resourceType)); + assertNotNull("Custom resource value " + value + " is null!", value); + resourceValues.put(name, value); + } + + return resourceValues; + } + + private void verify() throws JSONException { + assertEquals("nodeLabelExpression doesn't match", + resourceRequest.getNodeLabelExpression(), + requestInfo.getString("nodeLabelExpression")); + assertEquals("numContainers doesn't match", + resourceRequest.getNumContainers(), + requestInfo.getInt("numContainers")); + assertEquals("relaxLocality doesn't match", + resourceRequest.getRelaxLocality(), + requestInfo.getBoolean("relaxLocality")); + assertEquals("priority does not match", + resourceRequest.getPriority().getPriority(), + requestInfo.getInt("priority")); + assertEquals("resourceName does not match", + resourceRequest.getResourceName(), + requestInfo.getString("resourceName")); + assertEquals("memory does not match", + resourceRequest.getCapability().getMemorySize(), + requestInfo.getJSONObject("capability").getLong("memory")); + assertEquals("vCores does not match", + resourceRequest.getCapability().getVirtualCores(), + requestInfo.getJSONObject("capability").getLong("vCores")); + + verifyAtLeastOneCustomResourceIsSerialized(); + + JSONObject executionTypeRequest = + requestInfo.getJSONObject("executionTypeRequest"); + assertEquals("executionType does not match", + resourceRequest.getExecutionTypeRequest().getExecutionType().name(), + executionTypeRequest.getString("executionType")); + assertEquals("enforceExecutionType does not match", + resourceRequest.getExecutionTypeRequest().getEnforceExecutionType(), + executionTypeRequest.getBoolean("enforceExecutionType")); + } + + /** + * JSON serialization produces "invalid JSON" by default as maps are + * serialized like this: + * "customResources":{"entry":{"key":"customResource-1","value":"0"}} + * If the map has multiple keys then multiple entries will be serialized. + * Our json parser in tests cannot handle duplicates therefore only one + * custom resource will be in the parsed json. See: + * https://issues.apache.org/jira/browse/YARN-7505 + */ + private void verifyAtLeastOneCustomResourceIsSerialized() { + boolean resourceFound = false; + for (String expectedCustomResourceType : expectedCustomResourceTypes) { + if (customResourceTypes.containsKey(expectedCustomResourceType)) { + resourceFound = true; + Long resourceValue = + customResourceTypes.get(expectedCustomResourceType); + assertNotNull("Resource value should not be null!", resourceValue); + } + } + assertTrue("No custom resource type can be found in the response!", + resourceFound); + } + + /** + * Builder class for {@link ResourceRequestsJsonVerifications}. + */ + public static final class Builder { + private List expectedCustomResourceTypes = Lists.newArrayList(); + private Map customResourceTypes; + private ResourceRequest resourceRequest; + private JSONObject requestInfo; + + Builder() { + } + + public static Builder create() { + return new Builder(); + } + + Builder withExpectedCustomResourceTypes( + List expectedCustomResourceTypes) { + this.expectedCustomResourceTypes = expectedCustomResourceTypes; + return this; + } + + Builder withCustomResourceTypes( + Map customResourceTypes) { + this.customResourceTypes = customResourceTypes; + return this; + } + + Builder withRequest(ResourceRequest resourceRequest) { + this.resourceRequest = resourceRequest; + return this; + } + + Builder withRequestInfoJson(JSONObject requestInfo) { + this.requestInfo = requestInfo; + return this; + } + + public ResourceRequestsJsonVerifications build() { + return new ResourceRequestsJsonVerifications(this); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java new file mode 100644 index 00000000000..af9b0f35103 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java @@ -0,0 +1,215 @@ +/* + * 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.yarn.server.resourcemanager.webapp.helper; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.w3c.dom.Element; +import org.w3c.dom.NodeList; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static junit.framework.TestCase.assertTrue; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.XmlCustomResourceTypeTestCase.toXml; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlBoolean; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlInt; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlLong; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Performs value verifications on + * {@link org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceRequestInfo} + * objects against the values of {@link ResourceRequest}. With the help of the + * {@link Builder}, users can also make verifications of the custom resource + * types and its values. + */ +public class ResourceRequestsXmlVerifications { + private final ResourceRequest resourceRequest; + private final Element requestInfo; + private final Map customResourceTypes; + private final List expectedCustomResourceTypes; + + ResourceRequestsXmlVerifications(Builder builder) { + this.resourceRequest = builder.resourceRequest; + this.requestInfo = builder.requestInfo; + this.customResourceTypes = builder.customResourceTypes; + this.expectedCustomResourceTypes = builder.expectedCustomResourceTypes; + } + + public static void verifyWithCustomResourceTypes(Element requestInfo, + ResourceRequest resourceRequest, List expectedResourceTypes) { + + createDefaultBuilder(requestInfo, resourceRequest) + .withExpectedCustomResourceTypes(expectedResourceTypes) + .withCustomResourceTypes(extractActualCustomResourceType(requestInfo, + expectedResourceTypes)) + .build().verify(); + } + + private static Builder createDefaultBuilder(Element requestInfo, + ResourceRequest resourceRequest) { + return new ResourceRequestsXmlVerifications.Builder() + .withRequest(resourceRequest).withRequestInfo(requestInfo); + } + + private static Map extractActualCustomResourceType( + Element requestInfo, List expectedResourceTypes) { + Element capability = + (Element) requestInfo.getElementsByTagName("capability").item(0); + + return extractCustomResorceTypes(capability, + Sets.newHashSet(expectedResourceTypes)); + } + + private static Map extractCustomResorceTypes(Element capability, + Set expectedResourceTypes) { + assertEquals( + toXml(capability) + " should have only one resourceInformations child!", + 1, capability.getElementsByTagName("resourceInformations").getLength()); + Element resourceInformations = (Element) capability + .getElementsByTagName("resourceInformations").item(0); + + NodeList customResources = + resourceInformations.getElementsByTagName("resourceInformation"); + + // customResources will include vcores / memory as well + assertEquals( + "Different number of custom resource types found than expected", + expectedResourceTypes.size(), customResources.getLength() - 2); + + Map resourceTypesAndValues = Maps.newHashMap(); + for (int i = 0; i < customResources.getLength(); i++) { + Element customResource = (Element) customResources.item(i); + String name = getXmlString(customResource, "name"); + String unit = getXmlString(customResource, "units"); + String resourceType = getXmlString(customResource, "resourceType"); + Long value = getXmlLong(customResource, "value"); + + if (ResourceInformation.MEMORY_URI.equals(name) + || ResourceInformation.VCORES_URI.equals(name)) { + continue; + } + + assertTrue("Custom resource type " + name + " not found", + expectedResourceTypes.contains(name)); + assertEquals("k", unit); + assertEquals(ResourceTypes.COUNTABLE, + ResourceTypes.valueOf(resourceType)); + assertNotNull("Resource value should not be null for resource type " + + resourceType + ", listing xml contents: " + toXml(customResource), + value); + resourceTypesAndValues.put(name, value); + } + + return resourceTypesAndValues; + } + + private void verify() { + assertEquals("nodeLabelExpression doesn't match", + resourceRequest.getNodeLabelExpression(), + getXmlString(requestInfo, "nodeLabelExpression")); + assertEquals("numContainers doesn't match", + resourceRequest.getNumContainers(), + getXmlInt(requestInfo, "numContainers")); + assertEquals("relaxLocality doesn't match", + resourceRequest.getRelaxLocality(), + getXmlBoolean(requestInfo, "relaxLocality")); + assertEquals("priority does not match", + resourceRequest.getPriority().getPriority(), + getXmlInt(requestInfo, "priority")); + assertEquals("resourceName does not match", + resourceRequest.getResourceName(), + getXmlString(requestInfo, "resourceName")); + Element capability = (Element) requestInfo + .getElementsByTagName("capability").item(0); + assertEquals("memory does not match", + resourceRequest.getCapability().getMemorySize(), + getXmlLong(capability, "memory")); + assertEquals("vCores does not match", + resourceRequest.getCapability().getVirtualCores(), + getXmlLong(capability, "vCores")); + + for (String expectedCustomResourceType : expectedCustomResourceTypes) { + assertTrue( + "Custom resource type " + expectedCustomResourceType + + " cannot be found!", + customResourceTypes.containsKey(expectedCustomResourceType)); + + Long resourceValue = customResourceTypes.get(expectedCustomResourceType); + assertNotNull("Resource value should not be null!", resourceValue); + } + + Element executionTypeRequest = (Element) requestInfo + .getElementsByTagName("executionTypeRequest").item(0); + assertEquals("executionType does not match", + resourceRequest.getExecutionTypeRequest().getExecutionType().name(), + getXmlString(executionTypeRequest, "executionType")); + assertEquals("enforceExecutionType does not match", + resourceRequest.getExecutionTypeRequest().getEnforceExecutionType(), + getXmlBoolean(executionTypeRequest, "enforceExecutionType")); + } + + /** + * Builder class for {@link ResourceRequestsXmlVerifications}. + */ + public static final class Builder { + private List expectedCustomResourceTypes = Lists.newArrayList(); + private Map customResourceTypes; + private ResourceRequest resourceRequest; + private Element requestInfo; + + Builder() { + } + + public static Builder create() { + return new Builder(); + } + + Builder withExpectedCustomResourceTypes( + List expectedCustomResourceTypes) { + this.expectedCustomResourceTypes = expectedCustomResourceTypes; + return this; + } + + Builder withCustomResourceTypes(Map customResourceTypes) { + this.customResourceTypes = customResourceTypes; + return this; + } + + Builder withRequest(ResourceRequest resourceRequest) { + this.resourceRequest = resourceRequest; + return this; + } + + Builder withRequestInfo(Element requestInfo) { + this.requestInfo = requestInfo; + return this; + } + + public ResourceRequestsXmlVerifications build() { + return new ResourceRequestsXmlVerifications(this); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java new file mode 100644 index 00000000000..29260aad1df --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java @@ -0,0 +1,112 @@ +/** + * 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.yarn.server.resourcemanager.webapp.helper; + +import com.sun.jersey.api.client.WebResource; +import org.apache.hadoop.http.JettyUtils; +import org.codehaus.jettison.json.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.w3c.dom.Document; +import org.w3c.dom.Node; +import org.xml.sax.InputSource; + +import javax.ws.rs.core.MediaType; +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.transform.*; +import javax.xml.transform.dom.DOMSource; +import javax.xml.transform.stream.StreamResult; +import java.io.StringReader; +import java.io.StringWriter; +import java.util.function.Consumer; + +import static org.junit.Assert.assertEquals; + +/** + * This class hides the implementation details of how to verify the structure of + * XML responses. Tests should only provide the path of the + * {@link WebResource}, the response from the resource and + * the verifier Consumer to + * {@link XmlCustomResourceTypeTestCase#verify(Consumer)}. An instance of + * {@link JSONObject} will be passed to that consumer to be able to + * verify the response. + */ +public class XmlCustomResourceTypeTestCase { + private static final Logger LOG = + LoggerFactory.getLogger(XmlCustomResourceTypeTestCase.class); + + private WebResource path; + private BufferedClientResponse response; + private Document parsedResponse; + + public XmlCustomResourceTypeTestCase(WebResource path, + BufferedClientResponse response) { + this.path = path; + this.response = response; + } + + public void verify(Consumer verifier) { + assertEquals(MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8, + response.getType().toString()); + + parsedResponse = parseXml(response); + logResponse(parsedResponse); + verifier.accept(parsedResponse); + } + + private Document parseXml(BufferedClientResponse response) { + try { + String xml = response.getEntity(String.class); + DocumentBuilder db = + DocumentBuilderFactory.newInstance().newDocumentBuilder(); + InputSource is = new InputSource(); + is.setCharacterStream(new StringReader(xml)); + + return db.parse(is); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private void logResponse(Document doc) { + String responseStr = response.getEntity(String.class); + LOG.info("Raw response from service URL {}: {}", path.toString(), + responseStr); + LOG.info("Parsed response from service URL {}: {}", path.toString(), + toXml(doc)); + } + + public static String toXml(Node node) { + StringWriter writer; + try { + TransformerFactory tf = TransformerFactory.newInstance(); + Transformer transformer = tf.newTransformer(); + transformer.setOutputProperty(OutputKeys.INDENT, "yes"); + transformer.setOutputProperty( + "{http://xml.apache.org/xslt}indent" + "-amount", "2"); + writer = new StringWriter(); + transformer.transform(new DOMSource(node), new StreamResult(writer)); + } catch (TransformerException e) { + throw new RuntimeException(e); + } + + return writer.getBuffer().toString(); + } +} From 0d9804dcef2eab5ebf84667d9ca49bb035d9a731 Mon Sep 17 00:00:00 2001 From: Giovanni Matteo Fumarola Date: Thu, 5 Jul 2018 10:54:31 -0700 Subject: [PATCH 31/47] YARN-8435. Fix NPE when the same client simultaneously contact for the first time Yarn Router. Contributed by Rang Jiaheng. --- .../clientrm/RouterClientRMService.java | 53 ++++++++------- .../router/rmadmin/RouterRMAdminService.java | 51 +++++++-------- .../router/webapp/RouterWebServices.java | 48 +++++++------- .../clientrm/TestRouterClientRMService.java | 60 +++++++++++++++++ .../rmadmin/TestRouterRMAdminService.java | 60 +++++++++++++++++ .../router/webapp/TestRouterWebServices.java | 65 +++++++++++++++++++ 6 files changed, 259 insertions(+), 78 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java index 73cc18558d1..bbb8047d98d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java @@ -430,13 +430,15 @@ public class RouterClientRMService extends AbstractService return pipeline.getRootInterceptor().getResourceTypeInfo(request); } - private RequestInterceptorChainWrapper getInterceptorChain() + @VisibleForTesting + protected RequestInterceptorChainWrapper getInterceptorChain() throws IOException { String user = UserGroupInformation.getCurrentUser().getUserName(); - if (!userPipelineMap.containsKey(user)) { - initializePipeline(user); + RequestInterceptorChainWrapper chain = userPipelineMap.get(user); + if (chain != null && chain.getRootInterceptor() != null) { + return chain; } - return userPipelineMap.get(user); + return initializePipeline(user); } /** @@ -503,36 +505,33 @@ public class RouterClientRMService extends AbstractService * * @param user */ - private void initializePipeline(String user) { - RequestInterceptorChainWrapper chainWrapper = null; + private RequestInterceptorChainWrapper initializePipeline(String user) { synchronized (this.userPipelineMap) { if (this.userPipelineMap.containsKey(user)) { LOG.info("Request to start an already existing user: {}" + " was received, so ignoring.", user); - return; + return userPipelineMap.get(user); + } + + RequestInterceptorChainWrapper chainWrapper = + new RequestInterceptorChainWrapper(); + try { + // We should init the pipeline instance after it is created and then + // add to the map, to ensure thread safe. + LOG.info("Initializing request processing pipeline for application " + + "for the user: {}", user); + + ClientRequestInterceptor interceptorChain = + this.createRequestInterceptorChain(); + interceptorChain.init(user); + chainWrapper.init(interceptorChain); + } catch (Exception e) { + LOG.error("Init ClientRequestInterceptor error for user: " + user, e); + throw e; } - chainWrapper = new RequestInterceptorChainWrapper(); this.userPipelineMap.put(user, chainWrapper); - } - - // We register the pipeline instance in the map first and then initialize it - // later because chain initialization can be expensive and we would like to - // release the lock as soon as possible to prevent other applications from - // blocking when one application's chain is initializing - LOG.info("Initializing request processing pipeline for application " - + "for the user: {}", user); - - try { - ClientRequestInterceptor interceptorChain = - this.createRequestInterceptorChain(); - interceptorChain.init(user); - chainWrapper.init(interceptorChain); - } catch (Exception e) { - synchronized (this.userPipelineMap) { - this.userPipelineMap.remove(user); - } - throw e; + return chainWrapper; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java index b8b7ad818f3..ef30613f50c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java @@ -165,13 +165,15 @@ public class RouterRMAdminService extends AbstractService return interceptorClassNames; } - private RequestInterceptorChainWrapper getInterceptorChain() + @VisibleForTesting + protected RequestInterceptorChainWrapper getInterceptorChain() throws IOException { String user = UserGroupInformation.getCurrentUser().getUserName(); - if (!userPipelineMap.containsKey(user)) { - initializePipeline(user); + RequestInterceptorChainWrapper chain = userPipelineMap.get(user); + if (chain != null && chain.getRootInterceptor() != null) { + return chain; } - return userPipelineMap.get(user); + return initializePipeline(user); } /** @@ -239,35 +241,32 @@ public class RouterRMAdminService extends AbstractService * * @param user */ - private void initializePipeline(String user) { - RequestInterceptorChainWrapper chainWrapper = null; + private RequestInterceptorChainWrapper initializePipeline(String user) { synchronized (this.userPipelineMap) { if (this.userPipelineMap.containsKey(user)) { LOG.info("Request to start an already existing user: {}" + " was received, so ignoring.", user); - return; + return userPipelineMap.get(user); + } + + RequestInterceptorChainWrapper chainWrapper = + new RequestInterceptorChainWrapper(); + try { + // We should init the pipeline instance after it is created and then + // add to the map, to ensure thread safe. + LOG.info("Initializing request processing pipeline for user: {}", user); + + RMAdminRequestInterceptor interceptorChain = + this.createRequestInterceptorChain(); + interceptorChain.init(user); + chainWrapper.init(interceptorChain); + } catch (Exception e) { + LOG.error("Init RMAdminRequestInterceptor error for user: " + user, e); + throw e; } - chainWrapper = new RequestInterceptorChainWrapper(); this.userPipelineMap.put(user, chainWrapper); - } - - // We register the pipeline instance in the map first and then initialize it - // later because chain initialization can be expensive and we would like to - // release the lock as soon as possible to prevent other applications from - // blocking when one application's chain is initializing - LOG.info("Initializing request processing pipeline for the user: {}", user); - - try { - RMAdminRequestInterceptor interceptorChain = - this.createRequestInterceptorChain(); - interceptorChain.init(user); - chainWrapper.init(interceptorChain); - } catch (Exception e) { - synchronized (this.userPipelineMap) { - this.userPipelineMap.remove(user); - } - throw e; + return chainWrapper; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java index ae57f1cdc88..49de588ba85 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java @@ -173,10 +173,11 @@ public class RouterWebServices implements RMWebServiceProtocol { } catch (IOException e) { LOG.error("Cannot get user: {}", e.getMessage()); } - if (!userPipelineMap.containsKey(user)) { - initializePipeline(user); + RequestInterceptorChainWrapper chain = userPipelineMap.get(user); + if (chain != null && chain.getRootInterceptor() != null) { + return chain; } - return userPipelineMap.get(user); + return initializePipeline(user); } /** @@ -242,35 +243,32 @@ public class RouterWebServices implements RMWebServiceProtocol { * * @param user */ - private void initializePipeline(String user) { - RequestInterceptorChainWrapper chainWrapper = null; + private RequestInterceptorChainWrapper initializePipeline(String user) { synchronized (this.userPipelineMap) { if (this.userPipelineMap.containsKey(user)) { LOG.info("Request to start an already existing user: {}" + " was received, so ignoring.", user); - return; + return userPipelineMap.get(user); + } + + RequestInterceptorChainWrapper chainWrapper = + new RequestInterceptorChainWrapper(); + try { + // We should init the pipeline instance after it is created and then + // add to the map, to ensure thread safe. + LOG.info("Initializing request processing pipeline for user: {}", user); + + RESTRequestInterceptor interceptorChain = + this.createRequestInterceptorChain(); + interceptorChain.init(user); + chainWrapper.init(interceptorChain); + } catch (Exception e) { + LOG.error("Init RESTRequestInterceptor error for user: " + user, e); + throw e; } - chainWrapper = new RequestInterceptorChainWrapper(); this.userPipelineMap.put(user, chainWrapper); - } - - // We register the pipeline instance in the map first and then initialize it - // later because chain initialization can be expensive and we would like to - // release the lock as soon as possible to prevent other applications from - // blocking when one application's chain is initializing - LOG.info("Initializing request processing pipeline for the user: {}", user); - - try { - RESTRequestInterceptor interceptorChain = - this.createRequestInterceptorChain(); - interceptorChain.init(user); - chainWrapper.init(interceptorChain); - } catch (Exception e) { - synchronized (this.userPipelineMap) { - this.userPipelineMap.remove(user); - } - throw e; + return chainWrapper; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterClientRMService.java index a9c37293f69..b03059decfa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterClientRMService.java @@ -19,8 +19,10 @@ package org.apache.hadoop.yarn.server.router.clientrm; import java.io.IOException; +import java.security.PrivilegedExceptionAction; import java.util.Map; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; @@ -207,4 +209,62 @@ public class TestRouterClientRMService extends BaseRouterClientRMTest { Assert.assertNull("test2 should have been evicted", chain); } + /** + * This test validates if the ClientRequestInterceptor chain for the user + * can build and init correctly when a multi-client process begins to + * request RouterClientRMService for the same user simultaneously. + */ + @Test + public void testClientPipelineConcurrent() throws InterruptedException { + final String user = "test1"; + + /* + * ClientTestThread is a thread to simulate a client request to get a + * ClientRequestInterceptor for the user. + */ + class ClientTestThread extends Thread { + private ClientRequestInterceptor interceptor; + @Override public void run() { + try { + interceptor = pipeline(); + } catch (IOException | InterruptedException e) { + e.printStackTrace(); + } + } + private ClientRequestInterceptor pipeline() + throws IOException, InterruptedException { + return UserGroupInformation.createRemoteUser(user).doAs( + new PrivilegedExceptionAction() { + @Override + public ClientRequestInterceptor run() throws Exception { + RequestInterceptorChainWrapper wrapper = + getRouterClientRMService().getInterceptorChain(); + ClientRequestInterceptor interceptor = + wrapper.getRootInterceptor(); + Assert.assertNotNull(interceptor); + LOG.info("init client interceptor success for user " + user); + return interceptor; + } + }); + } + } + + /* + * We start the first thread. It should not finish initing a chainWrapper + * before the other thread starts. In this way, the second thread can + * init at the same time of the first one. In the end, we validate that + * the 2 threads get the same chainWrapper without going into error. + */ + ClientTestThread client1 = new ClientTestThread(); + ClientTestThread client2 = new ClientTestThread(); + client1.start(); + client2.start(); + client1.join(); + client2.join(); + + Assert.assertNotNull(client1.interceptor); + Assert.assertNotNull(client2.interceptor); + Assert.assertTrue(client1.interceptor == client2.interceptor); + } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestRouterRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestRouterRMAdminService.java index 11786e6f980..07ef73c3cdb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestRouterRMAdminService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestRouterRMAdminService.java @@ -19,8 +19,10 @@ package org.apache.hadoop.yarn.server.router.rmadmin; import java.io.IOException; +import java.security.PrivilegedExceptionAction; import java.util.Map; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse; @@ -216,4 +218,62 @@ public class TestRouterRMAdminService extends BaseRouterRMAdminTest { Assert.assertNull("test2 should have been evicted", chain); } + /** + * This test validates if the RMAdminRequestInterceptor chain for the user + * can build and init correctly when a multi-client process begins to + * request RouterRMAdminService for the same user simultaneously. + */ + @Test + public void testRMAdminPipelineConcurrent() throws InterruptedException { + final String user = "test1"; + + /* + * ClientTestThread is a thread to simulate a client request to get a + * RMAdminRequestInterceptor for the user. + */ + class ClientTestThread extends Thread { + private RMAdminRequestInterceptor interceptor; + @Override public void run() { + try { + interceptor = pipeline(); + } catch (IOException | InterruptedException e) { + e.printStackTrace(); + } + } + private RMAdminRequestInterceptor pipeline() + throws IOException, InterruptedException { + return UserGroupInformation.createRemoteUser(user).doAs( + new PrivilegedExceptionAction() { + @Override + public RMAdminRequestInterceptor run() throws Exception { + RequestInterceptorChainWrapper wrapper = + getRouterRMAdminService().getInterceptorChain(); + RMAdminRequestInterceptor interceptor = + wrapper.getRootInterceptor(); + Assert.assertNotNull(interceptor); + LOG.info("init rm admin interceptor success for user" + user); + return interceptor; + } + }); + } + } + + /* + * We start the first thread. It should not finish initing a chainWrapper + * before the other thread starts. In this way, the second thread can + * init at the same time of the first one. In the end, we validate that + * the 2 threads get the same chainWrapper without going into error. + */ + ClientTestThread client1 = new ClientTestThread(); + ClientTestThread client2 = new ClientTestThread(); + client1.start(); + client2.start(); + client1.join(); + client2.join(); + + Assert.assertNotNull(client1.interceptor); + Assert.assertNotNull(client2.interceptor); + Assert.assertTrue(client1.interceptor == client2.interceptor); + } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.java index c96575c21a4..14652435dac 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.java @@ -19,10 +19,12 @@ package org.apache.hadoop.yarn.server.router.webapp; import java.io.IOException; +import java.security.PrivilegedExceptionAction; import java.util.Map; import javax.ws.rs.core.Response; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo; @@ -49,12 +51,17 @@ import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo; import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo; import org.junit.Assert; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Test class to validate the WebService interceptor model inside the Router. */ public class TestRouterWebServices extends BaseRouterWebServicesTest { + private static final Logger LOG = + LoggerFactory.getLogger(TestRouterWebServices.class); + private String user = "test1"; /** @@ -266,4 +273,62 @@ public class TestRouterWebServices extends BaseRouterWebServicesTest { Assert.assertNull("test2 should have been evicted", chain); } + /** + * This test validates if the RESTRequestInterceptor chain for the user + * can build and init correctly when a multi-client process begins to + * request RouterWebServices for the same user simultaneously. + */ + @Test + public void testWebPipelineConcurrent() throws InterruptedException { + final String user = "test1"; + + /* + * ClientTestThread is a thread to simulate a client request to get a + * RESTRequestInterceptor for the user. + */ + class ClientTestThread extends Thread { + private RESTRequestInterceptor interceptor; + @Override public void run() { + try { + interceptor = pipeline(); + } catch (IOException | InterruptedException e) { + e.printStackTrace(); + } + } + private RESTRequestInterceptor pipeline() + throws IOException, InterruptedException { + return UserGroupInformation.createRemoteUser(user).doAs( + new PrivilegedExceptionAction() { + @Override + public RESTRequestInterceptor run() throws Exception { + RequestInterceptorChainWrapper wrapper = + getInterceptorChain(user); + RESTRequestInterceptor interceptor = + wrapper.getRootInterceptor(); + Assert.assertNotNull(interceptor); + LOG.info("init web interceptor success for user" + user); + return interceptor; + } + }); + } + } + + /* + * We start the first thread. It should not finish initing a chainWrapper + * before the other thread starts. In this way, the second thread can + * init at the same time of the first one. In the end, we validate that + * the 2 threads get the same chainWrapper without going into error. + */ + ClientTestThread client1 = new ClientTestThread(); + ClientTestThread client2 = new ClientTestThread(); + client1.start(); + client2.start(); + client1.join(); + client2.join(); + + Assert.assertNotNull(client1.interceptor); + Assert.assertNotNull(client2.interceptor); + Assert.assertTrue(client1.interceptor == client2.interceptor); + } + } From 39ad98903a5f042573b97a2e5438bc57af7cc7a1 Mon Sep 17 00:00:00 2001 From: Anu Engineer Date: Thu, 5 Jul 2018 12:22:18 -0700 Subject: [PATCH 32/47] Revert "Merge branch 'trunk' of https://git-wip-us.apache.org/repos/asf/hadoop into trunk" This reverts commit c163d1797ade0f47d35b4a44381b8ef1dfec5b60, reversing changes made to 0d9804dcef2eab5ebf84667d9ca49bb035d9a731. --- .../dev-support/findbugs-exclude.xml | 17 +- .../hadoop/yarn/api/records/Resource.java | 13 - .../api/records/impl/LightWeightResource.java | 23 +- .../scheduler/fair/ConfigurableResource.java | 69 +---- .../fair/FairSchedulerConfiguration.java | 174 ++--------- .../allocation/AllocationFileQueueParser.java | 2 +- .../resourcemanager/webapp/dao/AppInfo.java | 2 +- .../webapp/dao/SchedulerInfo.java | 8 +- .../fair/TestFairSchedulerConfiguration.java | 158 +++------- .../webapp/TestRMWebServices.java | 31 +- .../webapp/TestRMWebServicesApps.java | 14 +- ...tRMWebServicesAppsCustomResourceTypes.java | 242 ---------------- .../TestRMWebServicesCapacitySched.java | 30 +- ...estRMWebServicesConfigurationMutation.java | 5 - .../TestRMWebServicesFairScheduler.java | 95 +++--- .../TestRMWebServicesSchedulerActivities.java | 2 +- ...tomResourceTypesConfigurationProvider.java | 138 --------- .../FairSchedulerJsonVerifications.java | 139 --------- .../FairSchedulerXmlVerifications.java | 153 ---------- ...vicesFairSchedulerCustomResourceTypes.java | 271 ------------------ .../helper/AppInfoJsonVerifications.java | 123 -------- .../helper/AppInfoXmlVerifications.java | 132 --------- .../webapp/helper/BufferedClientResponse.java | 57 ---- .../JsonCustomResourceTypeTestcase.java | 77 ----- .../ResourceRequestsJsonVerifications.java | 252 ---------------- .../ResourceRequestsXmlVerifications.java | 215 -------------- .../helper/XmlCustomResourceTypeTestCase.java | 112 -------- .../src/site/markdown/FairScheduler.md | 6 +- 28 files changed, 156 insertions(+), 2404 deletions(-) delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml index 5cc81e57ec6..58413618df0 100644 --- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml @@ -65,6 +65,11 @@ + + + + + @@ -113,18 +118,6 @@ - - - - - - - - - - - - diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java index 173d4c9e777..71a6b54352b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java @@ -30,7 +30,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; import org.apache.hadoop.yarn.api.records.impl.LightWeightResource; @@ -76,18 +75,6 @@ public abstract class Resource implements Comparable { @Private public static final int VCORES_INDEX = 1; - /** - * Return a new {@link Resource} instance with all resource values - * initialized to {@code value}. - * @param value the value to use for all resources - * @return a new {@link Resource} instance - */ - @Private - @Unstable - public static Resource newInstance(long value) { - return new LightWeightResource(value); - } - @Public @Stable public static Resource newInstance(int memory, int vCores) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java index 77f77f312ce..a6e6432976f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java @@ -18,8 +18,9 @@ package org.apache.hadoop.yarn.api.records.impl; -import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.util.resource.ResourceUtils; @@ -57,29 +58,13 @@ import static org.apache.hadoop.yarn.api.records.ResourceInformation.*; * * @see Resource */ -@Private +@InterfaceAudience.Private @Unstable public class LightWeightResource extends Resource { private ResourceInformation memoryResInfo; private ResourceInformation vcoresResInfo; - /** - * Create a new {@link LightWeightResource} instance with all resource values - * initialized to {@code value}. - * @param value the value to use for all resources - */ - public LightWeightResource(long value) { - ResourceInformation[] types = ResourceUtils.getResourceTypesArray(); - initResourceInformations(value, value, types.length); - - for (int i = 2; i < types.length; i++) { - resources[i] = new ResourceInformation(); - ResourceInformation.copy(types[i], resources[i]); - resources[i].setValue(value); - } - } - public LightWeightResource(long memory, int vcores) { int numberOfKnownResourceTypes = ResourceUtils .getNumberOfKnownResourceTypes(); @@ -106,7 +91,7 @@ public class LightWeightResource extends Resource { } } - private void initResourceInformations(long memory, long vcores, + private void initResourceInformations(long memory, int vcores, int numberOfKnownResourceTypes) { this.memoryResInfo = newDefaultInformation(MEMORY_URI, MEMORY_MB.getUnits(), memory); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java index 0c3b0ddc6b6..ecdd0111a6b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java @@ -18,13 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; -import java.util.Arrays; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.ResourceInformation; -import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException; -import org.apache.hadoop.yarn.util.resource.ResourceUtils; /** * A {@code ConfigurableResource} object represents an entity that is used to @@ -37,53 +33,29 @@ public class ConfigurableResource { private final Resource resource; private final double[] percentages; - ConfigurableResource() { - this(getOneHundredPercentArray()); - } - - ConfigurableResource(double[] percentages) { + public ConfigurableResource(double[] percentages) { this.percentages = percentages.clone(); this.resource = null; } - ConfigurableResource(long value) { - this(Resource.newInstance(value)); - } - public ConfigurableResource(Resource resource) { this.percentages = null; this.resource = resource; } - private static double[] getOneHundredPercentArray() { - double[] resourcePercentages = - new double[ResourceUtils.getNumberOfKnownResourceTypes()]; - Arrays.fill(resourcePercentages, 1.0); - - return resourcePercentages; - } - /** * Get resource by multiplying the cluster resource and the percentage of * each resource respectively. Return the absolute resource if either * {@code percentages} or {@code clusterResource} is null. * * @param clusterResource the cluster resource - * @return resource the resulting resource + * @return resource */ public Resource getResource(Resource clusterResource) { if (percentages != null && clusterResource != null) { long memory = (long) (clusterResource.getMemorySize() * percentages[0]); int vcore = (int) (clusterResource.getVirtualCores() * percentages[1]); - Resource res = Resource.newInstance(memory, vcore); - ResourceInformation[] clusterInfo = clusterResource.getResources(); - - for (int i = 2; i < clusterInfo.length; i++) { - res.setResourceValue(i, - (long)(clusterInfo[i].getValue() * percentages[i])); - } - - return res; + return Resource.newInstance(memory, vcore); } else { return resource; } @@ -97,39 +69,4 @@ public class ConfigurableResource { public Resource getResource() { return resource; } - - /** - * Set the value of the wrapped resource if this object isn't setup to use - * percentages. If this object is set to use percentages, this method has - * no effect. - * - * @param name the name of the resource - * @param value the value - */ - void setValue(String name, long value) { - if (resource != null) { - resource.setResourceValue(name, value); - } - } - - /** - * Set the percentage of the resource if this object is setup to use - * percentages. If this object is set to use percentages, this method has - * no effect. - * - * @param name the name of the resource - * @param value the percentage - */ - void setPercentage(String name, double value) { - if (percentages != null) { - Integer index = ResourceUtils.getResourceTypeIndex().get(name); - - if (index != null) { - percentages[index] = value; - } else { - throw new ResourceNotFoundException("The requested resource, \"" - + name + "\", could not be found."); - } - } - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java index 8c4932bfe67..b50e4bbe317 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java @@ -33,7 +33,6 @@ import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.UnitsConversionUtil; import org.apache.hadoop.yarn.util.resource.ResourceUtils; @@ -214,9 +213,6 @@ public class FairSchedulerConfiguration extends Configuration { CONF_PREFIX + "reservable-nodes"; public static final float RESERVABLE_NODES_DEFAULT = 0.05f; - private static final String INVALID_RESOURCE_DEFINITION_PREFIX = - "Error reading resource config--invalid resource definition: "; - public FairSchedulerConfiguration() { super(); } @@ -411,167 +407,54 @@ public class FairSchedulerConfiguration extends Configuration { } /** - * Parses a resource config value in one of three forms: - *

    - *
  1. Percentage: "50%" or "40% memory, 60% cpu"
  2. - *
  3. New style resources: "vcores=10, memory-mb=1024" - * or "vcores=60%, memory-mb=40%"
  4. - *
  5. Old style resources: "1024 mb, 10 vcores"
  6. - *
- * In new style resources, any resource that is not specified will be - * set to {@link Long#MAX_VALUE} or 100%, as appropriate. Also, in the new - * style resources, units are not allowed. Units are assumed from the resource - * manager's settings for the resources when the value isn't a percentage. - * - * @param value the resource definition to parse - * @return a {@link ConfigurableResource} that represents the parsed value - * @throws AllocationConfigurationException if the raw value is not a valid - * resource definition + * Parses a resource config value of a form like "1024", "1024 mb", + * or "1024 mb, 3 vcores". If no units are given, megabytes are assumed. + * + * @throws AllocationConfigurationException */ - public static ConfigurableResource parseResourceConfigValue(String value) + public static ConfigurableResource parseResourceConfigValue(String val) throws AllocationConfigurationException { - return parseResourceConfigValue(value, Long.MAX_VALUE); - } - - /** - * Parses a resource config value in one of three forms: - *
    - *
  1. Percentage: "50%" or "40% memory, 60% cpu"
  2. - *
  3. New style resources: "vcores=10, memory-mb=1024" - * or "vcores=60%, memory-mb=40%"
  4. - *
  5. Old style resources: "1024 mb, 10 vcores"
  6. - *
- * In new style resources, any resource that is not specified will be - * set to {@code missing} or 0%, as appropriate. Also, in the new style - * resources, units are not allowed. Units are assumed from the resource - * manager's settings for the resources when the value isn't a percentage. - * - * The {@code missing} parameter is only used in the case of new style - * resources without percentages. With new style resources with percentages, - * any missing resources will be assumed to be 100% because percentages are - * only used with maximum resource limits. - * - * @param value the resource definition to parse - * @param missing the value to use for any unspecified resources - * @return a {@link ConfigurableResource} that represents the parsed value - * @throws AllocationConfigurationException if the raw value is not a valid - * resource definition - */ - public static ConfigurableResource parseResourceConfigValue(String value, - long missing) throws AllocationConfigurationException { ConfigurableResource configurableResource; - - if (value.trim().isEmpty()) { - throw new AllocationConfigurationException("Error reading resource " - + "config--the resource string is empty."); - } - try { - if (value.contains("=")) { - configurableResource = parseNewStyleResource(value, missing); - } else if (value.contains("%")) { - configurableResource = parseOldStyleResourceAsPercentage(value); + val = StringUtils.toLowerCase(val); + if (val.contains("%")) { + configurableResource = new ConfigurableResource( + getResourcePercentage(val)); } else { - configurableResource = parseOldStyleResource(value); + int memory = findResource(val, "mb"); + int vcores = findResource(val, "vcores"); + configurableResource = new ConfigurableResource( + BuilderUtils.newResource(memory, vcores)); } - } catch (RuntimeException ex) { + } catch (AllocationConfigurationException ex) { + throw ex; + } catch (Exception ex) { throw new AllocationConfigurationException( "Error reading resource config", ex); } - return configurableResource; } - private static ConfigurableResource parseNewStyleResource(String value, - long missing) throws AllocationConfigurationException { - - final ConfigurableResource configurableResource; - boolean asPercent = value.contains("%"); - if (asPercent) { - configurableResource = new ConfigurableResource(); - } else { - configurableResource = new ConfigurableResource(missing); - } - - String[] resources = value.split(","); - for (String resource : resources) { - String[] parts = resource.split("="); - - if (parts.length != 2) { - throw createConfigException(value, - "Every resource must be of the form: name=value."); - } - - String resourceName = parts[0].trim(); - String resourceValue = parts[1].trim(); - try { - if (asPercent) { - configurableResource.setPercentage(resourceName, - findPercentage(resourceValue, "")); - } else { - configurableResource.setValue(resourceName, - Long.parseLong(resourceValue)); - } - } catch (ResourceNotFoundException ex) { - throw createConfigException(value, "The " - + "resource name, \"" + resourceName + "\" was not " - + "recognized. Please check the value of " - + YarnConfiguration.RESOURCE_TYPES + " in the Resource " - + "Manager's configuration files.", ex); - } catch (NumberFormatException ex) { - // This only comes from Long.parseLong() - throw createConfigException(value, "The " - + "resource values must all be integers. \"" + resourceValue - + "\" is not an integer.", ex); - } catch (AllocationConfigurationException ex) { - // This only comes from findPercentage() - throw createConfigException(value, "The " - + "resource values must all be percentages. \"" - + resourceValue + "\" is either not a number or does not " - + "include the '%' symbol.", ex); - } - } - return configurableResource; - } - - private static ConfigurableResource parseOldStyleResourceAsPercentage( - String value) throws AllocationConfigurationException { - return new ConfigurableResource( - getResourcePercentage(StringUtils.toLowerCase(value))); - } - - private static ConfigurableResource parseOldStyleResource(String value) - throws AllocationConfigurationException { - final String lCaseValue = StringUtils.toLowerCase(value); - int memory = findResource(lCaseValue, "mb"); - int vcores = findResource(lCaseValue, "vcores"); - - return new ConfigurableResource( - BuilderUtils.newResource(memory, vcores)); - } - private static double[] getResourcePercentage( String val) throws AllocationConfigurationException { int numberOfKnownResourceTypes = ResourceUtils .getNumberOfKnownResourceTypes(); double[] resourcePercentage = new double[numberOfKnownResourceTypes]; String[] strings = val.split(","); - if (strings.length == 1) { double percentage = findPercentage(strings[0], ""); for (int i = 0; i < numberOfKnownResourceTypes; i++) { - resourcePercentage[i] = percentage; + resourcePercentage[i] = percentage/100; } } else { - resourcePercentage[0] = findPercentage(val, "memory"); - resourcePercentage[1] = findPercentage(val, "cpu"); + resourcePercentage[0] = findPercentage(val, "memory")/100; + resourcePercentage[1] = findPercentage(val, "cpu")/100; } - return resourcePercentage; } private static double findPercentage(String val, String units) - throws AllocationConfigurationException { + throws AllocationConfigurationException { final Pattern pattern = Pattern.compile("((\\d+)(\\.\\d*)?)\\s*%\\s*" + units); Matcher matcher = pattern.matcher(val); @@ -584,22 +467,7 @@ public class FairSchedulerConfiguration extends Configuration { units); } } - return Double.parseDouble(matcher.group(1)) / 100.0; - } - - private static AllocationConfigurationException createConfigException( - String value, String message) { - return createConfigException(value, message, null); - } - - private static AllocationConfigurationException createConfigException( - String value, String message, Throwable t) { - String msg = INVALID_RESOURCE_DEFINITION_PREFIX + value + ". " + message; - if (t != null) { - return new AllocationConfigurationException(msg, t); - } else { - return new AllocationConfigurationException(msg); - } + return Double.parseDouble(matcher.group(1)); } public long getUpdateInterval() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java index 441c34a1aa1..d5a436ed0a9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java @@ -134,7 +134,7 @@ public class AllocationFileQueueParser { if (MIN_RESOURCES.equals(field.getTagName())) { String text = getTrimmedTextData(field); ConfigurableResource val = - FairSchedulerConfiguration.parseResourceConfigValue(text, 0L); + FairSchedulerConfiguration.parseResourceConfigValue(text); builder.minQueueResources(queueName, val.getResource()); } else if (MAX_RESOURCES.equals(field.getTagName())) { String text = getTrimmedTextData(field); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java index 9d82bc78c76..d47f13d9586 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java @@ -479,7 +479,7 @@ public class AppInfo { public int getNumNonAMContainersPreempted() { return numNonAMContainerPreempted; } - + public int getNumAMContainersPreempted() { return numAMContainerPreempted; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java index 163f707253a..81491b14ce1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java @@ -41,9 +41,8 @@ public class SchedulerInfo { protected EnumSet schedulingResourceTypes; protected int maximumClusterPriority; - // JAXB needs this public SchedulerInfo() { - } + } // JAXB needs this public SchedulerInfo(final ResourceManager rm) { ResourceScheduler rs = rm.getResourceScheduler(); @@ -75,10 +74,7 @@ public class SchedulerInfo { } public String getSchedulerResourceTypes() { - if (minAllocResource != null) { - return Arrays.toString(minAllocResource.getResource().getResources()); - } - return null; + return Arrays.toString(minAllocResource.getResource().getResources()); } public int getMaxClusterLevelAppPriority() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java index 70f83ab3095..481645bb494 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java @@ -48,9 +48,6 @@ import org.apache.log4j.spi.LoggingEvent; import org.junit.Assert; import org.junit.Test; -/** - * Tests fair scheduler configuration. - */ public class TestFairSchedulerConfiguration { private static final String A_CUSTOM_RESOURCE = "a-custom-resource"; @@ -105,152 +102,67 @@ public class TestFairSchedulerConfiguration { @Test public void testParseResourceConfigValue() throws Exception { - Resource expected = BuilderUtils.newResource(5 * 1024, 2); - Resource clusterResource = BuilderUtils.newResource(10 * 1024, 4); + assertEquals(BuilderUtils.newResource(1024, 2), + parseResourceConfigValue("2 vcores, 1024 mb").getResource()); + assertEquals(BuilderUtils.newResource(1024, 2), + parseResourceConfigValue("1024 mb, 2 vcores").getResource()); + assertEquals(BuilderUtils.newResource(1024, 2), + parseResourceConfigValue("2vcores,1024mb").getResource()); + assertEquals(BuilderUtils.newResource(1024, 2), + parseResourceConfigValue("1024mb,2vcores").getResource()); + assertEquals(BuilderUtils.newResource(1024, 2), + parseResourceConfigValue("1024 mb, 2 vcores").getResource()); + assertEquals(BuilderUtils.newResource(1024, 2), + parseResourceConfigValue("1024 Mb, 2 vCores").getResource()); + assertEquals(BuilderUtils.newResource(1024, 2), + parseResourceConfigValue(" 1024 mb, 2 vcores ").getResource()); + assertEquals(BuilderUtils.newResource(1024, 2), + parseResourceConfigValue(" 1024.3 mb, 2.35 vcores ").getResource()); + assertEquals(BuilderUtils.newResource(1024, 2), + parseResourceConfigValue(" 1024. mb, 2. vcores ").getResource()); - assertEquals(expected, - parseResourceConfigValue("2 vcores, 5120 mb").getResource()); - assertEquals(expected, - parseResourceConfigValue("5120 mb, 2 vcores").getResource()); - assertEquals(expected, - parseResourceConfigValue("2vcores,5120mb").getResource()); - assertEquals(expected, - parseResourceConfigValue("5120mb,2vcores").getResource()); - assertEquals(expected, - parseResourceConfigValue("5120mb mb, 2 vcores").getResource()); - assertEquals(expected, - parseResourceConfigValue("5120 Mb, 2 vCores").getResource()); - assertEquals(expected, - parseResourceConfigValue(" 5120 mb, 2 vcores ").getResource()); - assertEquals(expected, - parseResourceConfigValue(" 5120.3 mb, 2.35 vcores ").getResource()); - assertEquals(expected, - parseResourceConfigValue(" 5120. mb, 2. vcores ").getResource()); - - assertEquals(expected, + Resource clusterResource = BuilderUtils.newResource(2048, 4); + assertEquals(BuilderUtils.newResource(1024, 2), parseResourceConfigValue("50% memory, 50% cpu"). getResource(clusterResource)); - assertEquals(expected, + assertEquals(BuilderUtils.newResource(1024, 2), parseResourceConfigValue("50% Memory, 50% CpU"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(5 * 1024, 4), + assertEquals(BuilderUtils.newResource(1024, 2), + parseResourceConfigValue("50%").getResource(clusterResource)); + assertEquals(BuilderUtils.newResource(1024, 4), parseResourceConfigValue("50% memory, 100% cpu"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(5 * 1024, 4), + assertEquals(BuilderUtils.newResource(1024, 4), parseResourceConfigValue(" 100% cpu, 50% memory"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(5 * 1024, 0), + assertEquals(BuilderUtils.newResource(1024, 0), parseResourceConfigValue("50% memory, 0% cpu"). getResource(clusterResource)); - assertEquals(expected, + assertEquals(BuilderUtils.newResource(1024, 2), parseResourceConfigValue("50 % memory, 50 % cpu"). getResource(clusterResource)); - assertEquals(expected, + assertEquals(BuilderUtils.newResource(1024, 2), parseResourceConfigValue("50%memory,50%cpu"). getResource(clusterResource)); - assertEquals(expected, + assertEquals(BuilderUtils.newResource(1024, 2), parseResourceConfigValue(" 50 % memory, 50 % cpu "). getResource(clusterResource)); - assertEquals(expected, + assertEquals(BuilderUtils.newResource(1024, 2), parseResourceConfigValue("50.% memory, 50.% cpu"). getResource(clusterResource)); + + clusterResource = BuilderUtils.newResource(1024 * 10, 4); assertEquals(BuilderUtils.newResource((int)(1024 * 10 * 0.109), 2), parseResourceConfigValue("10.9% memory, 50.6% cpu"). getResource(clusterResource)); - assertEquals(expected, - parseResourceConfigValue("50%").getResource(clusterResource)); - - Configuration conf = new Configuration(); - - conf.set(YarnConfiguration.RESOURCE_TYPES, "test1"); - ResourceUtils.resetResourceTypes(conf); - - clusterResource = BuilderUtils.newResource(10 * 1024, 4); - expected = BuilderUtils.newResource(5 * 1024, 2); - expected.setResourceValue("test1", Long.MAX_VALUE); - - assertEquals(expected, - parseResourceConfigValue("vcores=2, memory-mb=5120").getResource()); - assertEquals(expected, - parseResourceConfigValue("memory-mb=5120, vcores=2").getResource()); - assertEquals(expected, - parseResourceConfigValue("vcores=2,memory-mb=5120").getResource()); - assertEquals(expected, parseResourceConfigValue(" vcores = 2 , " - + "memory-mb = 5120 ").getResource()); - - expected.setResourceValue("test1", 0L); - - assertEquals(expected, - parseResourceConfigValue("vcores=2, memory-mb=5120", 0L).getResource()); - assertEquals(expected, - parseResourceConfigValue("memory-mb=5120, vcores=2", 0L).getResource()); - assertEquals(expected, - parseResourceConfigValue("vcores=2,memory-mb=5120", 0L).getResource()); - assertEquals(expected, - parseResourceConfigValue(" vcores = 2 , memory-mb = 5120 ", - 0L).getResource()); - - clusterResource.setResourceValue("test1", 8L); - expected.setResourceValue("test1", 4L); - - assertEquals(expected, - parseResourceConfigValue("50%").getResource(clusterResource)); - assertEquals(expected, - parseResourceConfigValue("vcores=2, memory-mb=5120, " - + "test1=4").getResource()); - assertEquals(expected, - parseResourceConfigValue("test1=4, vcores=2, " - + "memory-mb=5120").getResource()); - assertEquals(expected, - parseResourceConfigValue("memory-mb=5120, test1=4, " - + "vcores=2").getResource()); - assertEquals(expected, - parseResourceConfigValue("vcores=2,memory-mb=5120," - + "test1=4").getResource()); - assertEquals(expected, - parseResourceConfigValue(" vcores = 2 , memory-mb = 5120 , " - + "test1 = 4 ").getResource()); - - expected = BuilderUtils.newResource(4 * 1024, 3); - expected.setResourceValue("test1", 8L); - - assertEquals(expected, - parseResourceConfigValue("vcores=75%, " - + "memory-mb=40%").getResource(clusterResource)); - assertEquals(expected, - parseResourceConfigValue("memory-mb=40%, " - + "vcores=75%").getResource(clusterResource)); - assertEquals(expected, - parseResourceConfigValue("vcores=75%," - + "memory-mb=40%").getResource(clusterResource)); - assertEquals(expected, - parseResourceConfigValue(" vcores = 75 % , " - + "memory-mb = 40 % ").getResource(clusterResource)); - - expected.setResourceValue("test1", 4L); - - assertEquals(expected, - parseResourceConfigValue("vcores=75%, memory-mb=40%, " - + "test1=50%").getResource(clusterResource)); - assertEquals(expected, - parseResourceConfigValue("test1=50%, vcores=75%, " - + "memory-mb=40%").getResource(clusterResource)); - assertEquals(expected, - parseResourceConfigValue("memory-mb=40%, test1=50%, " - + "vcores=75%").getResource(clusterResource)); - assertEquals(expected, - parseResourceConfigValue("vcores=75%,memory-mb=40%," - + "test1=50%").getResource(clusterResource)); - assertEquals(expected, - parseResourceConfigValue(" vcores = 75 % , memory-mb = 40 % , " - + "test1 = 50 % ").getResource(clusterResource)); } - + @Test(expected = AllocationConfigurationException.class) public void testNoUnits() throws Exception { parseResourceConfigValue("1024"); } - + @Test(expected = AllocationConfigurationException.class) public void testOnlyMemory() throws Exception { parseResourceConfigValue("1024mb"); @@ -260,7 +172,7 @@ public class TestFairSchedulerConfiguration { public void testOnlyCPU() throws Exception { parseResourceConfigValue("1024vcores"); } - + @Test(expected = AllocationConfigurationException.class) public void testGibberish() throws Exception { parseResourceConfigValue("1o24vc0res"); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java index 3902889a4a1..0702d652a02 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java @@ -53,7 +53,11 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.*; +import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService; +import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; @@ -72,12 +76,11 @@ import org.apache.hadoop.yarn.webapp.JerseyTestBase; import org.apache.hadoop.yarn.webapp.WebServicesTestUtils; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; +import org.eclipse.jetty.server.Response; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -93,8 +96,6 @@ import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; import com.sun.jersey.test.framework.WebAppDescriptor; public class TestRMWebServices extends JerseyTestBase { - private static final Logger LOG = - LoggerFactory.getLogger(TestRMWebServices.class); private static MockRM rm; @@ -471,19 +472,19 @@ public class TestRMWebServices extends JerseyTestBase { QueueMetrics metrics = rs.getRootQueueMetrics(); ClusterMetrics clusterMetrics = ClusterMetrics.getMetrics(); - long totalMBExpect = + long totalMBExpect = metrics.getAvailableMB() + metrics.getAllocatedMB(); - long totalVirtualCoresExpect = + long totalVirtualCoresExpect = metrics.getAvailableVirtualCores() + metrics.getAllocatedVirtualCores(); - assertEquals("appsSubmitted doesn't match", + assertEquals("appsSubmitted doesn't match", metrics.getAppsSubmitted(), submittedApps); - assertEquals("appsCompleted doesn't match", + assertEquals("appsCompleted doesn't match", metrics.getAppsCompleted(), completedApps); assertEquals("reservedMB doesn't match", metrics.getReservedMB(), reservedMB); - assertEquals("availableMB doesn't match", + assertEquals("availableMB doesn't match", metrics.getAvailableMB(), availableMB); - assertEquals("allocatedMB doesn't match", + assertEquals("allocatedMB doesn't match", metrics.getAllocatedMB(), allocMB); assertEquals("reservedVirtualCores doesn't match", metrics.getReservedVirtualCores(), reservedVirtualCores); @@ -596,13 +597,11 @@ public class TestRMWebServices extends JerseyTestBase { public void verifyClusterSchedulerFifo(JSONObject json) throws JSONException, Exception { - assertEquals("incorrect number of elements in: " + json, 1, json.length()); + assertEquals("incorrect number of elements", 1, json.length()); JSONObject info = json.getJSONObject("scheduler"); - assertEquals("incorrect number of elements in: " + info, 1, info.length()); + assertEquals("incorrect number of elements", 1, info.length()); info = info.getJSONObject("schedulerInfo"); - - LOG.debug("schedulerInfo: {}", info); - assertEquals("incorrect number of elements in: " + info, 11, info.length()); + assertEquals("incorrect number of elements", 11, info.length()); verifyClusterSchedulerFifoGeneric(info.getString("type"), info.getString("qstate"), (float) info.getDouble("capacity"), diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java index 15f94e1bb00..6c6f400a623 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java @@ -79,7 +79,7 @@ import com.sun.jersey.test.framework.WebAppDescriptor; public class TestRMWebServicesApps extends JerseyTestBase { private static MockRM rm; - + private static final int CONTAINER_MB = 1024; private static class WebServletModule extends ServletModule { @@ -324,7 +324,7 @@ public class TestRMWebServicesApps extends JerseyTestBase { assertEquals("incorrect number of elements", 1, apps.length()); array = apps.getJSONArray("app"); assertEquals("incorrect number of elements", 2, array.length()); - assertTrue("both app states of ACCEPTED and KILLED are not present", + assertTrue("both app states of ACCEPTED and KILLED are not present", (array.getJSONObject(0).getString("state").equals("ACCEPTED") && array.getJSONObject(1).getString("state").equals("KILLED")) || (array.getJSONObject(0).getString("state").equals("KILLED") && @@ -375,12 +375,12 @@ public class TestRMWebServicesApps extends JerseyTestBase { assertEquals("incorrect number of elements", 1, apps.length()); array = apps.getJSONArray("app"); assertEquals("incorrect number of elements", 2, array.length()); - assertTrue("both app states of ACCEPTED and KILLED are not present", + assertTrue("both app states of ACCEPTED and KILLED are not present", (array.getJSONObject(0).getString("state").equals("ACCEPTED") && array.getJSONObject(1).getString("state").equals("KILLED")) || (array.getJSONObject(0).getString("state").equals("KILLED") && array.getJSONObject(1).getString("state").equals("ACCEPTED"))); - + rm.stop(); } @@ -511,8 +511,7 @@ public class TestRMWebServicesApps extends JerseyTestBase { WebResource r = resource(); ClientResponse response = r.path("ws").path("v1").path("cluster") - .path("apps").queryParam("finalStatus", - FinalApplicationStatus.UNDEFINED.toString()) + .path("apps").queryParam("finalStatus", FinalApplicationStatus.UNDEFINED.toString()) .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); @@ -1805,8 +1804,7 @@ public class TestRMWebServicesApps extends JerseyTestBase { int numAttempt = 1; while (true) { // fail the AM by sending CONTAINER_FINISHED event without registering. - amNodeManager.nodeHeartbeat(am.getApplicationAttemptId(), 1, - ContainerState.COMPLETE); + amNodeManager.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE); rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FAILED); if (numAttempt == maxAppAttempts) { rm.waitForState(app1.getApplicationId(), RMAppState.FAILED); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java deleted file mode 100644 index 83e00567eca..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java +++ /dev/null @@ -1,242 +0,0 @@ -/** - * 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.yarn.server.resourcemanager.webapp; - -import com.google.inject.Guice; -import com.google.inject.servlet.ServletModule; -import com.sun.jersey.api.client.ClientResponse; -import com.sun.jersey.api.client.WebResource; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import com.sun.jersey.test.framework.WebAppDescriptor; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.records.ResourceRequest; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.MockAM; -import org.apache.hadoop.yarn.server.resourcemanager.MockNM; -import org.apache.hadoop.yarn.server.resourcemanager.MockRM; -import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.fairscheduler.CustomResourceTypesConfigurationProvider; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.AppInfoJsonVerifications; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.AppInfoXmlVerifications; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.BufferedClientResponse; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.JsonCustomResourceTypeTestcase; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.ResourceRequestsJsonVerifications; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.ResourceRequestsXmlVerifications; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.XmlCustomResourceTypeTestCase; -import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; -import org.apache.hadoop.yarn.webapp.GuiceServletConfig; -import org.apache.hadoop.yarn.webapp.JerseyTestBase; -import org.codehaus.jettison.json.JSONArray; -import org.codehaus.jettison.json.JSONException; -import org.codehaus.jettison.json.JSONObject; -import org.junit.Before; -import org.junit.Test; -import org.w3c.dom.Element; -import org.w3c.dom.Node; -import org.w3c.dom.NodeList; - -import javax.ws.rs.core.MediaType; -import java.util.ArrayList; - -import static org.junit.Assert.assertEquals; - -/** - * This test verifies that custom resource types are correctly serialized to XML - * and JSON when HTTP GET request is sent to the resource: ws/v1/cluster/apps. - */ -public class TestRMWebServicesAppsCustomResourceTypes extends JerseyTestBase { - - private static MockRM rm; - private static final int CONTAINER_MB = 1024; - - private static class WebServletModule extends ServletModule { - @Override - protected void configureServlets() { - bind(JAXBContextResolver.class); - bind(RMWebServices.class); - bind(GenericExceptionHandler.class); - Configuration conf = new Configuration(); - conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, - YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); - conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class, - ResourceScheduler.class); - initResourceTypes(conf); - rm = new MockRM(conf); - bind(ResourceManager.class).toInstance(rm); - serve("/*").with(GuiceContainer.class); - } - - private void initResourceTypes(Configuration conf) { - conf.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS, - CustomResourceTypesConfigurationProvider.class.getName()); - ResourceUtils.resetResourceTypes(conf); - } - } - - @Before - @Override - public void setUp() throws Exception { - super.setUp(); - createInjectorForWebServletModule(); - } - - private void createInjectorForWebServletModule() { - GuiceServletConfig - .setInjector(Guice.createInjector(new WebServletModule())); - } - - public TestRMWebServicesAppsCustomResourceTypes() { - super(new WebAppDescriptor.Builder( - "org.apache.hadoop.yarn.server.resourcemanager.webapp") - .contextListenerClass(GuiceServletConfig.class) - .filterClass(com.google.inject.servlet.GuiceFilter.class) - .contextPath("jersey-guice-filter").servletPath("/").build()); - } - - @Test - public void testRunningAppXml() throws Exception { - rm.start(); - MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048); - RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1"); - MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, amNodeManager); - am1.allocate("*", 2048, 1, new ArrayList<>()); - amNodeManager.nodeHeartbeat(true); - - WebResource r = resource(); - WebResource path = r.path("ws").path("v1").path("cluster").path("apps"); - ClientResponse response = - path.accept(MediaType.APPLICATION_XML).get(ClientResponse.class); - - XmlCustomResourceTypeTestCase testCase = - new XmlCustomResourceTypeTestCase(path, - new BufferedClientResponse(response)); - testCase.verify(document -> { - NodeList apps = document.getElementsByTagName("apps"); - assertEquals("incorrect number of apps elements", 1, apps.getLength()); - - NodeList appArray = ((Element)(apps.item(0))) - .getElementsByTagName("app"); - assertEquals("incorrect number of app elements", 1, appArray.getLength()); - - verifyAppsXML(appArray, app1); - }); - - rm.stop(); - } - - @Test - public void testRunningAppJson() throws Exception { - rm.start(); - MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048); - RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1"); - MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, amNodeManager); - am1.allocate("*", 2048, 1, new ArrayList<>()); - amNodeManager.nodeHeartbeat(true); - - WebResource r = resource(); - WebResource path = r.path("ws").path("v1").path("cluster").path("apps"); - ClientResponse response = - path.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - - JsonCustomResourceTypeTestcase testCase = - new JsonCustomResourceTypeTestcase(path, - new BufferedClientResponse(response)); - testCase.verify(json -> { - try { - assertEquals("incorrect number of apps elements", 1, json.length()); - JSONObject apps = json.getJSONObject("apps"); - assertEquals("incorrect number of app elements", 1, apps.length()); - JSONArray array = apps.getJSONArray("app"); - assertEquals("incorrect count of app", 1, array.length()); - - verifyAppInfoJson(array.getJSONObject(0), app1); - } catch (JSONException e) { - throw new RuntimeException(e); - } - }); - - rm.stop(); - } - - private void verifyAppsXML(NodeList appArray, RMApp app) { - for (int i = 0; i < appArray.getLength(); i++) { - Element element = (Element) appArray.item(i); - AppInfoXmlVerifications.verify(element, app); - - NodeList resourceRequests = - element.getElementsByTagName("resourceRequests"); - assertEquals(1, resourceRequests.getLength()); - Node resourceRequest = resourceRequests.item(0); - ResourceRequest rr = - ((AbstractYarnScheduler) rm.getRMContext().getScheduler()) - .getApplicationAttempt( - app.getCurrentAppAttempt().getAppAttemptId()) - .getAppSchedulingInfo().getAllResourceRequests().get(0); - ResourceRequestsXmlVerifications.verifyWithCustomResourceTypes( - (Element) resourceRequest, rr, - CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); - } - } - - private void verifyAppInfoJson(JSONObject info, RMApp app) throws - JSONException { - int expectedNumberOfElements = getExpectedNumberOfElements(app); - - assertEquals("incorrect number of elements", expectedNumberOfElements, - info.length()); - - AppInfoJsonVerifications.verify(info, app); - - JSONArray resourceRequests = info.getJSONArray("resourceRequests"); - JSONObject requestInfo = resourceRequests.getJSONObject(0); - ResourceRequest rr = - ((AbstractYarnScheduler) rm.getRMContext().getScheduler()) - .getApplicationAttempt(app.getCurrentAppAttempt().getAppAttemptId()) - .getAppSchedulingInfo().getAllResourceRequests().get(0); - - ResourceRequestsJsonVerifications.verifyWithCustomResourceTypes( - requestInfo, rr, - CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); - } - - private int getExpectedNumberOfElements(RMApp app) { - int expectedNumberOfElements = 40 + 2; // 2 -> resourceRequests - if (app.getApplicationSubmissionContext() - .getNodeLabelExpression() != null) { - expectedNumberOfElements++; - } - - if (app.getAMResourceRequests().get(0).getNodeLabelExpression() != null) { - expectedNumberOfElements++; - } - - if (AppInfo - .getAmRPCAddressFromRMAppAttempt(app.getCurrentAppAttempt()) != null) { - expectedNumberOfElements++; - } - return expectedNumberOfElements; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java index 46d0a6614fa..e37f76fa25f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java @@ -146,7 +146,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { config.setUserLimitFactor(B2, 100.0f); config.setCapacity(B3, 0.5f); config.setUserLimitFactor(B3, 100.0f); - + config.setQueues(A1, new String[] {"a1a", "a1b"}); final String A1A = A1 + ".a1a"; config.setCapacity(A1A, 85); @@ -254,7 +254,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { } } - public void verifySubQueueXML(Element qElem, String q, + public void verifySubQueueXML(Element qElem, String q, float parentAbsCapacity, float parentAbsMaxCapacity) throws Exception { NodeList children = qElem.getChildNodes(); @@ -317,34 +317,30 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { private void verifyClusterScheduler(JSONObject json) throws JSONException, Exception { - assertEquals("incorrect number of elements in: " + json, 1, json.length()); + assertEquals("incorrect number of elements", 1, json.length()); JSONObject info = json.getJSONObject("scheduler"); - assertEquals("incorrect number of elements in: " + info, 1, info.length()); + assertEquals("incorrect number of elements", 1, info.length()); info = info.getJSONObject("schedulerInfo"); - assertEquals("incorrect number of elements in: " + info, 8, info.length()); + assertEquals("incorrect number of elements", 8, info.length()); verifyClusterSchedulerGeneric(info.getString("type"), (float) info.getDouble("usedCapacity"), (float) info.getDouble("capacity"), (float) info.getDouble("maxCapacity"), info.getString("queueName")); JSONObject health = info.getJSONObject("health"); assertNotNull(health); - assertEquals("incorrect number of elements in: " + health, 3, - health.length()); + assertEquals("incorrect number of elements", 3, health.length()); JSONArray operationsInfo = health.getJSONArray("operationsInfo"); - assertEquals("incorrect number of elements in: " + health, 4, - operationsInfo.length()); + assertEquals("incorrect number of elements", 4, operationsInfo.length()); JSONArray lastRunDetails = health.getJSONArray("lastRunDetails"); - assertEquals("incorrect number of elements in: " + health, 3, - lastRunDetails.length()); + assertEquals("incorrect number of elements", 3, lastRunDetails.length()); JSONArray arr = info.getJSONObject("queues").getJSONArray("queue"); - assertEquals("incorrect number of elements in: " + arr, 2, arr.length()); + assertEquals("incorrect number of elements", 2, arr.length()); // test subqueues for (int i = 0; i < arr.length(); i++) { JSONObject obj = arr.getJSONObject(i); - String q = CapacitySchedulerConfiguration.ROOT + "." + - obj.getString("queueName"); + String q = CapacitySchedulerConfiguration.ROOT + "." + obj.getString("queueName"); verifySubQueue(obj, q, 100, 100); } } @@ -359,7 +355,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { assertTrue("queueName doesn't match", "root".matches(queueName)); } - private void verifySubQueue(JSONObject info, String q, + private void verifySubQueue(JSONObject info, String q, float parentAbsCapacity, float parentAbsMaxCapacity) throws JSONException, Exception { int numExpectedElements = 20; @@ -468,7 +464,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { csConf.getUserLimitFactor(q), info.userLimitFactor, 1e-3f); } - //Return a child Node of node with the tagname or null if none exists + //Return a child Node of node with the tagname or null if none exists private Node getChildNodeByName(Node node, String tagname) { NodeList nodeList = node.getChildNodes(); for (int i=0; i < nodeList.getLength(); ++i) { @@ -518,7 +514,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { for (int j=0; j - * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * + * 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. @@ -16,14 +16,13 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.server.resourcemanager.webapp.fairscheduler; +package org.apache.hadoop.yarn.server.resourcemanager.webapp; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import javax.ws.rs.core.MediaType; -import com.google.inject.Guice; -import com.google.inject.servlet.ServletModule; -import com.sun.jersey.api.client.ClientResponse; -import com.sun.jersey.api.client.WebResource; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import com.sun.jersey.test.framework.WebAppDescriptor; import org.apache.hadoop.http.JettyUtils; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.MockRM; @@ -31,9 +30,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueueManager; - -import org.apache.hadoop.yarn.server.resourcemanager.webapp.JAXBContextResolver; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServices; import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; import org.apache.hadoop.yarn.webapp.GuiceServletConfig; import org.apache.hadoop.yarn.webapp.JerseyTestBase; @@ -42,18 +38,18 @@ import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; import org.junit.Before; import org.junit.Test; -import javax.ws.rs.core.MediaType; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import com.google.inject.Guice; +import com.google.inject.servlet.ServletModule; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; +import com.sun.jersey.test.framework.WebAppDescriptor; -/** - * Tests RM Webservices fair scheduler resources. - */ public class TestRMWebServicesFairScheduler extends JerseyTestBase { private static MockRM rm; private static YarnConfiguration conf; - + private static class WebServletModule extends ServletModule { @Override protected void configureServlets() { @@ -62,7 +58,7 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase { bind(GenericExceptionHandler.class); conf = new YarnConfiguration(); conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class, - ResourceScheduler.class); + ResourceScheduler.class); rm = new MockRM(conf); bind(ResourceManager.class).toInstance(rm); serve("/*").with(GuiceContainer.class); @@ -70,32 +66,32 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase { } static { - GuiceServletConfig - .setInjector(Guice.createInjector(new WebServletModule())); + GuiceServletConfig.setInjector( + Guice.createInjector(new WebServletModule())); } @Before @Override public void setUp() throws Exception { super.setUp(); - GuiceServletConfig - .setInjector(Guice.createInjector(new WebServletModule())); + GuiceServletConfig.setInjector( + Guice.createInjector(new WebServletModule())); } public TestRMWebServicesFairScheduler() { super(new WebAppDescriptor.Builder( "org.apache.hadoop.yarn.server.resourcemanager.webapp") - .contextListenerClass(GuiceServletConfig.class) - .filterClass(com.google.inject.servlet.GuiceFilter.class) - .contextPath("jersey-guice-filter").servletPath("/").build()); + .contextListenerClass(GuiceServletConfig.class) + .filterClass(com.google.inject.servlet.GuiceFilter.class) + .contextPath("jersey-guice-filter").servletPath("/").build()); } - + @Test - public void testClusterScheduler() throws JSONException { + public void testClusterScheduler() throws JSONException, Exception { WebResource r = resource(); - ClientResponse response = - r.path("ws").path("v1").path("cluster").path("scheduler") - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + ClientResponse response = r.path("ws").path("v1").path("cluster") + .path("scheduler").accept(MediaType.APPLICATION_JSON) + .get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); JSONObject json = response.getEntity(JSONObject.class); @@ -103,51 +99,52 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase { } @Test - public void testClusterSchedulerSlash() throws JSONException { + public void testClusterSchedulerSlash() throws JSONException, Exception { WebResource r = resource(); - ClientResponse response = - r.path("ws").path("v1").path("cluster").path("scheduler/") - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + ClientResponse response = r.path("ws").path("v1").path("cluster") + .path("scheduler/").accept(MediaType.APPLICATION_JSON) + .get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); JSONObject json = response.getEntity(JSONObject.class); verifyClusterScheduler(json); } - + @Test - public void testClusterSchedulerWithSubQueues() - throws JSONException { - FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); + public void testClusterSchedulerWithSubQueues() throws JSONException, + Exception { + FairScheduler scheduler = (FairScheduler)rm.getResourceScheduler(); QueueManager queueManager = scheduler.getQueueManager(); // create LeafQueue queueManager.getLeafQueue("root.q.subqueue1", true); queueManager.getLeafQueue("root.q.subqueue2", true); WebResource r = resource(); - ClientResponse response = - r.path("ws").path("v1").path("cluster").path("scheduler") - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + ClientResponse response = r.path("ws").path("v1").path("cluster") + .path("scheduler").accept(MediaType.APPLICATION_JSON) + .get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); JSONObject json = response.getEntity(JSONObject.class); JSONArray subQueueInfo = json.getJSONObject("scheduler") .getJSONObject("schedulerInfo").getJSONObject("rootQueue") - .getJSONObject("childQueues").getJSONArray("queue").getJSONObject(1) - .getJSONObject("childQueues").getJSONArray("queue"); + .getJSONObject("childQueues").getJSONArray("queue") + .getJSONObject(1).getJSONObject("childQueues").getJSONArray("queue"); // subQueueInfo is consist of subqueue1 and subqueue2 info assertEquals(2, subQueueInfo.length()); // Verify 'childQueues' field is omitted from FairSchedulerLeafQueueInfo. try { subQueueInfo.getJSONObject(1).getJSONObject("childQueues"); - fail("FairSchedulerQueueInfo should omit field 'childQueues'" - + "if child queue is empty."); + fail("FairSchedulerQueueInfo should omit field 'childQueues'" + + "if child queue is empty."); } catch (JSONException je) { assertEquals("JSONObject[\"childQueues\"] not found.", je.getMessage()); } } - private void verifyClusterScheduler(JSONObject json) throws JSONException { + private void verifyClusterScheduler(JSONObject json) throws JSONException, + Exception { assertEquals("incorrect number of elements", 1, json.length()); JSONObject info = json.getJSONObject("scheduler"); assertEquals("incorrect number of elements", 1, info.length()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java index 40cf483cd3a..1e61186c3ab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java @@ -457,7 +457,7 @@ public class TestRMWebServicesSchedulerActivities if (object.getClass() == JSONObject.class) { assertEquals("Number of allocations is wrong", 1, realValue); } else if (object.getClass() == JSONArray.class) { - assertEquals("Number of allocations is wrong in: " + object, + assertEquals("Number of allocations is wrong", ((JSONArray) object).length(), realValue); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java deleted file mode 100644 index bb1fce05a46..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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.yarn.server.resourcemanager.webapp.fairscheduler; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.LocalConfigurationProvider; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; - -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import static java.util.stream.Collectors.toList; - -/** - * This class can generate an XML configuration file of custom resource types. - * See createInitialResourceTypes for the default values. All custom resource - * type is prefixed with CUSTOM_RESOURCE_PREFIX. Please use the - * getConfigurationInputStream method to get an InputStream of the XML. If you - * want to have different number of resources in your tests, please see usages - * of this class in this test class: - * {@link TestRMWebServicesFairSchedulerCustomResourceTypes} - * - */ -public class CustomResourceTypesConfigurationProvider - extends LocalConfigurationProvider { - - private static class CustomResourceTypes { - private int count; - private String xml; - - CustomResourceTypes(String xml, int count) { - this.xml = xml; - this.count = count; - } - - public int getCount() { - return count; - } - - public String getXml() { - return xml; - } - } - - private static final String CUSTOM_RESOURCE_PREFIX = "customResource-"; - - private static CustomResourceTypes customResourceTypes = - createInitialResourceTypes(); - - private static CustomResourceTypes createInitialResourceTypes() { - return createCustomResourceTypes(2); - } - - private static CustomResourceTypes createCustomResourceTypes(int count) { - List resourceTypeNames = generateResourceTypeNames(count); - - List resourceUnitXmlElements = IntStream.range(0, count) - .boxed() - .map(i -> getResourceUnitsXml(resourceTypeNames.get(i))) - .collect(toList()); - - StringBuilder sb = new StringBuilder("\n"); - sb.append(getResourceTypesXml(resourceTypeNames)); - - for (String resourceUnitXml : resourceUnitXmlElements) { - sb.append(resourceUnitXml); - - } - sb.append(""); - - return new CustomResourceTypes(sb.toString(), count); - } - - private static List generateResourceTypeNames(int count) { - return IntStream.range(0, count) - .boxed() - .map(i -> CUSTOM_RESOURCE_PREFIX + i) - .collect(toList()); - } - - private static String getResourceUnitsXml(String resource) { - return "\n" + "yarn.resource-types." + resource - + ".units\n" + "k\n" + "\n"; - } - - private static String getResourceTypesXml(List resources) { - final String resourceTypes = makeCommaSeparatedString(resources); - - return "\n" + "yarn.resource-types\n" + "" - + resourceTypes + "\n" + "\n"; - } - - private static String makeCommaSeparatedString(List resources) { - return resources.stream().collect(Collectors.joining(",")); - } - - @Override - public InputStream getConfigurationInputStream(Configuration bootstrapConf, - String name) throws YarnException, IOException { - if (YarnConfiguration.RESOURCE_TYPES_CONFIGURATION_FILE.equals(name)) { - return new ByteArrayInputStream( - customResourceTypes.getXml().getBytes()); - } else { - return super.getConfigurationInputStream(bootstrapConf, name); - } - } - - public static void reset() { - customResourceTypes = createInitialResourceTypes(); - } - - public static void setNumberOfResourceTypes(int count) { - customResourceTypes = createCustomResourceTypes(count); - } - - public static List getCustomResourceTypes() { - return generateResourceTypeNames(customResourceTypes.getCount()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java deleted file mode 100644 index 924411a4de0..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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.yarn.server.resourcemanager.webapp.fairscheduler; - -import com.google.common.collect.Sets; -import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; -import org.apache.hadoop.yarn.api.records.ResourceInformation; -import org.codehaus.jettison.json.JSONArray; -import org.codehaus.jettison.json.JSONException; -import org.codehaus.jettison.json.JSONObject; - -import java.util.List; -import java.util.Set; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - -/** - * This test helper class is primarily used by - * {@link TestRMWebServicesFairSchedulerCustomResourceTypes}. - */ -public class FairSchedulerJsonVerifications { - - private static final Set RESOURCE_FIELDS = - Sets.newHashSet("minResources", "amUsedResources", "amMaxResources", - "fairResources", "clusterResources", "reservedResources", - "maxResources", "usedResources", "steadyFairResources", - "demandResources"); - private final Set customResourceTypes; - - FairSchedulerJsonVerifications(List customResourceTypes) { - this.customResourceTypes = Sets.newHashSet(customResourceTypes); - } - - public void verify(JSONObject jsonObject) { - try { - verifyResourcesContainDefaultResourceTypes(jsonObject, RESOURCE_FIELDS); - verifyResourcesContainCustomResourceTypes(jsonObject, RESOURCE_FIELDS); - } catch (JSONException e) { - throw new RuntimeException(e); - } - } - - private void verifyResourcesContainDefaultResourceTypes(JSONObject queue, - Set resourceCategories) throws JSONException { - for (String resourceCategory : resourceCategories) { - boolean hasResourceCategory = queue.has(resourceCategory); - assertTrue("Queue " + queue + " does not have resource category key: " - + resourceCategory, hasResourceCategory); - verifyResourceContainsDefaultResourceTypes( - queue.getJSONObject(resourceCategory)); - } - } - - private void verifyResourceContainsDefaultResourceTypes( - JSONObject jsonObject) { - Object memory = jsonObject.opt("memory"); - Object vCores = jsonObject.opt("vCores"); - - assertNotNull("Key 'memory' not found in: " + jsonObject, memory); - assertNotNull("Key 'vCores' not found in: " + jsonObject, vCores); - } - - private void verifyResourcesContainCustomResourceTypes(JSONObject queue, - Set resourceCategories) throws JSONException { - for (String resourceCategory : resourceCategories) { - assertTrue("Queue " + queue + " does not have resource category key: " - + resourceCategory, queue.has(resourceCategory)); - verifyResourceContainsAllCustomResourceTypes( - queue.getJSONObject(resourceCategory)); - } - } - - private void verifyResourceContainsAllCustomResourceTypes( - JSONObject resourceCategory) throws JSONException { - assertTrue("resourceCategory does not have resourceInformations: " - + resourceCategory, resourceCategory.has("resourceInformations")); - - JSONObject resourceInformations = - resourceCategory.getJSONObject("resourceInformations"); - assertTrue( - "resourceInformations does not have resourceInformation object: " - + resourceInformations, - resourceInformations.has("resourceInformation")); - JSONArray customResources = - resourceInformations.getJSONArray("resourceInformation"); - - // customResources will include vcores / memory as well - assertEquals( - "Different number of custom resource types found than expected", - customResourceTypes.size(), customResources.length() - 2); - - for (int i = 0; i < customResources.length(); i++) { - JSONObject customResource = customResources.getJSONObject(i); - assertTrue("Resource type does not have name field: " + customResource, - customResource.has("name")); - assertTrue("Resource type does not have name resourceType field: " - + customResource, customResource.has("resourceType")); - assertTrue( - "Resource type does not have name units field: " + customResource, - customResource.has("units")); - assertTrue( - "Resource type does not have name value field: " + customResource, - customResource.has("value")); - - String name = customResource.getString("name"); - String unit = customResource.getString("units"); - String resourceType = customResource.getString("resourceType"); - Long value = customResource.getLong("value"); - - if (ResourceInformation.MEMORY_URI.equals(name) - || ResourceInformation.VCORES_URI.equals(name)) { - continue; - } - - assertTrue("Custom resource type " + name + " not found", - customResourceTypes.contains(name)); - assertEquals("k", unit); - assertEquals(ResourceTypes.COUNTABLE, - ResourceTypes.valueOf(resourceType)); - assertNotNull("Custom resource value " + value + " is null!", value); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java deleted file mode 100644 index 63ae7b74f1c..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * 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.yarn.server.resourcemanager.webapp.fairscheduler; - - -import com.google.common.collect.Sets; -import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; -import org.apache.hadoop.yarn.api.records.ResourceInformation; -import org.w3c.dom.Document; -import org.w3c.dom.Element; -import org.w3c.dom.Node; -import org.w3c.dom.NodeList; - -import java.util.List; -import java.util.Set; - -import static org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.XmlCustomResourceTypeTestCase.toXml; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlLong; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlString; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - -/** - * This test helper class is primarily used by - * {@link TestRMWebServicesFairSchedulerCustomResourceTypes}. - */ -public class FairSchedulerXmlVerifications { - - private static final Set RESOURCE_FIELDS = Sets.newHashSet( - "minResources", "amUsedResources", "amMaxResources", "fairResources", - "clusterResources", "reservedResources", "maxResources", "usedResources", - "steadyFairResources", "demandResources"); - private final Set customResourceTypes; - - FairSchedulerXmlVerifications(List customResourceTypes) { - this.customResourceTypes = Sets.newHashSet(customResourceTypes); - } - - public void verify(Element element) { - verifyResourcesContainDefaultResourceTypes(element, RESOURCE_FIELDS); - verifyResourcesContainCustomResourceTypes(element, RESOURCE_FIELDS); - } - - private void verifyResourcesContainDefaultResourceTypes(Element queue, - Set resourceCategories) { - for (String resourceCategory : resourceCategories) { - boolean hasResourceCategory = hasChild(queue, resourceCategory); - assertTrue("Queue " + queue + " does not have resource category key: " - + resourceCategory, hasResourceCategory); - verifyResourceContainsDefaultResourceTypes( - (Element) queue.getElementsByTagName(resourceCategory).item(0)); - } - } - - private void verifyResourceContainsDefaultResourceTypes( - Element element) { - Object memory = opt(element, "memory"); - Object vCores = opt(element, "vCores"); - - assertNotNull("Key 'memory' not found in: " + element, memory); - assertNotNull("Key 'vCores' not found in: " + element, vCores); - } - - private void verifyResourcesContainCustomResourceTypes(Element queue, - Set resourceCategories) { - for (String resourceCategory : resourceCategories) { - assertTrue("Queue " + queue + " does not have key for resourceCategory: " - + resourceCategory, hasChild(queue, resourceCategory)); - verifyResourceContainsCustomResourceTypes( - (Element) queue.getElementsByTagName(resourceCategory).item(0)); - } - } - - private void verifyResourceContainsCustomResourceTypes( - Element resourceCategory) { - assertEquals( - toXml(resourceCategory) - + " should have only one resourceInformations child!", - 1, resourceCategory.getElementsByTagName("resourceInformations") - .getLength()); - Element resourceInformations = (Element) resourceCategory - .getElementsByTagName("resourceInformations").item(0); - - NodeList customResources = - resourceInformations.getElementsByTagName("resourceInformation"); - - // customResources will include vcores / memory as well - assertEquals( - "Different number of custom resource types found than expected", - customResourceTypes.size(), customResources.getLength() - 2); - - for (int i = 0; i < customResources.getLength(); i++) { - Element customResource = (Element) customResources.item(i); - String name = getXmlString(customResource, "name"); - String unit = getXmlString(customResource, "units"); - String resourceType = getXmlString(customResource, "resourceType"); - Long value = getXmlLong(customResource, "value"); - - if (ResourceInformation.MEMORY_URI.equals(name) - || ResourceInformation.VCORES_URI.equals(name)) { - continue; - } - - assertTrue("Custom resource type " + name + " not found", - customResourceTypes.contains(name)); - assertEquals("k", unit); - assertEquals(ResourceTypes.COUNTABLE, - ResourceTypes.valueOf(resourceType)); - assertNotNull("Resource value should not be null for resource type " - + resourceType + ", listing xml contents: " + toXml(customResource), - value); - } - } - - private Object opt(Node node, String child) { - NodeList nodes = getElementsByTagNameInternal(node, child); - if (nodes.getLength() > 0) { - return nodes.item(0); - } - - return null; - } - - private boolean hasChild(Node node, String child) { - return getElementsByTagNameInternal(node, child).getLength() > 0; - } - - private NodeList getElementsByTagNameInternal(Node node, String child) { - if (node instanceof Element) { - return ((Element) node).getElementsByTagName(child); - } else if (node instanceof Document) { - return ((Document) node).getElementsByTagName(child); - } else { - throw new IllegalStateException("Unknown type of wrappedObject: " + node - + ", type: " + node.getClass()); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java deleted file mode 100644 index de4d5a10d1f..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java +++ /dev/null @@ -1,271 +0,0 @@ -/* - * 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.yarn.server.resourcemanager.webapp.fairscheduler; - -import com.google.inject.Guice; -import com.google.inject.servlet.ServletModule; -import com.sun.jersey.api.client.ClientResponse; -import com.sun.jersey.api.client.WebResource; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import com.sun.jersey.test.framework.WebAppDescriptor; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.MockRM; -import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueueManager; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.JAXBContextResolver; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServices; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.*; -import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; -import org.apache.hadoop.yarn.webapp.GuiceServletConfig; -import org.apache.hadoop.yarn.webapp.JerseyTestBase; -import org.codehaus.jettison.json.JSONArray; -import org.codehaus.jettison.json.JSONException; -import org.codehaus.jettison.json.JSONObject; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.w3c.dom.Element; -import javax.ws.rs.core.MediaType; -import java.lang.reflect.Method; -import java.util.List; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.Collectors; - -import static org.junit.Assert.assertEquals; - -/** - * This class is to test response representations of queue resources, - * explicitly setting custom resource types. with the help of - * {@link CustomResourceTypesConfigurationProvider} - */ -public class TestRMWebServicesFairSchedulerCustomResourceTypes - extends JerseyTestBase { - private static MockRM rm; - private static YarnConfiguration conf; - - private static class WebServletModule extends ServletModule { - @Override - protected void configureServlets() { - bind(JAXBContextResolver.class); - bind(RMWebServices.class); - bind(GenericExceptionHandler.class); - conf = new YarnConfiguration(); - conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class, - ResourceScheduler.class); - initResourceTypes(conf); - rm = new MockRM(conf); - bind(ResourceManager.class).toInstance(rm); - serve("/*").with(GuiceContainer.class); - } - - private void initResourceTypes(YarnConfiguration conf) { - conf.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS, - CustomResourceTypesConfigurationProvider.class.getName()); - ResourceUtils.resetResourceTypes(conf); - } - } - - @Before - @Override - public void setUp() throws Exception { - super.setUp(); - createInjectorForWebServletModule(); - } - - @After - public void tearDown() { - ResourceUtils.resetResourceTypes(new Configuration()); - } - - private void createInjectorForWebServletModule() { - GuiceServletConfig - .setInjector(Guice.createInjector(new WebServletModule())); - } - - @After - public void teardown() { - CustomResourceTypesConfigurationProvider.reset(); - } - - public TestRMWebServicesFairSchedulerCustomResourceTypes() { - super(new WebAppDescriptor.Builder( - "org.apache.hadoop.yarn.server.resourcemanager.webapp") - .contextListenerClass(GuiceServletConfig.class) - .filterClass(com.google.inject.servlet.GuiceFilter.class) - .contextPath("jersey-guice-filter").servletPath("/").build()); - } - - @Test - public void testClusterSchedulerWithCustomResourceTypesJson() { - FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); - QueueManager queueManager = scheduler.getQueueManager(); - // create LeafQueues - queueManager.getLeafQueue("root.q.subqueue1", true); - queueManager.getLeafQueue("root.q.subqueue2", true); - - FSLeafQueue subqueue1 = - queueManager.getLeafQueue("root.q.subqueue1", false); - incrementUsedResourcesOnQueue(subqueue1, 33L); - - WebResource path = - resource().path("ws").path("v1").path("cluster").path("scheduler"); - ClientResponse response = - path.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - - verifyJsonResponse(path, response, - CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); - } - - @Test - public void testClusterSchedulerWithCustomResourceTypesXml() { - FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); - QueueManager queueManager = scheduler.getQueueManager(); - // create LeafQueues - queueManager.getLeafQueue("root.q.subqueue1", true); - queueManager.getLeafQueue("root.q.subqueue2", true); - - FSLeafQueue subqueue1 = - queueManager.getLeafQueue("root.q.subqueue1", false); - incrementUsedResourcesOnQueue(subqueue1, 33L); - - WebResource path = - resource().path("ws").path("v1").path("cluster").path("scheduler"); - ClientResponse response = - path.accept(MediaType.APPLICATION_XML).get(ClientResponse.class); - - verifyXmlResponse(path, response, - CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); - } - - @Test - public void testClusterSchedulerWithElevenCustomResourceTypesXml() { - CustomResourceTypesConfigurationProvider.setNumberOfResourceTypes(11); - createInjectorForWebServletModule(); - - FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); - QueueManager queueManager = scheduler.getQueueManager(); - // create LeafQueues - queueManager.getLeafQueue("root.q.subqueue1", true); - queueManager.getLeafQueue("root.q.subqueue2", true); - - FSLeafQueue subqueue1 = - queueManager.getLeafQueue("root.q.subqueue1", false); - incrementUsedResourcesOnQueue(subqueue1, 33L); - - WebResource path = - resource().path("ws").path("v1").path("cluster").path("scheduler"); - ClientResponse response = - path.accept(MediaType.APPLICATION_XML).get(ClientResponse.class); - - verifyXmlResponse(path, response, - CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); - } - - @Test - public void testClusterSchedulerElevenWithCustomResourceTypesJson() { - CustomResourceTypesConfigurationProvider.setNumberOfResourceTypes(11); - createInjectorForWebServletModule(); - - FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); - QueueManager queueManager = scheduler.getQueueManager(); - // create LeafQueues - queueManager.getLeafQueue("root.q.subqueue1", true); - queueManager.getLeafQueue("root.q.subqueue2", true); - - FSLeafQueue subqueue1 = - queueManager.getLeafQueue("root.q.subqueue1", false); - incrementUsedResourcesOnQueue(subqueue1, 33L); - - WebResource path = - resource().path("ws").path("v1").path("cluster").path("scheduler"); - ClientResponse response = - path.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - - verifyJsonResponse(path, response, - CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); - } - - private void verifyJsonResponse(WebResource path, ClientResponse response, - List customResourceTypes) { - JsonCustomResourceTypeTestcase testCase = - new JsonCustomResourceTypeTestcase(path, - new BufferedClientResponse(response)); - testCase.verify(json -> { - try { - JSONArray queues = json.getJSONObject("scheduler") - .getJSONObject("schedulerInfo").getJSONObject("rootQueue") - .getJSONObject("childQueues").getJSONArray("queue"); - - // childQueueInfo consists of subqueue1 and subqueue2 info - assertEquals(2, queues.length()); - JSONObject firstChildQueue = queues.getJSONObject(0); - new FairSchedulerJsonVerifications(customResourceTypes) - .verify(firstChildQueue); - } catch (JSONException e) { - throw new RuntimeException(e); - } - }); - } - - private void verifyXmlResponse(WebResource path, ClientResponse response, - List customResourceTypes) { - XmlCustomResourceTypeTestCase testCase = new XmlCustomResourceTypeTestCase( - path, new BufferedClientResponse(response)); - - testCase.verify(xml -> { - Element scheduler = - (Element) xml.getElementsByTagName("scheduler").item(0); - Element schedulerInfo = - (Element) scheduler.getElementsByTagName("schedulerInfo").item(0); - Element rootQueue = - (Element) schedulerInfo.getElementsByTagName("rootQueue").item(0); - - Element childQueues = - (Element) rootQueue.getElementsByTagName("childQueues").item(0); - Element queue = - (Element) childQueues.getElementsByTagName("queue").item(0); - new FairSchedulerXmlVerifications(customResourceTypes).verify(queue); - }); - } - - private void incrementUsedResourcesOnQueue(final FSLeafQueue queue, - final long value) { - try { - Method incUsedResourceMethod = queue.getClass().getSuperclass() - .getDeclaredMethod("incUsedResource", Resource.class); - incUsedResourceMethod.setAccessible(true); - - Map customResources = - CustomResourceTypesConfigurationProvider.getCustomResourceTypes() - .stream() - .collect(Collectors.toMap(Function.identity(), v -> value)); - - incUsedResourceMethod.invoke(queue, - Resource.newInstance(20, 30, customResources)); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java deleted file mode 100644 index 4ab1443ce54..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * 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.yarn.server.resourcemanager.webapp.helper; - -import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo; -import org.codehaus.jettison.json.JSONException; -import org.codehaus.jettison.json.JSONObject; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.checkStringEqual; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.checkStringMatch; -import static org.junit.Assert.*; - -/** - * Contains all value verifications that are needed to verify {@link AppInfo} - * JSON objects. - */ -public final class AppInfoJsonVerifications { - - private AppInfoJsonVerifications() { - //utility class - } - - /** - * Tests whether {@link AppInfo} representation object contains the required - * values as per defined in the specified app parameter. - * @param app an RMApp instance that contains the required values - * to test against. - */ - public static void verify(JSONObject info, RMApp app) throws JSONException { - checkStringMatch("id", app.getApplicationId().toString(), - info.getString("id")); - checkStringMatch("user", app.getUser(), info.getString("user")); - checkStringMatch("name", app.getName(), info.getString("name")); - checkStringMatch("applicationType", app.getApplicationType(), - info.getString("applicationType")); - checkStringMatch("queue", app.getQueue(), info.getString("queue")); - assertEquals("priority doesn't match", 0, info.getInt("priority")); - checkStringMatch("state", app.getState().toString(), - info.getString("state")); - checkStringMatch("finalStatus", app.getFinalApplicationStatus().toString(), - info.getString("finalStatus")); - assertEquals("progress doesn't match", 0, - (float) info.getDouble("progress"), 0.0); - if ("UNASSIGNED".equals(info.getString("trackingUI"))) { - checkStringMatch("trackingUI", "UNASSIGNED", - info.getString("trackingUI")); - } - checkStringEqual("diagnostics", app.getDiagnostics().toString(), - info.getString("diagnostics")); - assertEquals("clusterId doesn't match", - ResourceManager.getClusterTimeStamp(), info.getLong("clusterId")); - assertEquals("startedTime doesn't match", app.getStartTime(), - info.getLong("startedTime")); - assertEquals("finishedTime doesn't match", app.getFinishTime(), - info.getLong("finishedTime")); - assertTrue("elapsed time not greater than 0", - info.getLong("elapsedTime") > 0); - checkStringMatch("amHostHttpAddress", - app.getCurrentAppAttempt().getMasterContainer().getNodeHttpAddress(), - info.getString("amHostHttpAddress")); - assertTrue("amContainerLogs doesn't match", - info.getString("amContainerLogs").startsWith("http://")); - assertTrue("amContainerLogs doesn't contain user info", - info.getString("amContainerLogs").endsWith("/" + app.getUser())); - assertEquals("allocatedMB doesn't match", 1024, info.getInt("allocatedMB")); - assertEquals("allocatedVCores doesn't match", 1, - info.getInt("allocatedVCores")); - assertEquals("queueUsagePerc doesn't match", 50.0f, - (float) info.getDouble("queueUsagePercentage"), 0.01f); - assertEquals("clusterUsagePerc doesn't match", 50.0f, - (float) info.getDouble("clusterUsagePercentage"), 0.01f); - assertEquals("numContainers doesn't match", 1, - info.getInt("runningContainers")); - assertNotNull("preemptedResourceSecondsMap should not be null", - info.getJSONObject("preemptedResourceSecondsMap")); - assertEquals("preemptedResourceMB doesn't match", - app.getRMAppMetrics().getResourcePreempted().getMemorySize(), - info.getInt("preemptedResourceMB")); - assertEquals("preemptedResourceVCores doesn't match", - app.getRMAppMetrics().getResourcePreempted().getVirtualCores(), - info.getInt("preemptedResourceVCores")); - assertEquals("numNonAMContainerPreempted doesn't match", - app.getRMAppMetrics().getNumNonAMContainersPreempted(), - info.getInt("numNonAMContainerPreempted")); - assertEquals("numAMContainerPreempted doesn't match", - app.getRMAppMetrics().getNumAMContainersPreempted(), - info.getInt("numAMContainerPreempted")); - assertEquals("Log aggregation Status doesn't match", - app.getLogAggregationStatusForAppReport().toString(), - info.getString("logAggregationStatus")); - assertEquals("unmanagedApplication doesn't match", - app.getApplicationSubmissionContext().getUnmanagedAM(), - info.getBoolean("unmanagedApplication")); - - if (app.getApplicationSubmissionContext() - .getNodeLabelExpression() != null) { - assertEquals("appNodeLabelExpression doesn't match", - app.getApplicationSubmissionContext().getNodeLabelExpression(), - info.getString("appNodeLabelExpression")); - } - assertEquals("amNodeLabelExpression doesn't match", - app.getAMResourceRequests().get(0).getNodeLabelExpression(), - info.getString("amNodeLabelExpression")); - assertEquals("amRPCAddress", - AppInfo.getAmRPCAddressFromRMAppAttempt(app.getCurrentAppAttempt()), - info.getString("amRPCAddress")); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java deleted file mode 100644 index 7c5b6dbdeb2..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * 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.yarn.server.resourcemanager.webapp.helper; - -import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo; -import org.apache.hadoop.yarn.webapp.WebServicesTestUtils; -import org.w3c.dom.Element; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.checkStringMatch; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlBoolean; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlFloat; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlInt; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlLong; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlString; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - -/** - * Contains all value verifications that are needed to verify {@link AppInfo} - * XML documents. - */ -public final class AppInfoXmlVerifications { - - private AppInfoXmlVerifications() { - //utility class - } - - /** - * Tests whether {@link AppInfo} representation object contains the required - * values as per defined in the specified app parameter. - * @param info - * @param app an RMApp instance that contains the required values - */ - public static void verify(Element info, RMApp app) { - checkStringMatch("id", app.getApplicationId() - .toString(), getXmlString(info, "id")); - checkStringMatch("user", app.getUser(), - getXmlString(info, "user")); - checkStringMatch("name", app.getName(), - getXmlString(info, "name")); - checkStringMatch("applicationType", - app.getApplicationType(), getXmlString(info, "applicationType")); - checkStringMatch("queue", app.getQueue(), - getXmlString(info, "queue")); - assertEquals("priority doesn't match", 0, getXmlInt(info, "priority")); - checkStringMatch("state", app.getState().toString(), - getXmlString(info, "state")); - checkStringMatch("finalStatus", app - .getFinalApplicationStatus().toString(), - getXmlString(info, "finalStatus")); - assertEquals("progress doesn't match", 0, getXmlFloat(info, "progress"), - 0.0); - if ("UNASSIGNED".equals(getXmlString(info, "trackingUI"))) { - checkStringMatch("trackingUI", "UNASSIGNED", - getXmlString(info, "trackingUI")); - } - WebServicesTestUtils.checkStringEqual("diagnostics", - app.getDiagnostics().toString(), getXmlString(info, "diagnostics")); - assertEquals("clusterId doesn't match", - ResourceManager.getClusterTimeStamp(), - getXmlLong(info, "clusterId")); - assertEquals("startedTime doesn't match", app.getStartTime(), - getXmlLong(info, "startedTime")); - assertEquals("finishedTime doesn't match", app.getFinishTime(), - getXmlLong(info, "finishedTime")); - assertTrue("elapsed time not greater than 0", - getXmlLong(info, "elapsedTime") > 0); - checkStringMatch("amHostHttpAddress", app - .getCurrentAppAttempt().getMasterContainer() - .getNodeHttpAddress(), - getXmlString(info, "amHostHttpAddress")); - assertTrue("amContainerLogs doesn't match", - getXmlString(info, "amContainerLogs").startsWith("http://")); - assertTrue("amContainerLogs doesn't contain user info", - getXmlString(info, "amContainerLogs").endsWith("/" + app.getUser())); - assertEquals("allocatedMB doesn't match", 1024, - getXmlInt(info, "allocatedMB")); - assertEquals("allocatedVCores doesn't match", 1, - getXmlInt(info, "allocatedVCores")); - assertEquals("queueUsagePerc doesn't match", 50.0f, - getXmlFloat(info, "queueUsagePercentage"), 0.01f); - assertEquals("clusterUsagePerc doesn't match", 50.0f, - getXmlFloat(info, "clusterUsagePercentage"), 0.01f); - assertEquals("numContainers doesn't match", 1, - getXmlInt(info, "runningContainers")); - assertNotNull("preemptedResourceSecondsMap should not be null", - info.getElementsByTagName("preemptedResourceSecondsMap")); - assertEquals("preemptedResourceMB doesn't match", app - .getRMAppMetrics().getResourcePreempted().getMemorySize(), - getXmlInt(info, "preemptedResourceMB")); - assertEquals("preemptedResourceVCores doesn't match", app - .getRMAppMetrics().getResourcePreempted().getVirtualCores(), - getXmlInt(info, "preemptedResourceVCores")); - assertEquals("numNonAMContainerPreempted doesn't match", app - .getRMAppMetrics().getNumNonAMContainersPreempted(), - getXmlInt(info, "numNonAMContainerPreempted")); - assertEquals("numAMContainerPreempted doesn't match", app - .getRMAppMetrics().getNumAMContainersPreempted(), - getXmlInt(info, "numAMContainerPreempted")); - assertEquals("Log aggregation Status doesn't match", app - .getLogAggregationStatusForAppReport().toString(), - getXmlString(info, "logAggregationStatus")); - assertEquals("unmanagedApplication doesn't match", app - .getApplicationSubmissionContext().getUnmanagedAM(), - getXmlBoolean(info, "unmanagedApplication")); - assertEquals("unmanagedApplication doesn't match", - app.getApplicationSubmissionContext().getNodeLabelExpression(), - getXmlString(info, "appNodeLabelExpression")); - assertEquals("unmanagedApplication doesn't match", - app.getAMResourceRequests().get(0).getNodeLabelExpression(), - getXmlString(info, "amNodeLabelExpression")); - assertEquals("amRPCAddress", - AppInfo.getAmRPCAddressFromRMAppAttempt(app.getCurrentAppAttempt()), - getXmlString(info, "amRPCAddress")); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.java deleted file mode 100644 index a8990ca695a..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.yarn.server.resourcemanager.webapp.helper; - - -import com.sun.jersey.api.client.ClientHandlerException; -import com.sun.jersey.api.client.ClientResponse; -import com.sun.jersey.api.client.UniformInterfaceException; - -import javax.ws.rs.core.MediaType; -import java.io.IOException; - -/** - * This class is merely a wrapper for {@link ClientResponse}. Given that the - * entity input stream of {@link ClientResponse} can be read only once by - * default and for some tests it is convenient to read the input stream many - * times, this class hides the details of how to do that and prevents - * unnecessary code duplication in tests. - */ -public class BufferedClientResponse { - private ClientResponse response; - - public BufferedClientResponse(ClientResponse response) { - response.bufferEntity(); - this.response = response; - } - - public T getEntity(Class clazz) - throws ClientHandlerException, UniformInterfaceException { - try { - response.getEntityInputStream().reset(); - } catch (IOException e) { - throw new RuntimeException(e); - } - return response.getEntity(clazz); - } - - public MediaType getType() { - return response.getType(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java deleted file mode 100644 index 9d6a111d7ac..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java +++ /dev/null @@ -1,77 +0,0 @@ -/** - * 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.yarn.server.resourcemanager.webapp.helper; - -import com.sun.jersey.api.client.WebResource; -import org.apache.hadoop.http.JettyUtils; -import org.codehaus.jettison.json.JSONObject; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.ws.rs.core.MediaType; - -import java.util.function.Consumer; - -import static org.junit.Assert.*; - -/** - * This class hides the implementation details of how to verify the structure of - * JSON responses. Tests should only provide the path of the - * {@link WebResource}, the response from the resource and - * the verifier Consumer to - * {@link JsonCustomResourceTypeTestcase#verify(Consumer)}. An instance of - * {@link JSONObject} will be passed to that consumer to be able to - * verify the response. - */ -public class JsonCustomResourceTypeTestcase { - private static final Logger LOG = - LoggerFactory.getLogger(JsonCustomResourceTypeTestcase.class); - - private final WebResource path; - private final BufferedClientResponse response; - private final JSONObject parsedResponse; - - public JsonCustomResourceTypeTestcase(WebResource path, - BufferedClientResponse response) { - this.path = path; - this.response = response; - this.parsedResponse = response.getEntity(JSONObject.class); - } - - public void verify(Consumer verifier) { - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - - logResponse(); - - String responseStr = response.getEntity(String.class); - if (responseStr == null || responseStr.isEmpty()) { - throw new IllegalStateException("Response is null or empty!"); - } - verifier.accept(parsedResponse); - } - - private void logResponse() { - String responseStr = response.getEntity(String.class); - LOG.info("Raw response from service URL {}: {}", path.toString(), - responseStr); - LOG.info("Parsed response from service URL {}: {}", path.toString(), - parsedResponse); - } -} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java deleted file mode 100644 index 6e58a89692b..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java +++ /dev/null @@ -1,252 +0,0 @@ -/* - * 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.yarn.server.resourcemanager.webapp.helper; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; -import org.apache.hadoop.yarn.api.records.ResourceInformation; -import org.apache.hadoop.yarn.api.records.ResourceRequest; -import org.codehaus.jettison.json.JSONArray; -import org.codehaus.jettison.json.JSONException; -import org.codehaus.jettison.json.JSONObject; - -import java.util.List; -import java.util.Map; - -import static junit.framework.TestCase.assertTrue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - -/** - * Performs value verifications on - * {@link org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceRequestInfo} - * objects against the values of {@link ResourceRequest}. With the help of the - * {@link Builder}, users can also make verifications of the custom resource - * types and its values. - */ -public class ResourceRequestsJsonVerifications { - private final ResourceRequest resourceRequest; - private final JSONObject requestInfo; - private final Map customResourceTypes; - private final List expectedCustomResourceTypes; - - ResourceRequestsJsonVerifications(Builder builder) { - this.resourceRequest = builder.resourceRequest; - this.requestInfo = builder.requestInfo; - this.customResourceTypes = builder.customResourceTypes; - this.expectedCustomResourceTypes = builder.expectedCustomResourceTypes; - } - - public static void verify(JSONObject requestInfo, ResourceRequest rr) - throws JSONException { - createDefaultBuilder(requestInfo, rr).build().verify(); - } - - public static void verifyWithCustomResourceTypes(JSONObject requestInfo, - ResourceRequest resourceRequest, List expectedResourceTypes) - throws JSONException { - - createDefaultBuilder(requestInfo, resourceRequest) - .withExpectedCustomResourceTypes(expectedResourceTypes) - .withCustomResourceTypes( - extractActualCustomResourceTypes(requestInfo, expectedResourceTypes)) - .build().verify(); - } - - private static Builder createDefaultBuilder(JSONObject requestInfo, - ResourceRequest resourceRequest) { - return new ResourceRequestsJsonVerifications.Builder() - .withRequest(resourceRequest) - .withRequestInfoJson(requestInfo); - } - - private static Map extractActualCustomResourceTypes( - JSONObject requestInfo, List expectedResourceTypes) - throws JSONException { - JSONObject capability = requestInfo.getJSONObject("capability"); - Map resourceAndValue = - extractCustomResorceTypeValues(capability, expectedResourceTypes); - Map.Entry resourceEntry = - resourceAndValue.entrySet().iterator().next(); - - assertTrue( - "Found resource type: " + resourceEntry.getKey() - + " is not in expected resource types: " + expectedResourceTypes, - expectedResourceTypes.contains(resourceEntry.getKey())); - - return resourceAndValue; - } - - private static Map extractCustomResorceTypeValues( - JSONObject capability, List expectedResourceTypes) - throws JSONException { - assertTrue( - "resourceCategory does not have resourceInformations: " + capability, - capability.has("resourceInformations")); - - JSONObject resourceInformations = - capability.getJSONObject("resourceInformations"); - assertTrue( - "resourceInformations does not have resourceInformation object: " - + resourceInformations, - resourceInformations.has("resourceInformation")); - JSONArray customResources = - resourceInformations.getJSONArray("resourceInformation"); - - // customResources will include vcores / memory as well - assertEquals( - "Different number of custom resource types found than expected", - expectedResourceTypes.size(), customResources.length() - 2); - - Map resourceValues = Maps.newHashMap(); - for (int i = 0; i < customResources.length(); i++) { - JSONObject customResource = customResources.getJSONObject(i); - assertTrue("Resource type does not have name field: " + customResource, - customResource.has("name")); - assertTrue("Resource type does not have name resourceType field: " - + customResource, customResource.has("resourceType")); - assertTrue( - "Resource type does not have name units field: " + customResource, - customResource.has("units")); - assertTrue( - "Resource type does not have name value field: " + customResource, - customResource.has("value")); - - String name = customResource.getString("name"); - String unit = customResource.getString("units"); - String resourceType = customResource.getString("resourceType"); - Long value = customResource.getLong("value"); - - if (ResourceInformation.MEMORY_URI.equals(name) - || ResourceInformation.VCORES_URI.equals(name)) { - continue; - } - - assertTrue("Custom resource type " + name + " not found", - expectedResourceTypes.contains(name)); - assertEquals("k", unit); - assertEquals(ResourceTypes.COUNTABLE, - ResourceTypes.valueOf(resourceType)); - assertNotNull("Custom resource value " + value + " is null!", value); - resourceValues.put(name, value); - } - - return resourceValues; - } - - private void verify() throws JSONException { - assertEquals("nodeLabelExpression doesn't match", - resourceRequest.getNodeLabelExpression(), - requestInfo.getString("nodeLabelExpression")); - assertEquals("numContainers doesn't match", - resourceRequest.getNumContainers(), - requestInfo.getInt("numContainers")); - assertEquals("relaxLocality doesn't match", - resourceRequest.getRelaxLocality(), - requestInfo.getBoolean("relaxLocality")); - assertEquals("priority does not match", - resourceRequest.getPriority().getPriority(), - requestInfo.getInt("priority")); - assertEquals("resourceName does not match", - resourceRequest.getResourceName(), - requestInfo.getString("resourceName")); - assertEquals("memory does not match", - resourceRequest.getCapability().getMemorySize(), - requestInfo.getJSONObject("capability").getLong("memory")); - assertEquals("vCores does not match", - resourceRequest.getCapability().getVirtualCores(), - requestInfo.getJSONObject("capability").getLong("vCores")); - - verifyAtLeastOneCustomResourceIsSerialized(); - - JSONObject executionTypeRequest = - requestInfo.getJSONObject("executionTypeRequest"); - assertEquals("executionType does not match", - resourceRequest.getExecutionTypeRequest().getExecutionType().name(), - executionTypeRequest.getString("executionType")); - assertEquals("enforceExecutionType does not match", - resourceRequest.getExecutionTypeRequest().getEnforceExecutionType(), - executionTypeRequest.getBoolean("enforceExecutionType")); - } - - /** - * JSON serialization produces "invalid JSON" by default as maps are - * serialized like this: - * "customResources":{"entry":{"key":"customResource-1","value":"0"}} - * If the map has multiple keys then multiple entries will be serialized. - * Our json parser in tests cannot handle duplicates therefore only one - * custom resource will be in the parsed json. See: - * https://issues.apache.org/jira/browse/YARN-7505 - */ - private void verifyAtLeastOneCustomResourceIsSerialized() { - boolean resourceFound = false; - for (String expectedCustomResourceType : expectedCustomResourceTypes) { - if (customResourceTypes.containsKey(expectedCustomResourceType)) { - resourceFound = true; - Long resourceValue = - customResourceTypes.get(expectedCustomResourceType); - assertNotNull("Resource value should not be null!", resourceValue); - } - } - assertTrue("No custom resource type can be found in the response!", - resourceFound); - } - - /** - * Builder class for {@link ResourceRequestsJsonVerifications}. - */ - public static final class Builder { - private List expectedCustomResourceTypes = Lists.newArrayList(); - private Map customResourceTypes; - private ResourceRequest resourceRequest; - private JSONObject requestInfo; - - Builder() { - } - - public static Builder create() { - return new Builder(); - } - - Builder withExpectedCustomResourceTypes( - List expectedCustomResourceTypes) { - this.expectedCustomResourceTypes = expectedCustomResourceTypes; - return this; - } - - Builder withCustomResourceTypes( - Map customResourceTypes) { - this.customResourceTypes = customResourceTypes; - return this; - } - - Builder withRequest(ResourceRequest resourceRequest) { - this.resourceRequest = resourceRequest; - return this; - } - - Builder withRequestInfoJson(JSONObject requestInfo) { - this.requestInfo = requestInfo; - return this; - } - - public ResourceRequestsJsonVerifications build() { - return new ResourceRequestsJsonVerifications(this); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java deleted file mode 100644 index af9b0f35103..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java +++ /dev/null @@ -1,215 +0,0 @@ -/* - * 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.yarn.server.resourcemanager.webapp.helper; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; -import org.apache.hadoop.yarn.api.records.ResourceInformation; -import org.apache.hadoop.yarn.api.records.ResourceRequest; -import org.w3c.dom.Element; -import org.w3c.dom.NodeList; - -import java.util.List; -import java.util.Map; -import java.util.Set; - -import static junit.framework.TestCase.assertTrue; -import static org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.XmlCustomResourceTypeTestCase.toXml; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlBoolean; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlInt; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlLong; -import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlString; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - -/** - * Performs value verifications on - * {@link org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceRequestInfo} - * objects against the values of {@link ResourceRequest}. With the help of the - * {@link Builder}, users can also make verifications of the custom resource - * types and its values. - */ -public class ResourceRequestsXmlVerifications { - private final ResourceRequest resourceRequest; - private final Element requestInfo; - private final Map customResourceTypes; - private final List expectedCustomResourceTypes; - - ResourceRequestsXmlVerifications(Builder builder) { - this.resourceRequest = builder.resourceRequest; - this.requestInfo = builder.requestInfo; - this.customResourceTypes = builder.customResourceTypes; - this.expectedCustomResourceTypes = builder.expectedCustomResourceTypes; - } - - public static void verifyWithCustomResourceTypes(Element requestInfo, - ResourceRequest resourceRequest, List expectedResourceTypes) { - - createDefaultBuilder(requestInfo, resourceRequest) - .withExpectedCustomResourceTypes(expectedResourceTypes) - .withCustomResourceTypes(extractActualCustomResourceType(requestInfo, - expectedResourceTypes)) - .build().verify(); - } - - private static Builder createDefaultBuilder(Element requestInfo, - ResourceRequest resourceRequest) { - return new ResourceRequestsXmlVerifications.Builder() - .withRequest(resourceRequest).withRequestInfo(requestInfo); - } - - private static Map extractActualCustomResourceType( - Element requestInfo, List expectedResourceTypes) { - Element capability = - (Element) requestInfo.getElementsByTagName("capability").item(0); - - return extractCustomResorceTypes(capability, - Sets.newHashSet(expectedResourceTypes)); - } - - private static Map extractCustomResorceTypes(Element capability, - Set expectedResourceTypes) { - assertEquals( - toXml(capability) + " should have only one resourceInformations child!", - 1, capability.getElementsByTagName("resourceInformations").getLength()); - Element resourceInformations = (Element) capability - .getElementsByTagName("resourceInformations").item(0); - - NodeList customResources = - resourceInformations.getElementsByTagName("resourceInformation"); - - // customResources will include vcores / memory as well - assertEquals( - "Different number of custom resource types found than expected", - expectedResourceTypes.size(), customResources.getLength() - 2); - - Map resourceTypesAndValues = Maps.newHashMap(); - for (int i = 0; i < customResources.getLength(); i++) { - Element customResource = (Element) customResources.item(i); - String name = getXmlString(customResource, "name"); - String unit = getXmlString(customResource, "units"); - String resourceType = getXmlString(customResource, "resourceType"); - Long value = getXmlLong(customResource, "value"); - - if (ResourceInformation.MEMORY_URI.equals(name) - || ResourceInformation.VCORES_URI.equals(name)) { - continue; - } - - assertTrue("Custom resource type " + name + " not found", - expectedResourceTypes.contains(name)); - assertEquals("k", unit); - assertEquals(ResourceTypes.COUNTABLE, - ResourceTypes.valueOf(resourceType)); - assertNotNull("Resource value should not be null for resource type " - + resourceType + ", listing xml contents: " + toXml(customResource), - value); - resourceTypesAndValues.put(name, value); - } - - return resourceTypesAndValues; - } - - private void verify() { - assertEquals("nodeLabelExpression doesn't match", - resourceRequest.getNodeLabelExpression(), - getXmlString(requestInfo, "nodeLabelExpression")); - assertEquals("numContainers doesn't match", - resourceRequest.getNumContainers(), - getXmlInt(requestInfo, "numContainers")); - assertEquals("relaxLocality doesn't match", - resourceRequest.getRelaxLocality(), - getXmlBoolean(requestInfo, "relaxLocality")); - assertEquals("priority does not match", - resourceRequest.getPriority().getPriority(), - getXmlInt(requestInfo, "priority")); - assertEquals("resourceName does not match", - resourceRequest.getResourceName(), - getXmlString(requestInfo, "resourceName")); - Element capability = (Element) requestInfo - .getElementsByTagName("capability").item(0); - assertEquals("memory does not match", - resourceRequest.getCapability().getMemorySize(), - getXmlLong(capability, "memory")); - assertEquals("vCores does not match", - resourceRequest.getCapability().getVirtualCores(), - getXmlLong(capability, "vCores")); - - for (String expectedCustomResourceType : expectedCustomResourceTypes) { - assertTrue( - "Custom resource type " + expectedCustomResourceType - + " cannot be found!", - customResourceTypes.containsKey(expectedCustomResourceType)); - - Long resourceValue = customResourceTypes.get(expectedCustomResourceType); - assertNotNull("Resource value should not be null!", resourceValue); - } - - Element executionTypeRequest = (Element) requestInfo - .getElementsByTagName("executionTypeRequest").item(0); - assertEquals("executionType does not match", - resourceRequest.getExecutionTypeRequest().getExecutionType().name(), - getXmlString(executionTypeRequest, "executionType")); - assertEquals("enforceExecutionType does not match", - resourceRequest.getExecutionTypeRequest().getEnforceExecutionType(), - getXmlBoolean(executionTypeRequest, "enforceExecutionType")); - } - - /** - * Builder class for {@link ResourceRequestsXmlVerifications}. - */ - public static final class Builder { - private List expectedCustomResourceTypes = Lists.newArrayList(); - private Map customResourceTypes; - private ResourceRequest resourceRequest; - private Element requestInfo; - - Builder() { - } - - public static Builder create() { - return new Builder(); - } - - Builder withExpectedCustomResourceTypes( - List expectedCustomResourceTypes) { - this.expectedCustomResourceTypes = expectedCustomResourceTypes; - return this; - } - - Builder withCustomResourceTypes(Map customResourceTypes) { - this.customResourceTypes = customResourceTypes; - return this; - } - - Builder withRequest(ResourceRequest resourceRequest) { - this.resourceRequest = resourceRequest; - return this; - } - - Builder withRequestInfo(Element requestInfo) { - this.requestInfo = requestInfo; - return this; - } - - public ResourceRequestsXmlVerifications build() { - return new ResourceRequestsXmlVerifications(this); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java deleted file mode 100644 index 29260aad1df..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java +++ /dev/null @@ -1,112 +0,0 @@ -/** - * 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.yarn.server.resourcemanager.webapp.helper; - -import com.sun.jersey.api.client.WebResource; -import org.apache.hadoop.http.JettyUtils; -import org.codehaus.jettison.json.JSONObject; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.w3c.dom.Document; -import org.w3c.dom.Node; -import org.xml.sax.InputSource; - -import javax.ws.rs.core.MediaType; -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.transform.*; -import javax.xml.transform.dom.DOMSource; -import javax.xml.transform.stream.StreamResult; -import java.io.StringReader; -import java.io.StringWriter; -import java.util.function.Consumer; - -import static org.junit.Assert.assertEquals; - -/** - * This class hides the implementation details of how to verify the structure of - * XML responses. Tests should only provide the path of the - * {@link WebResource}, the response from the resource and - * the verifier Consumer to - * {@link XmlCustomResourceTypeTestCase#verify(Consumer)}. An instance of - * {@link JSONObject} will be passed to that consumer to be able to - * verify the response. - */ -public class XmlCustomResourceTypeTestCase { - private static final Logger LOG = - LoggerFactory.getLogger(XmlCustomResourceTypeTestCase.class); - - private WebResource path; - private BufferedClientResponse response; - private Document parsedResponse; - - public XmlCustomResourceTypeTestCase(WebResource path, - BufferedClientResponse response) { - this.path = path; - this.response = response; - } - - public void verify(Consumer verifier) { - assertEquals(MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8, - response.getType().toString()); - - parsedResponse = parseXml(response); - logResponse(parsedResponse); - verifier.accept(parsedResponse); - } - - private Document parseXml(BufferedClientResponse response) { - try { - String xml = response.getEntity(String.class); - DocumentBuilder db = - DocumentBuilderFactory.newInstance().newDocumentBuilder(); - InputSource is = new InputSource(); - is.setCharacterStream(new StringReader(xml)); - - return db.parse(is); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - private void logResponse(Document doc) { - String responseStr = response.getEntity(String.class); - LOG.info("Raw response from service URL {}: {}", path.toString(), - responseStr); - LOG.info("Parsed response from service URL {}: {}", path.toString(), - toXml(doc)); - } - - public static String toXml(Node node) { - StringWriter writer; - try { - TransformerFactory tf = TransformerFactory.newInstance(); - Transformer transformer = tf.newTransformer(); - transformer.setOutputProperty(OutputKeys.INDENT, "yes"); - transformer.setOutputProperty( - "{http://xml.apache.org/xslt}indent" + "-amount", "2"); - writer = new StringWriter(); - transformer.transform(new DOMSource(node), new StreamResult(writer)); - } catch (TransformerException e) { - throw new RuntimeException(e); - } - - return writer.getBuffer().toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md index b5bcbf5c8e9..269f5b40f49 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md @@ -86,11 +86,11 @@ The allocation file must be in XML format. The format contains five types of ele * **Queue elements**: which represent queues. Queue elements can take an optional attribute 'type', which when set to 'parent' makes it a parent queue. This is useful when we want to create a parent queue without configuring any leaf queues. Each queue element may contain the following properties: - * **minResources**: minimum resources the queue is entitled to, in the form of "X mb, Y vcores" or "vcores=X, memory-mb=Y". The latter form is required when specifying resources other than memory and CPU. For the single-resource fairness policy, the vcores value is ignored. If a queue's minimum share is not satisfied, it will be offered available resources before any other queue under the same parent. Under the single-resource fairness policy, a queue is considered unsatisfied if its memory usage is below its minimum memory share. Under dominant resource fairness, a queue is considered unsatisfied if its usage for its dominant resource with respect to the cluster capacity is below its minimum share for that resource. If multiple queues are unsatisfied in this situation, resources go to the queue with the smallest ratio between relevant resource usage and its minimum. Note that it is possible for a queue that is below its minimum to not immediately get up to its minimum when an application is submitted to the queue, because already-running jobs may be using those resources. + * **minResources**: minimum resources the queue is entitled to, in the form "X mb, Y vcores". For the single-resource fairness policy, the vcores value is ignored. If a queue's minimum share is not satisfied, it will be offered available resources before any other queue under the same parent. Under the single-resource fairness policy, a queue is considered unsatisfied if its memory usage is below its minimum memory share. Under dominant resource fairness, a queue is considered unsatisfied if its usage for its dominant resource with respect to the cluster capacity is below its minimum share for that resource. If multiple queues are unsatisfied in this situation, resources go to the queue with the smallest ratio between relevant resource usage and minimum. Note that it is possible that a queue that is below its minimum may not immediately get up to its minimum when it submits an application, because already-running jobs may be using those resources. - * **maxResources**: maximum resources a queue will allocated, expressed in the form of "X%", "X% cpu, Y% memory", "X mb, Y vcores", or "vcores=X, memory-mb=Y". The last form is required when specifying resources other than memory and CPU. In the last form, X and Y can either be a percentage or an integer resource value without units. In the latter case the units will be inferred from the default units configured for that resource. A queue will not be assigned a container that would put its aggregate usage over this limit. + * **maxResources**: maximum resources a queue is allocated, expressed either in absolute values (X mb, Y vcores) or as a percentage of the cluster resources (X% memory, Y% cpu). A queue will not be assigned a container that would put its aggregate usage over this limit. - * **maxChildResources**: maximum resources an ad hoc child queue will allocated, expressed in the form of "X%", "X% cpu, Y% memory", "X mb, Y vcores", or "vcores=X, memory-mb=Y". The last form is required when specifying resources other than memory and CPU. In the last form, X and Y can either be a percentage or an integer resource value without units. In the latter case the units will be inferred from the default units configured for that resource. An ad hoc child queue will not be assigned a container that would put its aggregate usage over this limit. + * **maxChildResources**: maximum resources an ad hoc child queue is allocated, expressed either in absolute values (X mb, Y vcores) or as a percentage of the cluster resources (X% memory, Y% cpu). An ad hoc child queue will not be assigned a container that would put its aggregate usage over this limit. * **maxRunningApps**: limit the number of apps from the queue to run at once From 9edc74f64a31450af3c55c0dadf352862e4b359d Mon Sep 17 00:00:00 2001 From: Haibo Chen Date: Thu, 5 Jul 2018 10:42:39 -0700 Subject: [PATCH 33/47] YARN-7556. Fair scheduler configuration should allow resource types in the minResources and maxResources properties. (Daniel Templeton and Szilard Nemeth via Haibo Chen) --- .../dev-support/findbugs-exclude.xml | 17 +- .../hadoop/yarn/api/records/Resource.java | 13 ++ .../api/records/impl/LightWeightResource.java | 23 ++- .../scheduler/fair/ConfigurableResource.java | 69 ++++++- .../fair/FairSchedulerConfiguration.java | 174 +++++++++++++++--- .../allocation/AllocationFileQueueParser.java | 2 +- .../fair/TestFairSchedulerConfiguration.java | 149 +++++++++++---- .../src/site/markdown/FairScheduler.md | 6 +- 8 files changed, 384 insertions(+), 69 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml index 58413618df0..5cc81e57ec6 100644 --- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml @@ -65,11 +65,6 @@ - - - - - @@ -118,6 +113,18 @@ + + + + + + + + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java index 71a6b54352b..173d4c9e777 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java @@ -30,6 +30,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; import org.apache.hadoop.yarn.api.records.impl.LightWeightResource; @@ -75,6 +76,18 @@ public abstract class Resource implements Comparable { @Private public static final int VCORES_INDEX = 1; + /** + * Return a new {@link Resource} instance with all resource values + * initialized to {@code value}. + * @param value the value to use for all resources + * @return a new {@link Resource} instance + */ + @Private + @Unstable + public static Resource newInstance(long value) { + return new LightWeightResource(value); + } + @Public @Stable public static Resource newInstance(int memory, int vCores) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java index a6e6432976f..77f77f312ce 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java @@ -18,9 +18,8 @@ package org.apache.hadoop.yarn.api.records.impl; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.util.resource.ResourceUtils; @@ -58,13 +57,29 @@ import static org.apache.hadoop.yarn.api.records.ResourceInformation.*; * * @see Resource */ -@InterfaceAudience.Private +@Private @Unstable public class LightWeightResource extends Resource { private ResourceInformation memoryResInfo; private ResourceInformation vcoresResInfo; + /** + * Create a new {@link LightWeightResource} instance with all resource values + * initialized to {@code value}. + * @param value the value to use for all resources + */ + public LightWeightResource(long value) { + ResourceInformation[] types = ResourceUtils.getResourceTypesArray(); + initResourceInformations(value, value, types.length); + + for (int i = 2; i < types.length; i++) { + resources[i] = new ResourceInformation(); + ResourceInformation.copy(types[i], resources[i]); + resources[i].setValue(value); + } + } + public LightWeightResource(long memory, int vcores) { int numberOfKnownResourceTypes = ResourceUtils .getNumberOfKnownResourceTypes(); @@ -91,7 +106,7 @@ public class LightWeightResource extends Resource { } } - private void initResourceInformations(long memory, int vcores, + private void initResourceInformations(long memory, long vcores, int numberOfKnownResourceTypes) { this.memoryResInfo = newDefaultInformation(MEMORY_URI, MEMORY_MB.getUnits(), memory); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java index ecdd0111a6b..0c3b0ddc6b6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/ConfigurableResource.java @@ -18,9 +18,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; +import java.util.Arrays; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; /** * A {@code ConfigurableResource} object represents an entity that is used to @@ -33,29 +37,53 @@ public class ConfigurableResource { private final Resource resource; private final double[] percentages; - public ConfigurableResource(double[] percentages) { + ConfigurableResource() { + this(getOneHundredPercentArray()); + } + + ConfigurableResource(double[] percentages) { this.percentages = percentages.clone(); this.resource = null; } + ConfigurableResource(long value) { + this(Resource.newInstance(value)); + } + public ConfigurableResource(Resource resource) { this.percentages = null; this.resource = resource; } + private static double[] getOneHundredPercentArray() { + double[] resourcePercentages = + new double[ResourceUtils.getNumberOfKnownResourceTypes()]; + Arrays.fill(resourcePercentages, 1.0); + + return resourcePercentages; + } + /** * Get resource by multiplying the cluster resource and the percentage of * each resource respectively. Return the absolute resource if either * {@code percentages} or {@code clusterResource} is null. * * @param clusterResource the cluster resource - * @return resource + * @return resource the resulting resource */ public Resource getResource(Resource clusterResource) { if (percentages != null && clusterResource != null) { long memory = (long) (clusterResource.getMemorySize() * percentages[0]); int vcore = (int) (clusterResource.getVirtualCores() * percentages[1]); - return Resource.newInstance(memory, vcore); + Resource res = Resource.newInstance(memory, vcore); + ResourceInformation[] clusterInfo = clusterResource.getResources(); + + for (int i = 2; i < clusterInfo.length; i++) { + res.setResourceValue(i, + (long)(clusterInfo[i].getValue() * percentages[i])); + } + + return res; } else { return resource; } @@ -69,4 +97,39 @@ public class ConfigurableResource { public Resource getResource() { return resource; } + + /** + * Set the value of the wrapped resource if this object isn't setup to use + * percentages. If this object is set to use percentages, this method has + * no effect. + * + * @param name the name of the resource + * @param value the value + */ + void setValue(String name, long value) { + if (resource != null) { + resource.setResourceValue(name, value); + } + } + + /** + * Set the percentage of the resource if this object is setup to use + * percentages. If this object is set to use percentages, this method has + * no effect. + * + * @param name the name of the resource + * @param value the percentage + */ + void setPercentage(String name, double value) { + if (percentages != null) { + Integer index = ResourceUtils.getResourceTypeIndex().get(name); + + if (index != null) { + percentages[index] = value; + } else { + throw new ResourceNotFoundException("The requested resource, \"" + + name + "\", could not be found."); + } + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java index b50e4bbe317..8c4932bfe67 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java @@ -33,6 +33,7 @@ import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.UnitsConversionUtil; import org.apache.hadoop.yarn.util.resource.ResourceUtils; @@ -213,6 +214,9 @@ public class FairSchedulerConfiguration extends Configuration { CONF_PREFIX + "reservable-nodes"; public static final float RESERVABLE_NODES_DEFAULT = 0.05f; + private static final String INVALID_RESOURCE_DEFINITION_PREFIX = + "Error reading resource config--invalid resource definition: "; + public FairSchedulerConfiguration() { super(); } @@ -407,54 +411,167 @@ public class FairSchedulerConfiguration extends Configuration { } /** - * Parses a resource config value of a form like "1024", "1024 mb", - * or "1024 mb, 3 vcores". If no units are given, megabytes are assumed. - * - * @throws AllocationConfigurationException + * Parses a resource config value in one of three forms: + *

    + *
  1. Percentage: "50%" or "40% memory, 60% cpu"
  2. + *
  3. New style resources: "vcores=10, memory-mb=1024" + * or "vcores=60%, memory-mb=40%"
  4. + *
  5. Old style resources: "1024 mb, 10 vcores"
  6. + *
+ * In new style resources, any resource that is not specified will be + * set to {@link Long#MAX_VALUE} or 100%, as appropriate. Also, in the new + * style resources, units are not allowed. Units are assumed from the resource + * manager's settings for the resources when the value isn't a percentage. + * + * @param value the resource definition to parse + * @return a {@link ConfigurableResource} that represents the parsed value + * @throws AllocationConfigurationException if the raw value is not a valid + * resource definition */ - public static ConfigurableResource parseResourceConfigValue(String val) + public static ConfigurableResource parseResourceConfigValue(String value) throws AllocationConfigurationException { + return parseResourceConfigValue(value, Long.MAX_VALUE); + } + + /** + * Parses a resource config value in one of three forms: + *
    + *
  1. Percentage: "50%" or "40% memory, 60% cpu"
  2. + *
  3. New style resources: "vcores=10, memory-mb=1024" + * or "vcores=60%, memory-mb=40%"
  4. + *
  5. Old style resources: "1024 mb, 10 vcores"
  6. + *
+ * In new style resources, any resource that is not specified will be + * set to {@code missing} or 0%, as appropriate. Also, in the new style + * resources, units are not allowed. Units are assumed from the resource + * manager's settings for the resources when the value isn't a percentage. + * + * The {@code missing} parameter is only used in the case of new style + * resources without percentages. With new style resources with percentages, + * any missing resources will be assumed to be 100% because percentages are + * only used with maximum resource limits. + * + * @param value the resource definition to parse + * @param missing the value to use for any unspecified resources + * @return a {@link ConfigurableResource} that represents the parsed value + * @throws AllocationConfigurationException if the raw value is not a valid + * resource definition + */ + public static ConfigurableResource parseResourceConfigValue(String value, + long missing) throws AllocationConfigurationException { ConfigurableResource configurableResource; + + if (value.trim().isEmpty()) { + throw new AllocationConfigurationException("Error reading resource " + + "config--the resource string is empty."); + } + try { - val = StringUtils.toLowerCase(val); - if (val.contains("%")) { - configurableResource = new ConfigurableResource( - getResourcePercentage(val)); + if (value.contains("=")) { + configurableResource = parseNewStyleResource(value, missing); + } else if (value.contains("%")) { + configurableResource = parseOldStyleResourceAsPercentage(value); } else { - int memory = findResource(val, "mb"); - int vcores = findResource(val, "vcores"); - configurableResource = new ConfigurableResource( - BuilderUtils.newResource(memory, vcores)); + configurableResource = parseOldStyleResource(value); } - } catch (AllocationConfigurationException ex) { - throw ex; - } catch (Exception ex) { + } catch (RuntimeException ex) { throw new AllocationConfigurationException( "Error reading resource config", ex); } + return configurableResource; } + private static ConfigurableResource parseNewStyleResource(String value, + long missing) throws AllocationConfigurationException { + + final ConfigurableResource configurableResource; + boolean asPercent = value.contains("%"); + if (asPercent) { + configurableResource = new ConfigurableResource(); + } else { + configurableResource = new ConfigurableResource(missing); + } + + String[] resources = value.split(","); + for (String resource : resources) { + String[] parts = resource.split("="); + + if (parts.length != 2) { + throw createConfigException(value, + "Every resource must be of the form: name=value."); + } + + String resourceName = parts[0].trim(); + String resourceValue = parts[1].trim(); + try { + if (asPercent) { + configurableResource.setPercentage(resourceName, + findPercentage(resourceValue, "")); + } else { + configurableResource.setValue(resourceName, + Long.parseLong(resourceValue)); + } + } catch (ResourceNotFoundException ex) { + throw createConfigException(value, "The " + + "resource name, \"" + resourceName + "\" was not " + + "recognized. Please check the value of " + + YarnConfiguration.RESOURCE_TYPES + " in the Resource " + + "Manager's configuration files.", ex); + } catch (NumberFormatException ex) { + // This only comes from Long.parseLong() + throw createConfigException(value, "The " + + "resource values must all be integers. \"" + resourceValue + + "\" is not an integer.", ex); + } catch (AllocationConfigurationException ex) { + // This only comes from findPercentage() + throw createConfigException(value, "The " + + "resource values must all be percentages. \"" + + resourceValue + "\" is either not a number or does not " + + "include the '%' symbol.", ex); + } + } + return configurableResource; + } + + private static ConfigurableResource parseOldStyleResourceAsPercentage( + String value) throws AllocationConfigurationException { + return new ConfigurableResource( + getResourcePercentage(StringUtils.toLowerCase(value))); + } + + private static ConfigurableResource parseOldStyleResource(String value) + throws AllocationConfigurationException { + final String lCaseValue = StringUtils.toLowerCase(value); + int memory = findResource(lCaseValue, "mb"); + int vcores = findResource(lCaseValue, "vcores"); + + return new ConfigurableResource( + BuilderUtils.newResource(memory, vcores)); + } + private static double[] getResourcePercentage( String val) throws AllocationConfigurationException { int numberOfKnownResourceTypes = ResourceUtils .getNumberOfKnownResourceTypes(); double[] resourcePercentage = new double[numberOfKnownResourceTypes]; String[] strings = val.split(","); + if (strings.length == 1) { double percentage = findPercentage(strings[0], ""); for (int i = 0; i < numberOfKnownResourceTypes; i++) { - resourcePercentage[i] = percentage/100; + resourcePercentage[i] = percentage; } } else { - resourcePercentage[0] = findPercentage(val, "memory")/100; - resourcePercentage[1] = findPercentage(val, "cpu")/100; + resourcePercentage[0] = findPercentage(val, "memory"); + resourcePercentage[1] = findPercentage(val, "cpu"); } + return resourcePercentage; } private static double findPercentage(String val, String units) - throws AllocationConfigurationException { + throws AllocationConfigurationException { final Pattern pattern = Pattern.compile("((\\d+)(\\.\\d*)?)\\s*%\\s*" + units); Matcher matcher = pattern.matcher(val); @@ -467,7 +584,22 @@ public class FairSchedulerConfiguration extends Configuration { units); } } - return Double.parseDouble(matcher.group(1)); + return Double.parseDouble(matcher.group(1)) / 100.0; + } + + private static AllocationConfigurationException createConfigException( + String value, String message) { + return createConfigException(value, message, null); + } + + private static AllocationConfigurationException createConfigException( + String value, String message, Throwable t) { + String msg = INVALID_RESOURCE_DEFINITION_PREFIX + value + ". " + message; + if (t != null) { + return new AllocationConfigurationException(msg, t); + } else { + return new AllocationConfigurationException(msg); + } } public long getUpdateInterval() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java index d5a436ed0a9..441c34a1aa1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/allocation/AllocationFileQueueParser.java @@ -134,7 +134,7 @@ public class AllocationFileQueueParser { if (MIN_RESOURCES.equals(field.getTagName())) { String text = getTrimmedTextData(field); ConfigurableResource val = - FairSchedulerConfiguration.parseResourceConfigValue(text); + FairSchedulerConfiguration.parseResourceConfigValue(text, 0L); builder.minQueueResources(queueName, val.getResource()); } else if (MAX_RESOURCES.equals(field.getTagName())) { String text = getTrimmedTextData(field); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java index 481645bb494..76a5af50960 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java @@ -102,60 +102,145 @@ public class TestFairSchedulerConfiguration { @Test public void testParseResourceConfigValue() throws Exception { - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("2 vcores, 1024 mb").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("1024 mb, 2 vcores").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("2vcores,1024mb").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("1024mb,2vcores").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("1024 mb, 2 vcores").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("1024 Mb, 2 vCores").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue(" 1024 mb, 2 vcores ").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue(" 1024.3 mb, 2.35 vcores ").getResource()); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue(" 1024. mb, 2. vcores ").getResource()); + Resource expected = BuilderUtils.newResource(5 * 1024, 2); + Resource clusterResource = BuilderUtils.newResource(10 * 1024, 4); - Resource clusterResource = BuilderUtils.newResource(2048, 4); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, + parseResourceConfigValue("2 vcores, 5120 mb").getResource()); + assertEquals(expected, + parseResourceConfigValue("5120 mb, 2 vcores").getResource()); + assertEquals(expected, + parseResourceConfigValue("2vcores,5120mb").getResource()); + assertEquals(expected, + parseResourceConfigValue("5120mb,2vcores").getResource()); + assertEquals(expected, + parseResourceConfigValue("5120mb mb, 2 vcores").getResource()); + assertEquals(expected, + parseResourceConfigValue("5120 Mb, 2 vCores").getResource()); + assertEquals(expected, + parseResourceConfigValue(" 5120 mb, 2 vcores ").getResource()); + assertEquals(expected, + parseResourceConfigValue(" 5120.3 mb, 2.35 vcores ").getResource()); + assertEquals(expected, + parseResourceConfigValue(" 5120. mb, 2. vcores ").getResource()); + + assertEquals(expected, parseResourceConfigValue("50% memory, 50% cpu"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, parseResourceConfigValue("50% Memory, 50% CpU"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), - parseResourceConfigValue("50%").getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 4), + assertEquals(BuilderUtils.newResource(5 * 1024, 4), parseResourceConfigValue("50% memory, 100% cpu"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 4), + assertEquals(BuilderUtils.newResource(5 * 1024, 4), parseResourceConfigValue(" 100% cpu, 50% memory"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 0), + assertEquals(BuilderUtils.newResource(5 * 1024, 0), parseResourceConfigValue("50% memory, 0% cpu"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, parseResourceConfigValue("50 % memory, 50 % cpu"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, parseResourceConfigValue("50%memory,50%cpu"). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, parseResourceConfigValue(" 50 % memory, 50 % cpu "). getResource(clusterResource)); - assertEquals(BuilderUtils.newResource(1024, 2), + assertEquals(expected, parseResourceConfigValue("50.% memory, 50.% cpu"). getResource(clusterResource)); - - clusterResource = BuilderUtils.newResource(1024 * 10, 4); assertEquals(BuilderUtils.newResource((int)(1024 * 10 * 0.109), 2), parseResourceConfigValue("10.9% memory, 50.6% cpu"). getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("50%").getResource(clusterResource)); + + Configuration conf = new Configuration(); + + conf.set(YarnConfiguration.RESOURCE_TYPES, "test1"); + ResourceUtils.resetResourceTypes(conf); + + clusterResource = BuilderUtils.newResource(10 * 1024, 4); + expected = BuilderUtils.newResource(5 * 1024, 2); + expected.setResourceValue("test1", Long.MAX_VALUE); + + assertEquals(expected, + parseResourceConfigValue("vcores=2, memory-mb=5120").getResource()); + assertEquals(expected, + parseResourceConfigValue("memory-mb=5120, vcores=2").getResource()); + assertEquals(expected, + parseResourceConfigValue("vcores=2,memory-mb=5120").getResource()); + assertEquals(expected, parseResourceConfigValue(" vcores = 2 , " + + "memory-mb = 5120 ").getResource()); + + expected.setResourceValue("test1", 0L); + + assertEquals(expected, + parseResourceConfigValue("vcores=2, memory-mb=5120", 0L).getResource()); + assertEquals(expected, + parseResourceConfigValue("memory-mb=5120, vcores=2", 0L).getResource()); + assertEquals(expected, + parseResourceConfigValue("vcores=2,memory-mb=5120", 0L).getResource()); + assertEquals(expected, + parseResourceConfigValue(" vcores = 2 , memory-mb = 5120 ", + 0L).getResource()); + + clusterResource.setResourceValue("test1", 8L); + expected.setResourceValue("test1", 4L); + + assertEquals(expected, + parseResourceConfigValue("50%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("vcores=2, memory-mb=5120, " + + "test1=4").getResource()); + assertEquals(expected, + parseResourceConfigValue("test1=4, vcores=2, " + + "memory-mb=5120").getResource()); + assertEquals(expected, + parseResourceConfigValue("memory-mb=5120, test1=4, " + + "vcores=2").getResource()); + assertEquals(expected, + parseResourceConfigValue("vcores=2,memory-mb=5120," + + "test1=4").getResource()); + assertEquals(expected, + parseResourceConfigValue(" vcores = 2 , memory-mb = 5120 , " + + "test1 = 4 ").getResource()); + + expected = BuilderUtils.newResource(4 * 1024, 3); + expected.setResourceValue("test1", 8L); + + assertEquals(expected, + parseResourceConfigValue("vcores=75%, " + + "memory-mb=40%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("memory-mb=40%, " + + "vcores=75%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("vcores=75%," + + "memory-mb=40%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue(" vcores = 75 % , " + + "memory-mb = 40 % ").getResource(clusterResource)); + + expected.setResourceValue("test1", 4L); + + assertEquals(expected, + parseResourceConfigValue("vcores=75%, memory-mb=40%, " + + "test1=50%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("test1=50%, vcores=75%, " + + "memory-mb=40%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("memory-mb=40%, test1=50%, " + + "vcores=75%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue("vcores=75%,memory-mb=40%," + + "test1=50%").getResource(clusterResource)); + assertEquals(expected, + parseResourceConfigValue(" vcores = 75 % , memory-mb = 40 % , " + + "test1 = 50 % ").getResource(clusterResource)); } @Test(expected = AllocationConfigurationException.class) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md index 269f5b40f49..b5bcbf5c8e9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md @@ -86,11 +86,11 @@ The allocation file must be in XML format. The format contains five types of ele * **Queue elements**: which represent queues. Queue elements can take an optional attribute 'type', which when set to 'parent' makes it a parent queue. This is useful when we want to create a parent queue without configuring any leaf queues. Each queue element may contain the following properties: - * **minResources**: minimum resources the queue is entitled to, in the form "X mb, Y vcores". For the single-resource fairness policy, the vcores value is ignored. If a queue's minimum share is not satisfied, it will be offered available resources before any other queue under the same parent. Under the single-resource fairness policy, a queue is considered unsatisfied if its memory usage is below its minimum memory share. Under dominant resource fairness, a queue is considered unsatisfied if its usage for its dominant resource with respect to the cluster capacity is below its minimum share for that resource. If multiple queues are unsatisfied in this situation, resources go to the queue with the smallest ratio between relevant resource usage and minimum. Note that it is possible that a queue that is below its minimum may not immediately get up to its minimum when it submits an application, because already-running jobs may be using those resources. + * **minResources**: minimum resources the queue is entitled to, in the form of "X mb, Y vcores" or "vcores=X, memory-mb=Y". The latter form is required when specifying resources other than memory and CPU. For the single-resource fairness policy, the vcores value is ignored. If a queue's minimum share is not satisfied, it will be offered available resources before any other queue under the same parent. Under the single-resource fairness policy, a queue is considered unsatisfied if its memory usage is below its minimum memory share. Under dominant resource fairness, a queue is considered unsatisfied if its usage for its dominant resource with respect to the cluster capacity is below its minimum share for that resource. If multiple queues are unsatisfied in this situation, resources go to the queue with the smallest ratio between relevant resource usage and its minimum. Note that it is possible for a queue that is below its minimum to not immediately get up to its minimum when an application is submitted to the queue, because already-running jobs may be using those resources. - * **maxResources**: maximum resources a queue is allocated, expressed either in absolute values (X mb, Y vcores) or as a percentage of the cluster resources (X% memory, Y% cpu). A queue will not be assigned a container that would put its aggregate usage over this limit. + * **maxResources**: maximum resources a queue will allocated, expressed in the form of "X%", "X% cpu, Y% memory", "X mb, Y vcores", or "vcores=X, memory-mb=Y". The last form is required when specifying resources other than memory and CPU. In the last form, X and Y can either be a percentage or an integer resource value without units. In the latter case the units will be inferred from the default units configured for that resource. A queue will not be assigned a container that would put its aggregate usage over this limit. - * **maxChildResources**: maximum resources an ad hoc child queue is allocated, expressed either in absolute values (X mb, Y vcores) or as a percentage of the cluster resources (X% memory, Y% cpu). An ad hoc child queue will not be assigned a container that would put its aggregate usage over this limit. + * **maxChildResources**: maximum resources an ad hoc child queue will allocated, expressed in the form of "X%", "X% cpu, Y% memory", "X mb, Y vcores", or "vcores=X, memory-mb=Y". The last form is required when specifying resources other than memory and CPU. In the last form, X and Y can either be a percentage or an integer resource value without units. In the latter case the units will be inferred from the default units configured for that resource. An ad hoc child queue will not be assigned a container that would put its aggregate usage over this limit. * **maxRunningApps**: limit the number of apps from the queue to run at once From a129e3e74e16ed039d637dc1499dc3e5df317d94 Mon Sep 17 00:00:00 2001 From: Robert Kanter Date: Thu, 5 Jul 2018 10:54:19 -0700 Subject: [PATCH 34/47] YARN-7451. Add missing tests to verify the presence of custom resources of RM apps and scheduler webservice endpoints (snemeth via rkanter) --- .../resourcemanager/webapp/dao/AppInfo.java | 2 +- .../webapp/dao/SchedulerInfo.java | 8 +- .../fair/TestFairSchedulerConfiguration.java | 9 +- .../webapp/TestRMWebServices.java | 31 +- .../webapp/TestRMWebServicesApps.java | 14 +- ...tRMWebServicesAppsCustomResourceTypes.java | 242 ++++++++++++++++ .../TestRMWebServicesCapacitySched.java | 30 +- ...estRMWebServicesConfigurationMutation.java | 5 + .../TestRMWebServicesFairScheduler.java | 95 +++--- .../TestRMWebServicesSchedulerActivities.java | 2 +- ...tomResourceTypesConfigurationProvider.java | 138 +++++++++ .../FairSchedulerJsonVerifications.java | 139 +++++++++ .../FairSchedulerXmlVerifications.java | 153 ++++++++++ ...vicesFairSchedulerCustomResourceTypes.java | 271 ++++++++++++++++++ .../helper/AppInfoJsonVerifications.java | 123 ++++++++ .../helper/AppInfoXmlVerifications.java | 132 +++++++++ .../webapp/helper/BufferedClientResponse.java | 57 ++++ .../JsonCustomResourceTypeTestcase.java | 77 +++++ .../ResourceRequestsJsonVerifications.java | 252 ++++++++++++++++ .../ResourceRequestsXmlVerifications.java | 215 ++++++++++++++ .../helper/XmlCustomResourceTypeTestCase.java | 112 ++++++++ 21 files changed, 2020 insertions(+), 87 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java index d47f13d9586..9d82bc78c76 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java @@ -479,7 +479,7 @@ public class AppInfo { public int getNumNonAMContainersPreempted() { return numNonAMContainerPreempted; } - + public int getNumAMContainersPreempted() { return numAMContainerPreempted; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java index 81491b14ce1..163f707253a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java @@ -41,8 +41,9 @@ public class SchedulerInfo { protected EnumSet schedulingResourceTypes; protected int maximumClusterPriority; + // JAXB needs this public SchedulerInfo() { - } // JAXB needs this + } public SchedulerInfo(final ResourceManager rm) { ResourceScheduler rs = rm.getResourceScheduler(); @@ -74,7 +75,10 @@ public class SchedulerInfo { } public String getSchedulerResourceTypes() { - return Arrays.toString(minAllocResource.getResource().getResources()); + if (minAllocResource != null) { + return Arrays.toString(minAllocResource.getResource().getResources()); + } + return null; } public int getMaxClusterLevelAppPriority() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java index 76a5af50960..70f83ab3095 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java @@ -48,6 +48,9 @@ import org.apache.log4j.spi.LoggingEvent; import org.junit.Assert; import org.junit.Test; +/** + * Tests fair scheduler configuration. + */ public class TestFairSchedulerConfiguration { private static final String A_CUSTOM_RESOURCE = "a-custom-resource"; @@ -242,12 +245,12 @@ public class TestFairSchedulerConfiguration { parseResourceConfigValue(" vcores = 75 % , memory-mb = 40 % , " + "test1 = 50 % ").getResource(clusterResource)); } - + @Test(expected = AllocationConfigurationException.class) public void testNoUnits() throws Exception { parseResourceConfigValue("1024"); } - + @Test(expected = AllocationConfigurationException.class) public void testOnlyMemory() throws Exception { parseResourceConfigValue("1024mb"); @@ -257,7 +260,7 @@ public class TestFairSchedulerConfiguration { public void testOnlyCPU() throws Exception { parseResourceConfigValue("1024vcores"); } - + @Test(expected = AllocationConfigurationException.class) public void testGibberish() throws Exception { parseResourceConfigValue("1o24vc0res"); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java index 0702d652a02..3902889a4a1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java @@ -53,11 +53,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService; -import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics; -import org.apache.hadoop.yarn.server.resourcemanager.MockRM; -import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; -import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.*; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; @@ -76,11 +72,12 @@ import org.apache.hadoop.yarn.webapp.JerseyTestBase; import org.apache.hadoop.yarn.webapp.WebServicesTestUtils; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; -import org.eclipse.jetty.server.Response; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -96,6 +93,8 @@ import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; import com.sun.jersey.test.framework.WebAppDescriptor; public class TestRMWebServices extends JerseyTestBase { + private static final Logger LOG = + LoggerFactory.getLogger(TestRMWebServices.class); private static MockRM rm; @@ -472,19 +471,19 @@ public class TestRMWebServices extends JerseyTestBase { QueueMetrics metrics = rs.getRootQueueMetrics(); ClusterMetrics clusterMetrics = ClusterMetrics.getMetrics(); - long totalMBExpect = + long totalMBExpect = metrics.getAvailableMB() + metrics.getAllocatedMB(); - long totalVirtualCoresExpect = + long totalVirtualCoresExpect = metrics.getAvailableVirtualCores() + metrics.getAllocatedVirtualCores(); - assertEquals("appsSubmitted doesn't match", + assertEquals("appsSubmitted doesn't match", metrics.getAppsSubmitted(), submittedApps); - assertEquals("appsCompleted doesn't match", + assertEquals("appsCompleted doesn't match", metrics.getAppsCompleted(), completedApps); assertEquals("reservedMB doesn't match", metrics.getReservedMB(), reservedMB); - assertEquals("availableMB doesn't match", + assertEquals("availableMB doesn't match", metrics.getAvailableMB(), availableMB); - assertEquals("allocatedMB doesn't match", + assertEquals("allocatedMB doesn't match", metrics.getAllocatedMB(), allocMB); assertEquals("reservedVirtualCores doesn't match", metrics.getReservedVirtualCores(), reservedVirtualCores); @@ -597,11 +596,13 @@ public class TestRMWebServices extends JerseyTestBase { public void verifyClusterSchedulerFifo(JSONObject json) throws JSONException, Exception { - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals("incorrect number of elements in: " + json, 1, json.length()); JSONObject info = json.getJSONObject("scheduler"); - assertEquals("incorrect number of elements", 1, info.length()); + assertEquals("incorrect number of elements in: " + info, 1, info.length()); info = info.getJSONObject("schedulerInfo"); - assertEquals("incorrect number of elements", 11, info.length()); + + LOG.debug("schedulerInfo: {}", info); + assertEquals("incorrect number of elements in: " + info, 11, info.length()); verifyClusterSchedulerFifoGeneric(info.getString("type"), info.getString("qstate"), (float) info.getDouble("capacity"), diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java index 6c6f400a623..15f94e1bb00 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java @@ -79,7 +79,7 @@ import com.sun.jersey.test.framework.WebAppDescriptor; public class TestRMWebServicesApps extends JerseyTestBase { private static MockRM rm; - + private static final int CONTAINER_MB = 1024; private static class WebServletModule extends ServletModule { @@ -324,7 +324,7 @@ public class TestRMWebServicesApps extends JerseyTestBase { assertEquals("incorrect number of elements", 1, apps.length()); array = apps.getJSONArray("app"); assertEquals("incorrect number of elements", 2, array.length()); - assertTrue("both app states of ACCEPTED and KILLED are not present", + assertTrue("both app states of ACCEPTED and KILLED are not present", (array.getJSONObject(0).getString("state").equals("ACCEPTED") && array.getJSONObject(1).getString("state").equals("KILLED")) || (array.getJSONObject(0).getString("state").equals("KILLED") && @@ -375,12 +375,12 @@ public class TestRMWebServicesApps extends JerseyTestBase { assertEquals("incorrect number of elements", 1, apps.length()); array = apps.getJSONArray("app"); assertEquals("incorrect number of elements", 2, array.length()); - assertTrue("both app states of ACCEPTED and KILLED are not present", + assertTrue("both app states of ACCEPTED and KILLED are not present", (array.getJSONObject(0).getString("state").equals("ACCEPTED") && array.getJSONObject(1).getString("state").equals("KILLED")) || (array.getJSONObject(0).getString("state").equals("KILLED") && array.getJSONObject(1).getString("state").equals("ACCEPTED"))); - + rm.stop(); } @@ -511,7 +511,8 @@ public class TestRMWebServicesApps extends JerseyTestBase { WebResource r = resource(); ClientResponse response = r.path("ws").path("v1").path("cluster") - .path("apps").queryParam("finalStatus", FinalApplicationStatus.UNDEFINED.toString()) + .path("apps").queryParam("finalStatus", + FinalApplicationStatus.UNDEFINED.toString()) .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); @@ -1804,7 +1805,8 @@ public class TestRMWebServicesApps extends JerseyTestBase { int numAttempt = 1; while (true) { // fail the AM by sending CONTAINER_FINISHED event without registering. - amNodeManager.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE); + amNodeManager.nodeHeartbeat(am.getApplicationAttemptId(), 1, + ContainerState.COMPLETE); rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FAILED); if (numAttempt == maxAppAttempts) { rm.waitForState(app1.getApplicationId(), RMAppState.FAILED); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java new file mode 100644 index 00000000000..83e00567eca --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsCustomResourceTypes.java @@ -0,0 +1,242 @@ +/** + * 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.yarn.server.resourcemanager.webapp; + +import com.google.inject.Guice; +import com.google.inject.servlet.ServletModule; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; +import com.sun.jersey.test.framework.WebAppDescriptor; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.MockAM; +import org.apache.hadoop.yarn.server.resourcemanager.MockNM; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.fairscheduler.CustomResourceTypesConfigurationProvider; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.AppInfoJsonVerifications; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.AppInfoXmlVerifications; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.BufferedClientResponse; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.JsonCustomResourceTypeTestcase; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.ResourceRequestsJsonVerifications; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.ResourceRequestsXmlVerifications; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.XmlCustomResourceTypeTestCase; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; +import org.apache.hadoop.yarn.webapp.GuiceServletConfig; +import org.apache.hadoop.yarn.webapp.JerseyTestBase; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; +import org.junit.Before; +import org.junit.Test; +import org.w3c.dom.Element; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; + +import javax.ws.rs.core.MediaType; +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +/** + * This test verifies that custom resource types are correctly serialized to XML + * and JSON when HTTP GET request is sent to the resource: ws/v1/cluster/apps. + */ +public class TestRMWebServicesAppsCustomResourceTypes extends JerseyTestBase { + + private static MockRM rm; + private static final int CONTAINER_MB = 1024; + + private static class WebServletModule extends ServletModule { + @Override + protected void configureServlets() { + bind(JAXBContextResolver.class); + bind(RMWebServices.class); + bind(GenericExceptionHandler.class); + Configuration conf = new Configuration(); + conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); + conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class, + ResourceScheduler.class); + initResourceTypes(conf); + rm = new MockRM(conf); + bind(ResourceManager.class).toInstance(rm); + serve("/*").with(GuiceContainer.class); + } + + private void initResourceTypes(Configuration conf) { + conf.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS, + CustomResourceTypesConfigurationProvider.class.getName()); + ResourceUtils.resetResourceTypes(conf); + } + } + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + createInjectorForWebServletModule(); + } + + private void createInjectorForWebServletModule() { + GuiceServletConfig + .setInjector(Guice.createInjector(new WebServletModule())); + } + + public TestRMWebServicesAppsCustomResourceTypes() { + super(new WebAppDescriptor.Builder( + "org.apache.hadoop.yarn.server.resourcemanager.webapp") + .contextListenerClass(GuiceServletConfig.class) + .filterClass(com.google.inject.servlet.GuiceFilter.class) + .contextPath("jersey-guice-filter").servletPath("/").build()); + } + + @Test + public void testRunningAppXml() throws Exception { + rm.start(); + MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048); + RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, amNodeManager); + am1.allocate("*", 2048, 1, new ArrayList<>()); + amNodeManager.nodeHeartbeat(true); + + WebResource r = resource(); + WebResource path = r.path("ws").path("v1").path("cluster").path("apps"); + ClientResponse response = + path.accept(MediaType.APPLICATION_XML).get(ClientResponse.class); + + XmlCustomResourceTypeTestCase testCase = + new XmlCustomResourceTypeTestCase(path, + new BufferedClientResponse(response)); + testCase.verify(document -> { + NodeList apps = document.getElementsByTagName("apps"); + assertEquals("incorrect number of apps elements", 1, apps.getLength()); + + NodeList appArray = ((Element)(apps.item(0))) + .getElementsByTagName("app"); + assertEquals("incorrect number of app elements", 1, appArray.getLength()); + + verifyAppsXML(appArray, app1); + }); + + rm.stop(); + } + + @Test + public void testRunningAppJson() throws Exception { + rm.start(); + MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048); + RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, amNodeManager); + am1.allocate("*", 2048, 1, new ArrayList<>()); + amNodeManager.nodeHeartbeat(true); + + WebResource r = resource(); + WebResource path = r.path("ws").path("v1").path("cluster").path("apps"); + ClientResponse response = + path.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + + JsonCustomResourceTypeTestcase testCase = + new JsonCustomResourceTypeTestcase(path, + new BufferedClientResponse(response)); + testCase.verify(json -> { + try { + assertEquals("incorrect number of apps elements", 1, json.length()); + JSONObject apps = json.getJSONObject("apps"); + assertEquals("incorrect number of app elements", 1, apps.length()); + JSONArray array = apps.getJSONArray("app"); + assertEquals("incorrect count of app", 1, array.length()); + + verifyAppInfoJson(array.getJSONObject(0), app1); + } catch (JSONException e) { + throw new RuntimeException(e); + } + }); + + rm.stop(); + } + + private void verifyAppsXML(NodeList appArray, RMApp app) { + for (int i = 0; i < appArray.getLength(); i++) { + Element element = (Element) appArray.item(i); + AppInfoXmlVerifications.verify(element, app); + + NodeList resourceRequests = + element.getElementsByTagName("resourceRequests"); + assertEquals(1, resourceRequests.getLength()); + Node resourceRequest = resourceRequests.item(0); + ResourceRequest rr = + ((AbstractYarnScheduler) rm.getRMContext().getScheduler()) + .getApplicationAttempt( + app.getCurrentAppAttempt().getAppAttemptId()) + .getAppSchedulingInfo().getAllResourceRequests().get(0); + ResourceRequestsXmlVerifications.verifyWithCustomResourceTypes( + (Element) resourceRequest, rr, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + } + + private void verifyAppInfoJson(JSONObject info, RMApp app) throws + JSONException { + int expectedNumberOfElements = getExpectedNumberOfElements(app); + + assertEquals("incorrect number of elements", expectedNumberOfElements, + info.length()); + + AppInfoJsonVerifications.verify(info, app); + + JSONArray resourceRequests = info.getJSONArray("resourceRequests"); + JSONObject requestInfo = resourceRequests.getJSONObject(0); + ResourceRequest rr = + ((AbstractYarnScheduler) rm.getRMContext().getScheduler()) + .getApplicationAttempt(app.getCurrentAppAttempt().getAppAttemptId()) + .getAppSchedulingInfo().getAllResourceRequests().get(0); + + ResourceRequestsJsonVerifications.verifyWithCustomResourceTypes( + requestInfo, rr, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + + private int getExpectedNumberOfElements(RMApp app) { + int expectedNumberOfElements = 40 + 2; // 2 -> resourceRequests + if (app.getApplicationSubmissionContext() + .getNodeLabelExpression() != null) { + expectedNumberOfElements++; + } + + if (app.getAMResourceRequests().get(0).getNodeLabelExpression() != null) { + expectedNumberOfElements++; + } + + if (AppInfo + .getAmRPCAddressFromRMAppAttempt(app.getCurrentAppAttempt()) != null) { + expectedNumberOfElements++; + } + return expectedNumberOfElements; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java index e37f76fa25f..46d0a6614fa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java @@ -146,7 +146,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { config.setUserLimitFactor(B2, 100.0f); config.setCapacity(B3, 0.5f); config.setUserLimitFactor(B3, 100.0f); - + config.setQueues(A1, new String[] {"a1a", "a1b"}); final String A1A = A1 + ".a1a"; config.setCapacity(A1A, 85); @@ -254,7 +254,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { } } - public void verifySubQueueXML(Element qElem, String q, + public void verifySubQueueXML(Element qElem, String q, float parentAbsCapacity, float parentAbsMaxCapacity) throws Exception { NodeList children = qElem.getChildNodes(); @@ -317,30 +317,34 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { private void verifyClusterScheduler(JSONObject json) throws JSONException, Exception { - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals("incorrect number of elements in: " + json, 1, json.length()); JSONObject info = json.getJSONObject("scheduler"); - assertEquals("incorrect number of elements", 1, info.length()); + assertEquals("incorrect number of elements in: " + info, 1, info.length()); info = info.getJSONObject("schedulerInfo"); - assertEquals("incorrect number of elements", 8, info.length()); + assertEquals("incorrect number of elements in: " + info, 8, info.length()); verifyClusterSchedulerGeneric(info.getString("type"), (float) info.getDouble("usedCapacity"), (float) info.getDouble("capacity"), (float) info.getDouble("maxCapacity"), info.getString("queueName")); JSONObject health = info.getJSONObject("health"); assertNotNull(health); - assertEquals("incorrect number of elements", 3, health.length()); + assertEquals("incorrect number of elements in: " + health, 3, + health.length()); JSONArray operationsInfo = health.getJSONArray("operationsInfo"); - assertEquals("incorrect number of elements", 4, operationsInfo.length()); + assertEquals("incorrect number of elements in: " + health, 4, + operationsInfo.length()); JSONArray lastRunDetails = health.getJSONArray("lastRunDetails"); - assertEquals("incorrect number of elements", 3, lastRunDetails.length()); + assertEquals("incorrect number of elements in: " + health, 3, + lastRunDetails.length()); JSONArray arr = info.getJSONObject("queues").getJSONArray("queue"); - assertEquals("incorrect number of elements", 2, arr.length()); + assertEquals("incorrect number of elements in: " + arr, 2, arr.length()); // test subqueues for (int i = 0; i < arr.length(); i++) { JSONObject obj = arr.getJSONObject(i); - String q = CapacitySchedulerConfiguration.ROOT + "." + obj.getString("queueName"); + String q = CapacitySchedulerConfiguration.ROOT + "." + + obj.getString("queueName"); verifySubQueue(obj, q, 100, 100); } } @@ -355,7 +359,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { assertTrue("queueName doesn't match", "root".matches(queueName)); } - private void verifySubQueue(JSONObject info, String q, + private void verifySubQueue(JSONObject info, String q, float parentAbsCapacity, float parentAbsMaxCapacity) throws JSONException, Exception { int numExpectedElements = 20; @@ -464,7 +468,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { csConf.getUserLimitFactor(q), info.userLimitFactor, 1e-3f); } - //Return a child Node of node with the tagname or null if none exists + //Return a child Node of node with the tagname or null if none exists private Node getChildNodeByName(Node node, String tagname) { NodeList nodeList = node.getChildNodes(); for (int i=0; i < nodeList.getLength(); ++i) { @@ -514,7 +518,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { for (int j=0; j + * 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. @@ -16,13 +16,14 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.server.resourcemanager.webapp; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -import javax.ws.rs.core.MediaType; +package org.apache.hadoop.yarn.server.resourcemanager.webapp.fairscheduler; +import com.google.inject.Guice; +import com.google.inject.servlet.ServletModule; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; +import com.sun.jersey.test.framework.WebAppDescriptor; import org.apache.hadoop.http.JettyUtils; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.MockRM; @@ -30,6 +31,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueueManager; + +import org.apache.hadoop.yarn.server.resourcemanager.webapp.JAXBContextResolver; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServices; import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; import org.apache.hadoop.yarn.webapp.GuiceServletConfig; import org.apache.hadoop.yarn.webapp.JerseyTestBase; @@ -38,18 +42,18 @@ import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; import org.junit.Before; import org.junit.Test; +import javax.ws.rs.core.MediaType; -import com.google.inject.Guice; -import com.google.inject.servlet.ServletModule; -import com.sun.jersey.api.client.ClientResponse; -import com.sun.jersey.api.client.WebResource; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import com.sun.jersey.test.framework.WebAppDescriptor; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +/** + * Tests RM Webservices fair scheduler resources. + */ public class TestRMWebServicesFairScheduler extends JerseyTestBase { private static MockRM rm; private static YarnConfiguration conf; - + private static class WebServletModule extends ServletModule { @Override protected void configureServlets() { @@ -58,7 +62,7 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase { bind(GenericExceptionHandler.class); conf = new YarnConfiguration(); conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class, - ResourceScheduler.class); + ResourceScheduler.class); rm = new MockRM(conf); bind(ResourceManager.class).toInstance(rm); serve("/*").with(GuiceContainer.class); @@ -66,32 +70,32 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase { } static { - GuiceServletConfig.setInjector( - Guice.createInjector(new WebServletModule())); + GuiceServletConfig + .setInjector(Guice.createInjector(new WebServletModule())); } @Before @Override public void setUp() throws Exception { super.setUp(); - GuiceServletConfig.setInjector( - Guice.createInjector(new WebServletModule())); + GuiceServletConfig + .setInjector(Guice.createInjector(new WebServletModule())); } public TestRMWebServicesFairScheduler() { super(new WebAppDescriptor.Builder( "org.apache.hadoop.yarn.server.resourcemanager.webapp") - .contextListenerClass(GuiceServletConfig.class) - .filterClass(com.google.inject.servlet.GuiceFilter.class) - .contextPath("jersey-guice-filter").servletPath("/").build()); + .contextListenerClass(GuiceServletConfig.class) + .filterClass(com.google.inject.servlet.GuiceFilter.class) + .contextPath("jersey-guice-filter").servletPath("/").build()); } - + @Test - public void testClusterScheduler() throws JSONException, Exception { + public void testClusterScheduler() throws JSONException { WebResource r = resource(); - ClientResponse response = r.path("ws").path("v1").path("cluster") - .path("scheduler").accept(MediaType.APPLICATION_JSON) - .get(ClientResponse.class); + ClientResponse response = + r.path("ws").path("v1").path("cluster").path("scheduler") + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); JSONObject json = response.getEntity(JSONObject.class); @@ -99,52 +103,51 @@ public class TestRMWebServicesFairScheduler extends JerseyTestBase { } @Test - public void testClusterSchedulerSlash() throws JSONException, Exception { + public void testClusterSchedulerSlash() throws JSONException { WebResource r = resource(); - ClientResponse response = r.path("ws").path("v1").path("cluster") - .path("scheduler/").accept(MediaType.APPLICATION_JSON) - .get(ClientResponse.class); + ClientResponse response = + r.path("ws").path("v1").path("cluster").path("scheduler/") + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); JSONObject json = response.getEntity(JSONObject.class); verifyClusterScheduler(json); } - + @Test - public void testClusterSchedulerWithSubQueues() throws JSONException, - Exception { - FairScheduler scheduler = (FairScheduler)rm.getResourceScheduler(); + public void testClusterSchedulerWithSubQueues() + throws JSONException { + FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); QueueManager queueManager = scheduler.getQueueManager(); // create LeafQueue queueManager.getLeafQueue("root.q.subqueue1", true); queueManager.getLeafQueue("root.q.subqueue2", true); WebResource r = resource(); - ClientResponse response = r.path("ws").path("v1").path("cluster") - .path("scheduler").accept(MediaType.APPLICATION_JSON) - .get(ClientResponse.class); + ClientResponse response = + r.path("ws").path("v1").path("cluster").path("scheduler") + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); JSONObject json = response.getEntity(JSONObject.class); JSONArray subQueueInfo = json.getJSONObject("scheduler") .getJSONObject("schedulerInfo").getJSONObject("rootQueue") - .getJSONObject("childQueues").getJSONArray("queue") - .getJSONObject(1).getJSONObject("childQueues").getJSONArray("queue"); + .getJSONObject("childQueues").getJSONArray("queue").getJSONObject(1) + .getJSONObject("childQueues").getJSONArray("queue"); // subQueueInfo is consist of subqueue1 and subqueue2 info assertEquals(2, subQueueInfo.length()); // Verify 'childQueues' field is omitted from FairSchedulerLeafQueueInfo. try { subQueueInfo.getJSONObject(1).getJSONObject("childQueues"); - fail("FairSchedulerQueueInfo should omit field 'childQueues'" + - "if child queue is empty."); + fail("FairSchedulerQueueInfo should omit field 'childQueues'" + + "if child queue is empty."); } catch (JSONException je) { assertEquals("JSONObject[\"childQueues\"] not found.", je.getMessage()); } } - private void verifyClusterScheduler(JSONObject json) throws JSONException, - Exception { + private void verifyClusterScheduler(JSONObject json) throws JSONException { assertEquals("incorrect number of elements", 1, json.length()); JSONObject info = json.getJSONObject("scheduler"); assertEquals("incorrect number of elements", 1, info.length()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java index 1e61186c3ab..40cf483cd3a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java @@ -457,7 +457,7 @@ public class TestRMWebServicesSchedulerActivities if (object.getClass() == JSONObject.class) { assertEquals("Number of allocations is wrong", 1, realValue); } else if (object.getClass() == JSONArray.class) { - assertEquals("Number of allocations is wrong", + assertEquals("Number of allocations is wrong in: " + object, ((JSONArray) object).length(), realValue); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java new file mode 100644 index 00000000000..bb1fce05a46 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/CustomResourceTypesConfigurationProvider.java @@ -0,0 +1,138 @@ +/* + * 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.yarn.server.resourcemanager.webapp.fairscheduler; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.LocalConfigurationProvider; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static java.util.stream.Collectors.toList; + +/** + * This class can generate an XML configuration file of custom resource types. + * See createInitialResourceTypes for the default values. All custom resource + * type is prefixed with CUSTOM_RESOURCE_PREFIX. Please use the + * getConfigurationInputStream method to get an InputStream of the XML. If you + * want to have different number of resources in your tests, please see usages + * of this class in this test class: + * {@link TestRMWebServicesFairSchedulerCustomResourceTypes} + * + */ +public class CustomResourceTypesConfigurationProvider + extends LocalConfigurationProvider { + + private static class CustomResourceTypes { + private int count; + private String xml; + + CustomResourceTypes(String xml, int count) { + this.xml = xml; + this.count = count; + } + + public int getCount() { + return count; + } + + public String getXml() { + return xml; + } + } + + private static final String CUSTOM_RESOURCE_PREFIX = "customResource-"; + + private static CustomResourceTypes customResourceTypes = + createInitialResourceTypes(); + + private static CustomResourceTypes createInitialResourceTypes() { + return createCustomResourceTypes(2); + } + + private static CustomResourceTypes createCustomResourceTypes(int count) { + List resourceTypeNames = generateResourceTypeNames(count); + + List resourceUnitXmlElements = IntStream.range(0, count) + .boxed() + .map(i -> getResourceUnitsXml(resourceTypeNames.get(i))) + .collect(toList()); + + StringBuilder sb = new StringBuilder("\n"); + sb.append(getResourceTypesXml(resourceTypeNames)); + + for (String resourceUnitXml : resourceUnitXmlElements) { + sb.append(resourceUnitXml); + + } + sb.append(""); + + return new CustomResourceTypes(sb.toString(), count); + } + + private static List generateResourceTypeNames(int count) { + return IntStream.range(0, count) + .boxed() + .map(i -> CUSTOM_RESOURCE_PREFIX + i) + .collect(toList()); + } + + private static String getResourceUnitsXml(String resource) { + return "\n" + "yarn.resource-types." + resource + + ".units\n" + "k\n" + "\n"; + } + + private static String getResourceTypesXml(List resources) { + final String resourceTypes = makeCommaSeparatedString(resources); + + return "\n" + "yarn.resource-types\n" + "" + + resourceTypes + "\n" + "\n"; + } + + private static String makeCommaSeparatedString(List resources) { + return resources.stream().collect(Collectors.joining(",")); + } + + @Override + public InputStream getConfigurationInputStream(Configuration bootstrapConf, + String name) throws YarnException, IOException { + if (YarnConfiguration.RESOURCE_TYPES_CONFIGURATION_FILE.equals(name)) { + return new ByteArrayInputStream( + customResourceTypes.getXml().getBytes()); + } else { + return super.getConfigurationInputStream(bootstrapConf, name); + } + } + + public static void reset() { + customResourceTypes = createInitialResourceTypes(); + } + + public static void setNumberOfResourceTypes(int count) { + customResourceTypes = createCustomResourceTypes(count); + } + + public static List getCustomResourceTypes() { + return generateResourceTypeNames(customResourceTypes.getCount()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java new file mode 100644 index 00000000000..924411a4de0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerJsonVerifications.java @@ -0,0 +1,139 @@ +/* + * 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.yarn.server.resourcemanager.webapp.fairscheduler; + +import com.google.common.collect.Sets; +import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; + +import java.util.List; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * This test helper class is primarily used by + * {@link TestRMWebServicesFairSchedulerCustomResourceTypes}. + */ +public class FairSchedulerJsonVerifications { + + private static final Set RESOURCE_FIELDS = + Sets.newHashSet("minResources", "amUsedResources", "amMaxResources", + "fairResources", "clusterResources", "reservedResources", + "maxResources", "usedResources", "steadyFairResources", + "demandResources"); + private final Set customResourceTypes; + + FairSchedulerJsonVerifications(List customResourceTypes) { + this.customResourceTypes = Sets.newHashSet(customResourceTypes); + } + + public void verify(JSONObject jsonObject) { + try { + verifyResourcesContainDefaultResourceTypes(jsonObject, RESOURCE_FIELDS); + verifyResourcesContainCustomResourceTypes(jsonObject, RESOURCE_FIELDS); + } catch (JSONException e) { + throw new RuntimeException(e); + } + } + + private void verifyResourcesContainDefaultResourceTypes(JSONObject queue, + Set resourceCategories) throws JSONException { + for (String resourceCategory : resourceCategories) { + boolean hasResourceCategory = queue.has(resourceCategory); + assertTrue("Queue " + queue + " does not have resource category key: " + + resourceCategory, hasResourceCategory); + verifyResourceContainsDefaultResourceTypes( + queue.getJSONObject(resourceCategory)); + } + } + + private void verifyResourceContainsDefaultResourceTypes( + JSONObject jsonObject) { + Object memory = jsonObject.opt("memory"); + Object vCores = jsonObject.opt("vCores"); + + assertNotNull("Key 'memory' not found in: " + jsonObject, memory); + assertNotNull("Key 'vCores' not found in: " + jsonObject, vCores); + } + + private void verifyResourcesContainCustomResourceTypes(JSONObject queue, + Set resourceCategories) throws JSONException { + for (String resourceCategory : resourceCategories) { + assertTrue("Queue " + queue + " does not have resource category key: " + + resourceCategory, queue.has(resourceCategory)); + verifyResourceContainsAllCustomResourceTypes( + queue.getJSONObject(resourceCategory)); + } + } + + private void verifyResourceContainsAllCustomResourceTypes( + JSONObject resourceCategory) throws JSONException { + assertTrue("resourceCategory does not have resourceInformations: " + + resourceCategory, resourceCategory.has("resourceInformations")); + + JSONObject resourceInformations = + resourceCategory.getJSONObject("resourceInformations"); + assertTrue( + "resourceInformations does not have resourceInformation object: " + + resourceInformations, + resourceInformations.has("resourceInformation")); + JSONArray customResources = + resourceInformations.getJSONArray("resourceInformation"); + + // customResources will include vcores / memory as well + assertEquals( + "Different number of custom resource types found than expected", + customResourceTypes.size(), customResources.length() - 2); + + for (int i = 0; i < customResources.length(); i++) { + JSONObject customResource = customResources.getJSONObject(i); + assertTrue("Resource type does not have name field: " + customResource, + customResource.has("name")); + assertTrue("Resource type does not have name resourceType field: " + + customResource, customResource.has("resourceType")); + assertTrue( + "Resource type does not have name units field: " + customResource, + customResource.has("units")); + assertTrue( + "Resource type does not have name value field: " + customResource, + customResource.has("value")); + + String name = customResource.getString("name"); + String unit = customResource.getString("units"); + String resourceType = customResource.getString("resourceType"); + Long value = customResource.getLong("value"); + + if (ResourceInformation.MEMORY_URI.equals(name) + || ResourceInformation.VCORES_URI.equals(name)) { + continue; + } + + assertTrue("Custom resource type " + name + " not found", + customResourceTypes.contains(name)); + assertEquals("k", unit); + assertEquals(ResourceTypes.COUNTABLE, + ResourceTypes.valueOf(resourceType)); + assertNotNull("Custom resource value " + value + " is null!", value); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java new file mode 100644 index 00000000000..63ae7b74f1c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/FairSchedulerXmlVerifications.java @@ -0,0 +1,153 @@ +/* + * 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.yarn.server.resourcemanager.webapp.fairscheduler; + + +import com.google.common.collect.Sets; +import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; + +import java.util.List; +import java.util.Set; + +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.XmlCustomResourceTypeTestCase.toXml; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlLong; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * This test helper class is primarily used by + * {@link TestRMWebServicesFairSchedulerCustomResourceTypes}. + */ +public class FairSchedulerXmlVerifications { + + private static final Set RESOURCE_FIELDS = Sets.newHashSet( + "minResources", "amUsedResources", "amMaxResources", "fairResources", + "clusterResources", "reservedResources", "maxResources", "usedResources", + "steadyFairResources", "demandResources"); + private final Set customResourceTypes; + + FairSchedulerXmlVerifications(List customResourceTypes) { + this.customResourceTypes = Sets.newHashSet(customResourceTypes); + } + + public void verify(Element element) { + verifyResourcesContainDefaultResourceTypes(element, RESOURCE_FIELDS); + verifyResourcesContainCustomResourceTypes(element, RESOURCE_FIELDS); + } + + private void verifyResourcesContainDefaultResourceTypes(Element queue, + Set resourceCategories) { + for (String resourceCategory : resourceCategories) { + boolean hasResourceCategory = hasChild(queue, resourceCategory); + assertTrue("Queue " + queue + " does not have resource category key: " + + resourceCategory, hasResourceCategory); + verifyResourceContainsDefaultResourceTypes( + (Element) queue.getElementsByTagName(resourceCategory).item(0)); + } + } + + private void verifyResourceContainsDefaultResourceTypes( + Element element) { + Object memory = opt(element, "memory"); + Object vCores = opt(element, "vCores"); + + assertNotNull("Key 'memory' not found in: " + element, memory); + assertNotNull("Key 'vCores' not found in: " + element, vCores); + } + + private void verifyResourcesContainCustomResourceTypes(Element queue, + Set resourceCategories) { + for (String resourceCategory : resourceCategories) { + assertTrue("Queue " + queue + " does not have key for resourceCategory: " + + resourceCategory, hasChild(queue, resourceCategory)); + verifyResourceContainsCustomResourceTypes( + (Element) queue.getElementsByTagName(resourceCategory).item(0)); + } + } + + private void verifyResourceContainsCustomResourceTypes( + Element resourceCategory) { + assertEquals( + toXml(resourceCategory) + + " should have only one resourceInformations child!", + 1, resourceCategory.getElementsByTagName("resourceInformations") + .getLength()); + Element resourceInformations = (Element) resourceCategory + .getElementsByTagName("resourceInformations").item(0); + + NodeList customResources = + resourceInformations.getElementsByTagName("resourceInformation"); + + // customResources will include vcores / memory as well + assertEquals( + "Different number of custom resource types found than expected", + customResourceTypes.size(), customResources.getLength() - 2); + + for (int i = 0; i < customResources.getLength(); i++) { + Element customResource = (Element) customResources.item(i); + String name = getXmlString(customResource, "name"); + String unit = getXmlString(customResource, "units"); + String resourceType = getXmlString(customResource, "resourceType"); + Long value = getXmlLong(customResource, "value"); + + if (ResourceInformation.MEMORY_URI.equals(name) + || ResourceInformation.VCORES_URI.equals(name)) { + continue; + } + + assertTrue("Custom resource type " + name + " not found", + customResourceTypes.contains(name)); + assertEquals("k", unit); + assertEquals(ResourceTypes.COUNTABLE, + ResourceTypes.valueOf(resourceType)); + assertNotNull("Resource value should not be null for resource type " + + resourceType + ", listing xml contents: " + toXml(customResource), + value); + } + } + + private Object opt(Node node, String child) { + NodeList nodes = getElementsByTagNameInternal(node, child); + if (nodes.getLength() > 0) { + return nodes.item(0); + } + + return null; + } + + private boolean hasChild(Node node, String child) { + return getElementsByTagNameInternal(node, child).getLength() > 0; + } + + private NodeList getElementsByTagNameInternal(Node node, String child) { + if (node instanceof Element) { + return ((Element) node).getElementsByTagName(child); + } else if (node instanceof Document) { + return ((Document) node).getElementsByTagName(child); + } else { + throw new IllegalStateException("Unknown type of wrappedObject: " + node + + ", type: " + node.getClass()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java new file mode 100644 index 00000000000..de4d5a10d1f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/fairscheduler/TestRMWebServicesFairSchedulerCustomResourceTypes.java @@ -0,0 +1,271 @@ +/* + * 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.yarn.server.resourcemanager.webapp.fairscheduler; + +import com.google.inject.Guice; +import com.google.inject.servlet.ServletModule; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; +import com.sun.jersey.test.framework.WebAppDescriptor; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueueManager; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.JAXBContextResolver; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServices; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.*; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; +import org.apache.hadoop.yarn.webapp.GuiceServletConfig; +import org.apache.hadoop.yarn.webapp.JerseyTestBase; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.w3c.dom.Element; +import javax.ws.rs.core.MediaType; +import java.lang.reflect.Method; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; + +/** + * This class is to test response representations of queue resources, + * explicitly setting custom resource types. with the help of + * {@link CustomResourceTypesConfigurationProvider} + */ +public class TestRMWebServicesFairSchedulerCustomResourceTypes + extends JerseyTestBase { + private static MockRM rm; + private static YarnConfiguration conf; + + private static class WebServletModule extends ServletModule { + @Override + protected void configureServlets() { + bind(JAXBContextResolver.class); + bind(RMWebServices.class); + bind(GenericExceptionHandler.class); + conf = new YarnConfiguration(); + conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class, + ResourceScheduler.class); + initResourceTypes(conf); + rm = new MockRM(conf); + bind(ResourceManager.class).toInstance(rm); + serve("/*").with(GuiceContainer.class); + } + + private void initResourceTypes(YarnConfiguration conf) { + conf.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS, + CustomResourceTypesConfigurationProvider.class.getName()); + ResourceUtils.resetResourceTypes(conf); + } + } + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + createInjectorForWebServletModule(); + } + + @After + public void tearDown() { + ResourceUtils.resetResourceTypes(new Configuration()); + } + + private void createInjectorForWebServletModule() { + GuiceServletConfig + .setInjector(Guice.createInjector(new WebServletModule())); + } + + @After + public void teardown() { + CustomResourceTypesConfigurationProvider.reset(); + } + + public TestRMWebServicesFairSchedulerCustomResourceTypes() { + super(new WebAppDescriptor.Builder( + "org.apache.hadoop.yarn.server.resourcemanager.webapp") + .contextListenerClass(GuiceServletConfig.class) + .filterClass(com.google.inject.servlet.GuiceFilter.class) + .contextPath("jersey-guice-filter").servletPath("/").build()); + } + + @Test + public void testClusterSchedulerWithCustomResourceTypesJson() { + FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); + QueueManager queueManager = scheduler.getQueueManager(); + // create LeafQueues + queueManager.getLeafQueue("root.q.subqueue1", true); + queueManager.getLeafQueue("root.q.subqueue2", true); + + FSLeafQueue subqueue1 = + queueManager.getLeafQueue("root.q.subqueue1", false); + incrementUsedResourcesOnQueue(subqueue1, 33L); + + WebResource path = + resource().path("ws").path("v1").path("cluster").path("scheduler"); + ClientResponse response = + path.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + + verifyJsonResponse(path, response, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + + @Test + public void testClusterSchedulerWithCustomResourceTypesXml() { + FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); + QueueManager queueManager = scheduler.getQueueManager(); + // create LeafQueues + queueManager.getLeafQueue("root.q.subqueue1", true); + queueManager.getLeafQueue("root.q.subqueue2", true); + + FSLeafQueue subqueue1 = + queueManager.getLeafQueue("root.q.subqueue1", false); + incrementUsedResourcesOnQueue(subqueue1, 33L); + + WebResource path = + resource().path("ws").path("v1").path("cluster").path("scheduler"); + ClientResponse response = + path.accept(MediaType.APPLICATION_XML).get(ClientResponse.class); + + verifyXmlResponse(path, response, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + + @Test + public void testClusterSchedulerWithElevenCustomResourceTypesXml() { + CustomResourceTypesConfigurationProvider.setNumberOfResourceTypes(11); + createInjectorForWebServletModule(); + + FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); + QueueManager queueManager = scheduler.getQueueManager(); + // create LeafQueues + queueManager.getLeafQueue("root.q.subqueue1", true); + queueManager.getLeafQueue("root.q.subqueue2", true); + + FSLeafQueue subqueue1 = + queueManager.getLeafQueue("root.q.subqueue1", false); + incrementUsedResourcesOnQueue(subqueue1, 33L); + + WebResource path = + resource().path("ws").path("v1").path("cluster").path("scheduler"); + ClientResponse response = + path.accept(MediaType.APPLICATION_XML).get(ClientResponse.class); + + verifyXmlResponse(path, response, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + + @Test + public void testClusterSchedulerElevenWithCustomResourceTypesJson() { + CustomResourceTypesConfigurationProvider.setNumberOfResourceTypes(11); + createInjectorForWebServletModule(); + + FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler(); + QueueManager queueManager = scheduler.getQueueManager(); + // create LeafQueues + queueManager.getLeafQueue("root.q.subqueue1", true); + queueManager.getLeafQueue("root.q.subqueue2", true); + + FSLeafQueue subqueue1 = + queueManager.getLeafQueue("root.q.subqueue1", false); + incrementUsedResourcesOnQueue(subqueue1, 33L); + + WebResource path = + resource().path("ws").path("v1").path("cluster").path("scheduler"); + ClientResponse response = + path.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + + verifyJsonResponse(path, response, + CustomResourceTypesConfigurationProvider.getCustomResourceTypes()); + } + + private void verifyJsonResponse(WebResource path, ClientResponse response, + List customResourceTypes) { + JsonCustomResourceTypeTestcase testCase = + new JsonCustomResourceTypeTestcase(path, + new BufferedClientResponse(response)); + testCase.verify(json -> { + try { + JSONArray queues = json.getJSONObject("scheduler") + .getJSONObject("schedulerInfo").getJSONObject("rootQueue") + .getJSONObject("childQueues").getJSONArray("queue"); + + // childQueueInfo consists of subqueue1 and subqueue2 info + assertEquals(2, queues.length()); + JSONObject firstChildQueue = queues.getJSONObject(0); + new FairSchedulerJsonVerifications(customResourceTypes) + .verify(firstChildQueue); + } catch (JSONException e) { + throw new RuntimeException(e); + } + }); + } + + private void verifyXmlResponse(WebResource path, ClientResponse response, + List customResourceTypes) { + XmlCustomResourceTypeTestCase testCase = new XmlCustomResourceTypeTestCase( + path, new BufferedClientResponse(response)); + + testCase.verify(xml -> { + Element scheduler = + (Element) xml.getElementsByTagName("scheduler").item(0); + Element schedulerInfo = + (Element) scheduler.getElementsByTagName("schedulerInfo").item(0); + Element rootQueue = + (Element) schedulerInfo.getElementsByTagName("rootQueue").item(0); + + Element childQueues = + (Element) rootQueue.getElementsByTagName("childQueues").item(0); + Element queue = + (Element) childQueues.getElementsByTagName("queue").item(0); + new FairSchedulerXmlVerifications(customResourceTypes).verify(queue); + }); + } + + private void incrementUsedResourcesOnQueue(final FSLeafQueue queue, + final long value) { + try { + Method incUsedResourceMethod = queue.getClass().getSuperclass() + .getDeclaredMethod("incUsedResource", Resource.class); + incUsedResourceMethod.setAccessible(true); + + Map customResources = + CustomResourceTypesConfigurationProvider.getCustomResourceTypes() + .stream() + .collect(Collectors.toMap(Function.identity(), v -> value)); + + incUsedResourceMethod.invoke(queue, + Resource.newInstance(20, 30, customResources)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java new file mode 100644 index 00000000000..4ab1443ce54 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoJsonVerifications.java @@ -0,0 +1,123 @@ +/* + * 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.yarn.server.resourcemanager.webapp.helper; + +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.checkStringEqual; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.checkStringMatch; +import static org.junit.Assert.*; + +/** + * Contains all value verifications that are needed to verify {@link AppInfo} + * JSON objects. + */ +public final class AppInfoJsonVerifications { + + private AppInfoJsonVerifications() { + //utility class + } + + /** + * Tests whether {@link AppInfo} representation object contains the required + * values as per defined in the specified app parameter. + * @param app an RMApp instance that contains the required values + * to test against. + */ + public static void verify(JSONObject info, RMApp app) throws JSONException { + checkStringMatch("id", app.getApplicationId().toString(), + info.getString("id")); + checkStringMatch("user", app.getUser(), info.getString("user")); + checkStringMatch("name", app.getName(), info.getString("name")); + checkStringMatch("applicationType", app.getApplicationType(), + info.getString("applicationType")); + checkStringMatch("queue", app.getQueue(), info.getString("queue")); + assertEquals("priority doesn't match", 0, info.getInt("priority")); + checkStringMatch("state", app.getState().toString(), + info.getString("state")); + checkStringMatch("finalStatus", app.getFinalApplicationStatus().toString(), + info.getString("finalStatus")); + assertEquals("progress doesn't match", 0, + (float) info.getDouble("progress"), 0.0); + if ("UNASSIGNED".equals(info.getString("trackingUI"))) { + checkStringMatch("trackingUI", "UNASSIGNED", + info.getString("trackingUI")); + } + checkStringEqual("diagnostics", app.getDiagnostics().toString(), + info.getString("diagnostics")); + assertEquals("clusterId doesn't match", + ResourceManager.getClusterTimeStamp(), info.getLong("clusterId")); + assertEquals("startedTime doesn't match", app.getStartTime(), + info.getLong("startedTime")); + assertEquals("finishedTime doesn't match", app.getFinishTime(), + info.getLong("finishedTime")); + assertTrue("elapsed time not greater than 0", + info.getLong("elapsedTime") > 0); + checkStringMatch("amHostHttpAddress", + app.getCurrentAppAttempt().getMasterContainer().getNodeHttpAddress(), + info.getString("amHostHttpAddress")); + assertTrue("amContainerLogs doesn't match", + info.getString("amContainerLogs").startsWith("http://")); + assertTrue("amContainerLogs doesn't contain user info", + info.getString("amContainerLogs").endsWith("/" + app.getUser())); + assertEquals("allocatedMB doesn't match", 1024, info.getInt("allocatedMB")); + assertEquals("allocatedVCores doesn't match", 1, + info.getInt("allocatedVCores")); + assertEquals("queueUsagePerc doesn't match", 50.0f, + (float) info.getDouble("queueUsagePercentage"), 0.01f); + assertEquals("clusterUsagePerc doesn't match", 50.0f, + (float) info.getDouble("clusterUsagePercentage"), 0.01f); + assertEquals("numContainers doesn't match", 1, + info.getInt("runningContainers")); + assertNotNull("preemptedResourceSecondsMap should not be null", + info.getJSONObject("preemptedResourceSecondsMap")); + assertEquals("preemptedResourceMB doesn't match", + app.getRMAppMetrics().getResourcePreempted().getMemorySize(), + info.getInt("preemptedResourceMB")); + assertEquals("preemptedResourceVCores doesn't match", + app.getRMAppMetrics().getResourcePreempted().getVirtualCores(), + info.getInt("preemptedResourceVCores")); + assertEquals("numNonAMContainerPreempted doesn't match", + app.getRMAppMetrics().getNumNonAMContainersPreempted(), + info.getInt("numNonAMContainerPreempted")); + assertEquals("numAMContainerPreempted doesn't match", + app.getRMAppMetrics().getNumAMContainersPreempted(), + info.getInt("numAMContainerPreempted")); + assertEquals("Log aggregation Status doesn't match", + app.getLogAggregationStatusForAppReport().toString(), + info.getString("logAggregationStatus")); + assertEquals("unmanagedApplication doesn't match", + app.getApplicationSubmissionContext().getUnmanagedAM(), + info.getBoolean("unmanagedApplication")); + + if (app.getApplicationSubmissionContext() + .getNodeLabelExpression() != null) { + assertEquals("appNodeLabelExpression doesn't match", + app.getApplicationSubmissionContext().getNodeLabelExpression(), + info.getString("appNodeLabelExpression")); + } + assertEquals("amNodeLabelExpression doesn't match", + app.getAMResourceRequests().get(0).getNodeLabelExpression(), + info.getString("amNodeLabelExpression")); + assertEquals("amRPCAddress", + AppInfo.getAmRPCAddressFromRMAppAttempt(app.getCurrentAppAttempt()), + info.getString("amRPCAddress")); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java new file mode 100644 index 00000000000..7c5b6dbdeb2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/AppInfoXmlVerifications.java @@ -0,0 +1,132 @@ +/* + * 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.yarn.server.resourcemanager.webapp.helper; + +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo; +import org.apache.hadoop.yarn.webapp.WebServicesTestUtils; +import org.w3c.dom.Element; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.checkStringMatch; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlBoolean; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlFloat; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlInt; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlLong; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Contains all value verifications that are needed to verify {@link AppInfo} + * XML documents. + */ +public final class AppInfoXmlVerifications { + + private AppInfoXmlVerifications() { + //utility class + } + + /** + * Tests whether {@link AppInfo} representation object contains the required + * values as per defined in the specified app parameter. + * @param info + * @param app an RMApp instance that contains the required values + */ + public static void verify(Element info, RMApp app) { + checkStringMatch("id", app.getApplicationId() + .toString(), getXmlString(info, "id")); + checkStringMatch("user", app.getUser(), + getXmlString(info, "user")); + checkStringMatch("name", app.getName(), + getXmlString(info, "name")); + checkStringMatch("applicationType", + app.getApplicationType(), getXmlString(info, "applicationType")); + checkStringMatch("queue", app.getQueue(), + getXmlString(info, "queue")); + assertEquals("priority doesn't match", 0, getXmlInt(info, "priority")); + checkStringMatch("state", app.getState().toString(), + getXmlString(info, "state")); + checkStringMatch("finalStatus", app + .getFinalApplicationStatus().toString(), + getXmlString(info, "finalStatus")); + assertEquals("progress doesn't match", 0, getXmlFloat(info, "progress"), + 0.0); + if ("UNASSIGNED".equals(getXmlString(info, "trackingUI"))) { + checkStringMatch("trackingUI", "UNASSIGNED", + getXmlString(info, "trackingUI")); + } + WebServicesTestUtils.checkStringEqual("diagnostics", + app.getDiagnostics().toString(), getXmlString(info, "diagnostics")); + assertEquals("clusterId doesn't match", + ResourceManager.getClusterTimeStamp(), + getXmlLong(info, "clusterId")); + assertEquals("startedTime doesn't match", app.getStartTime(), + getXmlLong(info, "startedTime")); + assertEquals("finishedTime doesn't match", app.getFinishTime(), + getXmlLong(info, "finishedTime")); + assertTrue("elapsed time not greater than 0", + getXmlLong(info, "elapsedTime") > 0); + checkStringMatch("amHostHttpAddress", app + .getCurrentAppAttempt().getMasterContainer() + .getNodeHttpAddress(), + getXmlString(info, "amHostHttpAddress")); + assertTrue("amContainerLogs doesn't match", + getXmlString(info, "amContainerLogs").startsWith("http://")); + assertTrue("amContainerLogs doesn't contain user info", + getXmlString(info, "amContainerLogs").endsWith("/" + app.getUser())); + assertEquals("allocatedMB doesn't match", 1024, + getXmlInt(info, "allocatedMB")); + assertEquals("allocatedVCores doesn't match", 1, + getXmlInt(info, "allocatedVCores")); + assertEquals("queueUsagePerc doesn't match", 50.0f, + getXmlFloat(info, "queueUsagePercentage"), 0.01f); + assertEquals("clusterUsagePerc doesn't match", 50.0f, + getXmlFloat(info, "clusterUsagePercentage"), 0.01f); + assertEquals("numContainers doesn't match", 1, + getXmlInt(info, "runningContainers")); + assertNotNull("preemptedResourceSecondsMap should not be null", + info.getElementsByTagName("preemptedResourceSecondsMap")); + assertEquals("preemptedResourceMB doesn't match", app + .getRMAppMetrics().getResourcePreempted().getMemorySize(), + getXmlInt(info, "preemptedResourceMB")); + assertEquals("preemptedResourceVCores doesn't match", app + .getRMAppMetrics().getResourcePreempted().getVirtualCores(), + getXmlInt(info, "preemptedResourceVCores")); + assertEquals("numNonAMContainerPreempted doesn't match", app + .getRMAppMetrics().getNumNonAMContainersPreempted(), + getXmlInt(info, "numNonAMContainerPreempted")); + assertEquals("numAMContainerPreempted doesn't match", app + .getRMAppMetrics().getNumAMContainersPreempted(), + getXmlInt(info, "numAMContainerPreempted")); + assertEquals("Log aggregation Status doesn't match", app + .getLogAggregationStatusForAppReport().toString(), + getXmlString(info, "logAggregationStatus")); + assertEquals("unmanagedApplication doesn't match", app + .getApplicationSubmissionContext().getUnmanagedAM(), + getXmlBoolean(info, "unmanagedApplication")); + assertEquals("unmanagedApplication doesn't match", + app.getApplicationSubmissionContext().getNodeLabelExpression(), + getXmlString(info, "appNodeLabelExpression")); + assertEquals("unmanagedApplication doesn't match", + app.getAMResourceRequests().get(0).getNodeLabelExpression(), + getXmlString(info, "amNodeLabelExpression")); + assertEquals("amRPCAddress", + AppInfo.getAmRPCAddressFromRMAppAttempt(app.getCurrentAppAttempt()), + getXmlString(info, "amRPCAddress")); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.java new file mode 100644 index 00000000000..a8990ca695a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/BufferedClientResponse.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 + * + * 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.yarn.server.resourcemanager.webapp.helper; + + +import com.sun.jersey.api.client.ClientHandlerException; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.UniformInterfaceException; + +import javax.ws.rs.core.MediaType; +import java.io.IOException; + +/** + * This class is merely a wrapper for {@link ClientResponse}. Given that the + * entity input stream of {@link ClientResponse} can be read only once by + * default and for some tests it is convenient to read the input stream many + * times, this class hides the details of how to do that and prevents + * unnecessary code duplication in tests. + */ +public class BufferedClientResponse { + private ClientResponse response; + + public BufferedClientResponse(ClientResponse response) { + response.bufferEntity(); + this.response = response; + } + + public T getEntity(Class clazz) + throws ClientHandlerException, UniformInterfaceException { + try { + response.getEntityInputStream().reset(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return response.getEntity(clazz); + } + + public MediaType getType() { + return response.getType(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java new file mode 100644 index 00000000000..9d6a111d7ac --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/JsonCustomResourceTypeTestcase.java @@ -0,0 +1,77 @@ +/** + * 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.yarn.server.resourcemanager.webapp.helper; + +import com.sun.jersey.api.client.WebResource; +import org.apache.hadoop.http.JettyUtils; +import org.codehaus.jettison.json.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.ws.rs.core.MediaType; + +import java.util.function.Consumer; + +import static org.junit.Assert.*; + +/** + * This class hides the implementation details of how to verify the structure of + * JSON responses. Tests should only provide the path of the + * {@link WebResource}, the response from the resource and + * the verifier Consumer to + * {@link JsonCustomResourceTypeTestcase#verify(Consumer)}. An instance of + * {@link JSONObject} will be passed to that consumer to be able to + * verify the response. + */ +public class JsonCustomResourceTypeTestcase { + private static final Logger LOG = + LoggerFactory.getLogger(JsonCustomResourceTypeTestcase.class); + + private final WebResource path; + private final BufferedClientResponse response; + private final JSONObject parsedResponse; + + public JsonCustomResourceTypeTestcase(WebResource path, + BufferedClientResponse response) { + this.path = path; + this.response = response; + this.parsedResponse = response.getEntity(JSONObject.class); + } + + public void verify(Consumer verifier) { + assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, + response.getType().toString()); + + logResponse(); + + String responseStr = response.getEntity(String.class); + if (responseStr == null || responseStr.isEmpty()) { + throw new IllegalStateException("Response is null or empty!"); + } + verifier.accept(parsedResponse); + } + + private void logResponse() { + String responseStr = response.getEntity(String.class); + LOG.info("Raw response from service URL {}: {}", path.toString(), + responseStr); + LOG.info("Parsed response from service URL {}: {}", path.toString(), + parsedResponse); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java new file mode 100644 index 00000000000..6e58a89692b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsJsonVerifications.java @@ -0,0 +1,252 @@ +/* + * 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.yarn.server.resourcemanager.webapp.helper; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; + +import java.util.List; +import java.util.Map; + +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Performs value verifications on + * {@link org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceRequestInfo} + * objects against the values of {@link ResourceRequest}. With the help of the + * {@link Builder}, users can also make verifications of the custom resource + * types and its values. + */ +public class ResourceRequestsJsonVerifications { + private final ResourceRequest resourceRequest; + private final JSONObject requestInfo; + private final Map customResourceTypes; + private final List expectedCustomResourceTypes; + + ResourceRequestsJsonVerifications(Builder builder) { + this.resourceRequest = builder.resourceRequest; + this.requestInfo = builder.requestInfo; + this.customResourceTypes = builder.customResourceTypes; + this.expectedCustomResourceTypes = builder.expectedCustomResourceTypes; + } + + public static void verify(JSONObject requestInfo, ResourceRequest rr) + throws JSONException { + createDefaultBuilder(requestInfo, rr).build().verify(); + } + + public static void verifyWithCustomResourceTypes(JSONObject requestInfo, + ResourceRequest resourceRequest, List expectedResourceTypes) + throws JSONException { + + createDefaultBuilder(requestInfo, resourceRequest) + .withExpectedCustomResourceTypes(expectedResourceTypes) + .withCustomResourceTypes( + extractActualCustomResourceTypes(requestInfo, expectedResourceTypes)) + .build().verify(); + } + + private static Builder createDefaultBuilder(JSONObject requestInfo, + ResourceRequest resourceRequest) { + return new ResourceRequestsJsonVerifications.Builder() + .withRequest(resourceRequest) + .withRequestInfoJson(requestInfo); + } + + private static Map extractActualCustomResourceTypes( + JSONObject requestInfo, List expectedResourceTypes) + throws JSONException { + JSONObject capability = requestInfo.getJSONObject("capability"); + Map resourceAndValue = + extractCustomResorceTypeValues(capability, expectedResourceTypes); + Map.Entry resourceEntry = + resourceAndValue.entrySet().iterator().next(); + + assertTrue( + "Found resource type: " + resourceEntry.getKey() + + " is not in expected resource types: " + expectedResourceTypes, + expectedResourceTypes.contains(resourceEntry.getKey())); + + return resourceAndValue; + } + + private static Map extractCustomResorceTypeValues( + JSONObject capability, List expectedResourceTypes) + throws JSONException { + assertTrue( + "resourceCategory does not have resourceInformations: " + capability, + capability.has("resourceInformations")); + + JSONObject resourceInformations = + capability.getJSONObject("resourceInformations"); + assertTrue( + "resourceInformations does not have resourceInformation object: " + + resourceInformations, + resourceInformations.has("resourceInformation")); + JSONArray customResources = + resourceInformations.getJSONArray("resourceInformation"); + + // customResources will include vcores / memory as well + assertEquals( + "Different number of custom resource types found than expected", + expectedResourceTypes.size(), customResources.length() - 2); + + Map resourceValues = Maps.newHashMap(); + for (int i = 0; i < customResources.length(); i++) { + JSONObject customResource = customResources.getJSONObject(i); + assertTrue("Resource type does not have name field: " + customResource, + customResource.has("name")); + assertTrue("Resource type does not have name resourceType field: " + + customResource, customResource.has("resourceType")); + assertTrue( + "Resource type does not have name units field: " + customResource, + customResource.has("units")); + assertTrue( + "Resource type does not have name value field: " + customResource, + customResource.has("value")); + + String name = customResource.getString("name"); + String unit = customResource.getString("units"); + String resourceType = customResource.getString("resourceType"); + Long value = customResource.getLong("value"); + + if (ResourceInformation.MEMORY_URI.equals(name) + || ResourceInformation.VCORES_URI.equals(name)) { + continue; + } + + assertTrue("Custom resource type " + name + " not found", + expectedResourceTypes.contains(name)); + assertEquals("k", unit); + assertEquals(ResourceTypes.COUNTABLE, + ResourceTypes.valueOf(resourceType)); + assertNotNull("Custom resource value " + value + " is null!", value); + resourceValues.put(name, value); + } + + return resourceValues; + } + + private void verify() throws JSONException { + assertEquals("nodeLabelExpression doesn't match", + resourceRequest.getNodeLabelExpression(), + requestInfo.getString("nodeLabelExpression")); + assertEquals("numContainers doesn't match", + resourceRequest.getNumContainers(), + requestInfo.getInt("numContainers")); + assertEquals("relaxLocality doesn't match", + resourceRequest.getRelaxLocality(), + requestInfo.getBoolean("relaxLocality")); + assertEquals("priority does not match", + resourceRequest.getPriority().getPriority(), + requestInfo.getInt("priority")); + assertEquals("resourceName does not match", + resourceRequest.getResourceName(), + requestInfo.getString("resourceName")); + assertEquals("memory does not match", + resourceRequest.getCapability().getMemorySize(), + requestInfo.getJSONObject("capability").getLong("memory")); + assertEquals("vCores does not match", + resourceRequest.getCapability().getVirtualCores(), + requestInfo.getJSONObject("capability").getLong("vCores")); + + verifyAtLeastOneCustomResourceIsSerialized(); + + JSONObject executionTypeRequest = + requestInfo.getJSONObject("executionTypeRequest"); + assertEquals("executionType does not match", + resourceRequest.getExecutionTypeRequest().getExecutionType().name(), + executionTypeRequest.getString("executionType")); + assertEquals("enforceExecutionType does not match", + resourceRequest.getExecutionTypeRequest().getEnforceExecutionType(), + executionTypeRequest.getBoolean("enforceExecutionType")); + } + + /** + * JSON serialization produces "invalid JSON" by default as maps are + * serialized like this: + * "customResources":{"entry":{"key":"customResource-1","value":"0"}} + * If the map has multiple keys then multiple entries will be serialized. + * Our json parser in tests cannot handle duplicates therefore only one + * custom resource will be in the parsed json. See: + * https://issues.apache.org/jira/browse/YARN-7505 + */ + private void verifyAtLeastOneCustomResourceIsSerialized() { + boolean resourceFound = false; + for (String expectedCustomResourceType : expectedCustomResourceTypes) { + if (customResourceTypes.containsKey(expectedCustomResourceType)) { + resourceFound = true; + Long resourceValue = + customResourceTypes.get(expectedCustomResourceType); + assertNotNull("Resource value should not be null!", resourceValue); + } + } + assertTrue("No custom resource type can be found in the response!", + resourceFound); + } + + /** + * Builder class for {@link ResourceRequestsJsonVerifications}. + */ + public static final class Builder { + private List expectedCustomResourceTypes = Lists.newArrayList(); + private Map customResourceTypes; + private ResourceRequest resourceRequest; + private JSONObject requestInfo; + + Builder() { + } + + public static Builder create() { + return new Builder(); + } + + Builder withExpectedCustomResourceTypes( + List expectedCustomResourceTypes) { + this.expectedCustomResourceTypes = expectedCustomResourceTypes; + return this; + } + + Builder withCustomResourceTypes( + Map customResourceTypes) { + this.customResourceTypes = customResourceTypes; + return this; + } + + Builder withRequest(ResourceRequest resourceRequest) { + this.resourceRequest = resourceRequest; + return this; + } + + Builder withRequestInfoJson(JSONObject requestInfo) { + this.requestInfo = requestInfo; + return this; + } + + public ResourceRequestsJsonVerifications build() { + return new ResourceRequestsJsonVerifications(this); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java new file mode 100644 index 00000000000..af9b0f35103 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/ResourceRequestsXmlVerifications.java @@ -0,0 +1,215 @@ +/* + * 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.yarn.server.resourcemanager.webapp.helper; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.w3c.dom.Element; +import org.w3c.dom.NodeList; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static junit.framework.TestCase.assertTrue; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.helper.XmlCustomResourceTypeTestCase.toXml; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlBoolean; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlInt; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlLong; +import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.getXmlString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Performs value verifications on + * {@link org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceRequestInfo} + * objects against the values of {@link ResourceRequest}. With the help of the + * {@link Builder}, users can also make verifications of the custom resource + * types and its values. + */ +public class ResourceRequestsXmlVerifications { + private final ResourceRequest resourceRequest; + private final Element requestInfo; + private final Map customResourceTypes; + private final List expectedCustomResourceTypes; + + ResourceRequestsXmlVerifications(Builder builder) { + this.resourceRequest = builder.resourceRequest; + this.requestInfo = builder.requestInfo; + this.customResourceTypes = builder.customResourceTypes; + this.expectedCustomResourceTypes = builder.expectedCustomResourceTypes; + } + + public static void verifyWithCustomResourceTypes(Element requestInfo, + ResourceRequest resourceRequest, List expectedResourceTypes) { + + createDefaultBuilder(requestInfo, resourceRequest) + .withExpectedCustomResourceTypes(expectedResourceTypes) + .withCustomResourceTypes(extractActualCustomResourceType(requestInfo, + expectedResourceTypes)) + .build().verify(); + } + + private static Builder createDefaultBuilder(Element requestInfo, + ResourceRequest resourceRequest) { + return new ResourceRequestsXmlVerifications.Builder() + .withRequest(resourceRequest).withRequestInfo(requestInfo); + } + + private static Map extractActualCustomResourceType( + Element requestInfo, List expectedResourceTypes) { + Element capability = + (Element) requestInfo.getElementsByTagName("capability").item(0); + + return extractCustomResorceTypes(capability, + Sets.newHashSet(expectedResourceTypes)); + } + + private static Map extractCustomResorceTypes(Element capability, + Set expectedResourceTypes) { + assertEquals( + toXml(capability) + " should have only one resourceInformations child!", + 1, capability.getElementsByTagName("resourceInformations").getLength()); + Element resourceInformations = (Element) capability + .getElementsByTagName("resourceInformations").item(0); + + NodeList customResources = + resourceInformations.getElementsByTagName("resourceInformation"); + + // customResources will include vcores / memory as well + assertEquals( + "Different number of custom resource types found than expected", + expectedResourceTypes.size(), customResources.getLength() - 2); + + Map resourceTypesAndValues = Maps.newHashMap(); + for (int i = 0; i < customResources.getLength(); i++) { + Element customResource = (Element) customResources.item(i); + String name = getXmlString(customResource, "name"); + String unit = getXmlString(customResource, "units"); + String resourceType = getXmlString(customResource, "resourceType"); + Long value = getXmlLong(customResource, "value"); + + if (ResourceInformation.MEMORY_URI.equals(name) + || ResourceInformation.VCORES_URI.equals(name)) { + continue; + } + + assertTrue("Custom resource type " + name + " not found", + expectedResourceTypes.contains(name)); + assertEquals("k", unit); + assertEquals(ResourceTypes.COUNTABLE, + ResourceTypes.valueOf(resourceType)); + assertNotNull("Resource value should not be null for resource type " + + resourceType + ", listing xml contents: " + toXml(customResource), + value); + resourceTypesAndValues.put(name, value); + } + + return resourceTypesAndValues; + } + + private void verify() { + assertEquals("nodeLabelExpression doesn't match", + resourceRequest.getNodeLabelExpression(), + getXmlString(requestInfo, "nodeLabelExpression")); + assertEquals("numContainers doesn't match", + resourceRequest.getNumContainers(), + getXmlInt(requestInfo, "numContainers")); + assertEquals("relaxLocality doesn't match", + resourceRequest.getRelaxLocality(), + getXmlBoolean(requestInfo, "relaxLocality")); + assertEquals("priority does not match", + resourceRequest.getPriority().getPriority(), + getXmlInt(requestInfo, "priority")); + assertEquals("resourceName does not match", + resourceRequest.getResourceName(), + getXmlString(requestInfo, "resourceName")); + Element capability = (Element) requestInfo + .getElementsByTagName("capability").item(0); + assertEquals("memory does not match", + resourceRequest.getCapability().getMemorySize(), + getXmlLong(capability, "memory")); + assertEquals("vCores does not match", + resourceRequest.getCapability().getVirtualCores(), + getXmlLong(capability, "vCores")); + + for (String expectedCustomResourceType : expectedCustomResourceTypes) { + assertTrue( + "Custom resource type " + expectedCustomResourceType + + " cannot be found!", + customResourceTypes.containsKey(expectedCustomResourceType)); + + Long resourceValue = customResourceTypes.get(expectedCustomResourceType); + assertNotNull("Resource value should not be null!", resourceValue); + } + + Element executionTypeRequest = (Element) requestInfo + .getElementsByTagName("executionTypeRequest").item(0); + assertEquals("executionType does not match", + resourceRequest.getExecutionTypeRequest().getExecutionType().name(), + getXmlString(executionTypeRequest, "executionType")); + assertEquals("enforceExecutionType does not match", + resourceRequest.getExecutionTypeRequest().getEnforceExecutionType(), + getXmlBoolean(executionTypeRequest, "enforceExecutionType")); + } + + /** + * Builder class for {@link ResourceRequestsXmlVerifications}. + */ + public static final class Builder { + private List expectedCustomResourceTypes = Lists.newArrayList(); + private Map customResourceTypes; + private ResourceRequest resourceRequest; + private Element requestInfo; + + Builder() { + } + + public static Builder create() { + return new Builder(); + } + + Builder withExpectedCustomResourceTypes( + List expectedCustomResourceTypes) { + this.expectedCustomResourceTypes = expectedCustomResourceTypes; + return this; + } + + Builder withCustomResourceTypes(Map customResourceTypes) { + this.customResourceTypes = customResourceTypes; + return this; + } + + Builder withRequest(ResourceRequest resourceRequest) { + this.resourceRequest = resourceRequest; + return this; + } + + Builder withRequestInfo(Element requestInfo) { + this.requestInfo = requestInfo; + return this; + } + + public ResourceRequestsXmlVerifications build() { + return new ResourceRequestsXmlVerifications(this); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java new file mode 100644 index 00000000000..29260aad1df --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/helper/XmlCustomResourceTypeTestCase.java @@ -0,0 +1,112 @@ +/** + * 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.yarn.server.resourcemanager.webapp.helper; + +import com.sun.jersey.api.client.WebResource; +import org.apache.hadoop.http.JettyUtils; +import org.codehaus.jettison.json.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.w3c.dom.Document; +import org.w3c.dom.Node; +import org.xml.sax.InputSource; + +import javax.ws.rs.core.MediaType; +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.transform.*; +import javax.xml.transform.dom.DOMSource; +import javax.xml.transform.stream.StreamResult; +import java.io.StringReader; +import java.io.StringWriter; +import java.util.function.Consumer; + +import static org.junit.Assert.assertEquals; + +/** + * This class hides the implementation details of how to verify the structure of + * XML responses. Tests should only provide the path of the + * {@link WebResource}, the response from the resource and + * the verifier Consumer to + * {@link XmlCustomResourceTypeTestCase#verify(Consumer)}. An instance of + * {@link JSONObject} will be passed to that consumer to be able to + * verify the response. + */ +public class XmlCustomResourceTypeTestCase { + private static final Logger LOG = + LoggerFactory.getLogger(XmlCustomResourceTypeTestCase.class); + + private WebResource path; + private BufferedClientResponse response; + private Document parsedResponse; + + public XmlCustomResourceTypeTestCase(WebResource path, + BufferedClientResponse response) { + this.path = path; + this.response = response; + } + + public void verify(Consumer verifier) { + assertEquals(MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8, + response.getType().toString()); + + parsedResponse = parseXml(response); + logResponse(parsedResponse); + verifier.accept(parsedResponse); + } + + private Document parseXml(BufferedClientResponse response) { + try { + String xml = response.getEntity(String.class); + DocumentBuilder db = + DocumentBuilderFactory.newInstance().newDocumentBuilder(); + InputSource is = new InputSource(); + is.setCharacterStream(new StringReader(xml)); + + return db.parse(is); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private void logResponse(Document doc) { + String responseStr = response.getEntity(String.class); + LOG.info("Raw response from service URL {}: {}", path.toString(), + responseStr); + LOG.info("Parsed response from service URL {}: {}", path.toString(), + toXml(doc)); + } + + public static String toXml(Node node) { + StringWriter writer; + try { + TransformerFactory tf = TransformerFactory.newInstance(); + Transformer transformer = tf.newTransformer(); + transformer.setOutputProperty(OutputKeys.INDENT, "yes"); + transformer.setOutputProperty( + "{http://xml.apache.org/xslt}indent" + "-amount", "2"); + writer = new StringWriter(); + transformer.transform(new DOMSource(node), new StreamResult(writer)); + } catch (TransformerException e) { + throw new RuntimeException(e); + } + + return writer.getBuffer().toString(); + } +} From 498e3bfb6b93bf542e5581d83e64e920983fe87e Mon Sep 17 00:00:00 2001 From: Xiaoyu Yao Date: Thu, 5 Jul 2018 14:19:05 -0700 Subject: [PATCH 35/47] HADOOP-15571. Multiple FileContexts created with the same configuration object should be allowed to have different umask. Contributed by Vinod Kumar Vavilapalli. --- .../org/apache/hadoop/fs/FileContext.java | 9 ++-- .../org/apache/hadoop/fs/TestFileContext.java | 44 ++++++++++++++++++- .../logaggregation/AggregatedLogFormat.java | 6 +-- 3 files changed, 49 insertions(+), 10 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java index 5215c3cdee9..0b3889b0f31 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java @@ -219,10 +219,12 @@ public class FileContext { * The FileContext is defined by. * 1) defaultFS (slash) * 2) wd - * 3) umask (Obtained by FsPermission.getUMask(conf)) + * 3) umask (explicitly set via setUMask(), + * falling back to FsPermission.getUMask(conf)) */ private final AbstractFileSystem defaultFS; //default FS for this FileContext. private Path workingDir; // Fully qualified + private FsPermission umask; private final Configuration conf; private final UserGroupInformation ugi; final boolean resolveSymlinks; @@ -575,7 +577,7 @@ public class FileContext { * @return the umask of this FileContext */ public FsPermission getUMask() { - return FsPermission.getUMask(conf); + return (umask != null ? umask : FsPermission.getUMask(conf)); } /** @@ -583,10 +585,9 @@ public class FileContext { * @param newUmask the new umask */ public void setUMask(final FsPermission newUmask) { - FsPermission.setUMask(conf, newUmask); + this.umask = newUmask; } - /** * Resolve the path following any symlinks or mount points * @param f to be resolved diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileContext.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileContext.java index f5fb06f74ae..60b24c776c1 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileContext.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileContext.java @@ -17,13 +17,17 @@ */ package org.apache.hadoop.fs; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import java.net.URI; + import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.permission.FsPermission; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.fail; - public class TestFileContext { private static final Logger LOG = LoggerFactory.getLogger(TestFileContext .class); @@ -39,4 +43,40 @@ public class TestFileContext { LOG.info("Expected exception: ", ufse); } } + + @Test + public void testConfBasedAndAPIBasedSetUMask() throws Exception { + + Configuration conf = new Configuration(); + + String defaultlUMask = + conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY); + assertEquals("Default UMask changed!", "022", defaultlUMask); + + URI uri1 = new URI("file://mydfs:50070/"); + URI uri2 = new URI("file://tmp"); + + FileContext fc1 = FileContext.getFileContext(uri1, conf); + FileContext fc2 = FileContext.getFileContext(uri2, conf); + assertEquals("Umask for fc1 is incorrect", 022, fc1.getUMask().toShort()); + assertEquals("Umask for fc2 is incorrect", 022, fc2.getUMask().toShort()); + + // Till a user explicitly calls FileContext.setUMask(), the updates through + // configuration should be reflected.. + conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "011"); + assertEquals("Umask for fc1 is incorrect", 011, fc1.getUMask().toShort()); + assertEquals("Umask for fc2 is incorrect", 011, fc2.getUMask().toShort()); + + // Stop reflecting the conf update for specific FileContexts, once an + // explicit setUMask is done. + conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "066"); + fc1.setUMask(FsPermission.createImmutable((short) 00033)); + assertEquals("Umask for fc1 is incorrect", 033, fc1.getUMask().toShort()); + assertEquals("Umask for fc2 is incorrect", 066, fc2.getUMask().toShort()); + + conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077"); + fc2.setUMask(FsPermission.createImmutable((short) 00044)); + assertEquals("Umask for fc1 is incorrect", 033, fc1.getUMask().toShort()); + assertEquals("Umask for fc2 is incorrect", 044, fc2.getUMask().toShort()); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java index 81d50534979..4ee5c8ab24d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java @@ -96,9 +96,6 @@ public class AggregatedLogFormat { */ private static final FsPermission APP_LOG_FILE_UMASK = FsPermission .createImmutable((short) (0640 ^ 0777)); - /** Default permission for the log file. */ - private static final FsPermission APP_LOG_FILE_PERM = - FsPermission.getFileDefault().applyUMask(APP_LOG_FILE_UMASK); static { RESERVED_KEYS = new HashMap(); @@ -477,10 +474,11 @@ public class AggregatedLogFormat { @Override public FSDataOutputStream run() throws Exception { fc = FileContext.getFileContext(remoteAppLogFile.toUri(), conf); + fc.setUMask(APP_LOG_FILE_UMASK); return fc.create( remoteAppLogFile, EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), - Options.CreateOpts.perms(APP_LOG_FILE_PERM)); + new Options.CreateOpts[] {}); } }); } catch (InterruptedException e) { From e4bf38cf50943565796c00f8b5711a2882813488 Mon Sep 17 00:00:00 2001 From: Sunil G Date: Fri, 6 Jul 2018 12:05:32 -0700 Subject: [PATCH 36/47] YARN-8492. ATSv2 HBase tests are failing with ClassNotFoundException. Contributed by Rohith Sharma K S. --- .../pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml index 8c143d3c047..05a5c651918 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml @@ -133,6 +133,10 @@ org.apache.hadoop hadoop-auth + + org.apache.hadoop + hadoop-hdfs-client + @@ -355,6 +359,12 @@ jetty-webapp test + + + org.apache.commons + commons-lang3 + test + From 061b168529a9cd5d6a3a482c890bacdb49186368 Mon Sep 17 00:00:00 2001 From: Arpit Agarwal Date: Fri, 6 Jul 2018 12:09:05 -0700 Subject: [PATCH 37/47] HDDS-167. Rename KeySpaceManager to OzoneManager. Contributed by Arpit Agarwal. --- .gitignore | 4 + dev-support/bin/ozone-dist-layout-stitching | 2 +- .../hadoop-common/src/main/conf/hadoop-env.sh | 6 +- .../main/compose/ozone/docker-compose.yaml | 6 +- .../src/main/compose/ozone/docker-config | 2 +- .../src/main/compose/ozoneperf/README.md | 4 +- .../compose/ozoneperf/docker-compose.yaml | 6 +- .../src/main/compose/ozoneperf/docker-config | 2 +- .../apache/hadoop/hdds/scm/ScmConfigKeys.java | 2 +- .../org/apache/hadoop/ozone/OzoneConsts.java | 22 +- .../apache/hadoop/ozone/common/Storage.java | 6 +- .../main/proto/ScmBlockLocationProtocol.proto | 10 +- hadoop-hdds/common/src/main/proto/hdds.proto | 6 +- .../src/main/resources/ozone-default.xml | 120 ++++---- .../main/resources/webapps/static/ozone.js | 4 +- .../webapps/static/templates/config.html | 4 +- .../hdds/scm/block/DeletedBlockLog.java | 2 +- .../container/states/ContainerStateMap.java | 6 +- .../hadoop/hdds/scm/node/CommandQueue.java | 2 +- .../scm/server/SCMBlockProtocolServer.java | 2 +- .../hadoop/hdds/scm/cli/OzoneBaseCLI.java | 2 +- .../src/test/acceptance/basic/basic.robot | 6 +- .../test/acceptance/basic/docker-compose.yaml | 8 +- .../src/test/acceptance/basic/docker-config | 4 +- .../test/acceptance/basic/ozone-shell.robot | 18 +- .../src/test/acceptance/commonlib.robot | 4 +- .../acceptance/ozonefs/docker-compose.yaml | 8 +- .../src/test/acceptance/ozonefs/docker-config | 4 +- .../src/test/acceptance/ozonefs/ozonefs.robot | 6 +- .../hadoop/ozone/client/BucketArgs.java | 4 +- .../ozone/client/OzoneClientFactory.java | 86 +++--- .../apache/hadoop/ozone/client/OzoneKey.java | 2 +- .../hadoop/ozone/client/VolumeArgs.java | 4 +- .../client/io/ChunkGroupInputStream.java | 24 +- .../client/io/ChunkGroupOutputStream.java | 54 ++-- .../rest/DefaultRestServerSelector.java | 2 +- .../hadoop/ozone/client/rest/RestClient.java | 15 +- .../ozone/client/rest/RestServerSelector.java | 2 +- .../hadoop/ozone/client/rpc/RpcClient.java | 142 +++++---- .../ozone/client/TestHddsClientUtils.java | 24 +- hadoop-ozone/common/pom.xml | 2 +- hadoop-ozone/common/src/main/bin/ozone | 9 +- .../common/src/main/bin/start-ozone.sh | 16 +- .../common/src/main/bin/stop-ozone.sh | 16 +- .../ozone/{KsmUtils.java => OmUtils.java} | 51 ++-- .../hadoop/ozone/freon/OzoneGetConf.java | 16 +- .../hadoop/ozone/ksm/KSMConfigKeys.java | 81 ----- .../apache/hadoop/ozone/om/OMConfigKeys.java | 81 +++++ .../helpers/OmBucketArgs.java} | 46 +-- .../helpers/OmBucketInfo.java} | 42 +-- .../helpers/OmKeyArgs.java} | 14 +- .../helpers/OmKeyInfo.java} | 64 ++-- .../helpers/OmKeyLocationInfo.java} | 20 +- .../helpers/OmKeyLocationInfoGroup.java} | 42 +-- .../helpers/OmOzoneAclMap.java} | 16 +- .../helpers/OmVolumeArgs.java} | 40 +-- .../{ksm => om}/helpers/OpenKeySession.java | 8 +- .../{ksm => om}/helpers/ServiceInfo.java | 20 +- .../ozone/{ksm => om}/helpers/VolumeArgs.java | 2 +- .../{ksm => om}/helpers/package-info.java | 2 +- .../ozone/{ksm => om}/package-info.java | 4 +- .../protocol/OzoneManagerProtocol.java} | 66 ++--- .../{ksm => om}/protocol/package-info.java | 2 +- ...anagerProtocolClientSideTranslatorPB.java} | 180 ++++++------ .../protocolPB/OzoneManagerProtocolPB.java} | 12 +- .../{ksm => om}/protocolPB/package-info.java | 2 +- .../{KSMPBHelper.java => OMPBHelper.java} | 10 +- ...tocol.proto => OzoneManagerProtocol.proto} | 14 +- hadoop-ozone/docs/content/GettingStarted.md | 18 +- hadoop-ozone/docs/content/Metrics.md | 10 +- hadoop-ozone/docs/content/_index.md | 12 +- hadoop-ozone/docs/static/OzoneOverview.svg | 2 +- .../apache/hadoop/ozone/MiniOzoneCluster.java | 24 +- .../hadoop/ozone/MiniOzoneClusterImpl.java | 66 ++--- .../ozone/TestOzoneConfigurationFields.java | 4 +- .../ozone/TestStorageContainerManager.java | 18 +- .../TestStorageContainerManagerHelper.java | 12 +- .../client/rest/TestOzoneRestClient.java | 6 +- .../ozone/client/rpc/TestOzoneRpcClient.java | 18 +- .../commandhandler/TestBlockDeletion.java | 45 +-- .../TestCloseContainerByPipeline.java | 35 ++- .../TestCloseContainerHandler.java | 14 +- .../TestContainerReportWithKeys.java | 12 +- .../TestMultipleContainerReadWrite.java | 2 +- .../TestOmBlockVersioning.java} | 68 ++--- .../TestOmMetrics.java} | 187 ++++++------ .../TestOmSQLCli.java} | 16 +- .../TestOzoneManager.java} | 185 ++++++------ .../TestOzoneManagerRestInterface.java} | 32 +- .../hadoop/ozone/ozShell/TestOzoneShell.java | 14 +- .../hadoop/ozone/scm/TestContainerSQLCli.java | 3 +- .../web/TestDistributedOzoneVolumes.java | 12 +- .../hadoop/ozone/web/client/TestKeys.java | 58 ++-- .../webapps/{ksm => ozoneManager}/.gitkeep | 0 .../server/datanode/ObjectStoreHandler.java | 33 +-- .../web/handlers/KeyProcessTemplate.java | 4 +- .../web/handlers/VolumeProcessTemplate.java | 4 +- .../storage/DistributedStorageHandler.java | 153 +++++----- .../ozone/{ksm => om}/BucketManager.java | 20 +- .../ozone/{ksm => om}/BucketManagerImpl.java | 80 ++--- .../ozone/{ksm => om}/KeyDeletingService.java | 20 +- .../hadoop/ozone/{ksm => om}/KeyManager.java | 42 +-- .../ozone/{ksm => om}/KeyManagerImpl.java | 140 ++++----- .../{ksm/KSMMXBean.java => om/OMMXBean.java} | 6 +- .../OMMetadataManager.java} | 38 +-- .../KSMMetrics.java => om/OMMetrics.java} | 22 +- .../KSMStorage.java => om/OMStorage.java} | 46 +-- .../OmMetadataManagerImpl.java} | 126 ++++---- .../{ksm => om}/OpenKeyCleanupService.java | 10 +- .../OzoneManager.java} | 277 +++++++++--------- .../OzoneManagerHttpServer.java} | 30 +- .../{ksm => om}/ServiceListJSONServlet.java | 14 +- .../ozone/{ksm => om}/VolumeManager.java | 18 +- .../ozone/{ksm => om}/VolumeManagerImpl.java | 89 +++--- .../exceptions/OMException.java} | 22 +- .../{ksm => om}/exceptions/package-info.java | 4 +- .../ozone/{ksm => om}/package-info.java | 4 +- ...anagerProtocolServerSideTranslatorPB.java} | 180 ++++++------ .../hadoop/ozone/protocolPB/package-info.java | 2 +- .../webapps/{ksm => ozoneManager}/index.html | 12 +- .../webapps/{ksm => ozoneManager}/main.css | 0 .../webapps/{ksm => ozoneManager}/main.html | 0 .../om-metrics.html} | 2 +- .../ksm.js => ozoneManager/ozoneManager.js} | 14 +- .../{ksm => om}/TestBucketManagerImpl.java | 129 ++++---- .../ozone/{ksm => om}/TestChunkStreams.java | 2 +- .../TestOzoneManagerHttpServer.java} | 16 +- .../ozone/{ksm => om}/package-info.java | 4 +- .../fs/ozone/contract/OzoneContract.java | 4 +- .../apache/hadoop/ozone/scm/cli/SQLCLI.java | 48 +-- 130 files changed, 2009 insertions(+), 1994 deletions(-) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{KsmUtils.java => OmUtils.java} (63%) delete mode 100644 hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java create mode 100644 hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm/helpers/KsmBucketArgs.java => om/helpers/OmBucketArgs.java} (81%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm/helpers/KsmBucketInfo.java => om/helpers/OmBucketInfo.java} (83%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm/helpers/KsmKeyArgs.java => om/helpers/OmKeyArgs.java} (88%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm/helpers/KsmKeyInfo.java => om/helpers/OmKeyInfo.java} (79%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm/helpers/KsmKeyLocationInfo.java => om/helpers/OmKeyLocationInfo.java} (85%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm/helpers/KsmKeyLocationInfoGroup.java => om/helpers/OmKeyLocationInfoGroup.java} (70%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm/helpers/KsmOzoneAclMap.java => om/helpers/OmOzoneAclMap.java} (89%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm/helpers/KsmVolumeArgs.java => om/helpers/OmVolumeArgs.java} (83%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm => om}/helpers/OpenKeySession.java (89%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm => om}/helpers/ServiceInfo.java (89%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm => om}/helpers/VolumeArgs.java (98%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm => om}/helpers/package-info.java (94%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm => om}/package-info.java (94%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm/protocol/KeySpaceManagerProtocol.java => om/protocol/OzoneManagerProtocol.java} (76%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm => om}/protocol/package-info.java (94%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java => om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java} (79%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm/protocolPB/KeySpaceManagerProtocolPB.java => om/protocolPB/OzoneManagerProtocolPB.java} (77%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/{ksm => om}/protocolPB/package-info.java (94%) rename hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/{KSMPBHelper.java => OMPBHelper.java} (93%) rename hadoop-ozone/common/src/main/proto/{KeySpaceManagerProtocol.proto => OzoneManagerProtocol.proto} (96%) rename hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/{ksm => om}/TestContainerReportWithKeys.java (94%) rename hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/{ksm => om}/TestMultipleContainerReadWrite.java (99%) rename hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/{ksm/TestKsmBlockVersioning.java => om/TestOmBlockVersioning.java} (81%) rename hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/{ksm/TestKSMMetrcis.java => om/TestOmMetrics.java} (53%) rename hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/{ksm/TestKSMSQLCli.java => om/TestOmSQLCli.java} (96%) rename hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/{ksm/TestKeySpaceManager.java => om/TestOzoneManager.java} (90%) rename hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/{ksm/TestKeySpaceManagerRestInterface.java => om/TestOzoneManagerRestInterface.java} (83%) rename hadoop-ozone/integration-test/src/test/resources/webapps/{ksm => ozoneManager}/.gitkeep (100%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm => om}/BucketManager.java (78%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm => om}/BucketManagerImpl.java (79%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm => om}/KeyDeletingService.java (88%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm => om}/KeyManager.java (82%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm => om}/KeyManagerImpl.java (83%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm/KSMMXBean.java => om/OMMXBean.java} (86%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm/KSMMetadataManager.java => om/OMMetadataManager.java} (85%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm/KSMMetrics.java => om/OMMetrics.java} (96%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm/KSMStorage.java => om/OMStorage.java} (63%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm/KSMMetadataManagerImpl.java => om/OmMetadataManagerImpl.java} (79%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm => om}/OpenKeyCleanupService.java (93%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm/KeySpaceManager.java => om/OzoneManager.java} (74%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm/KeySpaceManagerHttpServer.java => om/OzoneManagerHttpServer.java} (68%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm => om}/ServiceListJSONServlet.java (89%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm => om}/VolumeManager.java (84%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm => om}/VolumeManagerImpl.java (82%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm/exceptions/KSMException.java => om/exceptions/OMException.java} (85%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm => om}/exceptions/package-info.java (91%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/{ksm => om}/package-info.java (89%) rename hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/{KeySpaceManagerProtocolServerSideTranslatorPB.java => OzoneManagerProtocolServerSideTranslatorPB.java} (76%) rename hadoop-ozone/ozone-manager/src/main/webapps/{ksm => ozoneManager}/index.html (88%) rename hadoop-ozone/ozone-manager/src/main/webapps/{ksm => ozoneManager}/main.css (100%) rename hadoop-ozone/ozone-manager/src/main/webapps/{ksm => ozoneManager}/main.html (100%) rename hadoop-ozone/ozone-manager/src/main/webapps/{ksm/ksm-metrics.html => ozoneManager/om-metrics.html} (98%) rename hadoop-ozone/ozone-manager/src/main/webapps/{ksm/ksm.js => ozoneManager/ozoneManager.js} (90%) rename hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/{ksm => om}/TestBucketManagerImpl.java (77%) rename hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/{ksm => om}/TestChunkStreams.java (99%) rename hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/{ksm/TestKeySpaceManagerHttpServer.java => om/TestOzoneManagerHttpServer.java} (91%) rename hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/{ksm => om}/package-info.java (94%) diff --git a/.gitignore b/.gitignore index 428950ba997..d5550364b8f 100644 --- a/.gitignore +++ b/.gitignore @@ -50,6 +50,10 @@ patchprocess/ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/package-lock.json hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/yarn-error.log +# Ignore files generated by HDDS acceptance tests. +hadoop-ozone/acceptance-test/docker-compose.log +hadoop-ozone/acceptance-test/junit-results.xml + #robotframework outputs log.html output.xml diff --git a/dev-support/bin/ozone-dist-layout-stitching b/dev-support/bin/ozone-dist-layout-stitching index be330d5aaaa..c30a37d2e5f 100755 --- a/dev-support/bin/ozone-dist-layout-stitching +++ b/dev-support/bin/ozone-dist-layout-stitching @@ -148,7 +148,7 @@ run copy "${ROOT}/hadoop-ozone/tools/target/hadoop-ozone-tools-${HDDS_VERSION}" mkdir -p "./share/hadoop/ozonefs" cp "${ROOT}/hadoop-ozone/ozonefs/target/hadoop-ozone-filesystem-${HDDS_VERSION}.jar" "./share/hadoop/ozonefs/hadoop-ozone-filesystem.jar" # Optional documentation, could be missing -cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/ozone/webapps/ksm/ +cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/ozone/webapps/ozoneManager/ cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/hdds/webapps/scm/ diff --git a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh index 6573a81eb5b..3826f67a5ea 100644 --- a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh +++ b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh @@ -404,13 +404,13 @@ esac # export HDFS_DFSROUTER_OPTS="" ### -# HDFS Key Space Manager specific parameters +# Ozone Manager specific parameters ### -# Specify the JVM options to be used when starting the HDFS Key Space Manager. +# Specify the JVM options to be used when starting the Ozone Manager. # These options will be appended to the options specified as HADOOP_OPTS # and therefore may override any similar flags set in HADOOP_OPTS # -# export HDFS_KSM_OPTS="" +# export HDFS_OM_OPTS="" ### # HDFS StorageContainerManager specific parameters diff --git a/hadoop-dist/src/main/compose/ozone/docker-compose.yaml b/hadoop-dist/src/main/compose/ozone/docker-compose.yaml index 512c649e210..bb5e8dd5354 100644 --- a/hadoop-dist/src/main/compose/ozone/docker-compose.yaml +++ b/hadoop-dist/src/main/compose/ozone/docker-compose.yaml @@ -25,17 +25,17 @@ services: command: ["/opt/hadoop/bin/ozone","datanode"] env_file: - ./docker-config - ksm: + ozoneManager: image: apache/hadoop-runner volumes: - ../../ozone:/opt/hadoop ports: - 9874:9874 environment: - ENSURE_KSM_INITIALIZED: /data/metadata/ksm/current/VERSION + ENSURE_OM_INITIALIZED: /data/metadata/ozoneManager/current/VERSION env_file: - ./docker-config - command: ["/opt/hadoop/bin/ozone","ksm"] + command: ["/opt/hadoop/bin/ozone","om"] scm: image: apache/hadoop-runner volumes: diff --git a/hadoop-dist/src/main/compose/ozone/docker-config b/hadoop-dist/src/main/compose/ozone/docker-config index 632f8701d27..50abb18e1ab 100644 --- a/hadoop-dist/src/main/compose/ozone/docker-config +++ b/hadoop-dist/src/main/compose/ozone/docker-config @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -OZONE-SITE.XML_ozone.ksm.address=ksm +OZONE-SITE.XML_ozone.om.address=ozoneManager OZONE-SITE.XML_ozone.scm.names=scm OZONE-SITE.XML_ozone.enabled=True OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id diff --git a/hadoop-dist/src/main/compose/ozoneperf/README.md b/hadoop-dist/src/main/compose/ozoneperf/README.md index a78f2087328..527ff418d37 100644 --- a/hadoop-dist/src/main/compose/ozoneperf/README.md +++ b/hadoop-dist/src/main/compose/ozoneperf/README.md @@ -67,7 +67,7 @@ http://localhost:9090/graph Example queries: ``` -Hadoop_KeySpaceManager_NumKeyCommits -rate(Hadoop_KeySpaceManager_NumKeyCommits[10m]) +Hadoop_OzoneManager_NumKeyCommits +rate(Hadoop_OzoneManager_NumKeyCommits[10m]) rate(Hadoop_Ozone_BYTES_WRITTEN[10m]) ``` diff --git a/hadoop-dist/src/main/compose/ozoneperf/docker-compose.yaml b/hadoop-dist/src/main/compose/ozoneperf/docker-compose.yaml index 3233c116413..6d1d9cadb34 100644 --- a/hadoop-dist/src/main/compose/ozoneperf/docker-compose.yaml +++ b/hadoop-dist/src/main/compose/ozoneperf/docker-compose.yaml @@ -26,7 +26,7 @@ services: command: ["/opt/hadoop/bin/ozone","datanode"] env_file: - ./docker-config - ksm: + ozoneManager: image: apache/hadoop-runner volumes: - ../../ozone:/opt/hadoop @@ -34,10 +34,10 @@ services: ports: - 9874:9874 environment: - ENSURE_KSM_INITIALIZED: /data/metadata/ksm/current/VERSION + ENSURE_OM_INITIALIZED: /data/metadata/ozoneManager/current/VERSION env_file: - ./docker-config - command: ["/opt/hadoop/bin/ozone","ksm"] + command: ["/opt/hadoop/bin/ozone","om"] scm: image: apache/hadoop-runner volumes: diff --git a/hadoop-dist/src/main/compose/ozoneperf/docker-config b/hadoop-dist/src/main/compose/ozoneperf/docker-config index 2be22a77923..253995014c7 100644 --- a/hadoop-dist/src/main/compose/ozoneperf/docker-config +++ b/hadoop-dist/src/main/compose/ozoneperf/docker-config @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -OZONE-SITE.XML_ozone.ksm.address=ksm +OZONE-SITE.XML_ozone.om.address=ozoneManager OZONE-SITE.XML_ozone.scm.names=scm OZONE-SITE.XML_ozone.enabled=True OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java index ad326dcb7f4..4f1b1c8e2c8 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java @@ -94,7 +94,7 @@ public final class ScmConfigKeys { "ozone.scm.datanode.port"; public static final int OZONE_SCM_DATANODE_PORT_DEFAULT = 9861; - // OZONE_KSM_PORT_DEFAULT = 9862 + // OZONE_OM_PORT_DEFAULT = 9862 public static final String OZONE_SCM_BLOCK_CLIENT_PORT_KEY = "ozone.scm.block.client.port"; public static final int OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT = 9863; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java index 08a5ffdb87f..4fad5d83a8c 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java @@ -93,7 +93,7 @@ public final class OzoneConsts { public static final String BLOCK_DB = "block.db"; public static final String OPEN_CONTAINERS_DB = "openContainers.db"; public static final String DELETED_BLOCK_DB = "deletedBlock.db"; - public static final String KSM_DB_NAME = "ksm.db"; + public static final String OM_DB_NAME = "om.db"; /** * Supports Bucket Versioning. @@ -119,13 +119,13 @@ public final class OzoneConsts { public static final String OPEN_KEY_ID_DELIMINATOR = "#"; /** - * KSM LevelDB prefixes. + * OM LevelDB prefixes. * - * KSM DB stores metadata as KV pairs with certain prefixes, + * OM DB stores metadata as KV pairs with certain prefixes, * prefix is used to improve the performance to get related * metadata. * - * KSM DB Schema: + * OM DB Schema: * ---------------------------------------------------------- * | KEY | VALUE | * ---------------------------------------------------------- @@ -140,13 +140,13 @@ public final class OzoneConsts { * | #deleting#/volumeName/bucketName/keyName | KeyInfo | * ---------------------------------------------------------- */ - public static final String KSM_VOLUME_PREFIX = "/#"; - public static final String KSM_BUCKET_PREFIX = "/#"; - public static final String KSM_KEY_PREFIX = "/"; - public static final String KSM_USER_PREFIX = "$"; + public static final String OM_VOLUME_PREFIX = "/#"; + public static final String OM_BUCKET_PREFIX = "/#"; + public static final String OM_KEY_PREFIX = "/"; + public static final String OM_USER_PREFIX = "$"; /** - * Max KSM Quota size of 1024 PB. + * Max OM Quota size of 1024 PB. */ public static final long MAX_QUOTA_IN_BYTES = 1024L * 1024 * TB; @@ -168,9 +168,9 @@ public final class OzoneConsts { public static final int INVALID_PORT = -1; - // The ServiceListJSONServlet context attribute where KeySpaceManager + // The ServiceListJSONServlet context attribute where OzoneManager // instance gets stored. - public static final String KSM_CONTEXT_ATTRIBUTE = "ozone.ksm"; + public static final String OM_CONTEXT_ATTRIBUTE = "ozone.om"; private OzoneConsts() { // Never Constructed diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java index fb30d921b8d..a32d5590236 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java @@ -38,7 +38,7 @@ import java.util.Properties; * Local storage information is stored in a separate file VERSION. * It contains type of the node, * the storage layout version, the SCM id, and - * the KSM/SCM state creation time. + * the OM/SCM state creation time. * */ @InterfaceAudience.Private @@ -127,7 +127,7 @@ public abstract class Storage { abstract protected Properties getNodeProperties(); /** - * Sets the Node properties spaecific to KSM/SCM. + * Sets the Node properties spaecific to OM/SCM. */ private void setNodeProperties() { Properties nodeProperties = getNodeProperties(); @@ -152,7 +152,7 @@ public abstract class Storage { * File {@code VERSION} contains the following fields: *

    *
  1. node type
  2. - *
  3. KSM/SCM state creation time
  4. + *
  5. OM/SCM state creation time
  6. *
  7. other fields specific for this node type
  8. *
* The version file is always written last during storage directory updates. diff --git a/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto b/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto index 7bea82ab860..53f408ae889 100644 --- a/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto +++ b/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto @@ -46,7 +46,7 @@ message AllocateScmBlockRequestProto { } /** - * A delete key request sent by KSM to SCM, it contains + * A delete key request sent by OM to SCM, it contains * multiple number of keys (and their blocks). */ message DeleteScmKeyBlocksRequestProto { @@ -56,9 +56,9 @@ message DeleteScmKeyBlocksRequestProto { /** * A object key and all its associated blocks. * We need to encapsulate object key name plus the blocks in this potocol - * because SCM needs to response KSM with the keys it has deleted. + * because SCM needs to response OM with the keys it has deleted. * If the response only contains blocks, it will be very expensive for - * KSM to figure out what keys have been deleted. + * OM to figure out what keys have been deleted. */ message KeyBlocks { required string key = 1; @@ -66,7 +66,7 @@ message KeyBlocks { } /** - * A delete key response from SCM to KSM, it contains multiple child-results. + * A delete key response from SCM to OM, it contains multiple child-results. * Each child-result represents a key deletion result, only if all blocks of * a key are successfully deleted, this key result is considered as succeed. */ @@ -111,7 +111,7 @@ message AllocateScmBlockResponseProto { } /** - * Protocol used from KeySpaceManager to StorageContainerManager. + * Protocol used from OzoneManager to StorageContainerManager. * See request and response messages for details of the RPC calls. */ service ScmBlockLocationProtocolService { diff --git a/hadoop-hdds/common/src/main/proto/hdds.proto b/hadoop-hdds/common/src/main/proto/hdds.proto index b9def2a3ecb..a5ce9949402 100644 --- a/hadoop-hdds/common/src/main/proto/hdds.proto +++ b/hadoop-hdds/common/src/main/proto/hdds.proto @@ -58,9 +58,9 @@ message KeyValue { * Type of the node. */ enum NodeType { - KSM = 1; - SCM = 2; - DATANODE = 3; + OM = 1; // Ozone Manager + SCM = 2; // Storage Container Manager + DATANODE = 3; // DataNode } // Should we rename NodeState to DatanodeState? diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 568d26712a7..530fb09b24a 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -21,7 +21,7 @@ - + @@ -254,122 +254,122 @@ Tells ozone which storage handler to use. The possible values are: distributed - The Ozone distributed storage handler, which speaks to - KSM/SCM on the backend and provides REST services to clients. + OM/SCM on the backend and provides REST services to clients. local - Local Storage handler strictly for testing - To be removed. ozone.key.deleting.limit.per.task 1000 - KSM, PERFORMANCE + OM, PERFORMANCE A maximum number of keys to be scanned by key deleting service - per time interval in KSM. Those keys are sent to delete metadata and + per time interval in OM. Those keys are sent to delete metadata and generate transactions in SCM for next async deletion between SCM and DataNode. - ozone.ksm.address + ozone.om.address - KSM, REQUIRED + OM, REQUIRED - The address of the Ozone KSM service. This allows clients to discover - the KSMs address. + The address of the Ozone OM service. This allows clients to discover + the address of the OM. - ozone.ksm.group.rights + ozone.om.group.rights READ_WRITE - KSM, SECURITY + OM, SECURITY - Default group permissions in Ozone KSM. + Default group permissions in Ozone OM. - ozone.ksm.handler.count.key + ozone.om.handler.count.key 20 - KSM, PERFORMANCE + OM, PERFORMANCE - The number of RPC handler threads for KSM service endpoints. + The number of RPC handler threads for OM service endpoints. - ozone.ksm.http-address + ozone.om.http-address 0.0.0.0:9874 - KSM, MANAGEMENT + OM, MANAGEMENT - The address and the base port where the KSM web UI will listen on. + The address and the base port where the OM web UI will listen on. If the port is 0, then the server will start on a free port. However, it is best to specify a well-known port, so it is easy to connect and see - the KSM management UI. + the OM management UI. - ozone.ksm.http-bind-host + ozone.om.http-bind-host 0.0.0.0 - KSM, MANAGEMENT + OM, MANAGEMENT - The actual address the KSM web server will bind to. If this optional + The actual address the OM web server will bind to. If this optional the address is set, it overrides only the hostname portion of - ozone.ksm.http-address. + ozone.om.http-address. - ozone.ksm.http.enabled + ozone.om.http.enabled true - KSM, MANAGEMENT + OM, MANAGEMENT - Property to enable or disable KSM web user interface. + Property to enable or disable OM web user interface. - ozone.ksm.https-address + ozone.om.https-address 0.0.0.0:9875 - KSM, MANAGEMENT, SECURITY + OM, MANAGEMENT, SECURITY - The address and the base port where the KSM web UI will listen + The address and the base port where the OM web UI will listen on using HTTPS. If the port is 0 then the server will start on a free port. - ozone.ksm.https-bind-host + ozone.om.https-bind-host 0.0.0.0 - KSM, MANAGEMENT, SECURITY + OM, MANAGEMENT, SECURITY - The actual address the KSM web server will bind to using HTTPS. + The actual address the OM web server will bind to using HTTPS. If this optional address is set, it overrides only the hostname portion of - ozone.ksm.http-address. + ozone.om.http-address. - ozone.ksm.keytab.file + ozone.om.keytab.file - KSM, SECURITY + OM, SECURITY - The keytab file for Kerberos authentication in KSM. + The keytab file for Kerberos authentication in OM. - ozone.ksm.db.cache.size.mb + ozone.om.db.cache.size.mb 128 - KSM, PERFORMANCE + OM, PERFORMANCE - The size of KSM DB cache in MB that used for caching files. + The size of OM DB cache in MB that used for caching files. This value is set to an abnormally low value in the default configuration. That is to make unit testing easy. Generally, this value should be set to something like 16GB or more, if you intend to use Ozone at scale. - A large value for this key allows a proportionally larger amount of KSM - metadata to be cached in memory. This makes KSM operations faster. + A large value for this key allows a proportionally larger amount of OM + metadata to be cached in memory. This makes OM operations faster. - ozone.ksm.user.max.volume + ozone.om.user.max.volume 1024 - KSM, MANAGEMENT + OM, MANAGEMENT The maximum number of volumes a user can have on a cluster.Increasing or decreasing this number has no real impact on ozone cluster. This is @@ -379,11 +379,11 @@ - ozone.ksm.user.rights + ozone.om.user.rights READ_WRITE - KSM, SECURITY + OM, SECURITY - Default user permissions used in KSM. + Default user permissions used in OM. @@ -393,20 +393,20 @@ This is used only for testing purposes. This value is used by the local storage handler to simulate a REST backend. This is useful only when - debugging the REST front end independent of KSM and SCM. To be removed. + debugging the REST front end independent of OM and SCM. To be removed. ozone.metadata.dirs - OZONE, KSM, SCM, CONTAINER, REQUIRED, STORAGE + OZONE, OM, SCM, CONTAINER, REQUIRED, STORAGE - Ozone metadata is shared among KSM, which acts as the namespace + Ozone metadata is shared among OM, which acts as the namespace manager for ozone, SCM which acts as the block manager and data nodes which maintain the name of the key(Key Name and BlockIDs). This replicated and distributed metadata store is maintained under the directory pointed by this key. Since metadata can be I/O intensive, at - least on KSM and SCM we recommend having SSDs. If you have the luxury + least on OM and SCM we recommend having SSDs. If you have the luxury of mapping this path to SSDs on all machines in the cluster, that will be excellent. @@ -417,10 +417,10 @@ ozone.metastore.impl RocksDB - OZONE, KSM, SCM, CONTAINER, STORAGE + OZONE, OM, SCM, CONTAINER, STORAGE Ozone metadata store implementation. Ozone metadata are well - distributed to multiple services such as ksm, scm. They are stored in + distributed to multiple services such as ozoneManager, scm. They are stored in some local key-value databases. This property determines which database library to use. Supported value is either LevelDB or RocksDB. @@ -429,7 +429,7 @@ ozone.metastore.rocksdb.statistics ALL - OZONE, KSM, SCM, STORAGE, PERFORMANCE + OZONE, OM, SCM, STORAGE, PERFORMANCE The statistics level of the rocksdb store. If you use any value from org.rocksdb.StatsLevel (eg. ALL or EXCEPT_DETAILED_TIMERS), the rocksdb @@ -672,7 +672,7 @@ The heartbeat interval from a data node to SCM. Yes, it is not three but 30, since most data nodes will heart beating via Ratis heartbeats. If a client is not able to talk to a data node, it will notify - KSM/SCM eventually. So a 30 second HB seems to work. This assumes that + OM/SCM eventually. So a 30 second HB seems to work. This assumes that replication strategy used is Ratis if not, this value should be set to something smaller like 3 seconds. @@ -808,7 +808,7 @@ OZONE, SECURITY - The server principal used by the SCM and KSM for web UI SPNEGO + The server principal used by the SCM and OM for web UI SPNEGO authentication when Kerberos security is enabled. This is typically set to HTTP/_HOST@REALM.TLD The SPNEGO server principal begins with the prefix HTTP/ by convention. @@ -867,9 +867,9 @@ ozone.key.preallocation.maxsize 134217728 - OZONE, KSM, PERFORMANCE + OZONE, OM, PERFORMANCE - When a new key write request is sent to KSM, if a size is requested, at most + When a new key write request is sent to OM, if a size is requested, at most 128MB of size is allocated at request time. If client needs more space for the write, separate block allocation requests will be made. @@ -938,7 +938,7 @@ ozone.open.key.cleanup.service.interval.seconds 86400 - OZONE, KSM, PERFORMANCE + OZONE, OM, PERFORMANCE A background job periodically checks open key entries and delete the expired ones. This entry controls the interval of this cleanup check. @@ -948,7 +948,7 @@ ozone.open.key.expire.threshold 86400 - OZONE, KSM, PERFORMANCE + OZONE, OM, PERFORMANCE Controls how long an open key operation is considered active. Specifically, if a key has been open longer than the value of this config entry, that open key is considered as @@ -958,12 +958,12 @@ hadoop.tags.custom - OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,KSM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE + OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,OM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE ozone.tags.system - OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,KSM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE + OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,OM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE diff --git a/hadoop-hdds/framework/src/main/resources/webapps/static/ozone.js b/hadoop-hdds/framework/src/main/resources/webapps/static/ozone.js index 411438aae40..c2ed2adce2f 100644 --- a/hadoop-hdds/framework/src/main/resources/webapps/static/ozone.js +++ b/hadoop-hdds/framework/src/main/resources/webapps/static/ozone.js @@ -270,7 +270,7 @@ $http.get("conf?cmd=getOzoneTags") .then(function(response) { ctrl.tags = response.data; - var excludedTags = ['CBLOCK', 'KSM', 'SCM']; + var excludedTags = ['CBLOCK', 'OM', 'SCM']; for (var i = 0; i < excludedTags.length; i++) { var idx = ctrl.tags.indexOf(excludedTags[i]); // Remove CBLOCK related properties @@ -302,7 +302,7 @@ } ctrl.loadAll = function() { - $http.get("conf?cmd=getPropertyByTag&tags=KSM,SCM," + ctrl.tags) + $http.get("conf?cmd=getPropertyByTag&tags=OM,SCM," + ctrl.tags) .then(function(response) { ctrl.convertToArray(response.data); diff --git a/hadoop-hdds/framework/src/main/resources/webapps/static/templates/config.html b/hadoop-hdds/framework/src/main/resources/webapps/static/templates/config.html index 6825750c336..b52f6533fc9 100644 --- a/hadoop-hdds/framework/src/main/resources/webapps/static/templates/config.html +++ b/hadoop-hdds/framework/src/main/resources/webapps/static/templates/config.html @@ -27,8 +27,8 @@ ng-click="$ctrl.switchto('All')">All KSM + ng-class="$ctrl.allSelected('OM') ? 'btn-primary' :'btn-secondary'" + ng-click="$ctrl.switchto('OM')">OM SCM diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java index 4f4c75563b7..28103bef954 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java @@ -28,7 +28,7 @@ import java.util.Map; /** * The DeletedBlockLog is a persisted log in SCM to keep tracking * container blocks which are under deletion. It maintains info - * about under-deletion container blocks that notified by KSM, + * about under-deletion container blocks that notified by OM, * and the state how it is processed. */ public interface DeletedBlockLog extends Closeable { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java index 3ada8fe1928..c23b1fd17dc 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java @@ -53,9 +53,9 @@ import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes * client to able to write to it. *

* 2. Owners - Each instance of Name service, for example, Namenode of HDFS or - * Key Space Manager (KSM) of Ozone or CBlockServer -- is an owner. It is - * possible to have many KSMs for a Ozone cluster and only one SCM. But SCM - * keeps the data from each KSM in separate bucket, never mixing them. To + * Ozone Manager (OM) of Ozone or CBlockServer -- is an owner. It is + * possible to have many OMs for a Ozone cluster and only one SCM. But SCM + * keeps the data from each OM in separate bucket, never mixing them. To * write data, often we have to find all open containers for a specific owner. *

* 3. ReplicationType - The clients are allowed to specify what kind of diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/CommandQueue.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/CommandQueue.java index edbcfa12f20..996478caaaf 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/CommandQueue.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/CommandQueue.java @@ -33,7 +33,7 @@ import java.util.concurrent.locks.ReentrantLock; /** * Command Queue is queue of commands for the datanode. *

- * Node manager, container Manager and key space managers can queue commands for + * Node manager, container Manager and Ozone managers can queue commands for * datanodes into this queue. These commands will be send in the order in which * there where queued. */ diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java index 98fe9a11374..3bb284e8d02 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java @@ -152,7 +152,7 @@ public class SCMBlockProtocolServer implements ScmBlockLocationProtocol { @Override public List deleteKeyBlocks( List keyBlocksInfoList) throws IOException { - LOG.info("SCM is informed by KSM to delete {} blocks", keyBlocksInfoList + LOG.info("SCM is informed by OM to delete {} blocks", keyBlocksInfoList .size()); List results = new ArrayList<>(); for (BlockGroup keyBlocks : keyBlocksInfoList) { diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneBaseCLI.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneBaseCLI.java index 727c81a0d83..782844517b6 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneBaseCLI.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneBaseCLI.java @@ -27,7 +27,7 @@ import java.io.IOException; import java.net.URISyntaxException; /** - * This class is the base CLI for scm, ksm and scmadm. + * This class is the base CLI for scm, om and scmadm. */ public abstract class OzoneBaseCLI extends Configured implements Tool { diff --git a/hadoop-ozone/acceptance-test/src/test/acceptance/basic/basic.robot b/hadoop-ozone/acceptance-test/src/test/acceptance/basic/basic.robot index c741588c190..6d6fea02733 100644 --- a/hadoop-ozone/acceptance-test/src/test/acceptance/basic/basic.robot +++ b/hadoop-ozone/acceptance-test/src/test/acceptance/basic/basic.robot @@ -39,12 +39,12 @@ Test rest interface Should contain ${result} 200 OK Check webui static resources - ${result} = Execute on scm curl -s -I http://localhost:9876/static/bootstrap-3.3.7/js/bootstrap.min.js + ${result} = Execute on scm curl -s -I http://localhost:9876/static/bootstrap-3.3.7/js/bootstrap.min.js Should contain ${result} 200 - ${result} = Execute on ksm curl -s -I http://localhost:9874/static/bootstrap-3.3.7/js/bootstrap.min.js + ${result} = Execute on ozoneManager curl -s -I http://localhost:9874/static/bootstrap-3.3.7/js/bootstrap.min.js Should contain ${result} 200 Start freon testing - ${result} = Execute on ksm ozone freon -numOfVolumes 5 -numOfBuckets 5 -numOfKeys 5 -numOfThreads 10 + ${result} = Execute on ozoneManager ozone freon -numOfVolumes 5 -numOfBuckets 5 -numOfKeys 5 -numOfThreads 10 Wait Until Keyword Succeeds 3min 10sec Should contain ${result} Number of Keys added: 125 Should Not Contain ${result} ERROR diff --git a/hadoop-ozone/acceptance-test/src/test/acceptance/basic/docker-compose.yaml b/hadoop-ozone/acceptance-test/src/test/acceptance/basic/docker-compose.yaml index b50f42d3e97..99f28310f15 100644 --- a/hadoop-ozone/acceptance-test/src/test/acceptance/basic/docker-compose.yaml +++ b/hadoop-ozone/acceptance-test/src/test/acceptance/basic/docker-compose.yaml @@ -25,18 +25,18 @@ services: command: ["/opt/hadoop/bin/ozone","datanode"] env_file: - ./docker-config - ksm: + ozoneManager: image: apache/hadoop-runner - hostname: ksm + hostname: ozoneManager volumes: - ${OZONEDIR}:/opt/hadoop ports: - 9874 environment: - ENSURE_KSM_INITIALIZED: /data/metadata/ksm/current/VERSION + ENSURE_OM_INITIALIZED: /data/metadata/ozoneManager/current/VERSION env_file: - ./docker-config - command: ["/opt/hadoop/bin/ozone","ksm"] + command: ["/opt/hadoop/bin/ozone","om"] scm: image: apache/hadoop-runner volumes: diff --git a/hadoop-ozone/acceptance-test/src/test/acceptance/basic/docker-config b/hadoop-ozone/acceptance-test/src/test/acceptance/basic/docker-config index c3ec2ef71b5..b72085b22fd 100644 --- a/hadoop-ozone/acceptance-test/src/test/acceptance/basic/docker-config +++ b/hadoop-ozone/acceptance-test/src/test/acceptance/basic/docker-config @@ -14,8 +14,8 @@ # See the License for the specific language governing permissions and # limitations under the License. -OZONE-SITE.XML_ozone.ksm.address=ksm -OZONE-SITE.XML_ozone.ksm.http-address=ksm:9874 +OZONE-SITE.XML_ozone.om.address=ozoneManager +OZONE-SITE.XML_ozone.om.http-address=ozoneManager:9874 OZONE-SITE.XML_ozone.scm.names=scm OZONE-SITE.XML_ozone.enabled=True OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id diff --git a/hadoop-ozone/acceptance-test/src/test/acceptance/basic/ozone-shell.robot b/hadoop-ozone/acceptance-test/src/test/acceptance/basic/ozone-shell.robot index 9521ad60bec..f4be3e0f6a0 100644 --- a/hadoop-ozone/acceptance-test/src/test/acceptance/basic/ozone-shell.robot +++ b/hadoop-ozone/acceptance-test/src/test/acceptance/basic/ozone-shell.robot @@ -28,22 +28,22 @@ ${PROJECTDIR} ${CURDIR}/../../../../../.. *** Test Cases *** RestClient without http port - Test ozone shell http:// ksm restwoport True + Test ozone shell http:// ozoneManager restwoport True RestClient with http port - Test ozone shell http:// ksm:9874 restwport True + Test ozone shell http:// ozoneManager:9874 restwport True RestClient without host name - Test ozone shell http:// ${EMPTY} restwohost True + Test ozone shell http:// ${EMPTY} restwohost True RpcClient with port - Test ozone shell o3:// ksm:9862 rpcwoport False + Test ozone shell o3:// ozoneManager:9862 rpcwoport False RpcClient without host - Test ozone shell o3:// ${EMPTY} rpcwport False + Test ozone shell o3:// ${EMPTY} rpcwport False RpcClient without scheme - Test ozone shell ${EMPTY} ${EMPTY} rpcwoscheme False + Test ozone shell ${EMPTY} ${EMPTY} rpcwoscheme False *** Keywords *** @@ -52,7 +52,7 @@ Test ozone shell ${result} = Execute on datanode ozone oz -createVolume ${protocol}${server}/${volume} -user bilbo -quota 100TB -root Should not contain ${result} Failed Should contain ${result} Creating Volume: ${volume} - ${result} = Execute on datanode ozone oz -listVolume o3://ksm -user bilbo | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.volumeName=="${volume}")' + ${result} = Execute on datanode ozone oz -listVolume o3://ozoneManager -user bilbo | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.volumeName=="${volume}")' Should contain ${result} createdOn Execute on datanode ozone oz -updateVolume ${protocol}${server}/${volume} -user bill -quota 10TB ${result} = Execute on datanode ozone oz -infoVolume ${protocol}${server}/${volume} | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.volumeName=="${volume}") | .owner | .name' @@ -66,7 +66,7 @@ Test ozone shell Should Be Equal ${result} GROUP ${result} = Execute on datanode ozone oz -updateBucket ${protocol}${server}/${volume}/bb1 -removeAcl group:samwise:r | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.bucketName=="bb1") | .acls | .[] | select(.name=="frodo") | .type' Should Be Equal ${result} USER - ${result} = Execute on datanode ozone oz -listBucket o3://ksm/${volume}/ | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.bucketName=="bb1") | .volumeName' + ${result} = Execute on datanode ozone oz -listBucket o3://ozoneManager/${volume}/ | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.bucketName=="bb1") | .volumeName' Should Be Equal ${result} ${volume} Run Keyword and Return If ${withkeytest} Test key handling ${protocol} ${server} ${volume} Execute on datanode ozone oz -deleteBucket ${protocol}${server}/${volume}/bb1 @@ -80,6 +80,6 @@ Test key handling Execute on datanode ls -l NOTICE.txt.1 ${result} = Execute on datanode ozone oz -infoKey ${protocol}${server}/${volume}/bb1/key1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.keyName=="key1")' Should contain ${result} createdOn - ${result} = Execute on datanode ozone oz -listKey o3://ksm/${volume}/bb1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.keyName=="key1") | .keyName' + ${result} = Execute on datanode ozone oz -listKey o3://ozoneManager/${volume}/bb1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.keyName=="key1") | .keyName' Should Be Equal ${result} key1 Execute on datanode ozone oz -deleteKey ${protocol}${server}/${volume}/bb1/key1 -v diff --git a/hadoop-ozone/acceptance-test/src/test/acceptance/commonlib.robot b/hadoop-ozone/acceptance-test/src/test/acceptance/commonlib.robot index a5ea30af34e..9235cd917ae 100644 --- a/hadoop-ozone/acceptance-test/src/test/acceptance/commonlib.robot +++ b/hadoop-ozone/acceptance-test/src/test/acceptance/commonlib.robot @@ -21,12 +21,12 @@ Startup Ozone cluster with size Run echo "Starting new docker-compose environment" >> docker-compose.log ${rc} ${output} = Run docker compose up -d Should Be Equal As Integers ${rc} 0 - Wait Until Keyword Succeeds 1min 5sec Is Daemon started ksm HTTP server of KSM is listening + Wait Until Keyword Succeeds 1min 5sec Is Daemon started ozoneManager HTTP server of OZONEMANAGER is listening Daemons are running without error Scale datanodes up 5 Daemons are running without error - Is daemon running without error ksm + Is daemon running without error ozoneManager Is daemon running without error scm Is daemon running without error datanode diff --git a/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/docker-compose.yaml b/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/docker-compose.yaml index 12022dfe61e..6b7b7bd946d 100644 --- a/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/docker-compose.yaml +++ b/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/docker-compose.yaml @@ -25,18 +25,18 @@ services: command: ["/opt/hadoop/bin/ozone","datanode"] env_file: - ./docker-config - ksm: + ozoneManager: image: apache/hadoop-runner - hostname: ksm + hostname: ozoneManager volumes: - ${OZONEDIR}:/opt/hadoop ports: - 9874 environment: - ENSURE_KSM_INITIALIZED: /data/metadata/ksm/current/VERSION + ENSURE_OM_INITIALIZED: /data/metadata/ozoneManager/current/VERSION env_file: - ./docker-config - command: ["/opt/hadoop/bin/ozone","ksm"] + command: ["/opt/hadoop/bin/ozone","om"] scm: image: apache/hadoop-runner volumes: diff --git a/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/docker-config b/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/docker-config index e06d434bb45..b0129bce0ff 100644 --- a/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/docker-config +++ b/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/docker-config @@ -15,8 +15,8 @@ # limitations under the License. CORE-SITE.XML_fs.o3.impl=org.apache.hadoop.fs.ozone.OzoneFileSystem -OZONE-SITE.XML_ozone.ksm.address=ksm -OZONE-SITE.XML_ozone.ksm.http-address=ksm:9874 +OZONE-SITE.XML_ozone.om.address=ozoneManager +OZONE-SITE.XML_ozone.om.http-address=ozoneManager:9874 OZONE-SITE.XML_ozone.scm.names=scm OZONE-SITE.XML_ozone.enabled=True OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id diff --git a/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/ozonefs.robot b/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/ozonefs.robot index 9e8a5d20043..ea473c0de71 100644 --- a/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/ozonefs.robot +++ b/hadoop-ozone/acceptance-test/src/test/acceptance/ozonefs/ozonefs.robot @@ -27,13 +27,13 @@ ${PROJECTDIR} ${CURDIR}/../../../../../.. *** Test Cases *** Create volume and bucket - Execute on datanode ozone oz -createVolume http://ksm/fstest -user bilbo -quota 100TB -root - Execute on datanode ozone oz -createBucket http://ksm/fstest/bucket1 + Execute on datanode ozone oz -createVolume http://ozoneManager/fstest -user bilbo -quota 100TB -root + Execute on datanode ozone oz -createBucket http://ozoneManager/fstest/bucket1 Check volume from ozonefs ${result} = Execute on hadooplast hdfs dfs -ls o3://bucket1.fstest/ Create directory from ozonefs Execute on hadooplast hdfs dfs -mkdir -p o3://bucket1.fstest/testdir/deep - ${result} = Execute on ksm ozone oz -listKey o3://ksm/fstest/bucket1 | grep -v WARN | jq -r '.[].keyName' + ${result} = Execute on ozoneManager ozone oz -listKey o3://ozoneManager/fstest/bucket1 | grep -v WARN | jq -r '.[].keyName' Should contain ${result} testdir/deep diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java index 39b7bb84037..0da52dc0339 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java @@ -81,7 +81,7 @@ public final class BucketArgs { } /** - * Returns new builder class that builds a KsmBucketInfo. + * Returns new builder class that builds a OmBucketInfo. * * @return Builder */ @@ -90,7 +90,7 @@ public final class BucketArgs { } /** - * Builder for KsmBucketInfo. + * Builder for OmBucketInfo. */ public static class Builder { private Boolean versioning; diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java index 3085b0daa68..de0d166abda 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java @@ -21,7 +21,7 @@ package org.apache.hadoop.ozone.client; import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.ozone.KsmUtils; +import org.apache.hadoop.ozone.OmUtils; import org.apache.hadoop.ozone.client.protocol.ClientProtocol; import org.apache.hadoop.ozone.client.rest.RestClient; import org.apache.hadoop.ozone.client.rpc.RpcClient; @@ -34,11 +34,9 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Proxy; -import static org.apache.hadoop.ozone.OzoneConfigKeys - .OZONE_CLIENT_PROTOCOL; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys - .OZONE_KSM_HTTP_ADDRESS_KEY; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_ADDRESS_KEY; +import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_PROTOCOL; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY; /** * Factory class to create different types of OzoneClients. @@ -97,46 +95,46 @@ public final class OzoneClientFactory { /** * Returns an OzoneClient which will use RPC protocol. * - * @param ksmHost - * hostname of KeySpaceManager to connect. + * @param omHost + * hostname of OzoneManager to connect. * * @return OzoneClient * * @throws IOException */ - public static OzoneClient getRpcClient(String ksmHost) + public static OzoneClient getRpcClient(String omHost) throws IOException { Configuration config = new OzoneConfiguration(); - int port = KsmUtils.getKsmRpcPort(config); - return getRpcClient(ksmHost, port, config); + int port = OmUtils.getOmRpcPort(config); + return getRpcClient(omHost, port, config); } /** * Returns an OzoneClient which will use RPC protocol. * - * @param ksmHost - * hostname of KeySpaceManager to connect. + * @param omHost + * hostname of OzoneManager to connect. * - * @param ksmRpcPort - * RPC port of KeySpaceManager. + * @param omRpcPort + * RPC port of OzoneManager. * * @return OzoneClient * * @throws IOException */ - public static OzoneClient getRpcClient(String ksmHost, Integer ksmRpcPort) + public static OzoneClient getRpcClient(String omHost, Integer omRpcPort) throws IOException { - return getRpcClient(ksmHost, ksmRpcPort, new OzoneConfiguration()); + return getRpcClient(omHost, omRpcPort, new OzoneConfiguration()); } /** * Returns an OzoneClient which will use RPC protocol. * - * @param ksmHost - * hostname of KeySpaceManager to connect. + * @param omHost + * hostname of OzoneManager to connect. * - * @param ksmRpcPort - * RPC port of KeySpaceManager. + * @param omRpcPort + * RPC port of OzoneManager. * * @param config * Configuration to be used for OzoneClient creation @@ -145,13 +143,13 @@ public final class OzoneClientFactory { * * @throws IOException */ - public static OzoneClient getRpcClient(String ksmHost, Integer ksmRpcPort, + public static OzoneClient getRpcClient(String omHost, Integer omRpcPort, Configuration config) throws IOException { - Preconditions.checkNotNull(ksmHost); - Preconditions.checkNotNull(ksmRpcPort); + Preconditions.checkNotNull(omHost); + Preconditions.checkNotNull(omRpcPort); Preconditions.checkNotNull(config); - config.set(OZONE_KSM_ADDRESS_KEY, ksmHost + ":" + ksmRpcPort); + config.set(OZONE_OM_ADDRESS_KEY, omHost + ":" + omRpcPort); return getRpcClient(config); } @@ -175,46 +173,46 @@ public final class OzoneClientFactory { /** * Returns an OzoneClient which will use REST protocol. * - * @param ksmHost - * hostname of KeySpaceManager to connect. + * @param omHost + * hostname of OzoneManager to connect. * * @return OzoneClient * * @throws IOException */ - public static OzoneClient getRestClient(String ksmHost) + public static OzoneClient getRestClient(String omHost) throws IOException { Configuration config = new OzoneConfiguration(); - int port = KsmUtils.getKsmRestPort(config); - return getRestClient(ksmHost, port, config); + int port = OmUtils.getOmRestPort(config); + return getRestClient(omHost, port, config); } /** * Returns an OzoneClient which will use REST protocol. * - * @param ksmHost - * hostname of KeySpaceManager to connect. + * @param omHost + * hostname of OzoneManager to connect. * - * @param ksmHttpPort - * HTTP port of KeySpaceManager. + * @param omHttpPort + * HTTP port of OzoneManager. * * @return OzoneClient * * @throws IOException */ - public static OzoneClient getRestClient(String ksmHost, Integer ksmHttpPort) + public static OzoneClient getRestClient(String omHost, Integer omHttpPort) throws IOException { - return getRestClient(ksmHost, ksmHttpPort, new OzoneConfiguration()); + return getRestClient(omHost, omHttpPort, new OzoneConfiguration()); } /** * Returns an OzoneClient which will use REST protocol. * - * @param ksmHost - * hostname of KeySpaceManager to connect. + * @param omHost + * hostname of OzoneManager to connect. * - * @param ksmHttpPort - * HTTP port of KeySpaceManager. + * @param omHttpPort + * HTTP port of OzoneManager. * * @param config * Configuration to be used for OzoneClient creation @@ -223,13 +221,13 @@ public final class OzoneClientFactory { * * @throws IOException */ - public static OzoneClient getRestClient(String ksmHost, Integer ksmHttpPort, + public static OzoneClient getRestClient(String omHost, Integer omHttpPort, Configuration config) throws IOException { - Preconditions.checkNotNull(ksmHost); - Preconditions.checkNotNull(ksmHttpPort); + Preconditions.checkNotNull(omHost); + Preconditions.checkNotNull(omHttpPort); Preconditions.checkNotNull(config); - config.set(OZONE_KSM_HTTP_ADDRESS_KEY, ksmHost + ":" + ksmHttpPort); + config.set(OZONE_OM_HTTP_ADDRESS_KEY, omHost + ":" + omHttpPort); return getRestClient(config); } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java index 0c723dd8ab5..7c93146abda 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java @@ -49,7 +49,7 @@ public class OzoneKey { private long modificationTime; /** - * Constructs OzoneKey from KsmKeyInfo. + * Constructs OzoneKey from OmKeyInfo. * */ public OzoneKey(String volumeName, String bucketName, diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/VolumeArgs.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/VolumeArgs.java index f1aa03108a5..ae1cfccd420 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/VolumeArgs.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/VolumeArgs.java @@ -77,7 +77,7 @@ public final class VolumeArgs { return acls; } /** - * Returns new builder class that builds a KsmVolumeArgs. + * Returns new builder class that builds a OmVolumeArgs. * * @return Builder */ @@ -86,7 +86,7 @@ public final class VolumeArgs { } /** - * Builder for KsmVolumeArgs. + * Builder for OmVolumeArgs. */ public static class Builder { private String adminName; diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java index edd85aabeec..b3a566e43f3 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java @@ -23,8 +23,8 @@ import org.apache.hadoop.fs.Seekable; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; @@ -255,28 +255,29 @@ public class ChunkGroupInputStream extends InputStream implements Seekable { } } - public static LengthInputStream getFromKsmKeyInfo(KsmKeyInfo keyInfo, + public static LengthInputStream getFromOmKeyInfo( + OmKeyInfo keyInfo, XceiverClientManager xceiverClientManager, StorageContainerLocationProtocolClientSideTranslatorPB - storageContainerLocationClient, String requestId) - throws IOException { + storageContainerLocationClient, + String requestId) throws IOException { long length = 0; long containerKey; ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream(); groupInputStream.key = keyInfo.getKeyName(); - List keyLocationInfos = + List keyLocationInfos = keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly(); groupInputStream.streamOffset = new long[keyLocationInfos.size()]; for (int i = 0; i < keyLocationInfos.size(); i++) { - KsmKeyLocationInfo ksmKeyLocationInfo = keyLocationInfos.get(i); - BlockID blockID = ksmKeyLocationInfo.getBlockID(); + OmKeyLocationInfo omKeyLocationInfo = keyLocationInfos.get(i); + BlockID blockID = omKeyLocationInfo.getBlockID(); long containerID = blockID.getContainerID(); ContainerWithPipeline containerWithPipeline = storageContainerLocationClient.getContainerWithPipeline(containerID); XceiverClientSpi xceiverClient = xceiverClientManager .acquireClient(containerWithPipeline.getPipeline(), containerID); boolean success = false; - containerKey = ksmKeyLocationInfo.getLocalID(); + containerKey = omKeyLocationInfo.getLocalID(); try { LOG.debug("get key accessing {} {}", containerID, containerKey); @@ -292,11 +293,10 @@ public class ChunkGroupInputStream extends InputStream implements Seekable { } success = true; ChunkInputStream inputStream = new ChunkInputStream( - ksmKeyLocationInfo.getBlockID(), xceiverClientManager, - xceiverClient, + omKeyLocationInfo.getBlockID(), xceiverClientManager, xceiverClient, chunks, requestId); groupInputStream.addStream(inputStream, - ksmKeyLocationInfo.getLength()); + omKeyLocationInfo.getLength()); } finally { if (!success) { xceiverClientManager.releaseClient(xceiverClient); diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java index d1a3b46b81c..94433179f99 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java @@ -24,15 +24,15 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; -import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession; -import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolClientSideTranslatorPB; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.container.common.helpers @@ -67,10 +67,10 @@ public class ChunkGroupOutputStream extends OutputStream { private final ArrayList streamEntries; private int currentStreamIndex; private long byteOffset; - private final KeySpaceManagerProtocolClientSideTranslatorPB ksmClient; + private final OzoneManagerProtocolClientSideTranslatorPB omClient; private final StorageContainerLocationProtocolClientSideTranslatorPB scmClient; - private final KsmKeyArgs keyArgs; + private final OmKeyArgs keyArgs; private final int openID; private final XceiverClientManager xceiverClientManager; private final int chunkSize; @@ -83,7 +83,7 @@ public class ChunkGroupOutputStream extends OutputStream { @VisibleForTesting public ChunkGroupOutputStream() { streamEntries = new ArrayList<>(); - ksmClient = null; + omClient = null; scmClient = null; keyArgs = null; openID = -1; @@ -113,16 +113,16 @@ public class ChunkGroupOutputStream extends OutputStream { public ChunkGroupOutputStream( OpenKeySession handler, XceiverClientManager xceiverClientManager, StorageContainerLocationProtocolClientSideTranslatorPB scmClient, - KeySpaceManagerProtocolClientSideTranslatorPB ksmClient, + OzoneManagerProtocolClientSideTranslatorPB omClient, int chunkSize, String requestId, ReplicationFactor factor, ReplicationType type) throws IOException { this.streamEntries = new ArrayList<>(); this.currentStreamIndex = 0; this.byteOffset = 0; - this.ksmClient = ksmClient; + this.omClient = omClient; this.scmClient = scmClient; - KsmKeyInfo info = handler.getKeyInfo(); - this.keyArgs = new KsmKeyArgs.Builder() + OmKeyInfo info = handler.getKeyInfo(); + this.keyArgs = new OmKeyArgs.Builder() .setVolumeName(info.getVolumeName()) .setBucketName(info.getBucketName()) .setKeyName(info.getKeyName()) @@ -150,19 +150,19 @@ public class ChunkGroupOutputStream extends OutputStream { * @param openVersion the version corresponding to the pre-allocation. * @throws IOException */ - public void addPreallocateBlocks(KsmKeyLocationInfoGroup version, + public void addPreallocateBlocks(OmKeyLocationInfoGroup version, long openVersion) throws IOException { // server may return any number of blocks, (0 to any) // only the blocks allocated in this open session (block createVersion // equals to open session version) - for (KsmKeyLocationInfo subKeyInfo : version.getLocationList()) { + for (OmKeyLocationInfo subKeyInfo : version.getLocationList()) { if (subKeyInfo.getCreateVersion() == openVersion) { checkKeyLocationInfo(subKeyInfo); } } } - private void checkKeyLocationInfo(KsmKeyLocationInfo subKeyInfo) + private void checkKeyLocationInfo(OmKeyLocationInfo subKeyInfo) throws IOException { ContainerWithPipeline containerWithPipeline = scmClient .getContainerWithPipeline(subKeyInfo.getContainerID()); @@ -210,7 +210,7 @@ public class ChunkGroupOutputStream extends OutputStream { checkNotClosed(); if (streamEntries.size() <= currentStreamIndex) { - Preconditions.checkNotNull(ksmClient); + Preconditions.checkNotNull(omClient); // allocate a new block, if a exception happens, log an error and // throw exception to the caller directly, and the write fails. try { @@ -258,7 +258,7 @@ public class ChunkGroupOutputStream extends OutputStream { int succeededAllocates = 0; while (len > 0) { if (streamEntries.size() <= currentStreamIndex) { - Preconditions.checkNotNull(ksmClient); + Preconditions.checkNotNull(omClient); // allocate a new block, if a exception happens, log an error and // throw exception to the caller directly, and the write fails. try { @@ -286,7 +286,7 @@ public class ChunkGroupOutputStream extends OutputStream { } /** - * Contact KSM to get a new block. Set the new block with the index (e.g. + * Contact OM to get a new block. Set the new block with the index (e.g. * first block has index = 0, second has index = 1 etc.) * * The returned block is made to new ChunkOutputStreamEntry to write. @@ -295,7 +295,7 @@ public class ChunkGroupOutputStream extends OutputStream { * @throws IOException */ private void allocateNewBlock(int index) throws IOException { - KsmKeyLocationInfo subKeyInfo = ksmClient.allocateBlock(keyArgs, openID); + OmKeyLocationInfo subKeyInfo = omClient.allocateBlock(keyArgs, openID); checkKeyLocationInfo(subKeyInfo); } @@ -311,7 +311,7 @@ public class ChunkGroupOutputStream extends OutputStream { } /** - * Commit the key to KSM, this will add the blocks as the new key blocks. + * Commit the key to OM, this will add the blocks as the new key blocks. * * @throws IOException */ @@ -329,7 +329,7 @@ public class ChunkGroupOutputStream extends OutputStream { if (keyArgs != null) { // in test, this could be null keyArgs.setDataSize(byteOffset); - ksmClient.commitKey(keyArgs, openID); + omClient.commitKey(keyArgs, openID); } else { LOG.warn("Closing ChunkGroupOutputStream, but key args is null"); } @@ -342,7 +342,7 @@ public class ChunkGroupOutputStream extends OutputStream { private OpenKeySession openHandler; private XceiverClientManager xceiverManager; private StorageContainerLocationProtocolClientSideTranslatorPB scmClient; - private KeySpaceManagerProtocolClientSideTranslatorPB ksmClient; + private OzoneManagerProtocolClientSideTranslatorPB omClient; private int chunkSize; private String requestID; private ReplicationType type; @@ -364,9 +364,9 @@ public class ChunkGroupOutputStream extends OutputStream { return this; } - public Builder setKsmClient( - KeySpaceManagerProtocolClientSideTranslatorPB client) { - this.ksmClient = client; + public Builder setOmClient( + OzoneManagerProtocolClientSideTranslatorPB client) { + this.omClient = client; return this; } @@ -392,7 +392,7 @@ public class ChunkGroupOutputStream extends OutputStream { public ChunkGroupOutputStream build() throws IOException { return new ChunkGroupOutputStream(openHandler, xceiverManager, scmClient, - ksmClient, chunkSize, requestID, factor, type); + omClient, chunkSize, requestID, factor, type); } } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/DefaultRestServerSelector.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/DefaultRestServerSelector.java index 93b3417b4b5..abdc2fbe19a 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/DefaultRestServerSelector.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/DefaultRestServerSelector.java @@ -18,7 +18,7 @@ package org.apache.hadoop.ozone.client.rest; -import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; import java.util.List; import java.util.Random; diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java index 6e3f617cd67..78fbe8d1f7d 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java @@ -45,10 +45,9 @@ import org.apache.hadoop.ozone.client.rest.headers.Header; import org.apache.hadoop.ozone.client.rest.response.BucketInfo; import org.apache.hadoop.ozone.client.rest.response.KeyInfo; import org.apache.hadoop.ozone.client.rest.response.VolumeInfo; -import org.apache.hadoop.ozone.ksm.KSMConfigKeys; -import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo; -import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ServicePort; +import org.apache.hadoop.ozone.om.OMConfigKeys; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort; import org.apache.hadoop.ozone.web.response.ListBuckets; import org.apache.hadoop.ozone.web.response.ListKeys; import org.apache.hadoop.ozone.web.response.ListVolumes; @@ -152,8 +151,8 @@ public class RestClient implements ClientProtocol { .build()) .build(); this.ugi = UserGroupInformation.getCurrentUser(); - this.userRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_USER_RIGHTS, - KSMConfigKeys.OZONE_KSM_USER_RIGHTS_DEFAULT); + this.userRights = conf.getEnum(OMConfigKeys.OZONE_OM_USER_RIGHTS, + OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT); // TODO: Add new configuration parameter to configure RestServerSelector. RestServerSelector defaultSelector = new DefaultRestServerSelector(); @@ -171,11 +170,11 @@ public class RestClient implements ClientProtocol { private InetSocketAddress getOzoneRestServerAddress( RestServerSelector selector) throws IOException { - String httpAddress = conf.get(KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY); + String httpAddress = conf.get(OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY); if (httpAddress == null) { throw new IllegalArgumentException( - KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY + " must be defined. See" + + OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY + " must be defined. See" + " https://wiki.apache.org/hadoop/Ozone#Configuration for" + " details on configuring Ozone."); } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestServerSelector.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestServerSelector.java index 54e219b92bc..fbd6eb8ea9a 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestServerSelector.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestServerSelector.java @@ -18,7 +18,7 @@ package org.apache.hadoop.ozone.client.rest; -import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; import java.util.List; diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java index 43b94a15296..fc705144532 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java @@ -27,7 +27,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ipc.Client; import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.ozone.KsmUtils; +import org.apache.hadoop.ozone.OmUtils; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.client.BucketArgs; import org.apache.hadoop.ozone.client.OzoneBucket; @@ -43,24 +43,22 @@ import org.apache.hadoop.ozone.client.io.LengthInputStream; import org.apache.hadoop.ozone.client.io.OzoneInputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.client.protocol.ClientProtocol; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; -import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession; -import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo; -import org.apache.hadoop.ozone.ksm.protocolPB - .KeySpaceManagerProtocolClientSideTranslatorPB; -import org.apache.hadoop.ozone.ksm.protocolPB - .KeySpaceManagerProtocolPB; +import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; +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.om.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; +import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB; +import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.ozone.OzoneAcl; -import org.apache.hadoop.ozone.ksm.KSMConfigKeys; +import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ServicePort; + .OzoneManagerProtocolProtos.ServicePort; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.ozone.protocolPB.KSMPBHelper; +import org.apache.hadoop.ozone.protocolPB.OMPBHelper; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.protocolPB @@ -80,7 +78,7 @@ import java.util.UUID; import java.util.stream.Collectors; /** - * Ozone RPC Client Implementation, it connects to KSM, SCM and DataNode + * Ozone RPC Client Implementation, it connects to OM, SCM and DataNode * to execute client calls. This uses RPC protocol for communication * with the servers. */ @@ -92,8 +90,8 @@ public class RpcClient implements ClientProtocol { private final OzoneConfiguration conf; private final StorageContainerLocationProtocolClientSideTranslatorPB storageContainerLocationClient; - private final KeySpaceManagerProtocolClientSideTranslatorPB - keySpaceManagerClient; + private final OzoneManagerProtocolClientSideTranslatorPB + ozoneManagerClient; private final XceiverClientManager xceiverClientManager; private final int chunkSize; private final UserGroupInformation ugi; @@ -109,20 +107,20 @@ public class RpcClient implements ClientProtocol { Preconditions.checkNotNull(conf); this.conf = new OzoneConfiguration(conf); this.ugi = UserGroupInformation.getCurrentUser(); - this.userRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_USER_RIGHTS, - KSMConfigKeys.OZONE_KSM_USER_RIGHTS_DEFAULT); - this.groupRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS, - KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS_DEFAULT); - long ksmVersion = - RPC.getProtocolVersion(KeySpaceManagerProtocolPB.class); - InetSocketAddress ksmAddress = KsmUtils - .getKsmAddressForClients(conf); - RPC.setProtocolEngine(conf, KeySpaceManagerProtocolPB.class, + this.userRights = conf.getEnum(OMConfigKeys.OZONE_OM_USER_RIGHTS, + OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT); + this.groupRights = conf.getEnum(OMConfigKeys.OZONE_OM_GROUP_RIGHTS, + OMConfigKeys.OZONE_OM_GROUP_RIGHTS_DEFAULT); + long omVersion = + RPC.getProtocolVersion(OzoneManagerProtocolPB.class); + InetSocketAddress omAddress = OmUtils + .getOmAddressForClients(conf); + RPC.setProtocolEngine(conf, OzoneManagerProtocolPB.class, ProtobufRpcEngine.class); - this.keySpaceManagerClient = - new KeySpaceManagerProtocolClientSideTranslatorPB( - RPC.getProxy(KeySpaceManagerProtocolPB.class, ksmVersion, - ksmAddress, UserGroupInformation.getCurrentUser(), conf, + this.ozoneManagerClient = + new OzoneManagerProtocolClientSideTranslatorPB( + RPC.getProxy(OzoneManagerProtocolPB.class, omVersion, + omAddress, UserGroupInformation.getCurrentUser(), conf, NetUtils.getDefaultSocketFactory(conf), Client.getRpcTimeout(conf))); @@ -155,7 +153,7 @@ public class RpcClient implements ClientProtocol { } private InetSocketAddress getScmAddressForClient() throws IOException { - List services = keySpaceManagerClient.getServiceList(); + List services = ozoneManagerClient.getServiceList(); ServiceInfo scmInfo = services.stream().filter( a -> a.getNodeType().equals(HddsProtos.NodeType.SCM)) .collect(Collectors.toList()).get(0); @@ -195,7 +193,7 @@ public class RpcClient implements ClientProtocol { listOfAcls.addAll(volArgs.getAcls()); } - KsmVolumeArgs.Builder builder = KsmVolumeArgs.newBuilder(); + OmVolumeArgs.Builder builder = OmVolumeArgs.newBuilder(); builder.setVolume(volumeName); builder.setAdminName(admin); builder.setOwnerName(owner); @@ -204,12 +202,12 @@ public class RpcClient implements ClientProtocol { //Remove duplicates and add ACLs for (OzoneAcl ozoneAcl : listOfAcls.stream().distinct().collect(Collectors.toList())) { - builder.addOzoneAcls(KSMPBHelper.convertOzoneAcl(ozoneAcl)); + builder.addOzoneAcls(OMPBHelper.convertOzoneAcl(ozoneAcl)); } LOG.info("Creating Volume: {}, with {} as owner and quota set to {} bytes.", volumeName, owner, quota); - keySpaceManagerClient.createVolume(builder.build()); + ozoneManagerClient.createVolume(builder.build()); } @Override @@ -217,7 +215,7 @@ public class RpcClient implements ClientProtocol { throws IOException { HddsClientUtils.verifyResourceName(volumeName); Preconditions.checkNotNull(owner); - keySpaceManagerClient.setOwner(volumeName, owner); + ozoneManagerClient.setOwner(volumeName, owner); } @Override @@ -226,14 +224,14 @@ public class RpcClient implements ClientProtocol { HddsClientUtils.verifyResourceName(volumeName); Preconditions.checkNotNull(quota); long quotaInBytes = quota.sizeInBytes(); - keySpaceManagerClient.setQuota(volumeName, quotaInBytes); + ozoneManagerClient.setQuota(volumeName, quotaInBytes); } @Override public OzoneVolume getVolumeDetails(String volumeName) throws IOException { HddsClientUtils.verifyResourceName(volumeName); - KsmVolumeArgs volume = keySpaceManagerClient.getVolumeInfo(volumeName); + OmVolumeArgs volume = ozoneManagerClient.getVolumeInfo(volumeName); return new OzoneVolume( conf, this, @@ -243,7 +241,7 @@ public class RpcClient implements ClientProtocol { volume.getQuotaInBytes(), volume.getCreationTime(), volume.getAclMap().ozoneAclGetProtobuf().stream(). - map(KSMPBHelper::convertOzoneAcl).collect(Collectors.toList())); + map(OMPBHelper::convertOzoneAcl).collect(Collectors.toList())); } @Override @@ -255,14 +253,14 @@ public class RpcClient implements ClientProtocol { @Override public void deleteVolume(String volumeName) throws IOException { HddsClientUtils.verifyResourceName(volumeName); - keySpaceManagerClient.deleteVolume(volumeName); + ozoneManagerClient.deleteVolume(volumeName); } @Override public List listVolumes(String volumePrefix, String prevVolume, int maxListResult) throws IOException { - List volumes = keySpaceManagerClient.listAllVolumes( + List volumes = ozoneManagerClient.listAllVolumes( volumePrefix, prevVolume, maxListResult); return volumes.stream().map(volume -> new OzoneVolume( @@ -274,7 +272,7 @@ public class RpcClient implements ClientProtocol { volume.getQuotaInBytes(), volume.getCreationTime(), volume.getAclMap().ozoneAclGetProtobuf().stream(). - map(KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()))) + map(OMPBHelper::convertOzoneAcl).collect(Collectors.toList()))) .collect(Collectors.toList()); } @@ -282,7 +280,7 @@ public class RpcClient implements ClientProtocol { public List listVolumes(String user, String volumePrefix, String prevVolume, int maxListResult) throws IOException { - List volumes = keySpaceManagerClient.listVolumeByUser( + List volumes = ozoneManagerClient.listVolumeByUser( user, volumePrefix, prevVolume, maxListResult); return volumes.stream().map(volume -> new OzoneVolume( @@ -294,7 +292,7 @@ public class RpcClient implements ClientProtocol { volume.getQuotaInBytes(), volume.getCreationTime(), volume.getAclMap().ozoneAclGetProtobuf().stream(). - map(KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()))) + map(OMPBHelper::convertOzoneAcl).collect(Collectors.toList()))) .collect(Collectors.toList()); } @@ -329,7 +327,7 @@ public class RpcClient implements ClientProtocol { listOfAcls.addAll(bucketArgs.getAcls()); } - KsmBucketInfo.Builder builder = KsmBucketInfo.newBuilder(); + OmBucketInfo.Builder builder = OmBucketInfo.newBuilder(); builder.setVolumeName(volumeName) .setBucketName(bucketName) .setIsVersionEnabled(isVersionEnabled) @@ -339,7 +337,7 @@ public class RpcClient implements ClientProtocol { LOG.info("Creating Bucket: {}/{}, with Versioning {} and " + "Storage Type set to {}", volumeName, bucketName, isVersionEnabled, storageType); - keySpaceManagerClient.createBucket(builder.build()); + ozoneManagerClient.createBucket(builder.build()); } @Override @@ -348,11 +346,11 @@ public class RpcClient implements ClientProtocol { throws IOException { HddsClientUtils.verifyResourceName(volumeName, bucketName); Preconditions.checkNotNull(addAcls); - KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder(); + OmBucketArgs.Builder builder = OmBucketArgs.newBuilder(); builder.setVolumeName(volumeName) .setBucketName(bucketName) .setAddAcls(addAcls); - keySpaceManagerClient.setBucketProperty(builder.build()); + ozoneManagerClient.setBucketProperty(builder.build()); } @Override @@ -361,11 +359,11 @@ public class RpcClient implements ClientProtocol { throws IOException { HddsClientUtils.verifyResourceName(volumeName, bucketName); Preconditions.checkNotNull(removeAcls); - KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder(); + OmBucketArgs.Builder builder = OmBucketArgs.newBuilder(); builder.setVolumeName(volumeName) .setBucketName(bucketName) .setRemoveAcls(removeAcls); - keySpaceManagerClient.setBucketProperty(builder.build()); + ozoneManagerClient.setBucketProperty(builder.build()); } @Override @@ -374,11 +372,11 @@ public class RpcClient implements ClientProtocol { throws IOException { HddsClientUtils.verifyResourceName(volumeName, bucketName); Preconditions.checkNotNull(versioning); - KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder(); + OmBucketArgs.Builder builder = OmBucketArgs.newBuilder(); builder.setVolumeName(volumeName) .setBucketName(bucketName) .setIsVersionEnabled(versioning); - keySpaceManagerClient.setBucketProperty(builder.build()); + ozoneManagerClient.setBucketProperty(builder.build()); } @Override @@ -387,18 +385,18 @@ public class RpcClient implements ClientProtocol { throws IOException { HddsClientUtils.verifyResourceName(volumeName, bucketName); Preconditions.checkNotNull(storageType); - KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder(); + OmBucketArgs.Builder builder = OmBucketArgs.newBuilder(); builder.setVolumeName(volumeName) .setBucketName(bucketName) .setStorageType(storageType); - keySpaceManagerClient.setBucketProperty(builder.build()); + ozoneManagerClient.setBucketProperty(builder.build()); } @Override public void deleteBucket( String volumeName, String bucketName) throws IOException { HddsClientUtils.verifyResourceName(volumeName, bucketName); - keySpaceManagerClient.deleteBucket(volumeName, bucketName); + ozoneManagerClient.deleteBucket(volumeName, bucketName); } @Override @@ -411,8 +409,8 @@ public class RpcClient implements ClientProtocol { public OzoneBucket getBucketDetails( String volumeName, String bucketName) throws IOException { HddsClientUtils.verifyResourceName(volumeName, bucketName); - KsmBucketInfo bucketArgs = - keySpaceManagerClient.getBucketInfo(volumeName, bucketName); + OmBucketInfo bucketArgs = + ozoneManagerClient.getBucketInfo(volumeName, bucketName); return new OzoneBucket( conf, this, @@ -428,7 +426,7 @@ public class RpcClient implements ClientProtocol { public List listBuckets(String volumeName, String bucketPrefix, String prevBucket, int maxListResult) throws IOException { - List buckets = keySpaceManagerClient.listBuckets( + List buckets = ozoneManagerClient.listBuckets( volumeName, prevBucket, bucketPrefix, maxListResult); return buckets.stream().map(bucket -> new OzoneBucket( @@ -451,7 +449,7 @@ public class RpcClient implements ClientProtocol { HddsClientUtils.verifyResourceName(volumeName, bucketName); HddsClientUtils.checkNotNull(keyName, type, factor); String requestId = UUID.randomUUID().toString(); - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(volumeName) .setBucketName(bucketName) .setKeyName(keyName) @@ -460,13 +458,13 @@ public class RpcClient implements ClientProtocol { .setFactor(HddsProtos.ReplicationFactor.valueOf(factor.getValue())) .build(); - OpenKeySession openKey = keySpaceManagerClient.openKey(keyArgs); + OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs); ChunkGroupOutputStream groupOutputStream = new ChunkGroupOutputStream.Builder() .setHandler(openKey) .setXceiverClientManager(xceiverClientManager) .setScmClient(storageContainerLocationClient) - .setKsmClient(keySpaceManagerClient) + .setOmClient(ozoneManagerClient) .setChunkSize(chunkSize) .setRequestID(requestId) .setType(HddsProtos.ReplicationType.valueOf(type.toString())) @@ -485,14 +483,14 @@ public class RpcClient implements ClientProtocol { HddsClientUtils.verifyResourceName(volumeName, bucketName); Preconditions.checkNotNull(keyName); String requestId = UUID.randomUUID().toString(); - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(volumeName) .setBucketName(bucketName) .setKeyName(keyName) .build(); - KsmKeyInfo keyInfo = keySpaceManagerClient.lookupKey(keyArgs); + OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs); LengthInputStream lengthInputStream = - ChunkGroupInputStream.getFromKsmKeyInfo( + ChunkGroupInputStream.getFromOmKeyInfo( keyInfo, xceiverClientManager, storageContainerLocationClient, requestId); return new OzoneInputStream( @@ -505,12 +503,12 @@ public class RpcClient implements ClientProtocol { throws IOException { HddsClientUtils.verifyResourceName(volumeName, bucketName); Preconditions.checkNotNull(keyName); - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(volumeName) .setBucketName(bucketName) .setKeyName(keyName) .build(); - keySpaceManagerClient.deleteKey(keyArgs); + ozoneManagerClient.deleteKey(keyArgs); } @Override @@ -518,12 +516,12 @@ public class RpcClient implements ClientProtocol { String fromKeyName, String toKeyName) throws IOException { HddsClientUtils.verifyResourceName(volumeName, bucketName); HddsClientUtils.checkNotNull(fromKeyName, toKeyName); - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(volumeName) .setBucketName(bucketName) .setKeyName(fromKeyName) .build(); - keySpaceManagerClient.renameKey(keyArgs, toKeyName); + ozoneManagerClient.renameKey(keyArgs, toKeyName); } @Override @@ -531,7 +529,7 @@ public class RpcClient implements ClientProtocol { String keyPrefix, String prevKey, int maxListResult) throws IOException { - List keys = keySpaceManagerClient.listKeys( + List keys = ozoneManagerClient.listKeys( volumeName, bucketName, prevKey, keyPrefix, maxListResult); return keys.stream().map(key -> new OzoneKey( @@ -551,12 +549,12 @@ public class RpcClient implements ClientProtocol { Preconditions.checkNotNull(volumeName); Preconditions.checkNotNull(bucketName); Preconditions.checkNotNull(keyName); - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(volumeName) .setBucketName(bucketName) .setKeyName(keyName) .build(); - KsmKeyInfo keyInfo = keySpaceManagerClient.lookupKey(keyArgs); + OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs); return new OzoneKey(keyInfo.getVolumeName(), keyInfo.getBucketName(), keyInfo.getKeyName(), @@ -568,7 +566,7 @@ public class RpcClient implements ClientProtocol { @Override public void close() throws IOException { IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient); - IOUtils.cleanupWithLogger(LOG, keySpaceManagerClient); + IOUtils.cleanupWithLogger(LOG, ozoneManagerClient); IOUtils.cleanupWithLogger(LOG, xceiverClientManager); } } diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestHddsClientUtils.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestHddsClientUtils.java index a270f61a818..3aefe8ac238 100644 --- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestHddsClientUtils.java +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestHddsClientUtils.java @@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.client; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.ozone.ksm.KSMConfigKeys; +import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.junit.Rule; import org.junit.Test; @@ -30,7 +30,7 @@ import org.junit.rules.Timeout; import java.net.InetSocketAddress; import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients; -import static org.apache.hadoop.ozone.KsmUtils.getKsmAddress; +import static org.apache.hadoop.ozone.OmUtils.getOmAddress; import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertThat; @@ -79,27 +79,27 @@ public class TestHddsClientUtils { } @Test - public void testGetKSMAddress() { + public void testGetOmAddress() { final Configuration conf = new OzoneConfiguration(); // First try a client address with just a host name. Verify it falls // back to the default port. - conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "1.2.3.4"); - InetSocketAddress addr = getKsmAddress(conf); + conf.set(OMConfigKeys.OZONE_OM_ADDRESS_KEY, "1.2.3.4"); + InetSocketAddress addr = getOmAddress(conf); assertThat(addr.getHostString(), is("1.2.3.4")); - assertThat(addr.getPort(), is(KSMConfigKeys.OZONE_KSM_PORT_DEFAULT)); + assertThat(addr.getPort(), is(OMConfigKeys.OZONE_OM_PORT_DEFAULT)); // Next try a client address with just a host name and port. Verify the port - // is ignored and the default KSM port is used. - conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "1.2.3.4:100"); - addr = getKsmAddress(conf); + // is ignored and the default OM port is used. + conf.set(OMConfigKeys.OZONE_OM_ADDRESS_KEY, "1.2.3.4:100"); + addr = getOmAddress(conf); assertThat(addr.getHostString(), is("1.2.3.4")); assertThat(addr.getPort(), is(100)); // Assert the we are able to use default configs if no value is specified. - conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, ""); - addr = getKsmAddress(conf); + conf.set(OMConfigKeys.OZONE_OM_ADDRESS_KEY, ""); + addr = getOmAddress(conf); assertThat(addr.getHostString(), is("0.0.0.0")); - assertThat(addr.getPort(), is(KSMConfigKeys.OZONE_KSM_PORT_DEFAULT)); + assertThat(addr.getPort(), is(OMConfigKeys.OZONE_OM_PORT_DEFAULT)); } } diff --git a/hadoop-ozone/common/pom.xml b/hadoop-ozone/common/pom.xml index d8581d1925a..83d023e9fdb 100644 --- a/hadoop-ozone/common/pom.xml +++ b/hadoop-ozone/common/pom.xml @@ -69,7 +69,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> ${basedir}/src/main/proto - KeySpaceManagerProtocol.proto + OzoneManagerProtocol.proto diff --git a/hadoop-ozone/common/src/main/bin/ozone b/hadoop-ozone/common/src/main/bin/ozone index 390f0895b78..9495eff0a9f 100755 --- a/hadoop-ozone/common/src/main/bin/ozone +++ b/hadoop-ozone/common/src/main/bin/ozone @@ -38,10 +38,9 @@ function hadoop_usage hadoop_add_subcommand "envvars" client "display computed Hadoop environment variables" hadoop_add_subcommand "freon" client "runs an ozone data generator" hadoop_add_subcommand "genesis" client "runs a collection of ozone benchmarks to help with tuning." - hadoop_add_subcommand "getozoneconf" client "get ozone config values from - configuration" + hadoop_add_subcommand "getozoneconf" client "get ozone config values from configuration" hadoop_add_subcommand "jmxget" admin "get JMX exported values from NameNode or DataNode." - hadoop_add_subcommand "ksm" daemon "Ozone keyspace manager" + hadoop_add_subcommand "om" daemon "Ozone Manager" hadoop_add_subcommand "o3" client "command line interface for ozone" hadoop_add_subcommand "noz" client "ozone debug tool, convert ozone metadata into relational data" hadoop_add_subcommand "scm" daemon "run the Storage Container Manager service" @@ -94,9 +93,9 @@ function ozonecmd_case getozoneconf) HADOOP_CLASSNAME=org.apache.hadoop.ozone.freon.OzoneGetConf; ;; - ksm) + om) HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true" - HADOOP_CLASSNAME=org.apache.hadoop.ozone.ksm.KeySpaceManager + HADOOP_CLASSNAME=org.apache.hadoop.ozone.om.OzoneManager ;; oz) HADOOP_CLASSNAME=org.apache.hadoop.ozone.web.ozShell.Shell diff --git a/hadoop-ozone/common/src/main/bin/start-ozone.sh b/hadoop-ozone/common/src/main/bin/start-ozone.sh index 92bc4a86914..29c36743369 100644 --- a/hadoop-ozone/common/src/main/bin/start-ozone.sh +++ b/hadoop-ozone/common/src/main/bin/start-ozone.sh @@ -179,19 +179,19 @@ if [[ "${AUTOHA_ENABLED}" = "true" ]]; then fi #--------------------------------------------------------- -# Ozone keyspacemanager nodes -KSM_NODES=$("${HADOOP_HDFS_HOME}/bin/ozone" getozoneconf -keyspacemanagers 2>/dev/null) -echo "Starting key space manager nodes [${KSM_NODES}]" -if [[ "${KSM_NODES}" == "0.0.0.0" ]]; then - KSM_NODES=$(hostname) +# Ozone ozonemanager nodes +OM_NODES=$("${HADOOP_HDFS_HOME}/bin/ozone" getozoneconf -ozonemanagers 2>/dev/null) +echo "Starting Ozone Manager nodes [${OM_NODES}]" +if [[ "${OM_NODES}" == "0.0.0.0" ]]; then + OM_NODES=$(hostname) fi -hadoop_uservar_su hdfs ksm "${HADOOP_HDFS_HOME}/bin/ozone" \ +hadoop_uservar_su hdfs om "${HADOOP_HDFS_HOME}/bin/ozone" \ --workers \ --config "${HADOOP_CONF_DIR}" \ - --hostnames "${KSM_NODES}" \ + --hostnames "${OM_NODES}" \ --daemon start \ - ksm + om HADOOP_JUMBO_RETCOUNTER=$? diff --git a/hadoop-ozone/common/src/main/bin/stop-ozone.sh b/hadoop-ozone/common/src/main/bin/stop-ozone.sh index be55be4e97f..5f5faf0153a 100644 --- a/hadoop-ozone/common/src/main/bin/stop-ozone.sh +++ b/hadoop-ozone/common/src/main/bin/stop-ozone.sh @@ -73,19 +73,19 @@ else fi #--------------------------------------------------------- -# Ozone keyspacemanager nodes -KSM_NODES=$("${HADOOP_HDFS_HOME}/bin/ozone" getozoneconf -keyspacemanagers 2>/dev/null) -echo "Stopping key space manager nodes [${KSM_NODES}]" -if [[ "${KSM_NODES}" == "0.0.0.0" ]]; then - KSM_NODES=$(hostname) +# Ozone Manager nodes +OM_NODES=$("${HADOOP_HDFS_HOME}/bin/ozone" getozoneconf -ozonemanagers 2>/dev/null) +echo "Stopping Ozone Manager nodes [${OM_NODES}]" +if [[ "${OM_NODES}" == "0.0.0.0" ]]; then + OM_NODES=$(hostname) fi -hadoop_uservar_su hdfs ksm "${HADOOP_HDFS_HOME}/bin/ozone" \ +hadoop_uservar_su hdfs om "${HADOOP_HDFS_HOME}/bin/ozone" \ --workers \ --config "${HADOOP_CONF_DIR}" \ - --hostnames "${KSM_NODES}" \ + --hostnames "${OM_NODES}" \ --daemon stop \ - ksm + om #--------------------------------------------------------- # Ozone storagecontainermanager nodes diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/KsmUtils.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java similarity index 63% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/KsmUtils.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java index 1025963aa87..097410405f0 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/KsmUtils.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java @@ -25,71 +25,70 @@ import org.apache.hadoop.net.NetUtils; import com.google.common.base.Optional; import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys; import static org.apache.hadoop.hdds.HddsUtils.getPortNumberFromConfigKeys; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_ADDRESS_KEY; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_HTTP_BIND_PORT_DEFAULT; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys - .OZONE_KSM_BIND_HOST_DEFAULT; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_PORT_DEFAULT; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_BIND_HOST_DEFAULT; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_BIND_PORT_DEFAULT; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_PORT_DEFAULT; /** - * Stateless helper functions for the server and client side of KSM + * Stateless helper functions for the server and client side of OM * communication. */ -public final class KsmUtils { +public final class OmUtils { - private KsmUtils() { + private OmUtils() { } /** - * Retrieve the socket address that is used by KSM. + * Retrieve the socket address that is used by OM. * @param conf * @return Target InetSocketAddress for the SCM service endpoint. */ - public static InetSocketAddress getKsmAddress( + public static InetSocketAddress getOmAddress( Configuration conf) { final Optional host = getHostNameFromConfigKeys(conf, - OZONE_KSM_ADDRESS_KEY); + OZONE_OM_ADDRESS_KEY); return NetUtils.createSocketAddr( - host.or(OZONE_KSM_BIND_HOST_DEFAULT) + ":" + - getKsmRpcPort(conf)); + host.or(OZONE_OM_BIND_HOST_DEFAULT) + ":" + + getOmRpcPort(conf)); } /** * Retrieve the socket address that should be used by clients to connect - * to KSM. + * to OM. * @param conf - * @return Target InetSocketAddress for the KSM service endpoint. + * @return Target InetSocketAddress for the OM service endpoint. */ - public static InetSocketAddress getKsmAddressForClients( + public static InetSocketAddress getOmAddressForClients( Configuration conf) { final Optional host = getHostNameFromConfigKeys(conf, - OZONE_KSM_ADDRESS_KEY); + OZONE_OM_ADDRESS_KEY); if (!host.isPresent()) { throw new IllegalArgumentException( - OZONE_KSM_ADDRESS_KEY + " must be defined. See" + + OZONE_OM_ADDRESS_KEY + " must be defined. See" + " https://wiki.apache.org/hadoop/Ozone#Configuration for" + " details on configuring Ozone."); } return NetUtils.createSocketAddr( - host.get() + ":" + getKsmRpcPort(conf)); + host.get() + ":" + getOmRpcPort(conf)); } - public static int getKsmRpcPort(Configuration conf) { + public static int getOmRpcPort(Configuration conf) { // If no port number is specified then we'll just try the defaultBindPort. final Optional port = getPortNumberFromConfigKeys(conf, - OZONE_KSM_ADDRESS_KEY); - return port.or(OZONE_KSM_PORT_DEFAULT); + OZONE_OM_ADDRESS_KEY); + return port.or(OZONE_OM_PORT_DEFAULT); } - public static int getKsmRestPort(Configuration conf) { + public static int getOmRestPort(Configuration conf) { // If no port number is specified then we'll just try the default // HTTP BindPort. final Optional port = - getPortNumberFromConfigKeys(conf, OZONE_KSM_HTTP_ADDRESS_KEY); - return port.or(OZONE_KSM_HTTP_BIND_PORT_DEFAULT); + getPortNumberFromConfigKeys(conf, OZONE_OM_HTTP_ADDRESS_KEY); + return port.or(OZONE_OM_HTTP_BIND_PORT_DEFAULT); } } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/freon/OzoneGetConf.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/freon/OzoneGetConf.java index d5f9093128f..ffbca6a2b5d 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/freon/OzoneGetConf.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/freon/OzoneGetConf.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.ozone.KsmUtils; +import org.apache.hadoop.ozone.OmUtils; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Tool; @@ -53,8 +53,8 @@ public class OzoneGetConf extends Configured implements Tool { EXCLUDE_FILE("-excludeFile", "gets the exclude file path that defines the datanodes " + "that need to decommissioned."), - KEYSPACEMANAGER("-keyspacemanagers", - "gets list of ozone key space manager nodes in the cluster"), + OZONEMANAGER("-ozonemanagers", + "gets list of Ozone Manager nodes in the cluster"), STORAGECONTAINERMANAGER("-storagecontainermanagers", "gets list of ozone storage container manager nodes in the cluster"), CONFKEY("-confKey [key]", "gets a specific key from the configuration"); @@ -63,8 +63,8 @@ public class OzoneGetConf extends Configured implements Tool { static { HANDLERS = new HashMap(); - HANDLERS.put(StringUtils.toLowerCase(KEYSPACEMANAGER.getName()), - new KeySpaceManagersCommandHandler()); + HANDLERS.put(StringUtils.toLowerCase(OZONEMANAGER.getName()), + new OzoneManagersCommandHandler()); HANDLERS.put(StringUtils.toLowerCase(STORAGECONTAINERMANAGER.getName()), new StorageContainerManagersCommandHandler()); HANDLERS.put(StringUtils.toLowerCase(CONFKEY.getName()), @@ -245,13 +245,13 @@ public class OzoneGetConf extends Configured implements Tool { } /** - * Handler for {@link Command#KEYSPACEMANAGER}. + * Handler for {@link Command#OZONEMANAGER}. */ - static class KeySpaceManagersCommandHandler extends CommandHandler { + static class OzoneManagersCommandHandler extends CommandHandler { @Override public int doWorkInternal(OzoneGetConf tool, String[] args) throws IOException { - tool.printOut(KsmUtils.getKsmAddress(tool.getConf()).getHostName()); + tool.printOut(OmUtils.getOmAddress(tool.getConf()).getHostName()); return 0; } } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java deleted file mode 100644 index 75cf613ba60..00000000000 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java +++ /dev/null @@ -1,81 +0,0 @@ -/** - * 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.ozone.ksm; - -import org.apache.hadoop.ozone.OzoneAcl; -/** - * KSM Constants. - */ -public final class KSMConfigKeys { - /** - * Never constructed. - */ - private KSMConfigKeys() { - } - - - public static final String OZONE_KSM_HANDLER_COUNT_KEY = - "ozone.ksm.handler.count.key"; - public static final int OZONE_KSM_HANDLER_COUNT_DEFAULT = 20; - - public static final String OZONE_KSM_ADDRESS_KEY = - "ozone.ksm.address"; - public static final String OZONE_KSM_BIND_HOST_DEFAULT = - "0.0.0.0"; - public static final int OZONE_KSM_PORT_DEFAULT = 9862; - - public static final String OZONE_KSM_HTTP_ENABLED_KEY = - "ozone.ksm.http.enabled"; - public static final String OZONE_KSM_HTTP_BIND_HOST_KEY = - "ozone.ksm.http-bind-host"; - public static final String OZONE_KSM_HTTPS_BIND_HOST_KEY = - "ozone.ksm.https-bind-host"; - public static final String OZONE_KSM_HTTP_ADDRESS_KEY = - "ozone.ksm.http-address"; - public static final String OZONE_KSM_HTTPS_ADDRESS_KEY = - "ozone.ksm.https-address"; - public static final String OZONE_KSM_KEYTAB_FILE = - "ozone.ksm.keytab.file"; - public static final String OZONE_KSM_HTTP_BIND_HOST_DEFAULT = "0.0.0.0"; - public static final int OZONE_KSM_HTTP_BIND_PORT_DEFAULT = 9874; - public static final int OZONE_KSM_HTTPS_BIND_PORT_DEFAULT = 9875; - - // LevelDB cache file uses an off-heap cache in LevelDB of 128 MB. - public static final String OZONE_KSM_DB_CACHE_SIZE_MB = - "ozone.ksm.db.cache.size.mb"; - public static final int OZONE_KSM_DB_CACHE_SIZE_DEFAULT = 128; - - public static final String OZONE_KSM_USER_MAX_VOLUME = - "ozone.ksm.user.max.volume"; - public static final int OZONE_KSM_USER_MAX_VOLUME_DEFAULT = 1024; - - // KSM Default user/group permissions - public static final String OZONE_KSM_USER_RIGHTS = - "ozone.ksm.user.rights"; - public static final OzoneAcl.OzoneACLRights OZONE_KSM_USER_RIGHTS_DEFAULT = - OzoneAcl.OzoneACLRights.READ_WRITE; - - public static final String OZONE_KSM_GROUP_RIGHTS = - "ozone.ksm.group.rights"; - public static final OzoneAcl.OzoneACLRights OZONE_KSM_GROUP_RIGHTS_DEFAULT = - OzoneAcl.OzoneACLRights.READ_WRITE; - - public static final String OZONE_KEY_DELETING_LIMIT_PER_TASK = - "ozone.key.deleting.limit.per.task"; - public static final int OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT = 1000; -} diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java new file mode 100644 index 00000000000..b9ca2966311 --- /dev/null +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java @@ -0,0 +1,81 @@ +/** + * 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.ozone.om; + +import org.apache.hadoop.ozone.OzoneAcl; +/** + * Ozone Manager Constants. + */ +public final class OMConfigKeys { + /** + * Never constructed. + */ + private OMConfigKeys() { + } + + + public static final String OZONE_OM_HANDLER_COUNT_KEY = + "ozone.om.handler.count.key"; + public static final int OZONE_OM_HANDLER_COUNT_DEFAULT = 20; + + public static final String OZONE_OM_ADDRESS_KEY = + "ozone.om.address"; + public static final String OZONE_OM_BIND_HOST_DEFAULT = + "0.0.0.0"; + public static final int OZONE_OM_PORT_DEFAULT = 9862; + + public static final String OZONE_OM_HTTP_ENABLED_KEY = + "ozone.om.http.enabled"; + public static final String OZONE_OM_HTTP_BIND_HOST_KEY = + "ozone.om.http-bind-host"; + public static final String OZONE_OM_HTTPS_BIND_HOST_KEY = + "ozone.om.https-bind-host"; + public static final String OZONE_OM_HTTP_ADDRESS_KEY = + "ozone.om.http-address"; + public static final String OZONE_OM_HTTPS_ADDRESS_KEY = + "ozone.om.https-address"; + public static final String OZONE_OM_KEYTAB_FILE = + "ozone.om.keytab.file"; + public static final String OZONE_OM_HTTP_BIND_HOST_DEFAULT = "0.0.0.0"; + public static final int OZONE_OM_HTTP_BIND_PORT_DEFAULT = 9874; + public static final int OZONE_OM_HTTPS_BIND_PORT_DEFAULT = 9875; + + // LevelDB cache file uses an off-heap cache in LevelDB of 128 MB. + public static final String OZONE_OM_DB_CACHE_SIZE_MB = + "ozone.om.db.cache.size.mb"; + public static final int OZONE_OM_DB_CACHE_SIZE_DEFAULT = 128; + + public static final String OZONE_OM_USER_MAX_VOLUME = + "ozone.om.user.max.volume"; + public static final int OZONE_OM_USER_MAX_VOLUME_DEFAULT = 1024; + + // OM Default user/group permissions + public static final String OZONE_OM_USER_RIGHTS = + "ozone.om.user.rights"; + public static final OzoneAcl.OzoneACLRights OZONE_OM_USER_RIGHTS_DEFAULT = + OzoneAcl.OzoneACLRights.READ_WRITE; + + public static final String OZONE_OM_GROUP_RIGHTS = + "ozone.om.group.rights"; + public static final OzoneAcl.OzoneACLRights OZONE_OM_GROUP_RIGHTS_DEFAULT = + OzoneAcl.OzoneACLRights.READ_WRITE; + + public static final String OZONE_KEY_DELETING_LIMIT_PER_TASK = + "ozone.key.deleting.limit.per.task"; + public static final int OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT = 1000; +} diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java similarity index 81% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketArgs.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java index 1211b50c5cf..6aabfef6b45 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketArgs.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.helpers; +package org.apache.hadoop.ozone.om.helpers; import java.util.List; import java.util.stream.Collectors; @@ -25,13 +25,13 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.BucketArgs; -import org.apache.hadoop.ozone.protocolPB.KSMPBHelper; + .OzoneManagerProtocolProtos.BucketArgs; +import org.apache.hadoop.ozone.protocolPB.OMPBHelper; /** * A class that encapsulates Bucket Arguments. */ -public final class KsmBucketArgs { +public final class OmBucketArgs { /** * Name of the volume in which the bucket belongs to. */ @@ -67,9 +67,9 @@ public final class KsmBucketArgs { * @param isVersionEnabled - Bucket version flag. * @param storageType - Storage type to be used. */ - private KsmBucketArgs(String volumeName, String bucketName, - List addAcls, List removeAcls, - Boolean isVersionEnabled, StorageType storageType) { + private OmBucketArgs(String volumeName, String bucketName, + List addAcls, List removeAcls, + Boolean isVersionEnabled, StorageType storageType) { this.volumeName = volumeName; this.bucketName = bucketName; this.addAcls = addAcls; @@ -127,7 +127,7 @@ public final class KsmBucketArgs { } /** - * Returns new builder class that builds a KsmBucketArgs. + * Returns new builder class that builds a OmBucketArgs. * * @return Builder */ @@ -136,7 +136,7 @@ public final class KsmBucketArgs { } /** - * Builder for KsmBucketArgs. + * Builder for OmBucketArgs. */ public static class Builder { private String volumeName; @@ -177,19 +177,19 @@ public final class KsmBucketArgs { } /** - * Constructs the KsmBucketArgs. - * @return instance of KsmBucketArgs. + * Constructs the OmBucketArgs. + * @return instance of OmBucketArgs. */ - public KsmBucketArgs build() { + public OmBucketArgs build() { Preconditions.checkNotNull(volumeName); Preconditions.checkNotNull(bucketName); - return new KsmBucketArgs(volumeName, bucketName, addAcls, + return new OmBucketArgs(volumeName, bucketName, addAcls, removeAcls, isVersionEnabled, storageType); } } /** - * Creates BucketArgs protobuf from KsmBucketArgs. + * Creates BucketArgs protobuf from OmBucketArgs. */ public BucketArgs getProtobuf() { BucketArgs.Builder builder = BucketArgs.newBuilder(); @@ -197,11 +197,11 @@ public final class KsmBucketArgs { .setBucketName(bucketName); if(addAcls != null && !addAcls.isEmpty()) { builder.addAllAddAcls(addAcls.stream().map( - KSMPBHelper::convertOzoneAcl).collect(Collectors.toList())); + OMPBHelper::convertOzoneAcl).collect(Collectors.toList())); } if(removeAcls != null && !removeAcls.isEmpty()) { builder.addAllRemoveAcls(removeAcls.stream().map( - KSMPBHelper::convertOzoneAcl).collect(Collectors.toList())); + OMPBHelper::convertOzoneAcl).collect(Collectors.toList())); } if(isVersionEnabled != null) { builder.setIsVersionEnabled(isVersionEnabled); @@ -214,20 +214,20 @@ public final class KsmBucketArgs { } /** - * Parses BucketInfo protobuf and creates KsmBucketArgs. + * Parses BucketInfo protobuf and creates OmBucketArgs. * @param bucketArgs - * @return instance of KsmBucketArgs + * @return instance of OmBucketArgs */ - public static KsmBucketArgs getFromProtobuf(BucketArgs bucketArgs) { - return new KsmBucketArgs(bucketArgs.getVolumeName(), + public static OmBucketArgs getFromProtobuf(BucketArgs bucketArgs) { + return new OmBucketArgs(bucketArgs.getVolumeName(), bucketArgs.getBucketName(), bucketArgs.getAddAclsList().stream().map( - KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()), + OMPBHelper::convertOzoneAcl).collect(Collectors.toList()), bucketArgs.getRemoveAclsList().stream().map( - KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()), + OMPBHelper::convertOzoneAcl).collect(Collectors.toList()), bucketArgs.hasIsVersionEnabled() ? bucketArgs.getIsVersionEnabled() : null, bucketArgs.hasStorageType() ? PBHelperClient.convertStorageType( bucketArgs.getStorageType()) : null); } -} \ No newline at end of file +} diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java similarity index 83% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketInfo.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java index a49137a7cfb..bf5abddc431 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketInfo.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.helpers; +package org.apache.hadoop.ozone.om.helpers; import com.google.common.base.Preconditions; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.BucketInfo; -import org.apache.hadoop.ozone.protocolPB.KSMPBHelper; + .OzoneManagerProtocolProtos.BucketInfo; +import org.apache.hadoop.ozone.protocolPB.OMPBHelper; import java.util.LinkedList; import java.util.List; @@ -32,7 +32,7 @@ import java.util.stream.Collectors; /** * A class that encapsulates Bucket Info. */ -public final class KsmBucketInfo { +public final class OmBucketInfo { /** * Name of the volume in which the bucket belongs to. */ @@ -68,9 +68,9 @@ public final class KsmBucketInfo { * @param storageType - Storage type to be used. * @param creationTime - Bucket creation time. */ - private KsmBucketInfo(String volumeName, String bucketName, - List acls, boolean isVersionEnabled, - StorageType storageType, long creationTime) { + private OmBucketInfo(String volumeName, String bucketName, + List acls, boolean isVersionEnabled, + StorageType storageType, long creationTime) { this.volumeName = volumeName; this.bucketName = bucketName; this.acls = acls; @@ -129,7 +129,7 @@ public final class KsmBucketInfo { } /** - * Returns new builder class that builds a KsmBucketInfo. + * Returns new builder class that builds a OmBucketInfo. * * @return Builder */ @@ -138,7 +138,7 @@ public final class KsmBucketInfo { } /** - * Builder for KsmBucketInfo. + * Builder for OmBucketInfo. */ public static class Builder { private String volumeName; @@ -186,30 +186,30 @@ public final class KsmBucketInfo { } /** - * Constructs the KsmBucketInfo. - * @return instance of KsmBucketInfo. + * Constructs the OmBucketInfo. + * @return instance of OmBucketInfo. */ - public KsmBucketInfo build() { + public OmBucketInfo build() { Preconditions.checkNotNull(volumeName); Preconditions.checkNotNull(bucketName); Preconditions.checkNotNull(acls); Preconditions.checkNotNull(isVersionEnabled); Preconditions.checkNotNull(storageType); - return new KsmBucketInfo(volumeName, bucketName, acls, + return new OmBucketInfo(volumeName, bucketName, acls, isVersionEnabled, storageType, creationTime); } } /** - * Creates BucketInfo protobuf from KsmBucketInfo. + * Creates BucketInfo protobuf from OmBucketInfo. */ public BucketInfo getProtobuf() { return BucketInfo.newBuilder() .setVolumeName(volumeName) .setBucketName(bucketName) .addAllAcls(acls.stream().map( - KSMPBHelper::convertOzoneAcl).collect(Collectors.toList())) + OMPBHelper::convertOzoneAcl).collect(Collectors.toList())) .setIsVersionEnabled(isVersionEnabled) .setStorageType(PBHelperClient.convertStorageType( storageType)) @@ -218,18 +218,18 @@ public final class KsmBucketInfo { } /** - * Parses BucketInfo protobuf and creates KsmBucketInfo. + * Parses BucketInfo protobuf and creates OmBucketInfo. * @param bucketInfo - * @return instance of KsmBucketInfo + * @return instance of OmBucketInfo */ - public static KsmBucketInfo getFromProtobuf(BucketInfo bucketInfo) { - return new KsmBucketInfo( + public static OmBucketInfo getFromProtobuf(BucketInfo bucketInfo) { + return new OmBucketInfo( bucketInfo.getVolumeName(), bucketInfo.getBucketName(), bucketInfo.getAclsList().stream().map( - KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()), + OMPBHelper::convertOzoneAcl).collect(Collectors.toList()), bucketInfo.getIsVersionEnabled(), PBHelperClient.convertStorageType( bucketInfo.getStorageType()), bucketInfo.getCreationTime()); } -} \ No newline at end of file +} diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java similarity index 88% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyArgs.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java index cd17e28b9ae..1f8ed5fb1e7 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyArgs.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.helpers; +package org.apache.hadoop.ozone.om.helpers; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; @@ -23,7 +23,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; * Args for key. Client use this to specify key's attributes on key creation * (putKey()). */ -public final class KsmKeyArgs { +public final class OmKeyArgs { private final String volumeName; private final String bucketName; private final String keyName; @@ -31,8 +31,8 @@ public final class KsmKeyArgs { private final ReplicationType type; private final ReplicationFactor factor; - private KsmKeyArgs(String volumeName, String bucketName, String keyName, - long dataSize, ReplicationType type, ReplicationFactor factor) { + private OmKeyArgs(String volumeName, String bucketName, String keyName, + long dataSize, ReplicationType type, ReplicationFactor factor) { this.volumeName = volumeName; this.bucketName = bucketName; this.keyName = keyName; @@ -70,7 +70,7 @@ public final class KsmKeyArgs { } /** - * Builder class of KsmKeyArgs. + * Builder class of OmKeyArgs. */ public static class Builder { private String volumeName; @@ -111,8 +111,8 @@ public final class KsmKeyArgs { return this; } - public KsmKeyArgs build() { - return new KsmKeyArgs(volumeName, bucketName, keyName, dataSize, + public OmKeyArgs build() { + return new OmKeyArgs(volumeName, bucketName, keyName, dataSize, type, factor); } } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java similarity index 79% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyInfo.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java index 5d6e63367a3..05c8d45fbf9 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyInfo.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.helpers; +package org.apache.hadoop.ozone.om.helpers; import com.google.common.base.Preconditions; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo; import org.apache.hadoop.util.Time; import java.io.IOException; @@ -28,36 +28,36 @@ import java.util.stream.Collectors; /** * Args for key block. The block instance for the key requested in putKey. - * This is returned from KSM to client, and client use class to talk to - * datanode. Also, this is the metadata written to ksm.db on server side. + * This is returned from OM to client, and client use class to talk to + * datanode. Also, this is the metadata written to om.db on server side. */ -public final class KsmKeyInfo { +public final class OmKeyInfo { private final String volumeName; private final String bucketName; // name of key client specified private String keyName; private long dataSize; - private List keyLocationVersions; + private List keyLocationVersions; private final long creationTime; private long modificationTime; private HddsProtos.ReplicationType type; private HddsProtos.ReplicationFactor factor; - private KsmKeyInfo(String volumeName, String bucketName, String keyName, - List versions, long dataSize, - long creationTime, long modificationTime, HddsProtos.ReplicationType type, - HddsProtos.ReplicationFactor factor) { + private OmKeyInfo(String volumeName, String bucketName, String keyName, + List versions, long dataSize, + long creationTime, long modificationTime, HddsProtos.ReplicationType type, + HddsProtos.ReplicationFactor factor) { this.volumeName = volumeName; this.bucketName = bucketName; this.keyName = keyName; this.dataSize = dataSize; // it is important that the versions are ordered from old to new. - // Do this sanity check when versions got loaded on creating KsmKeyInfo. + // Do this sanity check when versions got loaded on creating OmKeyInfo. // TODO : this is not necessary, here only because versioning is still a // work in-progress, remove this following check when versioning is // complete and prove correctly functioning long currentVersion = -1; - for (KsmKeyLocationInfoGroup version : versions) { + for (OmKeyLocationInfoGroup version : versions) { Preconditions.checkArgument( currentVersion + 1 == version.getVersion()); currentVersion = version.getVersion(); @@ -101,13 +101,13 @@ public final class KsmKeyInfo { this.dataSize = size; } - public synchronized KsmKeyLocationInfoGroup getLatestVersionLocations() + public synchronized OmKeyLocationInfoGroup getLatestVersionLocations() throws IOException { return keyLocationVersions.size() == 0? null : keyLocationVersions.get(keyLocationVersions.size() - 1); } - public List getKeyLocationVersions() { + public List getKeyLocationVersions() { return keyLocationVersions; } @@ -123,11 +123,11 @@ public final class KsmKeyInfo { * @throws IOException */ public synchronized void appendNewBlocks( - List newLocationList) throws IOException { + List newLocationList) throws IOException { if (keyLocationVersions.size() == 0) { throw new IOException("Appending new block, but no version exist"); } - KsmKeyLocationInfoGroup currentLatestVersion = + OmKeyLocationInfoGroup currentLatestVersion = keyLocationVersions.get(keyLocationVersions.size() - 1); currentLatestVersion.appendNewBlocks(newLocationList); setModificationTime(Time.now()); @@ -141,18 +141,18 @@ public final class KsmKeyInfo { * @throws IOException */ public synchronized long addNewVersion( - List newLocationList) throws IOException { + List newLocationList) throws IOException { long latestVersionNum; if (keyLocationVersions.size() == 0) { // no version exist, these blocks are the very first version. - keyLocationVersions.add(new KsmKeyLocationInfoGroup(0, newLocationList)); + keyLocationVersions.add(new OmKeyLocationInfoGroup(0, newLocationList)); latestVersionNum = 0; } else { // it is important that the new version are always at the tail of the list - KsmKeyLocationInfoGroup currentLatestVersion = + OmKeyLocationInfoGroup currentLatestVersion = keyLocationVersions.get(keyLocationVersions.size() - 1); // the new version is created based on the current latest version - KsmKeyLocationInfoGroup newVersion = + OmKeyLocationInfoGroup newVersion = currentLatestVersion.generateNextVersion(newLocationList); keyLocationVersions.add(newVersion); latestVersionNum = newVersion.getVersion(); @@ -174,14 +174,14 @@ public final class KsmKeyInfo { } /** - * Builder of KsmKeyInfo. + * Builder of OmKeyInfo. */ public static class Builder { private String volumeName; private String bucketName; private String keyName; private long dataSize; - private List ksmKeyLocationInfoGroups; + private List omKeyLocationInfoGroups; private long creationTime; private long modificationTime; private HddsProtos.ReplicationType type; @@ -202,9 +202,9 @@ public final class KsmKeyInfo { return this; } - public Builder setKsmKeyLocationInfos( - List ksmKeyLocationInfoList) { - this.ksmKeyLocationInfoGroups = ksmKeyLocationInfoList; + public Builder setOmKeyLocationInfos( + List omKeyLocationInfoList) { + this.omKeyLocationInfoGroups = omKeyLocationInfoList; return this; } @@ -233,9 +233,9 @@ public final class KsmKeyInfo { return this; } - public KsmKeyInfo build() { - return new KsmKeyInfo( - volumeName, bucketName, keyName, ksmKeyLocationInfoGroups, + public OmKeyInfo build() { + return new OmKeyInfo( + volumeName, bucketName, keyName, omKeyLocationInfoGroups, dataSize, creationTime, modificationTime, type, factor); } } @@ -251,7 +251,7 @@ public final class KsmKeyInfo { .setFactor(factor) .setType(type) .addAllKeyLocationList(keyLocationVersions.stream() - .map(KsmKeyLocationInfoGroup::getProtobuf) + .map(OmKeyLocationInfoGroup::getProtobuf) .collect(Collectors.toList())) .setLatestVersion(latestVersion) .setCreationTime(creationTime) @@ -259,13 +259,13 @@ public final class KsmKeyInfo { .build(); } - public static KsmKeyInfo getFromProtobuf(KeyInfo keyInfo) { - return new KsmKeyInfo( + public static OmKeyInfo getFromProtobuf(KeyInfo keyInfo) { + return new OmKeyInfo( keyInfo.getVolumeName(), keyInfo.getBucketName(), keyInfo.getKeyName(), keyInfo.getKeyLocationListList().stream() - .map(KsmKeyLocationInfoGroup::getFromProtobuf) + .map(OmKeyLocationInfoGroup::getFromProtobuf) .collect(Collectors.toList()), keyInfo.getDataSize(), keyInfo.getCreationTime(), diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java similarity index 85% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java index 45feda0b7ca..3f6666df814 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java @@ -14,16 +14,16 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm.helpers; +package org.apache.hadoop.ozone.om.helpers; import org.apache.hadoop.hdds.client.BlockID; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyLocation; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation; /** * One key can be too huge to fit in one container. In which case it gets split * into a number of subkeys. This class represents one such subkey instance. */ -public final class KsmKeyLocationInfo { +public final class OmKeyLocationInfo { private final BlockID blockID; private final boolean shouldCreateContainer; // the id of this subkey in all the subkeys. @@ -32,8 +32,8 @@ public final class KsmKeyLocationInfo { // the version number indicating when this block was added private long createVersion; - private KsmKeyLocationInfo(BlockID blockID, boolean shouldCreateContainer, - long length, long offset) { + private OmKeyLocationInfo(BlockID blockID, boolean shouldCreateContainer, + long length, long offset) { this.blockID = blockID; this.shouldCreateContainer = shouldCreateContainer; this.length = length; @@ -73,7 +73,7 @@ public final class KsmKeyLocationInfo { } /** - * Builder of KsmKeyLocationInfo. + * Builder of OmKeyLocationInfo. */ public static class Builder { private BlockID blockID; @@ -101,8 +101,8 @@ public final class KsmKeyLocationInfo { return this; } - public KsmKeyLocationInfo build() { - return new KsmKeyLocationInfo(blockID, + public OmKeyLocationInfo build() { + return new OmKeyLocationInfo(blockID, shouldCreateContainer, length, offset); } } @@ -117,8 +117,8 @@ public final class KsmKeyLocationInfo { .build(); } - public static KsmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) { - KsmKeyLocationInfo info = new KsmKeyLocationInfo( + public static OmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) { + OmKeyLocationInfo info = new OmKeyLocationInfo( BlockID.getFromProtobuf(keyLocation.getBlockID()), keyLocation.getShouldCreateContainer(), keyLocation.getLength(), diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfoGroup.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfoGroup.java similarity index 70% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfoGroup.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfoGroup.java index 0facf3ca41d..8bdcee3803c 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfoGroup.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfoGroup.java @@ -14,9 +14,9 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm.helpers; +package org.apache.hadoop.ozone.om.helpers; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyLocationList; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocationList; import java.io.IOException; import java.util.ArrayList; @@ -27,12 +27,12 @@ import java.util.stream.Collectors; * A list of key locations. This class represents one single version of the * blocks of a key. */ -public class KsmKeyLocationInfoGroup { +public class OmKeyLocationInfoGroup { private final long version; - private final List locationList; + private final List locationList; - public KsmKeyLocationInfoGroup(long version, - List locations) { + public OmKeyLocationInfoGroup(long version, + List locations) { this.version = version; this.locationList = locations; } @@ -42,8 +42,8 @@ public class KsmKeyLocationInfoGroup { * * @return the list of blocks that are created in the latest version. */ - public List getBlocksLatestVersionOnly() { - List list = new ArrayList<>(); + public List getBlocksLatestVersionOnly() { + List list = new ArrayList<>(); locationList.stream().filter(x -> x.getCreateVersion() == version) .forEach(list::add); return list; @@ -53,7 +53,7 @@ public class KsmKeyLocationInfoGroup { return version; } - public List getLocationList() { + public List getLocationList() { return locationList; } @@ -61,17 +61,17 @@ public class KsmKeyLocationInfoGroup { return KeyLocationList.newBuilder() .setVersion(version) .addAllKeyLocations( - locationList.stream().map(KsmKeyLocationInfo::getProtobuf) + locationList.stream().map(OmKeyLocationInfo::getProtobuf) .collect(Collectors.toList())) .build(); } - public static KsmKeyLocationInfoGroup getFromProtobuf( + public static OmKeyLocationInfoGroup getFromProtobuf( KeyLocationList keyLocationList) { - return new KsmKeyLocationInfoGroup( + return new OmKeyLocationInfoGroup( keyLocationList.getVersion(), keyLocationList.getKeyLocationsList().stream() - .map(KsmKeyLocationInfo::getFromProtobuf) + .map(OmKeyLocationInfo::getFromProtobuf) .collect(Collectors.toList())); } @@ -82,25 +82,25 @@ public class KsmKeyLocationInfoGroup { * @param newLocationList a list of new location to be added. * @return */ - KsmKeyLocationInfoGroup generateNextVersion( - List newLocationList) throws IOException { + OmKeyLocationInfoGroup generateNextVersion( + List newLocationList) throws IOException { // TODO : revisit if we can do this method more efficiently // one potential inefficiency here is that later version always include // older ones. e.g. v1 has B1, then v2, v3...will all have B1 and only add // more - List newList = new ArrayList<>(); + List newList = new ArrayList<>(); newList.addAll(locationList); - for (KsmKeyLocationInfo newInfo : newLocationList) { + for (OmKeyLocationInfo newInfo : newLocationList) { // all these new blocks will have addVersion of current version + 1 newInfo.setCreateVersion(version + 1); newList.add(newInfo); } - return new KsmKeyLocationInfoGroup(version + 1, newList); + return new OmKeyLocationInfoGroup(version + 1, newList); } - void appendNewBlocks(List newLocationList) + void appendNewBlocks(List newLocationList) throws IOException { - for (KsmKeyLocationInfo info : newLocationList) { + for (OmKeyLocationInfo info : newLocationList) { info.setCreateVersion(version); locationList.add(info); } @@ -110,7 +110,7 @@ public class KsmKeyLocationInfoGroup { public String toString() { StringBuilder sb = new StringBuilder(); sb.append("version:").append(version).append(" "); - for (KsmKeyLocationInfo kli : locationList) { + for (OmKeyLocationInfo kli : locationList) { sb.append(kli.getLocalID()).append(" || "); } return sb.toString(); diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmOzoneAclMap.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmOzoneAclMap.java similarity index 89% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmOzoneAclMap.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmOzoneAclMap.java index 7d9efad15a5..de75a05e9d9 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmOzoneAclMap.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmOzoneAclMap.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.helpers; +package org.apache.hadoop.ozone.om.helpers; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo; + .OzoneManagerProtocolProtos.OzoneAclInfo; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclRights; + .OzoneManagerProtocolProtos.OzoneAclInfo.OzoneAclRights; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclType; + .OzoneManagerProtocolProtos.OzoneAclInfo.OzoneAclType; import java.util.List; import java.util.LinkedList; @@ -34,11 +34,11 @@ import java.util.HashMap; /** * This helper class keeps a map of all user and their permissions. */ -public class KsmOzoneAclMap { +public class OmOzoneAclMap { // per Acl Type user:rights map private ArrayList> aclMaps; - KsmOzoneAclMap() { + OmOzoneAclMap() { aclMaps = new ArrayList<>(); for (OzoneAclType aclType : OzoneAclType.values()) { aclMaps.add(aclType.ordinal(), new HashMap<>()); @@ -99,9 +99,9 @@ public class KsmOzoneAclMap { } // Create map from list of OzoneAclInfos - public static KsmOzoneAclMap ozoneAclGetFromProtobuf( + public static OmOzoneAclMap ozoneAclGetFromProtobuf( List aclList) { - KsmOzoneAclMap aclMap = new KsmOzoneAclMap(); + OmOzoneAclMap aclMap = new OmOzoneAclMap(); for (OzoneAclInfo acl : aclList) { aclMap.addAcl(acl); } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java similarity index 83% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java index 6b42c279a03..c8b59b682db 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.helpers; +package org.apache.hadoop.ozone.om.helpers; import com.google.common.base.Preconditions; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo; + .OzoneManagerProtocolProtos.OzoneAclInfo; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.VolumeInfo; + .OzoneManagerProtocolProtos.VolumeInfo; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue; import java.io.IOException; @@ -33,16 +33,16 @@ import java.util.stream.Collectors; /** - * A class that encapsulates the KsmVolumeArgs Args. + * A class that encapsulates the OmVolumeArgs Args. */ -public final class KsmVolumeArgs { +public final class OmVolumeArgs { private final String adminName; private final String ownerName; private final String volume; private final long creationTime; private final long quotaInBytes; private final Map keyValueMap; - private final KsmOzoneAclMap aclMap; + private final OmOzoneAclMap aclMap; /** * Private constructor, constructed via builder. @@ -54,9 +54,9 @@ public final class KsmVolumeArgs { * @param aclMap - User to access rights map. * @param creationTime - Volume creation time. */ - private KsmVolumeArgs(String adminName, String ownerName, String volume, - long quotaInBytes, Map keyValueMap, - KsmOzoneAclMap aclMap, long creationTime) { + private OmVolumeArgs(String adminName, String ownerName, String volume, + long quotaInBytes, Map keyValueMap, + OmOzoneAclMap aclMap, long creationTime) { this.adminName = adminName; this.ownerName = ownerName; this.volume = volume; @@ -110,11 +110,11 @@ public final class KsmVolumeArgs { return keyValueMap; } - public KsmOzoneAclMap getAclMap() { + public OmOzoneAclMap getAclMap() { return aclMap; } /** - * Returns new builder class that builds a KsmVolumeArgs. + * Returns new builder class that builds a OmVolumeArgs. * * @return Builder */ @@ -123,7 +123,7 @@ public final class KsmVolumeArgs { } /** - * Builder for KsmVolumeArgs. + * Builder for OmVolumeArgs. */ public static class Builder { private String adminName; @@ -132,14 +132,14 @@ public final class KsmVolumeArgs { private long creationTime; private long quotaInBytes; private Map keyValueMap; - private KsmOzoneAclMap aclMap; + private OmOzoneAclMap aclMap; /** * Constructs a builder. */ Builder() { keyValueMap = new HashMap<>(); - aclMap = new KsmOzoneAclMap(); + aclMap = new OmOzoneAclMap(); } public Builder setAdminName(String admin) { @@ -181,11 +181,11 @@ public final class KsmVolumeArgs { * Constructs a CreateVolumeArgument. * @return CreateVolumeArgs. */ - public KsmVolumeArgs build() { + public OmVolumeArgs build() { Preconditions.checkNotNull(adminName); Preconditions.checkNotNull(ownerName); Preconditions.checkNotNull(volume); - return new KsmVolumeArgs(adminName, ownerName, volume, quotaInBytes, + return new OmVolumeArgs(adminName, ownerName, volume, quotaInBytes, keyValueMap, aclMap, creationTime); } } @@ -209,14 +209,14 @@ public final class KsmVolumeArgs { .build(); } - public static KsmVolumeArgs getFromProtobuf(VolumeInfo volInfo) { + public static OmVolumeArgs getFromProtobuf(VolumeInfo volInfo) { Map kvMap = volInfo.getMetadataList().stream() .collect(Collectors.toMap(KeyValue::getKey, KeyValue::getValue)); - KsmOzoneAclMap aclMap = - KsmOzoneAclMap.ozoneAclGetFromProtobuf(volInfo.getVolumeAclsList()); + OmOzoneAclMap aclMap = + OmOzoneAclMap.ozoneAclGetFromProtobuf(volInfo.getVolumeAclsList()); - return new KsmVolumeArgs(volInfo.getAdminName(), volInfo.getOwnerName(), + return new OmVolumeArgs(volInfo.getAdminName(), volInfo.getOwnerName(), volInfo.getVolume(), volInfo.getQuotaInBytes(), kvMap, aclMap, volInfo.getCreationTime()); } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/OpenKeySession.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java similarity index 89% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/OpenKeySession.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java index c19c04b1d83..bc364e665ff 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/OpenKeySession.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.helpers; +package org.apache.hadoop.ozone.om.helpers; /** * This class represents a open key "session". A session here means a key is @@ -24,13 +24,13 @@ package org.apache.hadoop.ozone.ksm.helpers; */ public class OpenKeySession { private final int id; - private final KsmKeyInfo keyInfo; + private final OmKeyInfo keyInfo; // the version of the key when it is being opened in this session. // a block that has a create version equals to open version means it will // be committed only when this open session is closed. private long openVersion; - public OpenKeySession(int id, KsmKeyInfo info, long version) { + public OpenKeySession(int id, OmKeyInfo info, long version) { this.id = id; this.keyInfo = info; this.openVersion = version; @@ -40,7 +40,7 @@ public class OpenKeySession { return this.openVersion; } - public KsmKeyInfo getKeyInfo() { + public OmKeyInfo getKeyInfo() { return keyInfo; } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/ServiceInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/ServiceInfo.java similarity index 89% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/ServiceInfo.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/ServiceInfo.java index e07232d3885..9b03aefe1a8 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/ServiceInfo.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/ServiceInfo.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.helpers; +package org.apache.hadoop.ozone.om.helpers; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -25,8 +25,8 @@ import com.fasterxml.jackson.databind.ObjectReader; import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.base.Preconditions; import org.apache.hadoop.ozone.client.rest.response.BucketInfo; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .ServicePort; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType; @@ -121,14 +121,14 @@ public final class ServiceInfo { } /** - * Converts {@link ServiceInfo} to KeySpaceManagerProtocolProtos.ServiceInfo. + * Converts {@link ServiceInfo} to OzoneManagerProtocolProtos.ServiceInfo. * - * @return KeySpaceManagerProtocolProtos.ServiceInfo + * @return OzoneManagerProtocolProtos.ServiceInfo */ @JsonIgnore - public KeySpaceManagerProtocolProtos.ServiceInfo getProtobuf() { - KeySpaceManagerProtocolProtos.ServiceInfo.Builder builder = - KeySpaceManagerProtocolProtos.ServiceInfo.newBuilder(); + public OzoneManagerProtocolProtos.ServiceInfo getProtobuf() { + OzoneManagerProtocolProtos.ServiceInfo.Builder builder = + OzoneManagerProtocolProtos.ServiceInfo.newBuilder(); builder.setNodeType(nodeType) .setHostname(hostname) .addAllServicePorts( @@ -143,13 +143,13 @@ public final class ServiceInfo { } /** - * Converts KeySpaceManagerProtocolProtos.ServiceInfo to {@link ServiceInfo}. + * Converts OzoneManagerProtocolProtos.ServiceInfo to {@link ServiceInfo}. * * @return {@link ServiceInfo} */ @JsonIgnore public static ServiceInfo getFromProtobuf( - KeySpaceManagerProtocolProtos.ServiceInfo serviceInfo) { + OzoneManagerProtocolProtos.ServiceInfo serviceInfo) { return new ServiceInfo(serviceInfo.getNodeType(), serviceInfo.getHostname(), serviceInfo.getServicePortsList()); diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/VolumeArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/VolumeArgs.java similarity index 98% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/VolumeArgs.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/VolumeArgs.java index 1a3d486d129..6fc7c8fcc53 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/VolumeArgs.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/VolumeArgs.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.helpers; +package org.apache.hadoop.ozone.om.helpers; import com.google.common.base.Preconditions; diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/package-info.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/package-info.java similarity index 94% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/package-info.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/package-info.java index ce627a5c394..b1211d8cb86 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/package-info.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/package-info.java @@ -15,4 +15,4 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.helpers; \ No newline at end of file +package org.apache.hadoop.ozone.om.helpers; \ No newline at end of file diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/package-info.java similarity index 94% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/package-info.java index 7698ee191ab..1744cffc134 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/package-info.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; /** - This package contains client side protocol library to communicate with KSM. + This package contains client side protocol library to communicate with OM. */ \ No newline at end of file diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/KeySpaceManagerProtocol.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java similarity index 76% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/KeySpaceManagerProtocol.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java index 54862d32412..b7a099d0286 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/KeySpaceManagerProtocol.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java @@ -15,32 +15,32 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.protocol; +package org.apache.hadoop.ozone.om.protocol; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; -import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession; -import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo; +import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; +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.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo; + .OzoneManagerProtocolProtos.OzoneAclInfo; import java.io.IOException; import java.util.List; /** - * Protocol to talk to KSM. + * Protocol to talk to OM. */ -public interface KeySpaceManagerProtocol { +public interface OzoneManagerProtocol { /** * Creates a volume. * @param args - Arguments to create Volume. * @throws IOException */ - void createVolume(KsmVolumeArgs args) throws IOException; + void createVolume(OmVolumeArgs args) throws IOException; /** * Changes the owner of a volume. @@ -75,7 +75,7 @@ public interface KeySpaceManagerProtocol { * @return VolumeArgs or exception is thrown. * @throws IOException */ - KsmVolumeArgs getVolumeInfo(String volume) throws IOException; + OmVolumeArgs getVolumeInfo(String volume) throws IOException; /** * Deletes an existing empty volume. @@ -93,7 +93,7 @@ public interface KeySpaceManagerProtocol { * @return List of Volumes. * @throws IOException */ - List listVolumeByUser(String userName, String prefix, String + List listVolumeByUser(String userName, String prefix, String prevKey, int maxKeys) throws IOException; /** @@ -104,7 +104,7 @@ public interface KeySpaceManagerProtocol { * @return List of Volumes. * @throws IOException */ - List listAllVolumes(String prefix, String + List listAllVolumes(String prefix, String prevKey, int maxKeys) throws IOException; /** @@ -112,16 +112,16 @@ public interface KeySpaceManagerProtocol { * @param bucketInfo - BucketInfo to create Bucket. * @throws IOException */ - void createBucket(KsmBucketInfo bucketInfo) throws IOException; + void createBucket(OmBucketInfo bucketInfo) throws IOException; /** * Gets the bucket information. * @param volumeName - Volume name. * @param bucketName - Bucket name. - * @return KsmBucketInfo or exception is thrown. + * @return OmBucketInfo or exception is thrown. * @throws IOException */ - KsmBucketInfo getBucketInfo(String volumeName, String bucketName) + OmBucketInfo getBucketInfo(String volumeName, String bucketName) throws IOException; /** @@ -129,7 +129,7 @@ public interface KeySpaceManagerProtocol { * @param args - BucketArgs. * @throws IOException */ - void setBucketProperty(KsmBucketArgs args) throws IOException; + void setBucketProperty(OmBucketArgs args) throws IOException; /** * Open the given key and return an open key session. @@ -138,7 +138,7 @@ public interface KeySpaceManagerProtocol { * @return OpenKeySession instance that client uses to talk to container. * @throws IOException */ - OpenKeySession openKey(KsmKeyArgs args) throws IOException; + OpenKeySession openKey(OmKeyArgs args) throws IOException; /** * Commit a key. This will make the change from the client visible. The client @@ -148,7 +148,7 @@ public interface KeySpaceManagerProtocol { * @param clientID the client identification * @throws IOException */ - void commitKey(KsmKeyArgs args, int clientID) throws IOException; + void commitKey(OmKeyArgs args, int clientID) throws IOException; /** * Allocate a new block, it is assumed that the client is having an open key @@ -159,24 +159,24 @@ public interface KeySpaceManagerProtocol { * @return an allocated block * @throws IOException */ - KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID) + OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID) throws IOException; /** * Look up for the container of an existing key. * * @param args the args of the key. - * @return KsmKeyInfo instance that client uses to talk to container. + * @return OmKeyInfo instance that client uses to talk to container. * @throws IOException */ - KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException; + OmKeyInfo lookupKey(OmKeyArgs args) throws IOException; /** * Rename an existing key within a bucket * @param args the args of the key. * @param toKeyName New name to be used for the Key */ - void renameKey(KsmKeyArgs args, String toKeyName) throws IOException; + void renameKey(OmKeyArgs args, String toKeyName) throws IOException; /** * Deletes an existing key. @@ -184,7 +184,7 @@ public interface KeySpaceManagerProtocol { * @param args the args of the key. * @throws IOException */ - void deleteKey(KsmKeyArgs args) throws IOException; + void deleteKey(OmKeyArgs args) throws IOException; /** * Deletes an existing empty bucket from volume. @@ -195,7 +195,7 @@ public interface KeySpaceManagerProtocol { void deleteBucket(String volume, String bucket) throws IOException; /** - * Returns a list of buckets represented by {@link KsmBucketInfo} + * Returns a list of buckets represented by {@link OmBucketInfo} * in the given volume. Argument volumeName is required, others * are optional. * @@ -213,12 +213,12 @@ public interface KeySpaceManagerProtocol { * @return a list of buckets. * @throws IOException */ - List listBuckets(String volumeName, - String startBucketName, String bucketPrefix, int maxNumOfBuckets) + List listBuckets(String volumeName, + String startBucketName, String bucketPrefix, int maxNumOfBuckets) throws IOException; /** - * Returns a list of keys represented by {@link KsmKeyInfo} + * Returns a list of keys represented by {@link OmKeyInfo} * in the given bucket. Argument volumeName, bucketName is required, * others are optional. * @@ -238,8 +238,8 @@ public interface KeySpaceManagerProtocol { * @return a list of keys. * @throws IOException */ - List listKeys(String volumeName, - String bucketName, String startKeyName, String keyPrefix, int maxKeys) + List listKeys(String volumeName, + String bucketName, String startKeyName, String keyPrefix, int maxKeys) throws IOException; /** diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/package-info.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/package-info.java similarity index 94% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/package-info.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/package-info.java index f77e5fd0477..9c7f3888d31 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/package-info.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/package-info.java @@ -16,4 +16,4 @@ * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.protocol; \ No newline at end of file +package org.apache.hadoop.ozone.om.protocol; \ No newline at end of file diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java similarity index 79% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java index 0f381692afd..37151fb659e 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.protocolPB; +package org.apache.hadoop.ozone.om.protocolPB; import com.google.common.base.Strings; import com.google.common.collect.Lists; @@ -24,95 +24,95 @@ import com.google.protobuf.ServiceException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.ipc.ProtobufHelper; import org.apache.hadoop.ipc.ProtocolTranslator; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; -import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession; -import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo; -import org.apache.hadoop.ozone.ksm.protocol.KeySpaceManagerProtocol; +import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; +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.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.AllocateBlockRequest; + .OzoneManagerProtocolProtos.AllocateBlockRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.AllocateBlockResponse; + .OzoneManagerProtocolProtos.AllocateBlockResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CommitKeyRequest; + .OzoneManagerProtocolProtos.CommitKeyRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CommitKeyResponse; + .OzoneManagerProtocolProtos.CommitKeyResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.BucketArgs; + .OzoneManagerProtocolProtos.BucketArgs; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.BucketInfo; + .OzoneManagerProtocolProtos.BucketInfo; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CreateBucketRequest; + .OzoneManagerProtocolProtos.CreateBucketRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CreateBucketResponse; + .OzoneManagerProtocolProtos.CreateBucketResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.InfoBucketRequest; + .OzoneManagerProtocolProtos.InfoBucketRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.InfoBucketResponse; + .OzoneManagerProtocolProtos.InfoBucketResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.SetBucketPropertyRequest; + .OzoneManagerProtocolProtos.SetBucketPropertyRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.SetBucketPropertyResponse; + .OzoneManagerProtocolProtos.SetBucketPropertyResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.DeleteBucketRequest; + .OzoneManagerProtocolProtos.DeleteBucketRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.DeleteBucketResponse; + .OzoneManagerProtocolProtos.DeleteBucketResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CreateVolumeRequest; + .OzoneManagerProtocolProtos.CreateVolumeRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CreateVolumeResponse; + .OzoneManagerProtocolProtos.CreateVolumeResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.LocateKeyRequest; + .OzoneManagerProtocolProtos.LocateKeyRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.LocateKeyResponse; + .OzoneManagerProtocolProtos.LocateKeyResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.RenameKeyRequest; + .OzoneManagerProtocolProtos.RenameKeyRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.RenameKeyResponse; + .OzoneManagerProtocolProtos.RenameKeyResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.KeyArgs; + .OzoneManagerProtocolProtos.KeyArgs; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.SetVolumePropertyRequest; + .OzoneManagerProtocolProtos.SetVolumePropertyRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.SetVolumePropertyResponse; + .OzoneManagerProtocolProtos.SetVolumePropertyResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.DeleteVolumeRequest; + .OzoneManagerProtocolProtos.DeleteVolumeRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.DeleteVolumeResponse; + .OzoneManagerProtocolProtos.DeleteVolumeResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.InfoVolumeRequest; + .OzoneManagerProtocolProtos.InfoVolumeRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.InfoVolumeResponse; + .OzoneManagerProtocolProtos.InfoVolumeResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CheckVolumeAccessRequest; + .OzoneManagerProtocolProtos.CheckVolumeAccessRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CheckVolumeAccessResponse; + .OzoneManagerProtocolProtos.CheckVolumeAccessResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListBucketsRequest; + .OzoneManagerProtocolProtos.ListBucketsRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListBucketsResponse; + .OzoneManagerProtocolProtos.ListBucketsResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListKeysRequest; + .OzoneManagerProtocolProtos.ListKeysRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListKeysResponse; + .OzoneManagerProtocolProtos.ListKeysResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.VolumeInfo; + .OzoneManagerProtocolProtos.VolumeInfo; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.Status; + .OzoneManagerProtocolProtos.Status; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo; + .OzoneManagerProtocolProtos.OzoneAclInfo; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListVolumeRequest; + .OzoneManagerProtocolProtos.ListVolumeRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListVolumeResponse; + .OzoneManagerProtocolProtos.ListVolumeResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ServiceListRequest; + .OzoneManagerProtocolProtos.ServiceListRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ServiceListResponse; + .OzoneManagerProtocolProtos.ServiceListResponse; import java.io.Closeable; import java.io.IOException; @@ -121,26 +121,26 @@ import java.util.ArrayList; import java.util.stream.Collectors; /** - * The client side implementation of KeySpaceManagerProtocol. + * The client side implementation of OzoneManagerProtocol. */ @InterfaceAudience.Private -public final class KeySpaceManagerProtocolClientSideTranslatorPB - implements KeySpaceManagerProtocol, ProtocolTranslator, Closeable { +public final class OzoneManagerProtocolClientSideTranslatorPB + implements OzoneManagerProtocol, ProtocolTranslator, Closeable { /** * RpcController is not used and hence is set to null. */ private static final RpcController NULL_RPC_CONTROLLER = null; - private final KeySpaceManagerProtocolPB rpcProxy; + private final OzoneManagerProtocolPB rpcProxy; /** * Constructor for KeySpaceManger Client. * @param rpcProxy */ - public KeySpaceManagerProtocolClientSideTranslatorPB( - KeySpaceManagerProtocolPB rpcProxy) { + public OzoneManagerProtocolClientSideTranslatorPB( + OzoneManagerProtocolPB rpcProxy) { this.rpcProxy = rpcProxy; } @@ -169,7 +169,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB * @throws IOException */ @Override - public void createVolume(KsmVolumeArgs args) throws IOException { + public void createVolume(OmVolumeArgs args) throws IOException { CreateVolumeRequest.Builder req = CreateVolumeRequest.newBuilder(); VolumeInfo volumeInfo = args.getProtobuf(); @@ -273,11 +273,11 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB * Gets the volume information. * * @param volume - Volume name. - * @return KsmVolumeArgs or exception is thrown. + * @return OmVolumeArgs or exception is thrown. * @throws IOException */ @Override - public KsmVolumeArgs getVolumeInfo(String volume) throws IOException { + public OmVolumeArgs getVolumeInfo(String volume) throws IOException { InfoVolumeRequest.Builder req = InfoVolumeRequest.newBuilder(); req.setVolumeName(volume); final InfoVolumeResponse resp; @@ -290,7 +290,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB throw new IOException("Info Volume failed, error:" + resp.getStatus()); } - return KsmVolumeArgs.getFromProtobuf(resp.getVolumeInfo()); + return OmVolumeArgs.getFromProtobuf(resp.getVolumeInfo()); } /** @@ -327,8 +327,8 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB * @throws IOException */ @Override - public List listVolumeByUser(String userName, String prefix, - String prevKey, int maxKeys) + public List listVolumeByUser(String userName, String prefix, + String prevKey, int maxKeys) throws IOException { ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder(); if (!Strings.isNullOrEmpty(prefix)) { @@ -354,8 +354,8 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB * @throws IOException */ @Override - public List listAllVolumes(String prefix, String prevKey, - int maxKeys) throws IOException { + public List listAllVolumes(String prefix, String prevKey, + int maxKeys) throws IOException { ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder(); if (!Strings.isNullOrEmpty(prefix)) { builder.setPrefix(prefix); @@ -368,7 +368,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB return listVolume(builder.build()); } - private List listVolume(ListVolumeRequest request) + private List listVolume(ListVolumeRequest request) throws IOException { final ListVolumeResponse resp; try { @@ -382,14 +382,14 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB + resp.getStatus()); } - List result = Lists.newArrayList(); + List result = Lists.newArrayList(); for (VolumeInfo volInfo : resp.getVolumeInfoList()) { - KsmVolumeArgs volArgs = KsmVolumeArgs.getFromProtobuf(volInfo); + OmVolumeArgs volArgs = OmVolumeArgs.getFromProtobuf(volInfo); result.add(volArgs); } return resp.getVolumeInfoList().stream() - .map(item -> KsmVolumeArgs.getFromProtobuf(item)) + .map(item -> OmVolumeArgs.getFromProtobuf(item)) .collect(Collectors.toList()); } @@ -400,7 +400,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB * @throws IOException */ @Override - public void createBucket(KsmBucketInfo bucketInfo) throws IOException { + public void createBucket(OmBucketInfo bucketInfo) throws IOException { CreateBucketRequest.Builder req = CreateBucketRequest.newBuilder(); BucketInfo bucketInfoProtobuf = bucketInfo.getProtobuf(); @@ -424,11 +424,11 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB * * @param volume - Volume name. * @param bucket - Bucket name. - * @return KsmBucketInfo or exception is thrown. + * @return OmBucketInfo or exception is thrown. * @throws IOException */ @Override - public KsmBucketInfo getBucketInfo(String volume, String bucket) + public OmBucketInfo getBucketInfo(String volume, String bucket) throws IOException { InfoBucketRequest.Builder req = InfoBucketRequest.newBuilder(); @@ -443,7 +443,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB throw ProtobufHelper.getRemoteException(e); } if (resp.getStatus() == Status.OK) { - return KsmBucketInfo.getFromProtobuf(resp.getBucketInfo()); + return OmBucketInfo.getFromProtobuf(resp.getBucketInfo()); } else { throw new IOException("Info Bucket failed, error: " + resp.getStatus()); @@ -456,7 +456,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB * @throws IOException */ @Override - public void setBucketProperty(KsmBucketArgs args) + public void setBucketProperty(OmBucketArgs args) throws IOException { SetBucketPropertyRequest.Builder req = SetBucketPropertyRequest.newBuilder(); @@ -486,9 +486,9 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB * @throws IOException */ @Override - public List listBuckets(String volumeName, - String startKey, String prefix, int count) throws IOException { - List buckets = new ArrayList<>(); + public List listBuckets(String volumeName, + String startKey, String prefix, int count) throws IOException { + List buckets = new ArrayList<>(); ListBucketsRequest.Builder reqBuilder = ListBucketsRequest.newBuilder(); reqBuilder.setVolumeName(volumeName); reqBuilder.setCount(count); @@ -509,7 +509,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB if (resp.getStatus() == Status.OK) { buckets.addAll( resp.getBucketInfoList().stream() - .map(KsmBucketInfo::getFromProtobuf) + .map(OmBucketInfo::getFromProtobuf) .collect(Collectors.toList())); return buckets; } else { @@ -526,7 +526,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB * @throws IOException */ @Override - public OpenKeySession openKey(KsmKeyArgs args) throws IOException { + public OpenKeySession openKey(OmKeyArgs args) throws IOException { LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder(); KeyArgs.Builder keyArgs = KeyArgs.newBuilder() .setVolumeName(args.getVolumeName()) @@ -549,11 +549,11 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB throw new IOException("Create key failed, error:" + resp.getStatus()); } return new OpenKeySession(resp.getID(), - KsmKeyInfo.getFromProtobuf(resp.getKeyInfo()), resp.getOpenVersion()); + OmKeyInfo.getFromProtobuf(resp.getKeyInfo()), resp.getOpenVersion()); } @Override - public KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID) + public OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID) throws IOException { AllocateBlockRequest.Builder req = AllocateBlockRequest.newBuilder(); KeyArgs keyArgs = KeyArgs.newBuilder() @@ -574,11 +574,11 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB throw new IOException("Allocate block failed, error:" + resp.getStatus()); } - return KsmKeyLocationInfo.getFromProtobuf(resp.getKeyLocation()); + return OmKeyLocationInfo.getFromProtobuf(resp.getKeyLocation()); } @Override - public void commitKey(KsmKeyArgs args, int clientID) + public void commitKey(OmKeyArgs args, int clientID) throws IOException { CommitKeyRequest.Builder req = CommitKeyRequest.newBuilder(); KeyArgs keyArgs = KeyArgs.newBuilder() @@ -603,7 +603,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB @Override - public KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException { + public OmKeyInfo lookupKey(OmKeyArgs args) throws IOException { LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder(); KeyArgs keyArgs = KeyArgs.newBuilder() .setVolumeName(args.getVolumeName()) @@ -622,11 +622,11 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB throw new IOException("Lookup key failed, error:" + resp.getStatus()); } - return KsmKeyInfo.getFromProtobuf(resp.getKeyInfo()); + return OmKeyInfo.getFromProtobuf(resp.getKeyInfo()); } @Override - public void renameKey(KsmKeyArgs args, String toKeyName) throws IOException { + public void renameKey(OmKeyArgs args, String toKeyName) throws IOException { RenameKeyRequest.Builder req = RenameKeyRequest.newBuilder(); KeyArgs keyArgs = KeyArgs.newBuilder() .setVolumeName(args.getVolumeName()) @@ -655,7 +655,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB * @throws IOException */ @Override - public void deleteKey(KsmKeyArgs args) throws IOException { + public void deleteKey(OmKeyArgs args) throws IOException { LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder(); KeyArgs keyArgs = KeyArgs.newBuilder() .setVolumeName(args.getVolumeName()) @@ -701,9 +701,9 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB * List keys in a bucket. */ @Override - public List listKeys(String volumeName, String bucketName, - String startKey, String prefix, int maxKeys) throws IOException { - List keys = new ArrayList<>(); + public List listKeys(String volumeName, String bucketName, + String startKey, String prefix, int maxKeys) throws IOException { + List keys = new ArrayList<>(); ListKeysRequest.Builder reqBuilder = ListKeysRequest.newBuilder(); reqBuilder.setVolumeName(volumeName); reqBuilder.setBucketName(bucketName); @@ -728,7 +728,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB if (resp.getStatus() == Status.OK) { keys.addAll( resp.getKeyInfoList().stream() - .map(KsmKeyInfo::getFromProtobuf) + .map(OmKeyInfo::getFromProtobuf) .collect(Collectors.toList())); return keys; } else { diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolPB.java similarity index 77% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolPB.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolPB.java index 8acca8adb7f..e0879d601ce 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolPB.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolPB.java @@ -15,20 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.protocolPB; +package org.apache.hadoop.ozone.om.protocolPB; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.ipc.ProtocolInfo; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.KeySpaceManagerService; + .OzoneManagerProtocolProtos.OzoneManagerService; /** - * Protocol used to communicate with KSM. + * Protocol used to communicate with OM. */ @ProtocolInfo(protocolName = - "org.apache.hadoop.ozone.protocol.KeySpaceManagerProtocol", + "org.apache.hadoop.ozone.protocol.OzoneManagerProtocol", protocolVersion = 1) @InterfaceAudience.Private -public interface KeySpaceManagerProtocolPB - extends KeySpaceManagerService.BlockingInterface { +public interface OzoneManagerProtocolPB + extends OzoneManagerService.BlockingInterface { } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/package-info.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/package-info.java similarity index 94% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/package-info.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/package-info.java index 67f9f7bc4f7..d595edf291a 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/package-info.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/package-info.java @@ -16,4 +16,4 @@ * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.protocolPB; \ No newline at end of file +package org.apache.hadoop.ozone.om.protocolPB; \ No newline at end of file diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java similarity index 93% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java index fdc3ce7a4e9..d57d32e0c45 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java @@ -19,18 +19,18 @@ package org.apache.hadoop.ozone.protocolPB; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo; + .OzoneManagerProtocolProtos.OzoneAclInfo; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclType; + .OzoneManagerProtocolProtos.OzoneAclInfo.OzoneAclType; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclRights; + .OzoneManagerProtocolProtos.OzoneAclInfo.OzoneAclRights; /** * Utilities for converting protobuf classes. */ -public final class KSMPBHelper { +public final class OMPBHelper { - private KSMPBHelper() { + private OMPBHelper() { /** Hidden constructor */ } diff --git a/hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto similarity index 96% rename from hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto rename to hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto index d3d1de69bd5..36b1c83efb1 100644 --- a/hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto +++ b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto @@ -23,14 +23,14 @@ */ option java_package = "org.apache.hadoop.ozone.protocol.proto"; -option java_outer_classname = "KeySpaceManagerProtocolProtos"; +option java_outer_classname = "OzoneManagerProtocolProtos"; option java_generic_services = true; option java_generate_equals_and_hash = true; package hadoop.ozone; /** This is file contains the protocol to communicate with -Ozone key space manager. Ozone KSM manages the namespace for ozone. +Ozone Manager. Ozone Manager manages the namespace for ozone. This is similar to Namenode for Ozone. */ @@ -53,6 +53,12 @@ enum Status { INVALID_KEY_NAME = 13; ACCESS_DENIED = 14; INTERNAL_ERROR = 15; + KEY_ALLOCATION_ERROR = 16; + KEY_DELETION_ERROR = 17; + KEY_RENAME_ERROR = 18; + METADATA_ERROR = 19; + OM_NOT_INITIALIZED = 20; + SCM_VERSION_MISMATCH_ERROR = 21; } @@ -355,9 +361,9 @@ message ServiceInfo { } /** - The KSM service that takes care of Ozone namespace. + The OM service that takes care of Ozone namespace. */ -service KeySpaceManagerService { +service OzoneManagerService { /** Creates a Volume. diff --git a/hadoop-ozone/docs/content/GettingStarted.md b/hadoop-ozone/docs/content/GettingStarted.md index 531d1924126..117a3071c58 100644 --- a/hadoop-ozone/docs/content/GettingStarted.md +++ b/hadoop-ozone/docs/content/GettingStarted.md @@ -194,12 +194,12 @@ This path will be created by datanodes if it doesn't exist already. Here is an ``` -1. **ozone.ksm.address** OM server address. This is used by OzoneClient and +1. **ozone.om.address** OM server address. This is used by OzoneClient and Ozone File System. ``` - ozone.ksm.address - ksm.hadoop.apache.org + ozone.om.address + om.hadoop.apache.org ``` @@ -210,10 +210,10 @@ Ozone File System. | ozone.enabled | True | This enables SCM and containers in HDFS cluster. | | ozone.metadata.dirs | file path | The metadata will be stored here. | | ozone.scm.names | SCM server name | Hostname:port or or IP:port address of SCM. | -| ozone.scm.block.client.address | SCM server name and port | Used by services like OM | +| ozone.scm.block.client.address | SCM server name and port | Used by services like OM | | ozone.scm.client.address | SCM server name and port | Used by client side | | ozone.scm.datanode.address | SCM server name and port | Used by datanode to talk to SCM | -| ozone.ksm.address | OM server name | Used by Ozone handler and Ozone file system. | +| ozone.om.address | OM server name | Used by Ozone handler and Ozone file system. | #### Sample ozone-site.xml @@ -253,7 +253,7 @@ Ozone File System. - ozone.ksm.address + ozone.om.address 127.0.0.1:9874 @@ -286,12 +286,12 @@ ozone --daemon start scm Once SCM gets started, OM must be initialized. ``` -ozone ksm -createObjectStore +ozone om -createObjectStore ``` Start OM. ``` -ozone --daemon start ksm +ozone --daemon start om ``` If you would like to start HDFS and Ozone together, you can do that by running @@ -349,7 +349,7 @@ log4j.additivity.org.apache.hadoop.ozone=false ``` On the SCM/OM side, you will be able to see -1. `hadoop-hdfs-ksm-hostname.log` +1. `hadoop-hdfs-om-hostname.log` 1. `hadoop-hdfs-scm-hostname.log` ## Reporting Bugs diff --git a/hadoop-ozone/docs/content/Metrics.md b/hadoop-ozone/docs/content/Metrics.md index dc58460517e..64a481fa8ce 100644 --- a/hadoop-ozone/docs/content/Metrics.md +++ b/hadoop-ozone/docs/content/Metrics.md @@ -131,10 +131,10 @@ Following are the counters for containers: ### Key Space Metrics -The metrics for various key space manager operations in HDFS Ozone. +The metrics for various Ozone Manager operations in HDFS Ozone. -key space manager (KSM) is a service that similar to the Namenode in HDFS. -In the current design of KSM, it maintains metadata of all volumes, buckets and keys. +The Ozone Manager (OM) is a service that similar to the Namenode in HDFS. +In the current design of OM, it maintains metadata of all volumes, buckets and keys. These metrics are only available when ozone is enabled. Following is the set of counters maintained for each key space operation. @@ -142,12 +142,12 @@ Following is the set of counters maintained for each key space operation. *Total number of operation* - We maintain an array which counts how many times a specific operation has been performed. Eg.`NumVolumeCreate` tells us how many times create volume has been -invoked in KSM. +invoked in OM. *Total number of failed operation* - This type operation is opposite to the above operation. Eg.`NumVolumeCreateFails` tells us how many times create volume has been invoked -failed in KSM. +failed in OM. Following are the counters for each of key space operations. diff --git a/hadoop-ozone/docs/content/_index.md b/hadoop-ozone/docs/content/_index.md index ab7eabe6446..383b2e0ef0a 100644 --- a/hadoop-ozone/docs/content/_index.md +++ b/hadoop-ozone/docs/content/_index.md @@ -56,14 +56,14 @@ This is like DFSClient in HDFS. This acts as the standard client to talk to Ozone. All other components that we have discussed so far rely on Ozone client (TODO: Add Ozone client documentation).
 -## Key Space Manager
 +## Ozone Manager -Key Space Manager(KSM) takes care of the Ozone's namespace. -All ozone entities like volumes, buckets and keys are managed by KSM -(TODO: Add KSM documentation). In Short, KSM is the metadata manager for Ozone. -KSM talks to blockManager(SCM) to get blocks and passes it on to the Ozone +Ozone Manager (OM) takes care of the Ozone's namespace. +All ozone entities like volumes, buckets and keys are managed by OM +(TODO: Add OM documentation). In short, OM is the metadata manager for Ozone. +OM talks to blockManager(SCM) to get blocks and passes it on to the Ozone client. Ozone client writes data to these blocks. -KSM will eventually be replicated via Apache Ratis for High Availability.
 +OM will eventually be replicated via Apache Ratis for High Availability.
 ## Storage Container Manager Storage Container Manager (SCM) is the block and cluster manager for Ozone. diff --git a/hadoop-ozone/docs/static/OzoneOverview.svg b/hadoop-ozone/docs/static/OzoneOverview.svg index 2e14d3fe66e..0120a5cc367 100644 --- a/hadoop-ozone/docs/static/OzoneOverview.svg +++ b/hadoop-ozone/docs/static/OzoneOverview.svg @@ -166,7 +166,7 @@ - + diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java index 091d7715929..b568672db34 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java @@ -21,7 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.ozone.client.OzoneClient; -import org.apache.hadoop.ozone.ksm.KeySpaceManager; +import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.hdds.scm.protocolPB .StorageContainerLocationProtocolClientSideTranslatorPB; import org.apache.hadoop.test.GenericTestUtils; @@ -82,12 +82,12 @@ public interface MiniOzoneCluster { StorageContainerManager getStorageContainerManager(); /** - * Returns {@link KeySpaceManager} associated with this + * Returns {@link OzoneManager} associated with this * {@link MiniOzoneCluster} instance. * - * @return {@link KeySpaceManager} instance + * @return {@link OzoneManager} instance */ - KeySpaceManager getKeySpaceManager(); + OzoneManager getOzoneManager(); /** * Returns the list of {@link HddsDatanodeService} which are part of this @@ -141,11 +141,11 @@ public interface MiniOzoneCluster { void restartStorageContainerManager() throws IOException; /** - * Restarts KeySpaceManager instance. + * Restarts OzoneManager instance. * * @throws IOException */ - void restartKeySpaceManager() throws IOException; + void restartOzoneManager() throws IOException; /** * Restart a particular HddsDatanode. @@ -184,13 +184,13 @@ public interface MiniOzoneCluster { protected Optional hbInterval = Optional.empty(); protected Optional hbProcessorInterval = Optional.empty(); protected Optional scmId = Optional.empty(); - protected Optional ksmId = Optional.empty(); + protected Optional omId = Optional.empty(); protected Boolean ozoneEnabled = true; protected Boolean randomContainerPort = true; // Use relative smaller number of handlers for testing - protected int numOfKsmHandlers = 20; + protected int numOfOmHandlers = 20; protected int numOfScmHandlers = 20; protected int numOfDatanodes = 1; @@ -226,14 +226,14 @@ public interface MiniOzoneCluster { } /** - * Sets the KSM id. + * Sets the OM id. * - * @param id KSM Id + * @param id OM Id * * @return MiniOzoneCluster.Builder */ - public Builder setKsmId(String id) { - ksmId = Optional.of(id); + public Builder setOmId(String id) { + omId = Optional.of(id); return this; } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java index f0bfef17be1..b3137bf3f4f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java @@ -34,10 +34,10 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.rest.OzoneException; -import org.apache.hadoop.ozone.ksm.KSMConfigKeys; -import org.apache.hadoop.ozone.ksm.KeySpaceManager; +import org.apache.hadoop.ozone.om.OMConfigKeys; +import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.hdds.scm.server.SCMStorage; -import org.apache.hadoop.ozone.ksm.KSMStorage; +import org.apache.hadoop.ozone.om.OMStorage; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.protocolPB .StorageContainerLocationProtocolClientSideTranslatorPB; @@ -73,7 +73,7 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys /** * MiniOzoneCluster creates a complete in-process Ozone cluster suitable for - * running tests. The cluster consists of a KeySpaceManager, + * running tests. The cluster consists of a OzoneManager, * StorageContainerManager and multiple DataNodes. */ @InterfaceAudience.Private @@ -84,7 +84,7 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster { private final OzoneConfiguration conf; private final StorageContainerManager scm; - private final KeySpaceManager ksm; + private final OzoneManager ozoneManager; private final List hddsDatanodes; /** @@ -93,11 +93,11 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster { * @throws IOException if there is an I/O error */ private MiniOzoneClusterImpl(OzoneConfiguration conf, - KeySpaceManager ksm, + OzoneManager ozoneManager, StorageContainerManager scm, List hddsDatanodes) { this.conf = conf; - this.ksm = ksm; + this.ozoneManager = ozoneManager; this.scm = scm; this.hddsDatanodes = hddsDatanodes; } @@ -147,8 +147,8 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster { } @Override - public KeySpaceManager getKeySpaceManager() { - return this.ksm; + public OzoneManager getOzoneManager() { + return this.ozoneManager; } @Override @@ -209,9 +209,9 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster { } @Override - public void restartKeySpaceManager() throws IOException { - ksm.stop(); - ksm.start(); + public void restartOzoneManager() throws IOException { + ozoneManager.stop(); + ozoneManager.start(); } @Override @@ -247,10 +247,10 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster { scm.getClientProtocolServer().getScmInfo().getClusterId())); FileUtils.deleteDirectory(baseDir); - if (ksm != null) { - LOG.info("Shutting down the keySpaceManager"); - ksm.stop(); - ksm.join(); + if (ozoneManager != null) { + LOG.info("Shutting down the OzoneManager"); + ozoneManager.stop(); + ozoneManager.join(); } if (scm != null) { @@ -291,11 +291,11 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster { initializeConfiguration(); StorageContainerManager scm = createSCM(); scm.start(); - KeySpaceManager ksm = createKSM(); - ksm.start(); + OzoneManager om = createOM(); + om.start(); List hddsDatanodes = createHddsDatanodes(scm); hddsDatanodes.forEach((datanode) -> datanode.start(null)); - return new MiniOzoneClusterImpl(conf, ksm, scm, hddsDatanodes); + return new MiniOzoneClusterImpl(conf, om, scm, hddsDatanodes); } /** @@ -331,20 +331,20 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster { } /** - * Creates a new KeySpaceManager instance. + * Creates a new OzoneManager instance. * - * @return {@link KeySpaceManager} + * @return {@link OzoneManager} * * @throws IOException */ - private KeySpaceManager createKSM() throws IOException { - configureKSM(); - KSMStorage ksmStore = new KSMStorage(conf); - ksmStore.setClusterId(clusterId); - ksmStore.setScmId(scmId.get()); - ksmStore.setKsmId(ksmId.orElse(UUID.randomUUID().toString())); - ksmStore.initialize(); - return KeySpaceManager.createKSM(null, conf); + private OzoneManager createOM() throws IOException { + configureOM(); + OMStorage omStore = new OMStorage(conf); + omStore.setClusterId(clusterId); + omStore.setScmId(scmId.get()); + omStore.setOmId(omId.orElse(UUID.randomUUID().toString())); + omStore.initialize(); + return OzoneManager.createOm(null, conf); } /** @@ -415,10 +415,10 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster { } - private void configureKSM() { - conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "127.0.0.1:0"); - conf.set(KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY, "127.0.0.1:0"); - conf.setInt(KSMConfigKeys.OZONE_KSM_HANDLER_COUNT_KEY, numOfKsmHandlers); + private void configureOM() { + conf.set(OMConfigKeys.OZONE_OM_ADDRESS_KEY, "127.0.0.1:0"); + conf.set(OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY, "127.0.0.1:0"); + conf.setInt(OMConfigKeys.OZONE_OM_HANDLER_COUNT_KEY, numOfOmHandlers); } private void configureHddsDatanodes() { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java index 4898a1b6c27..717bb68534b 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java @@ -18,7 +18,7 @@ package org.apache.hadoop.ozone; import org.apache.hadoop.conf.TestConfigurationFieldsBase; -import org.apache.hadoop.ozone.ksm.KSMConfigKeys; +import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys; /** @@ -31,7 +31,7 @@ public class TestOzoneConfigurationFields extends TestConfigurationFieldsBase { xmlFilename = new String("ozone-default.xml"); configurationClasses = new Class[] {OzoneConfigKeys.class, ScmConfigKeys.class, - KSMConfigKeys.class}; + OMConfigKeys.class}; errorIfMissingConfigProps = true; errorIfMissingXmlProps = true; xmlPropsToSkipCompare.add("hadoop.tags.custom"); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java index dd1a8de8903..cc367b3339d 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer; import org.apache.hadoop.hdds.scm.server.SCMStorage; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.ozone.container.ContainerTestHelper; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -60,8 +61,7 @@ import java.util.concurrent.TimeUnit; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.junit.rules.Timeout; @@ -211,7 +211,7 @@ public class TestStorageContainerManager { // Create {numKeys} random names keys. TestStorageContainerManagerHelper helper = new TestStorageContainerManagerHelper(cluster, conf); - Map keyLocations = helper.createKeys(numKeys, 4096); + Map keyLocations = helper.createKeys(numKeys, 4096); Map> containerBlocks = createDeleteTXLog(delLog, keyLocations, helper); @@ -293,7 +293,7 @@ public class TestStorageContainerManager { // Create {numKeys} random names keys. TestStorageContainerManagerHelper helper = new TestStorageContainerManagerHelper(cluster, conf); - Map keyLocations = helper.createKeys(numKeys, 4096); + Map keyLocations = helper.createKeys(numKeys, 4096); createDeleteTXLog(delLog, keyLocations, helper); // Verify a few TX gets created in the TX log. @@ -320,13 +320,13 @@ public class TestStorageContainerManager { } private Map> createDeleteTXLog(DeletedBlockLog delLog, - Map keyLocations, + Map keyLocations, TestStorageContainerManagerHelper helper) throws IOException { // These keys will be written into a bunch of containers, // gets a set of container names, verify container containerBlocks // on datanodes. Set containerNames = new HashSet<>(); - for (Map.Entry entry : keyLocations.entrySet()) { + for (Map.Entry entry : keyLocations.entrySet()) { entry.getValue().getLatestVersionLocations().getLocationList() .forEach(loc -> containerNames.add(loc.getContainerID())); } @@ -334,7 +334,7 @@ public class TestStorageContainerManager { // Total number of containerBlocks of these containers should be equal to // total number of containerBlocks via creation call. int totalCreatedBlocks = 0; - for (KsmKeyInfo info : keyLocations.values()) { + for (OmKeyInfo info : keyLocations.values()) { totalCreatedBlocks += info.getKeyLocationVersions().size(); } Assert.assertTrue(totalCreatedBlocks > 0); @@ -343,8 +343,8 @@ public class TestStorageContainerManager { // Create a deletion TX for each key. Map> containerBlocks = Maps.newHashMap(); - for (KsmKeyInfo info : keyLocations.values()) { - List list = + for (OmKeyInfo info : keyLocations.values()) { + List list = info.getLatestVersionLocations().getLocationList(); list.forEach(location -> { if (containerBlocks.containsKey(location.getContainerID())) { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java index 4c2a904ef04..a30c6f4ffee 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java @@ -30,8 +30,8 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; import org.apache.hadoop.ozone.container.common.helpers.KeyUtils; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.web.handlers.BucketArgs; import org.apache.hadoop.ozone.web.handlers.KeyArgs; import org.apache.hadoop.ozone.web.handlers.UserArgs; @@ -67,9 +67,9 @@ public class TestStorageContainerManagerHelper { storageHandler = new ObjectStoreHandler(conf).getStorageHandler(); } - public Map createKeys(int numOfKeys, int keySize) + public Map createKeys(int numOfKeys, int keySize) throws Exception { - Map keyLocationMap = Maps.newHashMap(); + Map keyLocationMap = Maps.newHashMap(); String volume = "volume" + RandomStringUtils.randomNumeric(5); String bucket = "bucket" + RandomStringUtils.randomNumeric(5); String userName = "user" + RandomStringUtils.randomNumeric(5); @@ -104,12 +104,12 @@ public class TestStorageContainerManagerHelper { } for (String key : keyNames) { - KsmKeyArgs arg = new KsmKeyArgs.Builder() + OmKeyArgs arg = new OmKeyArgs.Builder() .setVolumeName(volume) .setBucketName(bucket) .setKeyName(key) .build(); - KsmKeyInfo location = cluster.getKeySpaceManager() + OmKeyInfo location = cluster.getOzoneManager() .lookupKey(arg); keyLocationMap.put(key, location); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java index 9918d63668c..0dc039983b9 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java @@ -77,10 +77,10 @@ public class TestOzoneRestClient { OzoneConsts.OZONE_HANDLER_DISTRIBUTED); cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build(); cluster.waitForClusterToBeReady(); - InetSocketAddress ksmHttpAddress = cluster.getKeySpaceManager() + InetSocketAddress omHttpAddress = cluster.getOzoneManager() .getHttpServer().getHttpAddress(); - ozClient = OzoneClientFactory.getRestClient(ksmHttpAddress.getHostName(), - ksmHttpAddress.getPort(), conf); + ozClient = OzoneClientFactory.getRestClient(omHttpAddress.getHostName(), + omHttpAddress.getPort(), conf); store = ozClient.getObjectStore(); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java index 214382e5f65..2fbab361a79 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java @@ -39,10 +39,10 @@ import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.ozone.client.VolumeArgs; import org.apache.hadoop.ozone.client.io.OzoneInputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; -import org.apache.hadoop.ozone.ksm.KeySpaceManager; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.ozone.client.rest.OzoneException; import org.apache.hadoop.hdds.scm.ScmConfigKeys; @@ -73,7 +73,7 @@ public class TestOzoneRpcClient { private static MiniOzoneCluster cluster = null; private static OzoneClient ozClient = null; private static ObjectStore store = null; - private static KeySpaceManager keySpaceManager; + private static OzoneManager ozoneManager; private static StorageContainerLocationProtocolClientSideTranslatorPB storageContainerLocationClient; @@ -97,7 +97,7 @@ public class TestOzoneRpcClient { store = ozClient.getObjectStore(); storageContainerLocationClient = cluster.getStorageContainerLocationClient(); - keySpaceManager = cluster.getKeySpaceManager(); + ozoneManager = cluster.getOzoneManager(); } @Test @@ -376,7 +376,7 @@ public class TestOzoneRpcClient { private boolean verifyRatisReplication(String volumeName, String bucketName, String keyName, ReplicationType type, ReplicationFactor factor) throws IOException { - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(volumeName) .setBucketName(bucketName) .setKeyName(keyName) @@ -385,8 +385,8 @@ public class TestOzoneRpcClient { HddsProtos.ReplicationType.valueOf(type.toString()); HddsProtos.ReplicationFactor replicationFactor = HddsProtos.ReplicationFactor.valueOf(factor.getValue()); - KsmKeyInfo keyInfo = keySpaceManager.lookupKey(keyArgs); - for (KsmKeyLocationInfo info: + OmKeyInfo keyInfo = ozoneManager.lookupKey(keyArgs); + for (OmKeyLocationInfo info: keyInfo.getLatestVersionLocations().getLocationList()) { ContainerInfo container = storageContainerLocationClient.getContainer(info.getContainerID()); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java index 43e3f5095f3..62059ec85fa 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java @@ -37,10 +37,10 @@ import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; import org.apache.hadoop.ozone.container.common.helpers.KeyUtils; import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl; -import org.apache.hadoop.ozone.ksm.KeySpaceManager; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; import org.apache.hadoop.ozone.ozShell.TestOzoneShell; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.utils.MetadataStore; @@ -61,7 +61,7 @@ public class TestBlockDeletion { private static ObjectStore store; private static ContainerManagerImpl dnContainerManager = null; private static StorageContainerManager scm = null; - private static KeySpaceManager ksm = null; + private static OzoneManager om = null; private static Set containerIdsWithDeletedBlocks; @BeforeClass @@ -88,7 +88,7 @@ public class TestBlockDeletion { dnContainerManager = (ContainerManagerImpl) cluster.getHddsDatanodes().get(0) .getDatanodeStateMachine().getContainer().getContainerManager(); - ksm = cluster.getKeySpaceManager(); + om = cluster.getOzoneManager(); scm = cluster.getStorageContainerManager(); containerIdsWithDeletedBlocks = new HashSet<>(); } @@ -112,23 +112,23 @@ public class TestBlockDeletion { out.write(value.getBytes()); out.close(); - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder().setVolumeName(volumeName) + OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName) .setBucketName(bucketName).setKeyName(keyName).setDataSize(0) .setType(HddsProtos.ReplicationType.STAND_ALONE) .setFactor(HddsProtos.ReplicationFactor.ONE).build(); - List ksmKeyLocationInfoGroupList = - ksm.lookupKey(keyArgs).getKeyLocationVersions(); + List omKeyLocationInfoGroupList = + om.lookupKey(keyArgs).getKeyLocationVersions(); // verify key blocks were created in DN. - Assert.assertTrue(verifyBlocksCreated(ksmKeyLocationInfoGroupList)); + Assert.assertTrue(verifyBlocksCreated(omKeyLocationInfoGroupList)); // No containers with deleted blocks Assert.assertTrue(containerIdsWithDeletedBlocks.isEmpty()); // Delete transactionIds for the containers should be 0 matchContainerTransactionIds(); - ksm.deleteKey(keyArgs); + om.deleteKey(keyArgs); Thread.sleep(5000); // The blocks should be deleted in the DN. - Assert.assertTrue(verifyBlocksDeleted(ksmKeyLocationInfoGroupList)); + Assert.assertTrue(verifyBlocksDeleted(omKeyLocationInfoGroupList)); // Few containers with deleted blocks Assert.assertTrue(!containerIdsWithDeletedBlocks.isEmpty()); @@ -155,7 +155,7 @@ public class TestBlockDeletion { } private boolean verifyBlocksCreated( - List ksmKeyLocationInfoGroups) + List omKeyLocationInfoGroups) throws IOException { return performOperationOnKeyContainers((blockID) -> { try { @@ -166,11 +166,11 @@ public class TestBlockDeletion { } catch (IOException e) { e.printStackTrace(); } - }, ksmKeyLocationInfoGroups); + }, omKeyLocationInfoGroups); } private boolean verifyBlocksDeleted( - List ksmKeyLocationInfoGroups) + List omKeyLocationInfoGroups) throws IOException { return performOperationOnKeyContainers((blockID) -> { try { @@ -186,19 +186,20 @@ public class TestBlockDeletion { } catch (IOException e) { e.printStackTrace(); } - }, ksmKeyLocationInfoGroups); + }, omKeyLocationInfoGroups); } private boolean performOperationOnKeyContainers(Consumer consumer, - List ksmKeyLocationInfoGroups) + List omKeyLocationInfoGroups) throws IOException { try { - for (KsmKeyLocationInfoGroup ksmKeyLocationInfoGroup : ksmKeyLocationInfoGroups) { - List ksmKeyLocationInfos = - ksmKeyLocationInfoGroup.getLocationList(); - for (KsmKeyLocationInfo ksmKeyLocationInfo : ksmKeyLocationInfos) { - BlockID blockID = ksmKeyLocationInfo.getBlockID(); + for (OmKeyLocationInfoGroup omKeyLocationInfoGroup : + omKeyLocationInfoGroups) { + List omKeyLocationInfos = + omKeyLocationInfoGroup.getLocationList(); + for (OmKeyLocationInfo omKeyLocationInfo : omKeyLocationInfos) { + BlockID blockID = omKeyLocationInfo.getBlockID(); consumer.accept(blockID); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java index 3e514e7906a..58b831b0780 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java @@ -34,8 +34,8 @@ import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand; import org.apache.hadoop.test.GenericTestUtils; import org.junit.AfterClass; @@ -45,7 +45,6 @@ import org.junit.Test; import java.io.IOException; import java.util.List; -import java.util.Random; import java.util.concurrent.TimeoutException; public class TestCloseContainerByPipeline { @@ -98,17 +97,17 @@ public class TestCloseContainerByPipeline { key.close(); //get the name of a valid container - KsmKeyArgs keyArgs = - new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test") + OmKeyArgs keyArgs = + new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test") .setType(HddsProtos.ReplicationType.STAND_ALONE) .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024) .setKeyName("testCloseContainer").build(); - KsmKeyLocationInfo ksmKeyLocationInfo = - cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions() + OmKeyLocationInfo omKeyLocationInfo = + cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions() .get(0).getBlocksLatestVersionOnly().get(0); - long containerID = ksmKeyLocationInfo.getContainerID(); + long containerID = omKeyLocationInfo.getContainerID(); List datanodes = cluster.getStorageContainerManager() .getScmContainerManager().getContainerWithPipeline(containerID) .getPipeline().getMachines(); @@ -153,17 +152,17 @@ public class TestCloseContainerByPipeline { key.close(); //get the name of a valid container - KsmKeyArgs keyArgs = - new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test") + OmKeyArgs keyArgs = + new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test") .setType(HddsProtos.ReplicationType.STAND_ALONE) .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024) .setKeyName("standalone").build(); - KsmKeyLocationInfo ksmKeyLocationInfo = - cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions() + OmKeyLocationInfo omKeyLocationInfo = + cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions() .get(0).getBlocksLatestVersionOnly().get(0); - long containerID = ksmKeyLocationInfo.getContainerID(); + long containerID = omKeyLocationInfo.getContainerID(); List datanodes = cluster.getStorageContainerManager() .getScmContainerManager().getContainerWithPipeline(containerID) .getPipeline().getMachines(); @@ -207,16 +206,16 @@ public class TestCloseContainerByPipeline { key.close(); //get the name of a valid container - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder().setVolumeName("test"). + OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName("test"). setBucketName("test").setType(HddsProtos.ReplicationType.RATIS) .setFactor(HddsProtos.ReplicationFactor.THREE).setDataSize(1024) .setKeyName("ratis").build(); - KsmKeyLocationInfo ksmKeyLocationInfo = - cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions() + OmKeyLocationInfo omKeyLocationInfo = + cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions() .get(0).getBlocksLatestVersionOnly().get(0); - long containerID = ksmKeyLocationInfo.getContainerID(); + long containerID = omKeyLocationInfo.getContainerID(); List datanodes = cluster.getStorageContainerManager() .getScmContainerManager().getContainerWithPipeline(containerID) .getPipeline().getMachines(); @@ -232,7 +231,7 @@ public class TestCloseContainerByPipeline { .addDatanodeCommand(details.getUuid(), new CloseContainerCommand(containerID, HddsProtos.ReplicationType.RATIS)); - } + } for (DatanodeDetails datanodeDetails : datanodes) { GenericTestUtils.waitFor( diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java index efb734497fc..58a51540372 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java @@ -28,8 +28,8 @@ import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.client.rest.OzoneException; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB; @@ -69,17 +69,17 @@ public class TestCloseContainerHandler { key.close(); //get the name of a valid container - KsmKeyArgs keyArgs = - new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test") + OmKeyArgs keyArgs = + new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test") .setType(HddsProtos.ReplicationType.STAND_ALONE) .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024) .setKeyName("test").build(); - KsmKeyLocationInfo ksmKeyLocationInfo = - cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions() + OmKeyLocationInfo omKeyLocationInfo = + cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions() .get(0).getBlocksLatestVersionOnly().get(0); - long containerID = ksmKeyLocationInfo.getContainerID(); + long containerID = omKeyLocationInfo.getContainerID(); Assert.assertFalse(isContainerClosed(cluster, containerID)); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java similarity index 94% rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java index 1cc7ff8f225..54815061c00 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java @@ -14,7 +14,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.commons.lang3.RandomStringUtils; @@ -30,8 +30,8 @@ import org.apache.hadoop.ozone.client.*; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.container.common.helpers.ContainerData; import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.junit.AfterClass; @@ -104,7 +104,7 @@ public class TestContainerReportWithKeys { key.write(dataString.getBytes()); key.close(); - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(volumeName) .setBucketName(bucketName) .setKeyName(keyName) @@ -113,8 +113,8 @@ public class TestContainerReportWithKeys { .build(); - KsmKeyLocationInfo keyInfo = - cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions() + OmKeyLocationInfo keyInfo = + cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions() .get(0).getBlocksLatestVersionOnly().get(0); ContainerData cd = getContainerData(keyInfo.getContainerID()); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestMultipleContainerReadWrite.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java similarity index 99% rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestMultipleContainerReadWrite.java rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java index 1cb6e82c14a..1389cbaa986 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestMultipleContainerReadWrite.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java @@ -14,7 +14,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.fs.StorageType; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKsmBlockVersioning.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java similarity index 81% rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKsmBlockVersioning.java rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java index 15c3fd3b46d..15122b94cee 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKsmBlockVersioning.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java @@ -14,7 +14,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -24,11 +24,11 @@ import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup; -import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; import org.apache.hadoop.ozone.web.handlers.BucketArgs; import org.apache.hadoop.ozone.web.handlers.KeyArgs; import org.apache.hadoop.ozone.web.handlers.UserArgs; @@ -51,13 +51,13 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; /** - * This class tests the versioning of blocks from KSM side. + * This class tests the versioning of blocks from OM side. */ -public class TestKsmBlockVersioning { +public class TestOmBlockVersioning { private static MiniOzoneCluster cluster = null; private static UserArgs userArgs; private static OzoneConfiguration conf; - private static KeySpaceManager keySpaceManager; + private static OzoneManager ozoneManager; private static StorageHandler storageHandler; @Rule @@ -81,7 +81,7 @@ public class TestKsmBlockVersioning { storageHandler = new ObjectStoreHandler(conf).getStorageHandler(); userArgs = new UserArgs(null, OzoneUtils.getRequestID(), null, null, null, null); - keySpaceManager = cluster.getKeySpaceManager(); + ozoneManager = cluster.getOzoneManager(); } /** @@ -113,7 +113,7 @@ public class TestKsmBlockVersioning { bucketArgs.setStorageType(StorageType.DISK); storageHandler.createBucket(bucketArgs); - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(volumeName) .setBucketName(bucketName) .setKeyName(keyName) @@ -121,49 +121,49 @@ public class TestKsmBlockVersioning { .build(); // 1st update, version 0 - OpenKeySession openKey = keySpaceManager.openKey(keyArgs); - keySpaceManager.commitKey(keyArgs, openKey.getId()); + OpenKeySession openKey = ozoneManager.openKey(keyArgs); + ozoneManager.commitKey(keyArgs, openKey.getId()); - KsmKeyInfo keyInfo = keySpaceManager.lookupKey(keyArgs); - KsmKeyLocationInfoGroup highestVersion = + OmKeyInfo keyInfo = ozoneManager.lookupKey(keyArgs); + OmKeyLocationInfoGroup highestVersion = checkVersions(keyInfo.getKeyLocationVersions()); assertEquals(0, highestVersion.getVersion()); assertEquals(1, highestVersion.getLocationList().size()); // 2nd update, version 1 - openKey = keySpaceManager.openKey(keyArgs); - //KsmKeyLocationInfo locationInfo = - // keySpaceManager.allocateBlock(keyArgs, openKey.getId()); - keySpaceManager.commitKey(keyArgs, openKey.getId()); + openKey = ozoneManager.openKey(keyArgs); + //OmKeyLocationInfo locationInfo = + // ozoneManager.allocateBlock(keyArgs, openKey.getId()); + ozoneManager.commitKey(keyArgs, openKey.getId()); - keyInfo = keySpaceManager.lookupKey(keyArgs); + keyInfo = ozoneManager.lookupKey(keyArgs); highestVersion = checkVersions(keyInfo.getKeyLocationVersions()); assertEquals(1, highestVersion.getVersion()); assertEquals(2, highestVersion.getLocationList().size()); // 3rd update, version 2 - openKey = keySpaceManager.openKey(keyArgs); + openKey = ozoneManager.openKey(keyArgs); // this block will be appended to the latest version of version 2. - keySpaceManager.allocateBlock(keyArgs, openKey.getId()); - keySpaceManager.commitKey(keyArgs, openKey.getId()); + ozoneManager.allocateBlock(keyArgs, openKey.getId()); + ozoneManager.commitKey(keyArgs, openKey.getId()); - keyInfo = keySpaceManager.lookupKey(keyArgs); + keyInfo = ozoneManager.lookupKey(keyArgs); highestVersion = checkVersions(keyInfo.getKeyLocationVersions()); assertEquals(2, highestVersion.getVersion()); assertEquals(4, highestVersion.getLocationList().size()); } - private KsmKeyLocationInfoGroup checkVersions( - List versions) { - KsmKeyLocationInfoGroup currentVersion = null; - for (KsmKeyLocationInfoGroup version : versions) { + private OmKeyLocationInfoGroup checkVersions( + List versions) { + OmKeyLocationInfoGroup currentVersion = null; + for (OmKeyLocationInfoGroup version : versions) { if (currentVersion != null) { assertEquals(currentVersion.getVersion() + 1, version.getVersion()); - for (KsmKeyLocationInfo info : currentVersion.getLocationList()) { + for (OmKeyLocationInfo info : currentVersion.getLocationList()) { boolean found = false; // all the blocks from the previous version must present in the next // version - for (KsmKeyLocationInfo info2 : version.getLocationList()) { + for (OmKeyLocationInfo info2 : version.getLocationList()) { if (info.getLocalID() == info2.getLocalID()) { found = true; break; @@ -197,7 +197,7 @@ public class TestKsmBlockVersioning { bucketArgs.setStorageType(StorageType.DISK); storageHandler.createBucket(bucketArgs); - KsmKeyArgs ksmKeyArgs = new KsmKeyArgs.Builder() + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() .setVolumeName(volumeName) .setBucketName(bucketName) .setKeyName(keyName) @@ -214,7 +214,7 @@ public class TestKsmBlockVersioning { try (InputStream in = storageHandler.newKeyReader(keyArgs)) { in.read(data); } - KsmKeyInfo keyInfo = keySpaceManager.lookupKey(ksmKeyArgs); + OmKeyInfo keyInfo = ozoneManager.lookupKey(omKeyArgs); assertEquals(dataString, DFSUtil.bytes2String(data)); assertEquals(0, keyInfo.getLatestVersionLocations().getVersion()); assertEquals(1, @@ -230,7 +230,7 @@ public class TestKsmBlockVersioning { try (InputStream in = storageHandler.newKeyReader(keyArgs)) { in.read(data); } - keyInfo = keySpaceManager.lookupKey(ksmKeyArgs); + keyInfo = ozoneManager.lookupKey(omKeyArgs); assertEquals(dataString, DFSUtil.bytes2String(data)); assertEquals(1, keyInfo.getLatestVersionLocations().getVersion()); assertEquals(2, @@ -244,7 +244,7 @@ public class TestKsmBlockVersioning { try (InputStream in = storageHandler.newKeyReader(keyArgs)) { in.read(data); } - keyInfo = keySpaceManager.lookupKey(ksmKeyArgs); + keyInfo = ozoneManager.lookupKey(omKeyArgs); assertEquals(dataString, DFSUtil.bytes2String(data)); assertEquals(2, keyInfo.getLatestVersionLocations().getVersion()); assertEquals(3, diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMMetrcis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java similarity index 53% rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMMetrcis.java rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java index bf7d870bd45..8d0f4b2129e 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMMetrcis.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java @@ -14,7 +14,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import static org.apache.hadoop.test.MetricsAsserts.assertCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; @@ -26,18 +26,18 @@ import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.OzoneConsts; -import org.apache.hadoop.test.Whitebox; import org.junit.After; import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; /** - * Test for KSM metrics. + * Test for OM metrics. */ -public class TestKSMMetrcis { +@SuppressWarnings("deprecation") +public class TestOmMetrics { private MiniOzoneCluster cluster; - private KeySpaceManager ksmManager; + private OzoneManager ozoneManager; /** * The exception used for testing failure metrics. @@ -56,7 +56,7 @@ public class TestKSMMetrcis { OzoneConsts.OZONE_HANDLER_DISTRIBUTED); cluster = MiniOzoneCluster.newBuilder(conf).build(); cluster.waitForClusterToBeReady(); - ksmManager = cluster.getKeySpaceManager(); + ozoneManager = cluster.getOzoneManager(); } /** @@ -71,8 +71,9 @@ public class TestKSMMetrcis { @Test public void testVolumeOps() throws IOException { - VolumeManager volumeManager = (VolumeManager) Whitebox - .getInternalState(ksmManager, "volumeManager"); + VolumeManager volumeManager = + (VolumeManager) org.apache.hadoop.test.Whitebox + .getInternalState(ozoneManager, "volumeManager"); VolumeManager mockVm = Mockito.spy(volumeManager); Mockito.doNothing().when(mockVm).createVolume(null); @@ -82,17 +83,18 @@ public class TestKSMMetrcis { Mockito.doNothing().when(mockVm).setOwner(null, null); Mockito.doReturn(null).when(mockVm).listVolumes(null, null, null, 0); - Whitebox.setInternalState(ksmManager, "volumeManager", mockVm); + org.apache.hadoop.test.Whitebox.setInternalState( + ozoneManager, "volumeManager", mockVm); doVolumeOps(); - MetricsRecordBuilder ksmMetrics = getMetrics("KSMMetrics"); - assertCounter("NumVolumeOps", 6L, ksmMetrics); - assertCounter("NumVolumeCreates", 1L, ksmMetrics); - assertCounter("NumVolumeUpdates", 1L, ksmMetrics); - assertCounter("NumVolumeInfos", 1L, ksmMetrics); - assertCounter("NumVolumeCheckAccesses", 1L, ksmMetrics); - assertCounter("NumVolumeDeletes", 1L, ksmMetrics); - assertCounter("NumVolumeLists", 1L, ksmMetrics); + MetricsRecordBuilder omMetrics = getMetrics("OMMetrics"); + assertCounter("NumVolumeOps", 6L, omMetrics); + assertCounter("NumVolumeCreates", 1L, omMetrics); + assertCounter("NumVolumeUpdates", 1L, omMetrics); + assertCounter("NumVolumeInfos", 1L, omMetrics); + assertCounter("NumVolumeCheckAccesses", 1L, omMetrics); + assertCounter("NumVolumeDeletes", 1L, omMetrics); + assertCounter("NumVolumeLists", 1L, omMetrics); // inject exception to test for Failure Metrics Mockito.doThrow(exception).when(mockVm).createVolume(null); @@ -102,30 +104,31 @@ public class TestKSMMetrcis { Mockito.doThrow(exception).when(mockVm).setOwner(null, null); Mockito.doThrow(exception).when(mockVm).listVolumes(null, null, null, 0); - Whitebox.setInternalState(ksmManager, "volumeManager", mockVm); + org.apache.hadoop.test.Whitebox.setInternalState(ozoneManager, "volumeManager", mockVm); doVolumeOps(); - ksmMetrics = getMetrics("KSMMetrics"); - assertCounter("NumVolumeOps", 12L, ksmMetrics); - assertCounter("NumVolumeCreates", 2L, ksmMetrics); - assertCounter("NumVolumeUpdates", 2L, ksmMetrics); - assertCounter("NumVolumeInfos", 2L, ksmMetrics); - assertCounter("NumVolumeCheckAccesses", 2L, ksmMetrics); - assertCounter("NumVolumeDeletes", 2L, ksmMetrics); - assertCounter("NumVolumeLists", 2L, ksmMetrics); + omMetrics = getMetrics("OMMetrics"); + assertCounter("NumVolumeOps", 12L, omMetrics); + assertCounter("NumVolumeCreates", 2L, omMetrics); + assertCounter("NumVolumeUpdates", 2L, omMetrics); + assertCounter("NumVolumeInfos", 2L, omMetrics); + assertCounter("NumVolumeCheckAccesses", 2L, omMetrics); + assertCounter("NumVolumeDeletes", 2L, omMetrics); + assertCounter("NumVolumeLists", 2L, omMetrics); - assertCounter("NumVolumeCreateFails", 1L, ksmMetrics); - assertCounter("NumVolumeUpdateFails", 1L, ksmMetrics); - assertCounter("NumVolumeInfoFails", 1L, ksmMetrics); - assertCounter("NumVolumeCheckAccessFails", 1L, ksmMetrics); - assertCounter("NumVolumeDeleteFails", 1L, ksmMetrics); - assertCounter("NumVolumeListFails", 1L, ksmMetrics); + assertCounter("NumVolumeCreateFails", 1L, omMetrics); + assertCounter("NumVolumeUpdateFails", 1L, omMetrics); + assertCounter("NumVolumeInfoFails", 1L, omMetrics); + assertCounter("NumVolumeCheckAccessFails", 1L, omMetrics); + assertCounter("NumVolumeDeleteFails", 1L, omMetrics); + assertCounter("NumVolumeListFails", 1L, omMetrics); } @Test public void testBucketOps() throws IOException { - BucketManager bucketManager = (BucketManager) Whitebox - .getInternalState(ksmManager, "bucketManager"); + BucketManager bucketManager = + (BucketManager) org.apache.hadoop.test.Whitebox + .getInternalState(ozoneManager, "bucketManager"); BucketManager mockBm = Mockito.spy(bucketManager); Mockito.doNothing().when(mockBm).createBucket(null); @@ -134,16 +137,17 @@ public class TestKSMMetrcis { Mockito.doNothing().when(mockBm).setBucketProperty(null); Mockito.doReturn(null).when(mockBm).listBuckets(null, null, null, 0); - Whitebox.setInternalState(ksmManager, "bucketManager", mockBm); + org.apache.hadoop.test.Whitebox.setInternalState( + ozoneManager, "bucketManager", mockBm); doBucketOps(); - MetricsRecordBuilder ksmMetrics = getMetrics("KSMMetrics"); - assertCounter("NumBucketOps", 5L, ksmMetrics); - assertCounter("NumBucketCreates", 1L, ksmMetrics); - assertCounter("NumBucketUpdates", 1L, ksmMetrics); - assertCounter("NumBucketInfos", 1L, ksmMetrics); - assertCounter("NumBucketDeletes", 1L, ksmMetrics); - assertCounter("NumBucketLists", 1L, ksmMetrics); + MetricsRecordBuilder omMetrics = getMetrics("OMMetrics"); + assertCounter("NumBucketOps", 5L, omMetrics); + assertCounter("NumBucketCreates", 1L, omMetrics); + assertCounter("NumBucketUpdates", 1L, omMetrics); + assertCounter("NumBucketInfos", 1L, omMetrics); + assertCounter("NumBucketDeletes", 1L, omMetrics); + assertCounter("NumBucketLists", 1L, omMetrics); // inject exception to test for Failure Metrics Mockito.doThrow(exception).when(mockBm).createBucket(null); @@ -152,28 +156,29 @@ public class TestKSMMetrcis { Mockito.doThrow(exception).when(mockBm).setBucketProperty(null); Mockito.doThrow(exception).when(mockBm).listBuckets(null, null, null, 0); - Whitebox.setInternalState(ksmManager, "bucketManager", mockBm); + org.apache.hadoop.test.Whitebox.setInternalState( + ozoneManager, "bucketManager", mockBm); doBucketOps(); - ksmMetrics = getMetrics("KSMMetrics"); - assertCounter("NumBucketOps", 10L, ksmMetrics); - assertCounter("NumBucketCreates", 2L, ksmMetrics); - assertCounter("NumBucketUpdates", 2L, ksmMetrics); - assertCounter("NumBucketInfos", 2L, ksmMetrics); - assertCounter("NumBucketDeletes", 2L, ksmMetrics); - assertCounter("NumBucketLists", 2L, ksmMetrics); + omMetrics = getMetrics("OMMetrics"); + assertCounter("NumBucketOps", 10L, omMetrics); + assertCounter("NumBucketCreates", 2L, omMetrics); + assertCounter("NumBucketUpdates", 2L, omMetrics); + assertCounter("NumBucketInfos", 2L, omMetrics); + assertCounter("NumBucketDeletes", 2L, omMetrics); + assertCounter("NumBucketLists", 2L, omMetrics); - assertCounter("NumBucketCreateFails", 1L, ksmMetrics); - assertCounter("NumBucketUpdateFails", 1L, ksmMetrics); - assertCounter("NumBucketInfoFails", 1L, ksmMetrics); - assertCounter("NumBucketDeleteFails", 1L, ksmMetrics); - assertCounter("NumBucketListFails", 1L, ksmMetrics); + assertCounter("NumBucketCreateFails", 1L, omMetrics); + assertCounter("NumBucketUpdateFails", 1L, omMetrics); + assertCounter("NumBucketInfoFails", 1L, omMetrics); + assertCounter("NumBucketDeleteFails", 1L, omMetrics); + assertCounter("NumBucketListFails", 1L, omMetrics); } @Test public void testKeyOps() throws IOException { - KeyManager bucketManager = (KeyManager) Whitebox - .getInternalState(ksmManager, "keyManager"); + KeyManager bucketManager = (KeyManager) org.apache.hadoop.test.Whitebox + .getInternalState(ozoneManager, "keyManager"); KeyManager mockKm = Mockito.spy(bucketManager); Mockito.doReturn(null).when(mockKm).openKey(null); @@ -181,15 +186,16 @@ public class TestKSMMetrcis { Mockito.doReturn(null).when(mockKm).lookupKey(null); Mockito.doReturn(null).when(mockKm).listKeys(null, null, null, null, 0); - Whitebox.setInternalState(ksmManager, "keyManager", mockKm); + org.apache.hadoop.test.Whitebox.setInternalState( + ozoneManager, "keyManager", mockKm); doKeyOps(); - MetricsRecordBuilder ksmMetrics = getMetrics("KSMMetrics"); - assertCounter("NumKeyOps", 4L, ksmMetrics); - assertCounter("NumKeyAllocate", 1L, ksmMetrics); - assertCounter("NumKeyLookup", 1L, ksmMetrics); - assertCounter("NumKeyDeletes", 1L, ksmMetrics); - assertCounter("NumKeyLists", 1L, ksmMetrics); + MetricsRecordBuilder omMetrics = getMetrics("OMMetrics"); + assertCounter("NumKeyOps", 4L, omMetrics); + assertCounter("NumKeyAllocate", 1L, omMetrics); + assertCounter("NumKeyLookup", 1L, omMetrics); + assertCounter("NumKeyDeletes", 1L, omMetrics); + assertCounter("NumKeyLists", 1L, omMetrics); // inject exception to test for Failure Metrics Mockito.doThrow(exception).when(mockKm).openKey(null); @@ -198,20 +204,21 @@ public class TestKSMMetrcis { Mockito.doThrow(exception).when(mockKm).listKeys( null, null, null, null, 0); - Whitebox.setInternalState(ksmManager, "keyManager", mockKm); + org.apache.hadoop.test.Whitebox.setInternalState( + ozoneManager, "keyManager", mockKm); doKeyOps(); - ksmMetrics = getMetrics("KSMMetrics"); - assertCounter("NumKeyOps", 8L, ksmMetrics); - assertCounter("NumKeyAllocate", 2L, ksmMetrics); - assertCounter("NumKeyLookup", 2L, ksmMetrics); - assertCounter("NumKeyDeletes", 2L, ksmMetrics); - assertCounter("NumKeyLists", 2L, ksmMetrics); + omMetrics = getMetrics("OMMetrics"); + assertCounter("NumKeyOps", 8L, omMetrics); + assertCounter("NumKeyAllocate", 2L, omMetrics); + assertCounter("NumKeyLookup", 2L, omMetrics); + assertCounter("NumKeyDeletes", 2L, omMetrics); + assertCounter("NumKeyLists", 2L, omMetrics); - assertCounter("NumKeyAllocateFails", 1L, ksmMetrics); - assertCounter("NumKeyLookupFails", 1L, ksmMetrics); - assertCounter("NumKeyDeleteFails", 1L, ksmMetrics); - assertCounter("NumKeyListFails", 1L, ksmMetrics); + assertCounter("NumKeyAllocateFails", 1L, omMetrics); + assertCounter("NumKeyLookupFails", 1L, omMetrics); + assertCounter("NumKeyDeleteFails", 1L, omMetrics); + assertCounter("NumKeyListFails", 1L, omMetrics); } /** @@ -219,32 +226,32 @@ public class TestKSMMetrcis { */ private void doVolumeOps() { try { - ksmManager.createVolume(null); + ozoneManager.createVolume(null); } catch (IOException ignored) { } try { - ksmManager.deleteVolume(null); + ozoneManager.deleteVolume(null); } catch (IOException ignored) { } try { - ksmManager.getVolumeInfo(null); + ozoneManager.getVolumeInfo(null); } catch (IOException ignored) { } try { - ksmManager.checkVolumeAccess(null, null); + ozoneManager.checkVolumeAccess(null, null); } catch (IOException ignored) { } try { - ksmManager.setOwner(null, null); + ozoneManager.setOwner(null, null); } catch (IOException ignored) { } try { - ksmManager.listAllVolumes(null, null, 0); + ozoneManager.listAllVolumes(null, null, 0); } catch (IOException ignored) { } } @@ -254,27 +261,27 @@ public class TestKSMMetrcis { */ private void doBucketOps() { try { - ksmManager.createBucket(null); + ozoneManager.createBucket(null); } catch (IOException ignored) { } try { - ksmManager.deleteBucket(null, null); + ozoneManager.deleteBucket(null, null); } catch (IOException ignored) { } try { - ksmManager.getBucketInfo(null, null); + ozoneManager.getBucketInfo(null, null); } catch (IOException ignored) { } try { - ksmManager.setBucketProperty(null); + ozoneManager.setBucketProperty(null); } catch (IOException ignored) { } try { - ksmManager.listBuckets(null, null, null, 0); + ozoneManager.listBuckets(null, null, null, 0); } catch (IOException ignored) { } } @@ -284,22 +291,22 @@ public class TestKSMMetrcis { */ private void doKeyOps() { try { - ksmManager.openKey(null); + ozoneManager.openKey(null); } catch (IOException ignored) { } try { - ksmManager.deleteKey(null); + ozoneManager.deleteKey(null); } catch (IOException ignored) { } try { - ksmManager.lookupKey(null); + ozoneManager.lookupKey(null); } catch (IOException ignored) { } try { - ksmManager.listKeys(null, null, null, null, 0); + ozoneManager.listKeys(null, null, null, null, 0); } catch (IOException ignored) { } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMSQLCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java similarity index 96% rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMSQLCli.java rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java index 7b92ec75bcf..005a0124e8f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKSMSQLCli.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java @@ -14,7 +14,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler; import org.apache.hadoop.ozone.MiniOzoneCluster; @@ -51,17 +51,17 @@ import java.util.LinkedList; import java.util.List; import java.util.UUID; -import static org.apache.hadoop.ozone.OzoneConsts.KSM_DB_NAME; +import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; /** - * This class tests the CLI that transforms ksm.db into SQLite DB files. + * This class tests the CLI that transforms om.db into SQLite DB files. */ @RunWith(Parameterized.class) -public class TestKSMSQLCli { +public class TestOmSQLCli { private MiniOzoneCluster cluster = null; private StorageHandler storageHandler; private UserArgs userArgs; @@ -90,7 +90,7 @@ public class TestKSMSQLCli { private String metaStoreType; - public TestKSMSQLCli(String type) { + public TestOmSQLCli(String type) { metaStoreType = type; } @@ -152,7 +152,7 @@ public class TestKSMSQLCli { stream = storageHandler.newKeyWriter(keyArgs3); stream.close(); - cluster.getKeySpaceManager().stop(); + cluster.getOzoneManager().stop(); cluster.getStorageContainerManager().stop(); conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, metaStoreType); cli = new SQLCLI(conf); @@ -166,12 +166,12 @@ public class TestKSMSQLCli { } @Test - public void testKSMDB() throws Exception { + public void testOmDB() throws Exception { String dbOutPath = GenericTestUtils.getTempPath( UUID.randomUUID() + "/out_sql.db"); String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS); - String dbPath = dbRootPath + "/" + KSM_DB_NAME; + String dbPath = dbRootPath + "/" + OM_DB_NAME; String[] args = {"-p", dbPath, "-o", dbOutPath}; cli.run(args); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java similarity index 90% rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java index 8a16bfe86d2..7c8595ca029 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java @@ -14,7 +14,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.commons.lang3.RandomStringUtils; @@ -29,11 +29,11 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.common.BlockGroup; import org.apache.hadoop.ozone.client.rest.OzoneException; -import org.apache.hadoop.ozone.ksm.exceptions.KSMException; +import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.hdds.scm.server.SCMStorage; -import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ServicePort; + .OzoneManagerProtocolProtos.ServicePort; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.ozone.web.handlers.BucketArgs; import org.apache.hadoop.ozone.web.handlers.KeyArgs; @@ -50,7 +50,7 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ScmInfo; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.Status; + .OzoneManagerProtocolProtos.Status; import org.apache.hadoop.ozone.web.handlers.ListArgs; import org.apache.hadoop.ozone.web.response.ListBuckets; import org.apache.hadoop.ozone.web.response.ListKeys; @@ -74,7 +74,6 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.net.InetSocketAddress; import java.text.ParseException; -import java.util.HashSet; import java.util.LinkedList; import java.util.Map; import java.util.Random; @@ -86,22 +85,22 @@ import java.util.stream.Stream; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS; import static org.apache.hadoop.ozone.OzoneConsts.DELETING_KEY_PREFIX; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_ADDRESS_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; import static org.apache.hadoop.hdds.scm.ScmConfigKeys .OZONE_SCM_CLIENT_ADDRESS_KEY; /** - * Test Key Space Manager operation in distributed handler scenario. + * Test Ozone Manager operation in distributed handler scenario. */ -public class TestKeySpaceManager { +public class TestOzoneManager { private static MiniOzoneCluster cluster = null; private static StorageHandler storageHandler; private static UserArgs userArgs; - private static KSMMetrics ksmMetrics; + private static OMMetrics omMetrics; private static OzoneConfiguration conf; private static String clusterId; private static String scmId; - private static String ksmId; + private static String omId; @Rule public ExpectedException exception = ExpectedException.none(); @@ -119,20 +118,20 @@ public class TestKeySpaceManager { conf = new OzoneConfiguration(); clusterId = UUID.randomUUID().toString(); scmId = UUID.randomUUID().toString(); - ksmId = UUID.randomUUID().toString(); + omId = UUID.randomUUID().toString(); conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, OzoneConsts.OZONE_HANDLER_DISTRIBUTED); conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2); cluster = MiniOzoneCluster.newBuilder(conf) .setClusterId(clusterId) .setScmId(scmId) - .setKsmId(ksmId) + .setOmId(omId) .build(); cluster.waitForClusterToBeReady(); storageHandler = new ObjectStoreHandler(conf).getStorageHandler(); userArgs = new UserArgs(null, OzoneUtils.getRequestID(), null, null, null, null); - ksmMetrics = cluster.getKeySpaceManager().getMetrics(); + omMetrics = cluster.getOzoneManager().getMetrics(); } /** @@ -148,7 +147,7 @@ public class TestKeySpaceManager { // Create a volume and test its attribute after creating them @Test(timeout = 60000) public void testCreateVolume() throws IOException, OzoneException { - long volumeCreateFailCount = ksmMetrics.getNumVolumeCreateFails(); + long volumeCreateFailCount = omMetrics.getNumVolumeCreateFails(); String userName = "user" + RandomStringUtils.randomNumeric(5); String adminName = "admin" + RandomStringUtils.randomNumeric(5); String volumeName = "volume" + RandomStringUtils.randomNumeric(5); @@ -163,14 +162,14 @@ public class TestKeySpaceManager { Assert.assertTrue(retVolumeinfo.getVolumeName().equals(volumeName)); Assert.assertTrue(retVolumeinfo.getOwner().getName().equals(userName)); Assert.assertEquals(volumeCreateFailCount, - ksmMetrics.getNumVolumeCreateFails()); + omMetrics.getNumVolumeCreateFails()); } // Create a volume and modify the volume owner and then test its attributes @Test(timeout = 60000) public void testChangeVolumeOwner() throws IOException, OzoneException { - long volumeCreateFailCount = ksmMetrics.getNumVolumeCreateFails(); - long volumeInfoFailCount = ksmMetrics.getNumVolumeInfoFails(); + long volumeCreateFailCount = omMetrics.getNumVolumeCreateFails(); + long volumeInfoFailCount = omMetrics.getNumVolumeInfoFails(); String userName = "user" + RandomStringUtils.randomNumeric(5); String adminName = "admin" + RandomStringUtils.randomNumeric(5); String volumeName = "volume" + RandomStringUtils.randomNumeric(5); @@ -191,16 +190,16 @@ public class TestKeySpaceManager { Assert.assertFalse(retVolumeInfo.getOwner().getName().equals(userName)); Assert.assertTrue(retVolumeInfo.getOwner().getName().equals(newUserName)); Assert.assertEquals(volumeCreateFailCount, - ksmMetrics.getNumVolumeCreateFails()); + omMetrics.getNumVolumeCreateFails()); Assert.assertEquals(volumeInfoFailCount, - ksmMetrics.getNumVolumeInfoFails()); + omMetrics.getNumVolumeInfoFails()); } // Create a volume and modify the volume owner and then test its attributes @Test(timeout = 60000) public void testChangeVolumeQuota() throws IOException, OzoneException { - long numVolumeCreateFail = ksmMetrics.getNumVolumeCreateFails(); - long numVolumeInfoFail = ksmMetrics.getNumVolumeInfoFails(); + long numVolumeCreateFail = omMetrics.getNumVolumeCreateFails(); + long numVolumeInfoFail = omMetrics.getNumVolumeInfoFails(); String userName = "user" + RandomStringUtils.randomNumeric(5); String adminName = "admin" + RandomStringUtils.randomNumeric(5); String volumeName = "volume" + RandomStringUtils.randomNumeric(5); @@ -237,15 +236,15 @@ public class TestKeySpaceManager { Assert.assertEquals(OzoneConsts.MAX_QUOTA_IN_BYTES, retVolumeInfo.getQuota().sizeInBytes()); Assert.assertEquals(numVolumeCreateFail, - ksmMetrics.getNumVolumeCreateFails()); + omMetrics.getNumVolumeCreateFails()); Assert.assertEquals(numVolumeInfoFail, - ksmMetrics.getNumVolumeInfoFails()); + omMetrics.getNumVolumeInfoFails()); } // Create a volume and then delete it and then check for deletion @Test(timeout = 60000) public void testDeleteVolume() throws IOException, OzoneException { - long volumeCreateFailCount = ksmMetrics.getNumVolumeCreateFails(); + long volumeCreateFailCount = omMetrics.getNumVolumeCreateFails(); String userName = "user" + RandomStringUtils.randomNumeric(5); String adminName = "admin" + RandomStringUtils.randomNumeric(5); String volumeName = "volume" + RandomStringUtils.randomNumeric(5); @@ -270,7 +269,7 @@ public class TestKeySpaceManager { Assert.assertTrue(volumeInfo.getVolumeName().equals(volumeName1)); Assert.assertTrue(volumeInfo.getOwner().getName().equals(userName)); Assert.assertEquals(volumeCreateFailCount, - ksmMetrics.getNumVolumeCreateFails()); + omMetrics.getNumVolumeCreateFails()); // Volume with _A should be able to delete as it is empty. storageHandler.deleteVolume(volumeArgs); @@ -291,7 +290,7 @@ public class TestKeySpaceManager { // then delete it and then check for deletion failure @Test(timeout = 60000) public void testFailedDeleteVolume() throws IOException, OzoneException { - long numVolumeCreateFails = ksmMetrics.getNumVolumeCreateFails(); + long numVolumeCreateFails = omMetrics.getNumVolumeCreateFails(); String userName = "user" + RandomStringUtils.randomNumeric(5); String adminName = "admin" + RandomStringUtils.randomNumeric(5); String volumeName = "volume" + RandomStringUtils.randomNumeric(5); @@ -307,7 +306,7 @@ public class TestKeySpaceManager { Assert.assertTrue(retVolumeInfo.getVolumeName().equals(volumeName)); Assert.assertTrue(retVolumeInfo.getOwner().getName().equals(userName)); Assert.assertEquals(numVolumeCreateFails, - ksmMetrics.getNumVolumeCreateFails()); + omMetrics.getNumVolumeCreateFails()); BucketArgs bucketArgs = new BucketArgs(volumeName, bucketName, userArgs); storageHandler.createBucket(bucketArgs); @@ -366,15 +365,15 @@ public class TestKeySpaceManager { OzoneAcl.OzoneACLRights.READ); Assert.assertFalse(storageHandler.checkVolumeAccess(volumeName, worldAcl)); - Assert.assertEquals(0, ksmMetrics.getNumVolumeCheckAccessFails()); - Assert.assertEquals(0, ksmMetrics.getNumVolumeCreateFails()); + Assert.assertEquals(0, omMetrics.getNumVolumeCheckAccessFails()); + Assert.assertEquals(0, omMetrics.getNumVolumeCreateFails()); } @Test(timeout = 60000) public void testCreateBucket() throws IOException, OzoneException { - long numVolumeCreateFail = ksmMetrics.getNumVolumeCreateFails(); - long numBucketCreateFail = ksmMetrics.getNumBucketCreateFails(); - long numBucketInfoFail = ksmMetrics.getNumBucketInfoFails(); + long numVolumeCreateFail = omMetrics.getNumVolumeCreateFails(); + long numBucketCreateFail = omMetrics.getNumBucketCreateFails(); + long numBucketInfoFail = omMetrics.getNumBucketInfoFails(); String userName = "user" + RandomStringUtils.randomNumeric(5); String adminName = "admin" + RandomStringUtils.randomNumeric(5); String volumeName = "volume" + RandomStringUtils.randomNumeric(5); @@ -394,11 +393,11 @@ public class TestKeySpaceManager { Assert.assertTrue(bucketInfo.getVolumeName().equals(volumeName)); Assert.assertTrue(bucketInfo.getBucketName().equals(bucketName)); Assert.assertEquals(numVolumeCreateFail, - ksmMetrics.getNumVolumeCreateFails()); + omMetrics.getNumVolumeCreateFails()); Assert.assertEquals(numBucketCreateFail, - ksmMetrics.getNumBucketCreateFails()); + omMetrics.getNumBucketCreateFails()); Assert.assertEquals(numBucketInfoFail, - ksmMetrics.getNumBucketInfoFails()); + omMetrics.getNumBucketInfoFails()); } @Test(timeout = 60000) @@ -479,7 +478,7 @@ public class TestKeySpaceManager { } /** - * Basic test of both putKey and getKey from KSM, as one can not be tested + * Basic test of both putKey and getKey from OM, as one can not be tested * without the other. * * @throws IOException @@ -492,8 +491,8 @@ public class TestKeySpaceManager { String volumeName = "volume" + RandomStringUtils.randomNumeric(5); String bucketName = "bucket" + RandomStringUtils.randomNumeric(5); String keyName = "key" + RandomStringUtils.randomNumeric(5); - long numKeyAllocates = ksmMetrics.getNumKeyAllocates(); - long numKeyLookups = ksmMetrics.getNumKeyLookups(); + long numKeyAllocates = omMetrics.getNumKeyAllocates(); + long numKeyLookups = omMetrics.getNumKeyLookups(); VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs); createVolumeArgs.setUserName(userName); @@ -512,14 +511,14 @@ public class TestKeySpaceManager { try (OutputStream stream = storageHandler.newKeyWriter(keyArgs)) { stream.write(dataString.getBytes()); } - Assert.assertEquals(1 + numKeyAllocates, ksmMetrics.getNumKeyAllocates()); + Assert.assertEquals(1 + numKeyAllocates, omMetrics.getNumKeyAllocates()); byte[] data = new byte[dataString.length()]; try (InputStream in = storageHandler.newKeyReader(keyArgs)) { in.read(data); } Assert.assertEquals(dataString, DFSUtil.bytes2String(data)); - Assert.assertEquals(1 + numKeyLookups, ksmMetrics.getNumKeyLookups()); + Assert.assertEquals(1 + numKeyLookups, omMetrics.getNumKeyLookups()); } /** @@ -536,7 +535,7 @@ public class TestKeySpaceManager { String volumeName = "volume" + RandomStringUtils.randomNumeric(5); String bucketName = "bucket" + RandomStringUtils.randomNumeric(5); String keyName = "key" + RandomStringUtils.randomNumeric(5); - long numKeyAllocateFails = ksmMetrics.getNumKeyAllocateFails(); + long numKeyAllocateFails = omMetrics.getNumKeyAllocateFails(); VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs); createVolumeArgs.setUserName(userName); @@ -558,12 +557,12 @@ public class TestKeySpaceManager { // We allow the key overwrite to be successful. Please note : Till // HDFS-11922 is fixed this causes a data block leak on the data node side. - // That is this overwrite only overwrites the keys on KSM. We need to + // That is this overwrite only overwrites the keys on OM. We need to // garbage collect those blocks from datanode. KeyArgs keyArgs2 = new KeyArgs(volumeName, bucketName, keyName, userArgs); storageHandler.newKeyWriter(keyArgs2); Assert - .assertEquals(numKeyAllocateFails, ksmMetrics.getNumKeyAllocateFails()); + .assertEquals(numKeyAllocateFails, omMetrics.getNumKeyAllocateFails()); } /** @@ -579,7 +578,7 @@ public class TestKeySpaceManager { String volumeName = "volume" + RandomStringUtils.randomNumeric(5); String bucketName = "bucket" + RandomStringUtils.randomNumeric(5); String keyName = "key" + RandomStringUtils.randomNumeric(5); - long numKeyLookupFails = ksmMetrics.getNumKeyLookupFails(); + long numKeyLookupFails = omMetrics.getNumKeyLookupFails(); VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs); createVolumeArgs.setUserName(userName); @@ -598,11 +597,11 @@ public class TestKeySpaceManager { exception.expectMessage("KEY_NOT_FOUND"); storageHandler.newKeyReader(keyArgs); Assert.assertEquals(1 + numKeyLookupFails, - ksmMetrics.getNumKeyLookupFails()); + omMetrics.getNumKeyLookupFails()); } /** - * Test delete keys for ksm. + * Test delete keys for om. * * @throws IOException * @throws OzoneException @@ -614,8 +613,8 @@ public class TestKeySpaceManager { String volumeName = "volume" + RandomStringUtils.randomNumeric(5); String bucketName = "bucket" + RandomStringUtils.randomNumeric(5); String keyName = "key" + RandomStringUtils.randomNumeric(5); - long numKeyDeletes = ksmMetrics.getNumKeyDeletes(); - long numKeyDeleteFails = ksmMetrics.getNumKeyDeletesFails(); + long numKeyDeletes = omMetrics.getNumKeyDeletes(); + long numKeyDeleteFails = omMetrics.getNumKeyDeletesFails(); VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs); createVolumeArgs.setUserName(userName); @@ -633,10 +632,10 @@ public class TestKeySpaceManager { } storageHandler.deleteKey(keyArgs); - Assert.assertEquals(1 + numKeyDeletes, ksmMetrics.getNumKeyDeletes()); + Assert.assertEquals(1 + numKeyDeletes, omMetrics.getNumKeyDeletes()); // Make sure the deleted key has been renamed. - MetadataStore store = cluster.getKeySpaceManager(). + MetadataStore store = cluster.getOzoneManager(). getMetadataManager().getStore(); List> list = store.getRangeKVs(null, 10, new MetadataKeyFilters.KeyPrefixFilter() @@ -651,11 +650,11 @@ public class TestKeySpaceManager { Assert.assertTrue(ioe.getMessage().contains("KEY_NOT_FOUND")); } Assert.assertEquals(1 + numKeyDeleteFails, - ksmMetrics.getNumKeyDeletesFails()); + omMetrics.getNumKeyDeletesFails()); } /** - * Test rename key for ksm. + * Test rename key for om. * * @throws IOException * @throws OzoneException @@ -667,8 +666,8 @@ public class TestKeySpaceManager { String volumeName = "volume" + RandomStringUtils.randomNumeric(5); String bucketName = "bucket" + RandomStringUtils.randomNumeric(5); String keyName = "key" + RandomStringUtils.randomNumeric(5); - long numKeyRenames = ksmMetrics.getNumKeyRenames(); - long numKeyRenameFails = ksmMetrics.getNumKeyRenameFails(); + long numKeyRenames = omMetrics.getNumKeyRenames(); + long numKeyRenameFails = omMetrics.getNumKeyRenameFails(); int testRenameFails = 0; int testRenames = 0; IOException ioe = null; @@ -706,9 +705,9 @@ public class TestKeySpaceManager { testRenames++; storageHandler.renameKey(keyArgs, toKeyName); Assert.assertEquals(numKeyRenames + testRenames, - ksmMetrics.getNumKeyRenames()); + omMetrics.getNumKeyRenames()); Assert.assertEquals(numKeyRenameFails + testRenameFails, - ksmMetrics.getNumKeyRenameFails()); + omMetrics.getNumKeyRenameFails()); // Try to get the key, should fail as it has been renamed try { @@ -764,9 +763,9 @@ public class TestKeySpaceManager { Assert.assertTrue(ioe.getMessage().contains("Rename key failed, error")); Assert.assertEquals(numKeyRenames + testRenames, - ksmMetrics.getNumKeyRenames()); + omMetrics.getNumKeyRenames()); Assert.assertEquals(numKeyRenameFails + testRenameFails, - ksmMetrics.getNumKeyRenameFails()); + omMetrics.getNumKeyRenameFails()); } @Test(timeout = 60000) @@ -1183,7 +1182,7 @@ public class TestKeySpaceManager { */ @Test public void testGetScmInfo() throws IOException { - ScmInfo info = cluster.getKeySpaceManager().getScmInfo(); + ScmInfo info = cluster.getOzoneManager().getScmInfo(); Assert.assertEquals(clusterId, info.getClusterId()); Assert.assertEquals(scmId, info.getScmId()); } @@ -1192,7 +1191,7 @@ public class TestKeySpaceManager { @Test public void testExpiredOpenKey() throws Exception { BackgroundService openKeyCleanUpService = ((KeyManagerImpl)cluster - .getKeySpaceManager().getKeyManager()).getOpenKeyCleanupService(); + .getOzoneManager().getKeyManager()).getOpenKeyCleanupService(); String userName = "user" + RandomStringUtils.randomNumeric(5); String adminName = "admin" + RandomStringUtils.randomNumeric(5); @@ -1228,7 +1227,7 @@ public class TestKeySpaceManager { // Now all k1-k4 should be in open state, so ExpiredOpenKeys should not // contain these values. - openKeys = cluster.getKeySpaceManager() + openKeys = cluster.getOzoneManager() .getMetadataManager().getExpiredOpenKeys(); for (BlockGroup bg : openKeys) { @@ -1239,7 +1238,7 @@ public class TestKeySpaceManager { Thread.sleep(2000); // Now all k1-k4 should be in ExpiredOpenKeys - openKeys = cluster.getKeySpaceManager() + openKeys = cluster.getOzoneManager() .getMetadataManager().getExpiredOpenKeys(); for (BlockGroup bg : openKeys) { String[] subs = bg.getGroupID().split("/"); @@ -1258,7 +1257,7 @@ public class TestKeySpaceManager { // now all k1-k4 should have been removed by the clean-up task, only k5 // should be present in ExpiredOpenKeys. openKeys = - cluster.getKeySpaceManager().getMetadataManager().getExpiredOpenKeys(); + cluster.getOzoneManager().getMetadataManager().getExpiredOpenKeys(); System.out.println(openKeys); boolean key5found = false; Set removed = Stream.of( @@ -1276,68 +1275,68 @@ public class TestKeySpaceManager { } /** - * Tests the KSM Initialization. + * Tests the OM Initialization. * @throws IOException */ @Test - public void testKSMInitialization() throws IOException { - // Read the version file info from KSM version file - KSMStorage ksmStorage = cluster.getKeySpaceManager().getKsmStorage(); + public void testOmInitialization() throws IOException { + // Read the version file info from OM version file + OMStorage omStorage = cluster.getOzoneManager().getOmStorage(); SCMStorage scmStorage = new SCMStorage(conf); // asserts whether cluster Id and SCM ID are properly set in SCM Version // file. Assert.assertEquals(clusterId, scmStorage.getClusterID()); Assert.assertEquals(scmId, scmStorage.getScmId()); - // asserts whether KSM Id is properly set in KSM Version file. - Assert.assertEquals(ksmId, ksmStorage.getKsmId()); - // asserts whether the SCM info is correct in KSM Version file. - Assert.assertEquals(clusterId, ksmStorage.getClusterID()); - Assert.assertEquals(scmId, ksmStorage.getScmId()); + // asserts whether OM Id is properly set in OM Version file. + Assert.assertEquals(omId, omStorage.getOmId()); + // asserts whether the SCM info is correct in OM Version file. + Assert.assertEquals(clusterId, omStorage.getClusterID()); + Assert.assertEquals(scmId, omStorage.getScmId()); } /** - * Tests the KSM Initialization Failure. + * Tests the OM Initialization Failure. * @throws IOException */ @Test - public void testKSMInitializationFailure() throws Exception { + public void testOmInitializationFailure() throws Exception { OzoneConfiguration config = new OzoneConfiguration(); final String path = GenericTestUtils.getTempPath(UUID.randomUUID().toString()); - Path metaDirPath = Paths.get(path, "ksm-meta"); + Path metaDirPath = Paths.get(path, "om-meta"); config.set(OzoneConfigKeys.OZONE_METADATA_DIRS, metaDirPath.toString()); config.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true); config.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "127.0.0.1:0"); config.set(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY, conf.get(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY)); - exception.expect(KSMException.class); - exception.expectMessage("KSM not initialized."); - KeySpaceManager.createKSM(null, config); - KSMStorage ksmStore = new KSMStorage(config); - ksmStore.setClusterId("testClusterId"); - ksmStore.setScmId("testScmId"); + exception.expect(OMException.class); + exception.expectMessage("OM not initialized."); + OzoneManager.createOm(null, config); + OMStorage omStore = new OMStorage(config); + omStore.setClusterId("testClusterId"); + omStore.setScmId("testScmId"); // writes the version file properties - ksmStore.initialize(); - exception.expect(KSMException.class); + omStore.initialize(); + exception.expect(OMException.class); exception.expectMessage("SCM version info mismatch."); - KeySpaceManager.createKSM(null, conf); + OzoneManager.createOm(null, conf); } @Test public void testGetServiceList() throws IOException { - long numGetServiceListCalls = ksmMetrics.getNumGetServiceLists(); - List services = cluster.getKeySpaceManager().getServiceList(); + long numGetServiceListCalls = omMetrics.getNumGetServiceLists(); + List services = cluster.getOzoneManager().getServiceList(); Assert.assertEquals(numGetServiceListCalls + 1, - ksmMetrics.getNumGetServiceLists()); + omMetrics.getNumGetServiceLists()); - ServiceInfo ksmInfo = services.stream().filter( - a -> a.getNodeType().equals(HddsProtos.NodeType.KSM)) + ServiceInfo omInfo = services.stream().filter( + a -> a.getNodeType().equals(HddsProtos.NodeType.OM)) .collect(Collectors.toList()).get(0); - InetSocketAddress ksmAddress = new InetSocketAddress(ksmInfo.getHostname(), - ksmInfo.getPort(ServicePort.Type.RPC)); + InetSocketAddress omAddress = new InetSocketAddress(omInfo.getHostname(), + omInfo.getPort(ServicePort.Type.RPC)); Assert.assertEquals(NetUtils.createSocketAddr( - conf.get(OZONE_KSM_ADDRESS_KEY)), ksmAddress); + conf.get(OZONE_OM_ADDRESS_KEY)), omAddress); ServiceInfo scmInfo = services.stream().filter( a -> a.getNodeType().equals(HddsProtos.NodeType.SCM)) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerRestInterface.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerRestInterface.java similarity index 83% rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerRestInterface.java rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerRestInterface.java index feb83d3e5ab..8168d27a5d1 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerRestInterface.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerRestInterface.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.core.type.TypeReference; @@ -24,9 +24,9 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.ozone.MiniOzoneCluster; -import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ServicePort; + .OzoneManagerProtocolProtos.ServicePort; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.http.HttpResponse; import org.apache.http.client.HttpClient; @@ -44,12 +44,12 @@ import java.util.List; import java.util.Map; import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients; -import static org.apache.hadoop.ozone.KsmUtils.getKsmAddressForClients; +import static org.apache.hadoop.ozone.OmUtils.getOmAddressForClients; /** - * This class is to test the REST interface exposed by KeySpaceManager. + * This class is to test the REST interface exposed by OzoneManager. */ -public class TestKeySpaceManagerRestInterface { +public class TestOzoneManagerRestInterface { private static MiniOzoneCluster cluster; private static OzoneConfiguration conf; @@ -70,8 +70,8 @@ public class TestKeySpaceManagerRestInterface { @Test public void testGetServiceList() throws Exception { - KeySpaceManagerHttpServer server = - cluster.getKeySpaceManager().getHttpServer(); + OzoneManagerHttpServer server = + cluster.getOzoneManager().getHttpServer(); HttpClient client = HttpClients.createDefault(); String connectionUri = "http://" + NetUtils.getHostPortString(server.getHttpAddress()); @@ -89,15 +89,15 @@ public class TestKeySpaceManagerRestInterface { serviceMap.put(serviceInfo.getNodeType(), serviceInfo); } - InetSocketAddress ksmAddress = - getKsmAddressForClients(conf); - ServiceInfo ksmInfo = serviceMap.get(HddsProtos.NodeType.KSM); + InetSocketAddress omAddress = + getOmAddressForClients(conf); + ServiceInfo omInfo = serviceMap.get(HddsProtos.NodeType.OM); - Assert.assertEquals(ksmAddress.getHostName(), ksmInfo.getHostname()); - Assert.assertEquals(ksmAddress.getPort(), - ksmInfo.getPort(ServicePort.Type.RPC)); + Assert.assertEquals(omAddress.getHostName(), omInfo.getHostname()); + Assert.assertEquals(omAddress.getPort(), + omInfo.getPort(ServicePort.Type.RPC)); Assert.assertEquals(server.getHttpAddress().getPort(), - ksmInfo.getPort(ServicePort.Type.HTTP)); + omInfo.getPort(ServicePort.Type.HTTP)); InetSocketAddress scmAddress = getScmAddressForClients(conf); @@ -123,7 +123,7 @@ public class TestKeySpaceManagerRestInterface { ports.get(type)); break; default: - // KSM only sends Datanode's info port details + // OM only sends Datanode's info port details // i.e. HTTP or HTTPS // Other ports are not expected as of now. Assert.fail(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java index ed8f0d5c570..508287082e8 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java @@ -59,7 +59,7 @@ import org.apache.hadoop.ozone.client.protocol.ClientProtocol; import org.apache.hadoop.ozone.client.rest.OzoneException; import org.apache.hadoop.ozone.client.rest.RestClient; import org.apache.hadoop.ozone.client.rpc.RpcClient; -import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; import org.apache.hadoop.ozone.web.ozShell.Shell; import org.apache.hadoop.ozone.web.request.OzoneQuota; import org.apache.hadoop.ozone.web.response.BucketInfo; @@ -167,23 +167,23 @@ public class TestOzoneShell { System.setOut(new PrintStream(out)); System.setErr(new PrintStream(err)); if(clientProtocol.equals(RestClient.class)) { - String hostName = cluster.getKeySpaceManager().getHttpServer() + String hostName = cluster.getOzoneManager().getHttpServer() .getHttpAddress().getHostName(); int port = cluster - .getKeySpaceManager().getHttpServer().getHttpAddress().getPort(); + .getOzoneManager().getHttpServer().getHttpAddress().getPort(); url = String.format("http://" + hostName + ":" + port); } else { List services = null; try { - services = cluster.getKeySpaceManager().getServiceList(); + services = cluster.getOzoneManager().getServiceList(); } catch (IOException e) { - LOG.error("Could not get service list from KSM"); + LOG.error("Could not get service list from OM"); } String hostName = services.stream().filter( - a -> a.getNodeType().equals(HddsProtos.NodeType.KSM)) + a -> a.getNodeType().equals(HddsProtos.NodeType.OM)) .collect(Collectors.toList()).get(0).getHostname(); - String port = cluster.getKeySpaceManager().getRpcPort(); + String port = cluster.getOzoneManager().getRpcPort(); url = String.format("o3://" + hostName + ":" + port); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java index b4ed2b12c2f..1a1f37ca375 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacem import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementCapacity; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; -import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline; import org.apache.hadoop.ozone.scm.cli.SQLCLI; import org.apache.hadoop.test.GenericTestUtils; import org.junit.After; @@ -113,7 +112,7 @@ public class TestContainerSQLCli { cluster.waitForClusterToBeReady(); datanodeIpAddress = cluster.getHddsDatanodes().get(0) .getDatanodeDetails().getIpAddress(); - cluster.getKeySpaceManager().stop(); + cluster.getOzoneManager().stop(); cluster.getStorageContainerManager().stop(); nodeManager = cluster.getStorageContainerManager().getScmNodeManager(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java index 0e61391c634..e592d560d3f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java @@ -90,7 +90,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper { @Test public void testCreateVolumes() throws IOException { super.testCreateVolumes(port); - Assert.assertEquals(0, cluster.getKeySpaceManager() + Assert.assertEquals(0, cluster.getOzoneManager() .getMetrics().getNumVolumeCreateFails()); } @@ -102,7 +102,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper { @Test public void testCreateVolumesWithQuota() throws IOException { super.testCreateVolumesWithQuota(port); - Assert.assertEquals(0, cluster.getKeySpaceManager() + Assert.assertEquals(0, cluster.getOzoneManager() .getMetrics().getNumVolumeCreateFails()); } @@ -114,7 +114,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper { @Test public void testCreateVolumesWithInvalidQuota() throws IOException { super.testCreateVolumesWithInvalidQuota(port); - Assert.assertEquals(0, cluster.getKeySpaceManager() + Assert.assertEquals(0, cluster.getOzoneManager() .getMetrics().getNumVolumeCreateFails()); } @@ -128,7 +128,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper { @Test public void testCreateVolumesWithInvalidUser() throws IOException { super.testCreateVolumesWithInvalidUser(port); - Assert.assertEquals(0, cluster.getKeySpaceManager() + Assert.assertEquals(0, cluster.getOzoneManager() .getMetrics().getNumVolumeCreateFails()); } @@ -143,7 +143,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper { @Test public void testCreateVolumesWithOutAdminRights() throws IOException { super.testCreateVolumesWithOutAdminRights(port); - Assert.assertEquals(0, cluster.getKeySpaceManager() + Assert.assertEquals(0, cluster.getOzoneManager() .getMetrics().getNumVolumeCreateFails()); } @@ -155,7 +155,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper { @Test public void testCreateVolumesInLoop() throws IOException { super.testCreateVolumesInLoop(port); - Assert.assertEquals(0, cluster.getKeySpaceManager() + Assert.assertEquals(0, cluster.getOzoneManager() .getMetrics().getNumVolumeCreateFails()); } /** diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java index b86c57721c8..a95bd0e65f2 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java @@ -48,13 +48,13 @@ import org.apache.hadoop.ozone.container.common.helpers.ContainerData; import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils; import org.apache.hadoop.ozone.container.common.helpers.KeyData; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; -import org.apache.hadoop.ozone.ksm.KeySpaceManager; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos +import org.apache.hadoop.ozone.om.OzoneManager; +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.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .Status; import org.apache.hadoop.ozone.client.rest.OzoneException; import org.apache.hadoop.ozone.web.utils.OzoneUtils; @@ -644,15 +644,15 @@ public class TestKeys { } } - private int countKsmKeys(KeySpaceManager ksm) throws IOException { + private int countOmKeys(OzoneManager om) throws IOException { int totalCount = 0; - List volumes = - ksm.listAllVolumes(null, null, Integer.MAX_VALUE); - for (KsmVolumeArgs volume : volumes) { - List buckets = - ksm.listBuckets(volume.getVolume(), null, null, Integer.MAX_VALUE); - for (KsmBucketInfo bucket : buckets) { - List keys = ksm.listKeys(bucket.getVolumeName(), + List volumes = + om.listAllVolumes(null, null, Integer.MAX_VALUE); + for (OmVolumeArgs volume : volumes) { + List buckets = + om.listBuckets(volume.getVolume(), null, null, Integer.MAX_VALUE); + for (OmBucketInfo bucket : buckets) { + List keys = om.listKeys(bucket.getVolumeName(), bucket.getBucketName(), null, null, Integer.MAX_VALUE); totalCount += keys.size(); } @@ -662,10 +662,10 @@ public class TestKeys { @Test public void testDeleteKey() throws Exception { - KeySpaceManager ksm = ozoneCluster.getKeySpaceManager(); + OzoneManager ozoneManager = ozoneCluster.getOzoneManager(); // To avoid interference from other test cases, // we collect number of existing keys at the beginning - int numOfExistedKeys = countKsmKeys(ksm); + int numOfExistedKeys = countOmKeys(ozoneManager); // Keep tracking bucket keys info while creating them PutHelper helper = new PutHelper(client, path); @@ -689,15 +689,15 @@ public class TestKeys { // count the total number of created keys. Set> buckets = bucketKeys.getAllBuckets(); for (Pair buk : buckets) { - List createdKeys = - ksm.listKeys(buk.getKey(), buk.getValue(), null, null, 20); + List createdKeys = + ozoneManager.listKeys(buk.getKey(), buk.getValue(), null, null, 20); // Memorize chunks that has been created, // so we can verify actual deletions at DN side later. - for (KsmKeyInfo keyInfo : createdKeys) { - List locations = + for (OmKeyInfo keyInfo : createdKeys) { + List locations = keyInfo.getLatestVersionLocations().getLocationList(); - for (KsmKeyLocationInfo location : locations) { + for (OmKeyLocationInfo location : locations) { KeyData keyData = new KeyData(location.getBlockID()); KeyData blockInfo = cm.getContainerManager() .getKeyManager().getKey(keyData); @@ -721,9 +721,9 @@ public class TestKeys { // Ensure all keys are created. Assert.assertEquals(20, numOfCreatedKeys); - // Ensure all keys are visible from KSM. + // Ensure all keys are visible from OM. // Total number should be numOfCreated + numOfExisted - Assert.assertEquals(20 + numOfExistedKeys, countKsmKeys(ksm)); + Assert.assertEquals(20 + numOfExistedKeys, countOmKeys(ozoneManager)); // Delete 10 keys int delCount = 20; @@ -732,21 +732,21 @@ public class TestKeys { List bks = bucketKeys.getBucketKeys(bucketInfo.getValue()); for (String keyName : bks) { if (delCount > 0) { - KsmKeyArgs arg = - new KsmKeyArgs.Builder().setVolumeName(bucketInfo.getKey()) + OmKeyArgs arg = + new OmKeyArgs.Builder().setVolumeName(bucketInfo.getKey()) .setBucketName(bucketInfo.getValue()).setKeyName(keyName) .build(); - ksm.deleteKey(arg); + ozoneManager.deleteKey(arg); delCount--; } } } - // It should be pretty quick that keys are removed from KSM namespace, + // It should be pretty quick that keys are removed from OM namespace, // because actual deletion happens in async mode. GenericTestUtils.waitFor(() -> { try { - int num = countKsmKeys(ksm); + int num = countOmKeys(ozoneManager); return num == (numOfExistedKeys); } catch (IOException e) { return false; diff --git a/hadoop-ozone/integration-test/src/test/resources/webapps/ksm/.gitkeep b/hadoop-ozone/integration-test/src/test/resources/webapps/ozoneManager/.gitkeep similarity index 100% rename from hadoop-ozone/integration-test/src/test/resources/webapps/ksm/.gitkeep rename to hadoop-ozone/integration-test/src/test/resources/webapps/ozoneManager/.gitkeep diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java index 3128d31e808..2200cd8887a 100644 --- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java +++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.datanode; import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForBlockClients; import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients; -import static org.apache.hadoop.ozone.KsmUtils.getKsmAddress; +import static org.apache.hadoop.ozone.OmUtils.getOmAddress; import static org.apache.hadoop.ozone.OzoneConfigKeys.*; import static com.sun.jersey.api.core.ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS; import static com.sun.jersey.api.core.ResourceConfig.FEATURE_TRACE; @@ -34,9 +34,8 @@ import com.sun.jersey.api.container.ContainerFactory; import com.sun.jersey.api.core.ApplicationAdapter; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.ozone.ksm.protocolPB - .KeySpaceManagerProtocolClientSideTranslatorPB; -import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolPB; +import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB; +import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.web.ObjectStoreApplication; import org.apache.hadoop.ozone.web.handlers.ServiceFilter; @@ -72,8 +71,8 @@ public final class ObjectStoreHandler implements Closeable { LoggerFactory.getLogger(ObjectStoreHandler.class); private final ObjectStoreJerseyContainer objectStoreJerseyContainer; - private final KeySpaceManagerProtocolClientSideTranslatorPB - keySpaceManagerClient; + private final OzoneManagerProtocolClientSideTranslatorPB + ozoneManagerClient; private final StorageContainerLocationProtocolClientSideTranslatorPB storageContainerLocationClient; private final ScmBlockLocationProtocolClientSideTranslatorPB @@ -119,28 +118,28 @@ public final class ObjectStoreHandler implements Closeable { NetUtils.getDefaultSocketFactory(conf), Client.getRpcTimeout(conf))); - RPC.setProtocolEngine(conf, KeySpaceManagerProtocolPB.class, + RPC.setProtocolEngine(conf, OzoneManagerProtocolPB.class, ProtobufRpcEngine.class); - long ksmVersion = - RPC.getProtocolVersion(KeySpaceManagerProtocolPB.class); - InetSocketAddress ksmAddress = getKsmAddress(conf); - this.keySpaceManagerClient = - new KeySpaceManagerProtocolClientSideTranslatorPB( - RPC.getProxy(KeySpaceManagerProtocolPB.class, ksmVersion, - ksmAddress, UserGroupInformation.getCurrentUser(), conf, + long omVersion = + RPC.getProtocolVersion(OzoneManagerProtocolPB.class); + InetSocketAddress omAddress = getOmAddress(conf); + this.ozoneManagerClient = + new OzoneManagerProtocolClientSideTranslatorPB( + RPC.getProxy(OzoneManagerProtocolPB.class, omVersion, + omAddress, UserGroupInformation.getCurrentUser(), conf, NetUtils.getDefaultSocketFactory(conf), Client.getRpcTimeout(conf))); storageHandler = new DistributedStorageHandler( new OzoneConfiguration(conf), this.storageContainerLocationClient, - this.keySpaceManagerClient); + this.ozoneManagerClient); } else { if (OzoneConsts.OZONE_HANDLER_LOCAL.equalsIgnoreCase(shType)) { storageHandler = new LocalStorageHandler(conf); this.storageContainerLocationClient = null; this.scmBlockLocationClient = null; - this.keySpaceManagerClient = null; + this.ozoneManagerClient = null; } else { throw new IllegalArgumentException( String.format("Unrecognized value for %s: %s," @@ -186,6 +185,6 @@ public final class ObjectStoreHandler implements Closeable { storageHandler.close(); IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient); IOUtils.cleanupWithLogger(LOG, scmBlockLocationClient); - IOUtils.cleanupWithLogger(LOG, keySpaceManagerClient); + IOUtils.cleanupWithLogger(LOG, ozoneManagerClient); } } diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java index ef0293e7255..ad487873187 100644 --- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java +++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java @@ -21,7 +21,7 @@ package org.apache.hadoop.ozone.web.handlers; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.ozone.OzoneRestUtils; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.web.exceptions.ErrorTable; import org.apache.hadoop.ozone.client.rest.OzoneException; import org.apache.hadoop.ozone.client.rest.headers.Header; @@ -102,7 +102,7 @@ public abstract class KeyProcessTemplate { LOG.error("IOException:", fsExp); // Map KEY_NOT_FOUND to INVALID_KEY if (fsExp.getMessage().endsWith( - KeySpaceManagerProtocolProtos.Status.KEY_NOT_FOUND.name())) { + OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND.name())) { throw ErrorTable.newError(ErrorTable.INVALID_KEY, userArgs, fsExp); } diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java index 1d98400f318..fb95bb9a9f5 100644 --- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java +++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java @@ -30,7 +30,7 @@ import java.nio.file.NoSuchFileException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.ozone.OzoneRestUtils; import org.apache.hadoop.ozone.client.rest.OzoneException; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.web.exceptions.ErrorTable; import org.apache.hadoop.ozone.web.interfaces.StorageHandler; import org.apache.hadoop.ozone.web.interfaces.UserAuth; @@ -135,7 +135,7 @@ public abstract class VolumeProcessTemplate { OzoneException exp = null; if ((fsExp != null && fsExp.getMessage().endsWith( - KeySpaceManagerProtocolProtos.Status.VOLUME_ALREADY_EXISTS.name())) + OzoneManagerProtocolProtos.Status.VOLUME_ALREADY_EXISTS.name())) || fsExp instanceof FileAlreadyExistsException) { exp = ErrorTable .newError(ErrorTable.VOLUME_ALREADY_EXISTS, reqID, volume, hostName); diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java index fedc0f007a0..ec33990de43 100644 --- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java +++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java @@ -22,14 +22,13 @@ import com.google.common.base.Strings; import org.apache.hadoop.hdds.scm.client.HddsClientUtils; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.client.io.LengthInputStream; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; -import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession; -import org.apache.hadoop.ozone.ksm.protocolPB - .KeySpaceManagerProtocolClientSideTranslatorPB; +import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; +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.om.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.ozone.OzoneConsts; @@ -37,9 +36,9 @@ import org.apache.hadoop.ozone.OzoneConsts.Versioning; import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream; import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos; -import org.apache.hadoop.ozone.protocolPB.KSMPBHelper; -import org.apache.hadoop.ozone.ksm.KSMConfigKeys; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.apache.hadoop.ozone.protocolPB.OMPBHelper; +import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.web.request.OzoneQuota; import org.apache.hadoop.hdds.scm.ScmConfigKeys; @@ -77,8 +76,8 @@ public final class DistributedStorageHandler implements StorageHandler { private final StorageContainerLocationProtocolClientSideTranslatorPB storageContainerLocationClient; - private final KeySpaceManagerProtocolClientSideTranslatorPB - keySpaceManagerClient; + private final OzoneManagerProtocolClientSideTranslatorPB + ozoneManagerClient; private final XceiverClientManager xceiverClientManager; private final OzoneAcl.OzoneACLRights userRights; private final OzoneAcl.OzoneACLRights groupRights; @@ -92,14 +91,14 @@ public final class DistributedStorageHandler implements StorageHandler { * * @param conf configuration * @param storageContainerLocation StorageContainerLocationProtocol proxy - * @param keySpaceManagerClient KeySpaceManager proxy + * @param ozoneManagerClient OzoneManager proxy */ public DistributedStorageHandler(OzoneConfiguration conf, StorageContainerLocationProtocolClientSideTranslatorPB storageContainerLocation, - KeySpaceManagerProtocolClientSideTranslatorPB - keySpaceManagerClient) { - this.keySpaceManagerClient = keySpaceManagerClient; + OzoneManagerProtocolClientSideTranslatorPB + ozoneManagerClient) { + this.ozoneManagerClient = ozoneManagerClient; this.storageContainerLocationClient = storageContainerLocation; this.xceiverClientManager = new XceiverClientManager(conf); this.useRatis = conf.getBoolean( @@ -116,10 +115,10 @@ public final class DistributedStorageHandler implements StorageHandler { chunkSize = conf.getInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY, ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT); - userRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_USER_RIGHTS, - KSMConfigKeys.OZONE_KSM_USER_RIGHTS_DEFAULT); - groupRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS, - KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS_DEFAULT); + userRights = conf.getEnum(OMConfigKeys.OZONE_OM_USER_RIGHTS, + OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT); + groupRights = conf.getEnum(OMConfigKeys.OZONE_OM_GROUP_RIGHTS, + OMConfigKeys.OZONE_OM_GROUP_RIGHTS_DEFAULT); if(chunkSize > ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE) { LOG.warn("The chunk size ({}) is not allowed to be more than" + " the maximum size ({})," @@ -136,26 +135,26 @@ public final class DistributedStorageHandler implements StorageHandler { OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, args.getUserName(), userRights); - KsmVolumeArgs.Builder builder = KsmVolumeArgs.newBuilder(); + OmVolumeArgs.Builder builder = OmVolumeArgs.newBuilder(); builder.setAdminName(args.getAdminName()) .setOwnerName(args.getUserName()) .setVolume(args.getVolumeName()) .setQuotaInBytes(quota) - .addOzoneAcls(KSMPBHelper.convertOzoneAcl(userAcl)); + .addOzoneAcls(OMPBHelper.convertOzoneAcl(userAcl)); if (args.getGroups() != null) { for (String group : args.getGroups()) { OzoneAcl groupAcl = new OzoneAcl(OzoneAcl.OzoneACLType.GROUP, group, groupRights); - builder.addOzoneAcls(KSMPBHelper.convertOzoneAcl(groupAcl)); + builder.addOzoneAcls(OMPBHelper.convertOzoneAcl(groupAcl)); } } - keySpaceManagerClient.createVolume(builder.build()); + ozoneManagerClient.createVolume(builder.build()); } @Override public void setVolumeOwner(VolumeArgs args) throws IOException, OzoneException { - keySpaceManagerClient.setOwner(args.getVolumeName(), args.getUserName()); + ozoneManagerClient.setOwner(args.getVolumeName(), args.getUserName()); } @Override @@ -163,14 +162,14 @@ public final class DistributedStorageHandler implements StorageHandler { throws IOException, OzoneException { long quota = remove ? OzoneConsts.MAX_QUOTA_IN_BYTES : args.getQuota().sizeInBytes(); - keySpaceManagerClient.setQuota(args.getVolumeName(), quota); + ozoneManagerClient.setQuota(args.getVolumeName(), quota); } @Override public boolean checkVolumeAccess(String volume, OzoneAcl acl) throws IOException, OzoneException { - return keySpaceManagerClient - .checkVolumeAccess(volume, KSMPBHelper.convertOzoneAcl(acl)); + return ozoneManagerClient + .checkVolumeAccess(volume, OMPBHelper.convertOzoneAcl(acl)); } @Override @@ -185,9 +184,9 @@ public final class DistributedStorageHandler implements StorageHandler { OzoneConsts.MAX_LISTVOLUMES_SIZE, maxNumOfKeys)); } - List listResult; + List listResult; if (args.isRootScan()) { - listResult = keySpaceManagerClient.listAllVolumes(args.getPrefix(), + listResult = ozoneManagerClient.listAllVolumes(args.getPrefix(), args.getPrevKey(), args.getMaxKeys()); } else { UserArgs userArgs = args.getArgs(); @@ -195,16 +194,16 @@ public final class DistributedStorageHandler implements StorageHandler { throw new IllegalArgumentException("Illegal argument," + " missing user argument."); } - listResult = keySpaceManagerClient.listVolumeByUser( + listResult = ozoneManagerClient.listVolumeByUser( args.getArgs().getUserName(), args.getPrefix(), args.getPrevKey(), args.getMaxKeys()); } // TODO Add missing fields createdBy, bucketCount and bytesUsed ListVolumes result = new ListVolumes(); - for (KsmVolumeArgs volumeArgs : listResult) { + for (OmVolumeArgs volumeArgs : listResult) { VolumeInfo info = new VolumeInfo(); - KeySpaceManagerProtocolProtos.VolumeInfo + OzoneManagerProtocolProtos.VolumeInfo infoProto = volumeArgs.getProtobuf(); info.setOwner(new VolumeOwner(infoProto.getOwnerName())); info.setQuota(OzoneQuota.getOzoneQuota(infoProto.getQuotaInBytes())); @@ -220,14 +219,14 @@ public final class DistributedStorageHandler implements StorageHandler { @Override public void deleteVolume(VolumeArgs args) throws IOException, OzoneException { - keySpaceManagerClient.deleteVolume(args.getVolumeName()); + ozoneManagerClient.deleteVolume(args.getVolumeName()); } @Override public VolumeInfo getVolumeInfo(VolumeArgs args) throws IOException, OzoneException { - KsmVolumeArgs volumeArgs = - keySpaceManagerClient.getVolumeInfo(args.getVolumeName()); + OmVolumeArgs volumeArgs = + ozoneManagerClient.getVolumeInfo(args.getVolumeName()); //TODO: add support for createdOn and other fields in getVolumeInfo VolumeInfo volInfo = new VolumeInfo(volumeArgs.getVolume(), null, @@ -242,7 +241,7 @@ public final class DistributedStorageHandler implements StorageHandler { @Override public void createBucket(final BucketArgs args) throws IOException, OzoneException { - KsmBucketInfo.Builder builder = KsmBucketInfo.newBuilder(); + OmBucketInfo.Builder builder = OmBucketInfo.newBuilder(); builder.setVolumeName(args.getVolumeName()) .setBucketName(args.getBucketName()); if(args.getAddAcls() != null) { @@ -255,7 +254,7 @@ public final class DistributedStorageHandler implements StorageHandler { builder.setIsVersionEnabled(getBucketVersioningProtobuf( args.getVersioning())); } - keySpaceManagerClient.createBucket(builder.build()); + ozoneManagerClient.createBucket(builder.build()); } /** @@ -285,7 +284,7 @@ public final class DistributedStorageHandler implements StorageHandler { List removeAcls = args.getRemoveAcls(); List addAcls = args.getAddAcls(); if(removeAcls != null || addAcls != null) { - KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder(); + OmBucketArgs.Builder builder = OmBucketArgs.newBuilder(); builder.setVolumeName(args.getVolumeName()) .setBucketName(args.getBucketName()); if(removeAcls != null && !removeAcls.isEmpty()) { @@ -294,35 +293,35 @@ public final class DistributedStorageHandler implements StorageHandler { if(addAcls != null && !addAcls.isEmpty()) { builder.setAddAcls(args.getAddAcls()); } - keySpaceManagerClient.setBucketProperty(builder.build()); + ozoneManagerClient.setBucketProperty(builder.build()); } } @Override public void setBucketVersioning(BucketArgs args) throws IOException, OzoneException { - KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder(); + OmBucketArgs.Builder builder = OmBucketArgs.newBuilder(); builder.setVolumeName(args.getVolumeName()) .setBucketName(args.getBucketName()) .setIsVersionEnabled(getBucketVersioningProtobuf( args.getVersioning())); - keySpaceManagerClient.setBucketProperty(builder.build()); + ozoneManagerClient.setBucketProperty(builder.build()); } @Override public void setBucketStorageClass(BucketArgs args) throws IOException, OzoneException { - KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder(); + OmBucketArgs.Builder builder = OmBucketArgs.newBuilder(); builder.setVolumeName(args.getVolumeName()) .setBucketName(args.getBucketName()) .setStorageType(args.getStorageType()); - keySpaceManagerClient.setBucketProperty(builder.build()); + ozoneManagerClient.setBucketProperty(builder.build()); } @Override public void deleteBucket(BucketArgs args) throws IOException, OzoneException { - keySpaceManagerClient.deleteBucket(args.getVolumeName(), + ozoneManagerClient.deleteBucket(args.getVolumeName(), args.getBucketName()); } @@ -354,12 +353,12 @@ public final class DistributedStorageHandler implements StorageHandler { OzoneConsts.MAX_LISTBUCKETS_SIZE, maxNumOfKeys)); } - List buckets = - keySpaceManagerClient.listBuckets(va.getVolumeName(), + List buckets = + ozoneManagerClient.listBuckets(va.getVolumeName(), args.getPrevKey(), args.getPrefix(), args.getMaxKeys()); // Convert the result for the web layer. - for (KsmBucketInfo bucketInfo : buckets) { + for (OmBucketInfo bucketInfo : buckets) { BucketInfo bk = new BucketInfo(); bk.setVolumeName(bucketInfo.getVolumeName()); bk.setBucketName(bucketInfo.getBucketName()); @@ -382,26 +381,26 @@ public final class DistributedStorageHandler implements StorageHandler { throws IOException { String volumeName = args.getVolumeName(); String bucketName = args.getBucketName(); - KsmBucketInfo ksmBucketInfo = keySpaceManagerClient.getBucketInfo( + OmBucketInfo omBucketInfo = ozoneManagerClient.getBucketInfo( volumeName, bucketName); - BucketInfo bucketInfo = new BucketInfo(ksmBucketInfo.getVolumeName(), - ksmBucketInfo.getBucketName()); - if(ksmBucketInfo.getIsVersionEnabled()) { + BucketInfo bucketInfo = new BucketInfo(omBucketInfo.getVolumeName(), + omBucketInfo.getBucketName()); + if(omBucketInfo.getIsVersionEnabled()) { bucketInfo.setVersioning(Versioning.ENABLED); } else { bucketInfo.setVersioning(Versioning.DISABLED); } - bucketInfo.setStorageType(ksmBucketInfo.getStorageType()); - bucketInfo.setAcls(ksmBucketInfo.getAcls()); + bucketInfo.setStorageType(omBucketInfo.getStorageType()); + bucketInfo.setAcls(omBucketInfo.getAcls()); bucketInfo.setCreatedOn( - HddsClientUtils.formatDateTime(ksmBucketInfo.getCreationTime())); + HddsClientUtils.formatDateTime(omBucketInfo.getCreationTime())); return bucketInfo; } @Override public OutputStream newKeyWriter(KeyArgs args) throws IOException, OzoneException { - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(args.getVolumeName()) .setBucketName(args.getBucketName()) .setKeyName(args.getKeyName()) @@ -409,14 +408,14 @@ public final class DistributedStorageHandler implements StorageHandler { .setType(xceiverClientManager.getType()) .setFactor(xceiverClientManager.getFactor()) .build(); - // contact KSM to allocate a block for key. - OpenKeySession openKey = keySpaceManagerClient.openKey(keyArgs); + // contact OM to allocate a block for key. + OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs); ChunkGroupOutputStream groupOutputStream = new ChunkGroupOutputStream.Builder() .setHandler(openKey) .setXceiverClientManager(xceiverClientManager) .setScmClient(storageContainerLocationClient) - .setKsmClient(keySpaceManagerClient) + .setOmClient(ozoneManagerClient) .setChunkSize(chunkSize) .setRequestID(args.getRequestID()) .setType(xceiverClientManager.getType()) @@ -437,56 +436,56 @@ public final class DistributedStorageHandler implements StorageHandler { @Override public LengthInputStream newKeyReader(KeyArgs args) throws IOException, OzoneException { - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(args.getVolumeName()) .setBucketName(args.getBucketName()) .setKeyName(args.getKeyName()) .setDataSize(args.getSize()) .build(); - KsmKeyInfo keyInfo = keySpaceManagerClient.lookupKey(keyArgs); - return ChunkGroupInputStream.getFromKsmKeyInfo( + OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs); + return ChunkGroupInputStream.getFromOmKeyInfo( keyInfo, xceiverClientManager, storageContainerLocationClient, args.getRequestID()); } @Override public void deleteKey(KeyArgs args) throws IOException, OzoneException { - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(args.getVolumeName()) .setBucketName(args.getBucketName()) .setKeyName(args.getKeyName()) .build(); - keySpaceManagerClient.deleteKey(keyArgs); + ozoneManagerClient.deleteKey(keyArgs); } @Override public void renameKey(KeyArgs args, String toKeyName) throws IOException, OzoneException { - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(args.getVolumeName()) .setBucketName(args.getBucketName()) .setKeyName(args.getKeyName()) .build(); - keySpaceManagerClient.renameKey(keyArgs, toKeyName); + ozoneManagerClient.renameKey(keyArgs, toKeyName); } @Override public KeyInfo getKeyInfo(KeyArgs args) throws IOException, OzoneException { - KsmKeyArgs keyArgs = new KsmKeyArgs.Builder() + OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(args.getVolumeName()) .setBucketName(args.getBucketName()) .setKeyName(args.getKeyName()) .build(); - KsmKeyInfo ksmKeyInfo = keySpaceManagerClient.lookupKey(keyArgs); + OmKeyInfo omKeyInfo = ozoneManagerClient.lookupKey(keyArgs); KeyInfo keyInfo = new KeyInfo(); keyInfo.setVersion(0); - keyInfo.setKeyName(ksmKeyInfo.getKeyName()); - keyInfo.setSize(ksmKeyInfo.getDataSize()); + keyInfo.setKeyName(omKeyInfo.getKeyName()); + keyInfo.setSize(omKeyInfo.getDataSize()); keyInfo.setCreatedOn( - HddsClientUtils.formatDateTime(ksmKeyInfo.getCreationTime())); + HddsClientUtils.formatDateTime(omKeyInfo.getCreationTime())); keyInfo.setModifiedOn( - HddsClientUtils.formatDateTime(ksmKeyInfo.getModificationTime())); + HddsClientUtils.formatDateTime(omKeyInfo.getModificationTime())); return keyInfo; } @@ -515,13 +514,13 @@ public final class DistributedStorageHandler implements StorageHandler { OzoneConsts.MAX_LISTKEYS_SIZE, maxNumOfKeys)); } - List keys= - keySpaceManagerClient.listKeys(bucketArgs.getVolumeName(), + List keys= + ozoneManagerClient.listKeys(bucketArgs.getVolumeName(), bucketArgs.getBucketName(), args.getPrevKey(), args.getPrefix(), args.getMaxKeys()); // Convert the result for the web layer. - for (KsmKeyInfo info : keys) { + for (OmKeyInfo info : keys) { KeyInfo tempInfo = new KeyInfo(); tempInfo.setVersion(0); tempInfo.setKeyName(info.getKeyName()); @@ -547,7 +546,7 @@ public final class DistributedStorageHandler implements StorageHandler { @Override public void close() { IOUtils.cleanupWithLogger(LOG, xceiverClientManager); - IOUtils.cleanupWithLogger(LOG, keySpaceManagerClient); + IOUtils.cleanupWithLogger(LOG, ozoneManagerClient); IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient); } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/BucketManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManager.java similarity index 78% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/BucketManager.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManager.java index 6c756913d59..ddb2b0e26d3 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/BucketManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManager.java @@ -14,10 +14,10 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import java.io.IOException; import java.util.List; @@ -28,15 +28,15 @@ import java.util.List; public interface BucketManager { /** * Creates a bucket. - * @param bucketInfo - KsmBucketInfo for creating bucket. + * @param bucketInfo - OmBucketInfo for creating bucket. */ - void createBucket(KsmBucketInfo bucketInfo) throws IOException; + void createBucket(OmBucketInfo bucketInfo) throws IOException; /** * Returns Bucket Information. * @param volumeName - Name of the Volume. * @param bucketName - Name of the Bucket. */ - KsmBucketInfo getBucketInfo(String volumeName, String bucketName) + OmBucketInfo getBucketInfo(String volumeName, String bucketName) throws IOException; /** @@ -44,7 +44,7 @@ public interface BucketManager { * @param args - BucketArgs. * @throws IOException */ - void setBucketProperty(KsmBucketArgs args) throws IOException; + void setBucketProperty(OmBucketArgs args) throws IOException; /** * Deletes an existing empty bucket from volume. @@ -55,7 +55,7 @@ public interface BucketManager { void deleteBucket(String volumeName, String bucketName) throws IOException; /** - * Returns a list of buckets represented by {@link KsmBucketInfo} + * Returns a list of buckets represented by {@link OmBucketInfo} * in the given volume. * * @param volumeName @@ -73,7 +73,7 @@ public interface BucketManager { * @return a list of buckets. * @throws IOException */ - List listBuckets(String volumeName, - String startBucket, String bucketPrefix, int maxNumOfBuckets) + List listBuckets(String volumeName, + String startBucket, String bucketPrefix, int maxNumOfBuckets) throws IOException; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java similarity index 79% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java index 957a6d97778..4bbce8101bd 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java @@ -14,15 +14,15 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import com.google.common.base.Preconditions; import org.apache.hadoop.fs.StorageType; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; -import org.apache.hadoop.ozone.ksm.exceptions.KSMException; +import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.BucketInfo; + .OzoneManagerProtocolProtos.BucketInfo; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.util.Time; import org.iq80.leveldb.DBException; @@ -33,22 +33,22 @@ import java.io.IOException; import java.util.List; /** - * KSM bucket manager. + * OM bucket manager. */ public class BucketManagerImpl implements BucketManager { private static final Logger LOG = LoggerFactory.getLogger(BucketManagerImpl.class); /** - * KSMMetadataManager is used for accessing KSM MetadataDB and ReadWriteLock. + * OMMetadataManager is used for accessing OM MetadataDB and ReadWriteLock. */ - private final KSMMetadataManager metadataManager; + private final OMMetadataManager metadataManager; /** * Constructs BucketManager. * @param metadataManager */ - public BucketManagerImpl(KSMMetadataManager metadataManager){ + public BucketManagerImpl(OMMetadataManager metadataManager){ this.metadataManager = metadataManager; } @@ -73,10 +73,10 @@ public class BucketManagerImpl implements BucketManager { /** * Creates a bucket. - * @param bucketInfo - KsmBucketInfo. + * @param bucketInfo - OmBucketInfo. */ @Override - public void createBucket(KsmBucketInfo bucketInfo) throws IOException { + public void createBucket(OmBucketInfo bucketInfo) throws IOException { Preconditions.checkNotNull(bucketInfo); metadataManager.writeLock().lock(); String volumeName = bucketInfo.getVolumeName(); @@ -88,17 +88,17 @@ public class BucketManagerImpl implements BucketManager { //Check if the volume exists if (metadataManager.get(volumeKey) == null) { LOG.debug("volume: {} not found ", volumeName); - throw new KSMException("Volume doesn't exist", - KSMException.ResultCodes.FAILED_VOLUME_NOT_FOUND); + throw new OMException("Volume doesn't exist", + OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND); } //Check if bucket already exists if (metadataManager.get(bucketKey) != null) { LOG.debug("bucket: {} already exists ", bucketName); - throw new KSMException("Bucket already exist", - KSMException.ResultCodes.FAILED_BUCKET_ALREADY_EXISTS); + throw new OMException("Bucket already exist", + OMException.ResultCodes.FAILED_BUCKET_ALREADY_EXISTS); } - KsmBucketInfo ksmBucketInfo = KsmBucketInfo.newBuilder() + OmBucketInfo omBucketInfo = OmBucketInfo.newBuilder() .setVolumeName(bucketInfo.getVolumeName()) .setBucketName(bucketInfo.getBucketName()) .setAcls(bucketInfo.getAcls()) @@ -106,11 +106,11 @@ public class BucketManagerImpl implements BucketManager { .setIsVersionEnabled(bucketInfo.getIsVersionEnabled()) .setCreationTime(Time.now()) .build(); - metadataManager.put(bucketKey, ksmBucketInfo.getProtobuf().toByteArray()); + metadataManager.put(bucketKey, omBucketInfo.getProtobuf().toByteArray()); LOG.debug("created bucket: {} in volume: {}", bucketName, volumeName); } catch (IOException | DBException ex) { - if (!(ex instanceof KSMException)) { + if (!(ex instanceof OMException)) { LOG.error("Bucket creation failed for bucket:{} in volume:{}", bucketName, volumeName, ex); } @@ -127,7 +127,7 @@ public class BucketManagerImpl implements BucketManager { * @param bucketName - Name of the Bucket. */ @Override - public KsmBucketInfo getBucketInfo(String volumeName, String bucketName) + public OmBucketInfo getBucketInfo(String volumeName, String bucketName) throws IOException { Preconditions.checkNotNull(volumeName); Preconditions.checkNotNull(bucketName); @@ -138,12 +138,12 @@ public class BucketManagerImpl implements BucketManager { if (value == null) { LOG.debug("bucket: {} not found in volume: {}.", bucketName, volumeName); - throw new KSMException("Bucket not found", - KSMException.ResultCodes.FAILED_BUCKET_NOT_FOUND); + throw new OMException("Bucket not found", + OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND); } - return KsmBucketInfo.getFromProtobuf(BucketInfo.parseFrom(value)); + return OmBucketInfo.getFromProtobuf(BucketInfo.parseFrom(value)); } catch (IOException | DBException ex) { - if (!(ex instanceof KSMException)) { + if (!(ex instanceof OMException)) { LOG.error("Exception while getting bucket info for bucket: {}", bucketName, ex); } @@ -159,7 +159,7 @@ public class BucketManagerImpl implements BucketManager { * @throws IOException */ @Override - public void setBucketProperty(KsmBucketArgs args) throws IOException { + public void setBucketProperty(OmBucketArgs args) throws IOException { Preconditions.checkNotNull(args); metadataManager.writeLock().lock(); String volumeName = args.getVolumeName(); @@ -170,19 +170,19 @@ public class BucketManagerImpl implements BucketManager { if(metadataManager.get(metadataManager.getVolumeKey(volumeName)) == null) { LOG.debug("volume: {} not found ", volumeName); - throw new KSMException("Volume doesn't exist", - KSMException.ResultCodes.FAILED_VOLUME_NOT_FOUND); + throw new OMException("Volume doesn't exist", + OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND); } byte[] value = metadataManager.get(bucketKey); //Check if bucket exist if(value == null) { LOG.debug("bucket: {} not found ", bucketName); - throw new KSMException("Bucket doesn't exist", - KSMException.ResultCodes.FAILED_BUCKET_NOT_FOUND); + throw new OMException("Bucket doesn't exist", + OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND); } - KsmBucketInfo oldBucketInfo = KsmBucketInfo.getFromProtobuf( + OmBucketInfo oldBucketInfo = OmBucketInfo.getFromProtobuf( BucketInfo.parseFrom(value)); - KsmBucketInfo.Builder bucketInfoBuilder = KsmBucketInfo.newBuilder(); + OmBucketInfo.Builder bucketInfoBuilder = OmBucketInfo.newBuilder(); bucketInfoBuilder.setVolumeName(oldBucketInfo.getVolumeName()) .setBucketName(oldBucketInfo.getBucketName()); @@ -221,7 +221,7 @@ public class BucketManagerImpl implements BucketManager { metadataManager.put(bucketKey, bucketInfoBuilder.build().getProtobuf().toByteArray()); } catch (IOException | DBException ex) { - if (!(ex instanceof KSMException)) { + if (!(ex instanceof OMException)) { LOG.error("Setting bucket property failed for bucket:{} in volume:{}", bucketName, volumeName, ex); } @@ -269,24 +269,24 @@ public class BucketManagerImpl implements BucketManager { if (metadataManager.get(metadataManager.getVolumeKey(volumeName)) == null) { LOG.debug("volume: {} not found ", volumeName); - throw new KSMException("Volume doesn't exist", - KSMException.ResultCodes.FAILED_VOLUME_NOT_FOUND); + throw new OMException("Volume doesn't exist", + OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND); } //Check if bucket exist if (metadataManager.get(bucketKey) == null) { LOG.debug("bucket: {} not found ", bucketName); - throw new KSMException("Bucket doesn't exist", - KSMException.ResultCodes.FAILED_BUCKET_NOT_FOUND); + throw new OMException("Bucket doesn't exist", + OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND); } //Check if bucket is empty if (!metadataManager.isBucketEmpty(volumeName, bucketName)) { LOG.debug("bucket: {} is not empty ", bucketName); - throw new KSMException("Bucket is not empty", - KSMException.ResultCodes.FAILED_BUCKET_NOT_EMPTY); + throw new OMException("Bucket is not empty", + OMException.ResultCodes.FAILED_BUCKET_NOT_EMPTY); } metadataManager.delete(bucketKey); } catch (IOException ex) { - if (!(ex instanceof KSMException)) { + if (!(ex instanceof OMException)) { LOG.error("Delete bucket failed for bucket:{} in volume:{}", bucketName, volumeName, ex); } @@ -300,8 +300,8 @@ public class BucketManagerImpl implements BucketManager { * {@inheritDoc} */ @Override - public List listBuckets(String volumeName, - String startBucket, String bucketPrefix, int maxNumOfBuckets) + public List listBuckets(String volumeName, + String startBucket, String bucketPrefix, int maxNumOfBuckets) throws IOException { Preconditions.checkNotNull(volumeName); metadataManager.readLock().lock(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyDeletingService.java similarity index 88% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyDeletingService.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyDeletingService.java index e51ab28d028..ee23fe06fa0 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyDeletingService.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ozone.common.DeleteBlockGroupResult; @@ -35,12 +35,12 @@ import java.io.IOException; import java.util.List; import java.util.concurrent.TimeUnit; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KEY_DELETING_LIMIT_PER_TASK; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_KEY_DELETING_LIMIT_PER_TASK; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT; /** * This is the background service to delete keys. - * Scan the metadata of ksm periodically to get + * Scan the metadata of om periodically to get * the keys with prefix "#deleting" and ask scm to * delete metadata accordingly, if scm returns * success for keys, then clean up those keys. @@ -76,7 +76,7 @@ public class KeyDeletingService extends BackgroundService { } /** - * A key deleting task scans KSM DB and looking for a certain number + * A key deleting task scans OM DB and looking for a certain number * of pending-deletion keys, sends these keys along with their associated * blocks to SCM for deletion. Once SCM confirms keys are deleted (once * SCM persisted the blocks info in its deletedBlockLog), it removes @@ -97,15 +97,15 @@ public class KeyDeletingService extends BackgroundService { List keyBlocksList = manager .getPendingDeletionKeys(keyLimitPerTask); if (keyBlocksList.size() > 0) { - LOG.info("Found {} to-delete keys in KSM", keyBlocksList.size()); + LOG.info("Found {} to-delete keys in OM", keyBlocksList.size()); List results = scmClient.deleteKeyBlocks(keyBlocksList); for (DeleteBlockGroupResult result : results) { if (result.isSuccess()) { try { - // Purge key from KSM DB. + // Purge key from OM DB. manager.deletePendingDeletionKey(result.getObjectKey()); - LOG.debug("Key {} deleted from KSM DB", result.getObjectKey()); + LOG.debug("Key {} deleted from OM DB", result.getObjectKey()); } catch (IOException e) { // if a pending deletion key is failed to delete, // print a warning here and retain it in this state, @@ -123,14 +123,14 @@ public class KeyDeletingService extends BackgroundService { } if (!results.isEmpty()) { - LOG.info("Number of key deleted from KSM DB: {}," + LOG.info("Number of key deleted from OM DB: {}," + " task elapsed time: {}ms", results.size(), Time.monotonicNow() - startTime); } return results::size; } else { - LOG.debug("No pending deletion key found in KSM"); + LOG.debug("No pending deletion key found in OM"); } } catch (IOException e) { LOG.error("Unable to get pending deletion keys, retry in" diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java similarity index 82% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyManager.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java index 5ec1db8c5a5..226c07d6fe6 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java @@ -14,13 +14,13 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.hadoop.ozone.common.BlockGroup; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; -import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; import java.io.IOException; import java.util.List; @@ -49,7 +49,7 @@ public interface KeyManager { * @param clientID the client that is committing. * @throws IOException */ - void commitKey(KsmKeyArgs args, int clientID) throws IOException; + void commitKey(OmKeyArgs args, int clientID) throws IOException; /** * A client calls this on an open key, to request to allocate a new block, @@ -60,30 +60,30 @@ public interface KeyManager { * @return the reference to the new block. * @throws IOException */ - KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID) + OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID) throws IOException; /** * Given the args of a key to put, write an open key entry to meta data. * * In case that the container creation or key write failed on - * DistributedStorageHandler, this key's metadata will still stay in KSM. + * DistributedStorageHandler, this key's metadata will still stay in OM. * TODO garbage collect the open keys that never get closed * * @param args the args of the key provided by client. * @return a OpenKeySession instance client uses to talk to container. * @throws Exception */ - OpenKeySession openKey(KsmKeyArgs args) throws IOException; + OpenKeySession openKey(OmKeyArgs args) throws IOException; /** * Look up an existing key. Return the info of the key to client side, which * DistributedStorageHandler will use to access the data on datanode. * * @param args the args of the key provided by client. - * @return a KsmKeyInfo instance client uses to talk to container. + * @return a OmKeyInfo instance client uses to talk to container. * @throws IOException */ - KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException; + OmKeyInfo lookupKey(OmKeyArgs args) throws IOException; /** * Renames an existing key within a bucket. @@ -93,21 +93,21 @@ public interface KeyManager { * @throws IOException if specified key doesn't exist or * some other I/O errors while renaming the key. */ - void renameKey(KsmKeyArgs args, String toKeyName) throws IOException; + void renameKey(OmKeyArgs args, String toKeyName) throws IOException; /** * Deletes an object by an object key. The key will be immediately removed - * from KSM namespace and become invisible to clients. The object data + * from OM namespace and become invisible to clients. The object data * will be removed in async manner that might retain for some time. * * @param args the args of the key provided by client. * @throws IOException if specified key doesn't exist or * some other I/O errors while deleting an object. */ - void deleteKey(KsmKeyArgs args) throws IOException; + void deleteKey(OmKeyArgs args) throws IOException; /** - * Returns a list of keys represented by {@link KsmKeyInfo} + * Returns a list of keys represented by {@link OmKeyInfo} * in the given bucket. * * @param volumeName @@ -127,15 +127,15 @@ public interface KeyManager { * @return a list of keys. * @throws IOException */ - List listKeys(String volumeName, - String bucketName, String startKey, String keyPrefix, int maxKeys) + List listKeys(String volumeName, + String bucketName, String startKey, String keyPrefix, int maxKeys) throws IOException; /** * Returns a list of pending deletion key info that ups to the given count. * Each entry is a {@link BlockGroup}, which contains the info about the * key name and all its associated block IDs. A pending deletion key is - * stored with #deleting# prefix in KSM DB. + * stored with #deleting# prefix in OM DB. * * @param count max number of keys to return. * @return a list of {@link BlockGroup} representing keys and blocks. @@ -146,7 +146,7 @@ public interface KeyManager { /** * Deletes a pending deletion key by its name. This is often called when * key can be safely deleted from this layer. Once called, all footprints - * of the key will be purged from KSM DB. + * of the key will be purged from OM DB. * * @param objectKeyName object key name with #deleting# prefix. * @throws IOException if specified key doesn't exist or other I/O errors. @@ -156,7 +156,7 @@ public interface KeyManager { /** * Returns a list of all still open key info. Which contains the info about * the key name and all its associated block IDs. A pending open key has - * prefix #open# in KSM DB. + * prefix #open# in OM DB. * * @return a list of {@link BlockGroup} representing keys and blocks. * @throws IOException @@ -166,7 +166,7 @@ public interface KeyManager { /** * Deletes a expired open key by its name. Called when a hanging key has been * lingering for too long. Once called, the open key entries gets removed - * from KSM mdata data. + * from OM mdata data. * * @param objectKeyName object key name with #open# prefix. * @throws IOException if specified key doesn't exist or other I/O errors. diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java similarity index 83% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java index 0d4cfda45cb..ba92a29e817 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java @@ -14,23 +14,23 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.hdfs.DFSUtil; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.common.BlockGroup; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.ozone.ksm.exceptions.KSMException; -import org.apache.hadoop.ozone.ksm.exceptions.KSMException.ResultCodes; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup; -import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.KeyInfo; + .OzoneManagerProtocolProtos.KeyInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; import org.apache.hadoop.util.Time; @@ -88,7 +88,7 @@ public class KeyManagerImpl implements KeyManager { * A SCM block client, used to talk to SCM to allocate block during putKey. */ private final ScmBlockLocationProtocol scmBlockClient; - private final KSMMetadataManager metadataManager; + private final OMMetadataManager metadataManager; private final long scmBlockSize; private final boolean useRatis; private final BackgroundService keyDeletingService; @@ -96,11 +96,11 @@ public class KeyManagerImpl implements KeyManager { private final long preallocateMax; private final Random random; - private final String ksmId; + private final String omId; public KeyManagerImpl(ScmBlockLocationProtocol scmBlockClient, - KSMMetadataManager metadataManager, OzoneConfiguration conf, - String ksmId) { + OMMetadataManager metadataManager, OzoneConfiguration conf, + String omId) { this.scmBlockClient = scmBlockClient; this.metadataManager = metadataManager; this.scmBlockSize = conf.getLong(OZONE_SCM_BLOCK_SIZE_IN_MB, @@ -126,7 +126,7 @@ public class KeyManagerImpl implements KeyManager { openKeyCleanupService = new OpenKeyCleanupService( scmBlockClient, this, openkeyCheckInterval, serviceTimeout); random = new Random(); - this.ksmId = ksmId; + this.omId = omId; } @VisibleForTesting @@ -154,19 +154,19 @@ public class KeyManagerImpl implements KeyManager { //Check if the volume exists if(metadataManager.get(volumeKey) == null) { LOG.error("volume not found: {}", volumeName); - throw new KSMException("Volume not found", - KSMException.ResultCodes.FAILED_VOLUME_NOT_FOUND); + throw new OMException("Volume not found", + OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND); } //Check if bucket already exists if(metadataManager.get(bucketKey) == null) { LOG.error("bucket not found: {}/{} ", volumeName, bucketName); - throw new KSMException("Bucket not found", - KSMException.ResultCodes.FAILED_BUCKET_NOT_FOUND); + throw new OMException("Bucket not found", + OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND); } } @Override - public KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID) + public OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID) throws IOException { Preconditions.checkNotNull(args); metadataManager.writeLock().lock(); @@ -183,15 +183,15 @@ public class KeyManagerImpl implements KeyManager { if (keyData == null) { LOG.error("Allocate block for a key not in open status in meta store " + objectKey + " with ID " + clientID); - throw new KSMException("Open Key not found", - KSMException.ResultCodes.FAILED_KEY_NOT_FOUND); + throw new OMException("Open Key not found", + OMException.ResultCodes.FAILED_KEY_NOT_FOUND); } - KsmKeyInfo keyInfo = - KsmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(keyData)); + OmKeyInfo keyInfo = + OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(keyData)); AllocatedBlock allocatedBlock = scmBlockClient.allocateBlock(scmBlockSize, keyInfo.getType(), - keyInfo.getFactor(), ksmId); - KsmKeyLocationInfo info = new KsmKeyLocationInfo.Builder() + keyInfo.getFactor(), omId); + OmKeyLocationInfo info = new OmKeyLocationInfo.Builder() .setBlockID(allocatedBlock.getBlockID()) .setShouldCreateContainer(allocatedBlock.getCreateContainer()) .setLength(scmBlockSize) @@ -209,7 +209,7 @@ public class KeyManagerImpl implements KeyManager { } @Override - public OpenKeySession openKey(KsmKeyArgs args) throws IOException { + public OpenKeySession openKey(OmKeyArgs args) throws IOException { Preconditions.checkNotNull(args); metadataManager.writeLock().lock(); String volumeName = args.getVolumeName(); @@ -219,7 +219,7 @@ public class KeyManagerImpl implements KeyManager { ReplicationType type = args.getType(); // If user does not specify a replication strategy or - // replication factor, KSM will use defaults. + // replication factor, OM will use defaults. if(factor == null) { factor = useRatis ? ReplicationFactor.THREE: ReplicationFactor.ONE; } @@ -231,19 +231,19 @@ public class KeyManagerImpl implements KeyManager { try { validateBucket(volumeName, bucketName); long requestedSize = Math.min(preallocateMax, args.getDataSize()); - List locations = new ArrayList<>(); + List locations = new ArrayList<>(); String objectKey = metadataManager.getKeyWithDBPrefix( volumeName, bucketName, keyName); // requested size is not required but more like a optimization: // SCM looks at the requested, if it 0, no block will be allocated at // the point, if client needs more blocks, client can always call - // allocateBlock. But if requested size is not 0, KSM will preallocate + // allocateBlock. But if requested size is not 0, OM will preallocate // some blocks and piggyback to client, to save RPC calls. while (requestedSize > 0) { long allocateSize = Math.min(scmBlockSize, requestedSize); AllocatedBlock allocatedBlock = - scmBlockClient.allocateBlock(allocateSize, type, factor, ksmId); - KsmKeyLocationInfo subKeyInfo = new KsmKeyLocationInfo.Builder() + scmBlockClient.allocateBlock(allocateSize, type, factor, omId); + OmKeyLocationInfo subKeyInfo = new OmKeyLocationInfo.Builder() .setBlockID(allocatedBlock.getBlockID()) .setShouldCreateContainer(allocatedBlock.getCreateContainer()) .setLength(allocateSize) @@ -260,11 +260,11 @@ public class KeyManagerImpl implements KeyManager { byte[] keyKey = metadataManager.getDBKeyBytes( volumeName, bucketName, keyName); byte[] value = metadataManager.get(keyKey); - KsmKeyInfo keyInfo; + OmKeyInfo keyInfo; long openVersion; if (value != null) { // the key already exist, the new blocks will be added as new version - keyInfo = KsmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(value)); + keyInfo = OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(value)); // when locations.size = 0, the new version will have identical blocks // as its previous version openVersion = keyInfo.addNewVersion(locations); @@ -273,12 +273,12 @@ public class KeyManagerImpl implements KeyManager { // the key does not exist, create a new object, the new blocks are the // version 0 long currentTime = Time.now(); - keyInfo = new KsmKeyInfo.Builder() + keyInfo = new OmKeyInfo.Builder() .setVolumeName(args.getVolumeName()) .setBucketName(args.getBucketName()) .setKeyName(args.getKeyName()) - .setKsmKeyLocationInfos(Collections.singletonList( - new KsmKeyLocationInfoGroup(0, locations))) + .setOmKeyLocationInfos(Collections.singletonList( + new OmKeyLocationInfoGroup(0, locations))) .setCreationTime(currentTime) .setModificationTime(currentTime) .setDataSize(size) @@ -305,22 +305,22 @@ public class KeyManagerImpl implements KeyManager { LOG.debug("Key {} allocated in volume {} bucket {}", keyName, volumeName, bucketName); return new OpenKeySession(id, keyInfo, openVersion); - } catch (KSMException e) { + } catch (OMException e) { throw e; } catch (IOException ex) { - if (!(ex instanceof KSMException)) { + if (!(ex instanceof OMException)) { LOG.error("Key open failed for volume:{} bucket:{} key:{}", volumeName, bucketName, keyName, ex); } - throw new KSMException(ex.getMessage(), - KSMException.ResultCodes.FAILED_KEY_ALLOCATION); + throw new OMException(ex.getMessage(), + OMException.ResultCodes.FAILED_KEY_ALLOCATION); } finally { metadataManager.writeLock().unlock(); } } @Override - public void commitKey(KsmKeyArgs args, int clientID) throws IOException { + public void commitKey(OmKeyArgs args, int clientID) throws IOException { Preconditions.checkNotNull(args); metadataManager.writeLock().lock(); String volumeName = args.getVolumeName(); @@ -335,31 +335,31 @@ public class KeyManagerImpl implements KeyManager { byte[] openKey = metadataManager.getOpenKeyNameBytes(objectKey, clientID); byte[] openKeyData = metadataManager.get(openKey); if (openKeyData == null) { - throw new KSMException("Commit a key without corresponding entry " + + throw new OMException("Commit a key without corresponding entry " + DFSUtil.bytes2String(openKey), ResultCodes.FAILED_KEY_NOT_FOUND); } - KsmKeyInfo keyInfo = - KsmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(openKeyData)); + OmKeyInfo keyInfo = + OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(openKeyData)); keyInfo.setDataSize(args.getDataSize()); keyInfo.setModificationTime(Time.now()); BatchOperation batch = new BatchOperation(); batch.delete(openKey); batch.put(objectKeyBytes, keyInfo.getProtobuf().toByteArray()); metadataManager.writeBatch(batch); - } catch (KSMException e) { + } catch (OMException e) { throw e; } catch (IOException ex) { LOG.error("Key commit failed for volume:{} bucket:{} key:{}", volumeName, bucketName, keyName, ex); - throw new KSMException(ex.getMessage(), - KSMException.ResultCodes.FAILED_KEY_ALLOCATION); + throw new OMException(ex.getMessage(), + OMException.ResultCodes.FAILED_KEY_ALLOCATION); } finally { metadataManager.writeLock().unlock(); } } @Override - public KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException { + public OmKeyInfo lookupKey(OmKeyArgs args) throws IOException { Preconditions.checkNotNull(args); metadataManager.writeLock().lock(); String volumeName = args.getVolumeName(); @@ -372,22 +372,22 @@ public class KeyManagerImpl implements KeyManager { if (value == null) { LOG.debug("volume:{} bucket:{} Key:{} not found", volumeName, bucketName, keyName); - throw new KSMException("Key not found", - KSMException.ResultCodes.FAILED_KEY_NOT_FOUND); + throw new OMException("Key not found", + OMException.ResultCodes.FAILED_KEY_NOT_FOUND); } - return KsmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(value)); + return OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(value)); } catch (DBException ex) { LOG.error("Get key failed for volume:{} bucket:{} key:{}", volumeName, bucketName, keyName, ex); - throw new KSMException(ex.getMessage(), - KSMException.ResultCodes.FAILED_KEY_NOT_FOUND); + throw new OMException(ex.getMessage(), + OMException.ResultCodes.FAILED_KEY_NOT_FOUND); } finally { metadataManager.writeLock().unlock(); } } @Override - public void renameKey(KsmKeyArgs args, String toKeyName) throws IOException { + public void renameKey(OmKeyArgs args, String toKeyName) throws IOException { Preconditions.checkNotNull(args); Preconditions.checkNotNull(toKeyName); String volumeName = args.getVolumeName(); @@ -396,7 +396,7 @@ public class KeyManagerImpl implements KeyManager { if (toKeyName.length() == 0 || fromKeyName.length() == 0) { LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}.", volumeName, bucketName, fromKeyName, toKeyName); - throw new KSMException("Key name is empty", + throw new OMException("Key name is empty", ResultCodes.FAILED_INVALID_KEY_NAME); } @@ -412,8 +412,8 @@ public class KeyManagerImpl implements KeyManager { "Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}. " + "Key: {} not found.", volumeName, bucketName, fromKeyName, toKeyName, fromKeyName); - throw new KSMException("Key not found", - KSMException.ResultCodes.FAILED_KEY_NOT_FOUND); + throw new OMException("Key not found", + OMException.ResultCodes.FAILED_KEY_NOT_FOUND); } // toKeyName should not exist @@ -425,16 +425,16 @@ public class KeyManagerImpl implements KeyManager { "Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}. " + "Key: {} already exists.", volumeName, bucketName, fromKeyName, toKeyName, toKeyName); - throw new KSMException("Key not found", - KSMException.ResultCodes.FAILED_KEY_ALREADY_EXISTS); + throw new OMException("Key not found", + OMException.ResultCodes.FAILED_KEY_ALREADY_EXISTS); } if (fromKeyName.equals(toKeyName)) { return; } - KsmKeyInfo newKeyInfo = - KsmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(fromKeyValue)); + OmKeyInfo newKeyInfo = + OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(fromKeyValue)); newKeyInfo.setKeyName(toKeyName); newKeyInfo.updateModifcationTime(); BatchOperation batch = new BatchOperation(); @@ -444,7 +444,7 @@ public class KeyManagerImpl implements KeyManager { } catch (DBException ex) { LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}.", volumeName, bucketName, fromKeyName, toKeyName, ex); - throw new KSMException(ex.getMessage(), + throw new OMException(ex.getMessage(), ResultCodes.FAILED_KEY_RENAME); } finally { metadataManager.writeLock().unlock(); @@ -452,7 +452,7 @@ public class KeyManagerImpl implements KeyManager { } @Override - public void deleteKey(KsmKeyArgs args) throws IOException { + public void deleteKey(OmKeyArgs args) throws IOException { Preconditions.checkNotNull(args); metadataManager.writeLock().lock(); String volumeName = args.getVolumeName(); @@ -463,8 +463,8 @@ public class KeyManagerImpl implements KeyManager { volumeName, bucketName, keyName); byte[] objectValue = metadataManager.get(objectKey); if (objectValue == null) { - throw new KSMException("Key not found", - KSMException.ResultCodes.FAILED_KEY_NOT_FOUND); + throw new OMException("Key not found", + OMException.ResultCodes.FAILED_KEY_NOT_FOUND); } byte[] deletingKey = metadataManager.getDeletedKeyName(objectKey); BatchOperation batch = new BatchOperation(); @@ -474,7 +474,7 @@ public class KeyManagerImpl implements KeyManager { } catch (DBException ex) { LOG.error(String.format("Delete key failed for volume:%s " + "bucket:%s key:%s", volumeName, bucketName, keyName), ex); - throw new KSMException(ex.getMessage(), ex, + throw new OMException(ex.getMessage(), ex, ResultCodes.FAILED_KEY_DELETION); } finally { metadataManager.writeLock().unlock(); @@ -482,8 +482,8 @@ public class KeyManagerImpl implements KeyManager { } @Override - public List listKeys(String volumeName, String bucketName, - String startKey, String keyPrefix, int maxKeys) throws IOException { + public List listKeys(String volumeName, String bucketName, + String startKey, String keyPrefix, int maxKeys) throws IOException { Preconditions.checkNotNull(volumeName); Preconditions.checkNotNull(bucketName); @@ -516,7 +516,7 @@ public class KeyManagerImpl implements KeyManager { + " the name should be the key name with deleting prefix"); } - // Simply removes the entry from KSM DB. + // Simply removes the entry from OM DB. metadataManager.writeLock().lock(); try { byte[] pendingDelKey = DFSUtil.string2Bytes(objectKeyName); @@ -549,7 +549,7 @@ public class KeyManagerImpl implements KeyManager { + " the name should be the key name with open key prefix"); } - // Simply removes the entry from KSM DB. + // Simply removes the entry from OM DB. metadataManager.writeLock().lock(); try { byte[] openKey = DFSUtil.string2Bytes(objectKeyName); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMXBean.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMXBean.java similarity index 86% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMXBean.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMXBean.java index bf223324ae6..3ab9f47568d 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMXBean.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMXBean.java @@ -16,16 +16,16 @@ * limitations under the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdds.server.ServiceRuntimeInfo; /** - * This is the JMX management interface for ksm information. + * This is the JMX management interface for OM information. */ @InterfaceAudience.Private -public interface KSMMXBean extends ServiceRuntimeInfo { +public interface OMMXBean extends ServiceRuntimeInfo { String getRpcPort(); } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetadataManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java similarity index 85% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetadataManager.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java index f5a2d5bb0db..f2e78e661c4 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetadataManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java @@ -14,13 +14,13 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.common.BlockGroup; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; import org.apache.hadoop.utils.BatchOperation; import org.apache.hadoop.utils.MetadataStore; @@ -29,9 +29,9 @@ import java.util.List; import java.util.concurrent.locks.Lock; /** - * KSM metadata manager interface. + * OM metadata manager interface. */ -public interface KSMMetadataManager { +public interface OMMetadataManager { /** * Start metadata manager. */ @@ -117,7 +117,7 @@ public interface KSMMetadataManager { byte[] getDBKeyBytes(String volume, String bucket, String key); /** - * Returns the DB key name of a deleted key in KSM metadata store. + * Returns the DB key name of a deleted key in OM metadata store. * The name for a deleted key has prefix #deleting# followed by * the actual key name. * @param keyName - key name @@ -126,7 +126,7 @@ public interface KSMMetadataManager { byte[] getDeletedKeyName(byte[] keyName); /** - * Returns the DB key name of a open key in KSM metadata store. + * Returns the DB key name of a open key in OM metadata store. * Should be #open# prefix followed by actual key name. * @param keyName - key name * @param id - the id for this open @@ -163,7 +163,7 @@ public interface KSMMetadataManager { boolean isBucketEmpty(String volume, String bucket) throws IOException; /** - * Returns a list of buckets represented by {@link KsmBucketInfo} + * Returns a list of buckets represented by {@link OmBucketInfo} * in the given volume. * * @param volumeName @@ -182,11 +182,11 @@ public interface KSMMetadataManager { * @return a list of buckets. * @throws IOException */ - List listBuckets(String volumeName, String startBucket, - String bucketPrefix, int maxNumOfBuckets) throws IOException; + List listBuckets(String volumeName, String startBucket, + String bucketPrefix, int maxNumOfBuckets) throws IOException; /** - * Returns a list of keys represented by {@link KsmKeyInfo} + * Returns a list of keys represented by {@link OmKeyInfo} * in the given bucket. * * @param volumeName @@ -206,8 +206,8 @@ public interface KSMMetadataManager { * @return a list of keys. * @throws IOException */ - List listKeys(String volumeName, - String bucketName, String startKey, String keyPrefix, int maxKeys) + List listKeys(String volumeName, + String bucketName, String startKey, String keyPrefix, int maxKeys) throws IOException; /** @@ -223,17 +223,17 @@ public interface KSMMetadataManager { * this key is excluded from the result. * @param maxKeys * the maximum number of volumes to return. - * @return a list of {@link KsmVolumeArgs} + * @return a list of {@link OmVolumeArgs} * @throws IOException */ - List listVolumes(String userName, String prefix, - String startKey, int maxKeys) throws IOException; + List listVolumes(String userName, String prefix, + String startKey, int maxKeys) throws IOException; /** * Returns a list of pending deletion key info that ups to the given count. * Each entry is a {@link BlockGroup}, which contains the info about the * key name and all its associated block IDs. A pending deletion key is - * stored with #deleting# prefix in KSM DB. + * stored with #deleting# prefix in OM DB. * * @param count max number of keys to return. * @return a list of {@link BlockGroup} represent keys and blocks. @@ -244,7 +244,7 @@ public interface KSMMetadataManager { /** * Returns a list of all still open key info. Which contains the info about * the key name and all its associated block IDs. A pending open key has - * prefix #open# in KSM DB. + * prefix #open# in OM DB. * * @return a list of {@link BlockGroup} representing keys and blocks. * @throws IOException diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetrics.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java similarity index 96% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetrics.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java index 8ee67c3e278..2d044521ed6 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetrics.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import com.google.common.annotations.VisibleForTesting; @@ -27,20 +27,20 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.metrics2.lib.MutableCounterLong; /** - * This class is for maintaining KeySpaceManager statistics. + * This class is for maintaining Ozone Manager statistics. */ @InterfaceAudience.Private -@Metrics(about="Key Space Manager Metrics", context="dfs") -public class KSMMetrics { +@Metrics(about="Ozone Manager Metrics", context="dfs") +public class OMMetrics { private static final String SOURCE_NAME = - KSMMetrics.class.getSimpleName(); + OMMetrics.class.getSimpleName(); - // KSM request type op metrics + // OM request type op metrics private @Metric MutableCounterLong numVolumeOps; private @Metric MutableCounterLong numBucketOps; private @Metric MutableCounterLong numKeyOps; - // KSM op metrics + // OM op metrics private @Metric MutableCounterLong numVolumeCreates; private @Metric MutableCounterLong numVolumeUpdates; private @Metric MutableCounterLong numVolumeInfos; @@ -82,14 +82,14 @@ public class KSMMetrics { private @Metric MutableCounterLong numBlockAllocateCallFails; private @Metric MutableCounterLong numGetServiceListFails; - public KSMMetrics() { + public OMMetrics() { } - public static KSMMetrics create() { + public static OMMetrics create() { MetricsSystem ms = DefaultMetricsSystem.instance(); return ms.register(SOURCE_NAME, - "Key Space Manager Metrics", - new KSMMetrics()); + "Oozne Manager Metrics", + new OMMetrics()); } public void incNumVolumeCreates() { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMStorage.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStorage.java similarity index 63% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMStorage.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStorage.java index 015bed69864..3820aed76a3 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMStorage.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStorage.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import java.io.IOException; import java.util.Properties; @@ -29,35 +29,35 @@ import static org.apache.hadoop.ozone.OzoneConsts.SCM_ID; import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath; /** - * KSMStorage is responsible for management of the StorageDirectories used by - * the KSM. + * OMStorage is responsible for management of the StorageDirectories used by + * the Ozone Manager. */ -public class KSMStorage extends Storage { +public class OMStorage extends Storage { - public static final String STORAGE_DIR = "ksm"; - public static final String KSM_ID = "ksmUuid"; + public static final String STORAGE_DIR = "om"; + public static final String OM_ID = "omUuid"; /** - * Construct KSMStorage. + * Construct OMStorage. * @throws IOException if any directories are inaccessible. */ - public KSMStorage(OzoneConfiguration conf) throws IOException { - super(NodeType.KSM, getOzoneMetaDirPath(conf), STORAGE_DIR); + public OMStorage(OzoneConfiguration conf) throws IOException { + super(NodeType.OM, getOzoneMetaDirPath(conf), STORAGE_DIR); } public void setScmId(String scmId) throws IOException { if (getState() == StorageState.INITIALIZED) { - throw new IOException("KSM is already initialized."); + throw new IOException("OM is already initialized."); } else { getStorageInfo().setProperty(SCM_ID, scmId); } } - public void setKsmId(String ksmId) throws IOException { + public void setOmId(String omId) throws IOException { if (getState() == StorageState.INITIALIZED) { - throw new IOException("KSM is already initialized."); + throw new IOException("OM is already initialized."); } else { - getStorageInfo().setProperty(KSM_ID, ksmId); + getStorageInfo().setProperty(OM_ID, omId); } } @@ -70,21 +70,21 @@ public class KSMStorage extends Storage { } /** - * Retrieves the KSM ID from the version file. - * @return KSM_ID + * Retrieves the OM ID from the version file. + * @return OM_ID */ - public String getKsmId() { - return getStorageInfo().getProperty(KSM_ID); + public String getOmId() { + return getStorageInfo().getProperty(OM_ID); } @Override protected Properties getNodeProperties() { - String ksmId = getKsmId(); - if (ksmId == null) { - ksmId = UUID.randomUUID().toString(); + String omId = getOmId(); + if (omId == null) { + omId = UUID.randomUUID().toString(); } - Properties ksmProperties = new Properties(); - ksmProperties.setProperty(KSM_ID, ksmId); - return ksmProperties; + Properties omProperties = new Properties(); + omProperties.setProperty(OM_ID, omId); + return omProperties; } } \ No newline at end of file diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java similarity index 79% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetadataManagerImpl.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java index 6664a324620..21d24114c82 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KSMMetadataManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java @@ -14,7 +14,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; @@ -22,19 +22,19 @@ import com.google.common.collect.Lists; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdds.client.BlockID; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup; -import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; import org.apache.hadoop.ozone.common.BlockGroup; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.OzoneConsts; -import org.apache.hadoop.ozone.ksm.exceptions.KSMException; -import org.apache.hadoop.ozone.ksm.exceptions.KSMException.ResultCodes; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.BucketInfo; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyInfo; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.VolumeInfo; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.VolumeList; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList; import org.apache.hadoop.util.Time; import org.apache.hadoop.utils.BatchOperation; @@ -58,32 +58,32 @@ import java.util.stream.Collectors; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS_DEFAULT; import static org.apache.hadoop.ozone.OzoneConsts.DELETING_KEY_PREFIX; -import static org.apache.hadoop.ozone.OzoneConsts.KSM_DB_NAME; +import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME; import static org.apache.hadoop.ozone.OzoneConsts.OPEN_KEY_ID_DELIMINATOR; import static org.apache.hadoop.ozone.OzoneConsts.OPEN_KEY_PREFIX; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys - .OZONE_KSM_DB_CACHE_SIZE_DEFAULT; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys - .OZONE_KSM_DB_CACHE_SIZE_MB; +import static org.apache.hadoop.ozone.om.OMConfigKeys + .OZONE_OM_DB_CACHE_SIZE_DEFAULT; +import static org.apache.hadoop.ozone.om.OMConfigKeys + .OZONE_OM_DB_CACHE_SIZE_MB; import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath; /** - * KSM metadata manager interface. + * Ozone metadata manager interface. */ -public class KSMMetadataManagerImpl implements KSMMetadataManager { +public class OmMetadataManagerImpl implements OMMetadataManager { private final MetadataStore store; private final ReadWriteLock lock; private final long openKeyExpireThresholdMS; - public KSMMetadataManagerImpl(OzoneConfiguration conf) throws IOException { + public OmMetadataManagerImpl(OzoneConfiguration conf) throws IOException { File metaDir = getOzoneMetaDirPath(conf); - final int cacheSize = conf.getInt(OZONE_KSM_DB_CACHE_SIZE_MB, - OZONE_KSM_DB_CACHE_SIZE_DEFAULT); - File ksmDBFile = new File(metaDir.getPath(), KSM_DB_NAME); + final int cacheSize = conf.getInt(OZONE_OM_DB_CACHE_SIZE_MB, + OZONE_OM_DB_CACHE_SIZE_DEFAULT); + File omDBFile = new File(metaDir.getPath(), OM_DB_NAME); this.store = MetadataStoreBuilder.newBuilder() .setConf(conf) - .setDbFile(ksmDBFile) + .setDbFile(omDBFile) .setCacheSize(cacheSize * OzoneConsts.MB) .build(); this.lock = new ReentrantReadWriteLock(); @@ -125,7 +125,7 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { * @param volume - Volume name */ public byte[] getVolumeKey(String volume) { - String dbVolumeName = OzoneConsts.KSM_VOLUME_PREFIX + volume; + String dbVolumeName = OzoneConsts.OM_VOLUME_PREFIX + volume; return DFSUtil.string2Bytes(dbVolumeName); } @@ -134,7 +134,7 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { * @param user - User name */ public byte[] getUserKey(String user) { - String dbUserName = OzoneConsts.KSM_USER_PREFIX + user; + String dbUserName = OzoneConsts.OM_USER_PREFIX + user; return DFSUtil.string2Bytes(dbUserName); } @@ -144,8 +144,8 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { * @param bucket - Bucket name */ public byte[] getBucketKey(String volume, String bucket) { - String bucketKeyString = OzoneConsts.KSM_VOLUME_PREFIX + volume - + OzoneConsts.KSM_BUCKET_PREFIX + bucket; + String bucketKeyString = OzoneConsts.OM_VOLUME_PREFIX + volume + + OzoneConsts.OM_BUCKET_PREFIX + bucket; return DFSUtil.string2Bytes(bucketKeyString); } @@ -156,9 +156,9 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { */ private String getBucketWithDBPrefix(String volume, String bucket) { StringBuffer sb = new StringBuffer(); - sb.append(OzoneConsts.KSM_VOLUME_PREFIX) + sb.append(OzoneConsts.OM_VOLUME_PREFIX) .append(volume) - .append(OzoneConsts.KSM_BUCKET_PREFIX); + .append(OzoneConsts.OM_BUCKET_PREFIX); if (!Strings.isNullOrEmpty(bucket)) { sb.append(bucket); } @@ -167,9 +167,9 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { @Override public String getKeyWithDBPrefix(String volume, String bucket, String key) { - String keyVB = OzoneConsts.KSM_KEY_PREFIX + volume - + OzoneConsts.KSM_KEY_PREFIX + bucket - + OzoneConsts.KSM_KEY_PREFIX; + String keyVB = OzoneConsts.OM_KEY_PREFIX + volume + + OzoneConsts.OM_KEY_PREFIX + bucket + + OzoneConsts.OM_KEY_PREFIX; return Strings.isNullOrEmpty(key) ? keyVB : keyVB + key; } @@ -247,8 +247,8 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { * @return true if the volume is empty */ public boolean isVolumeEmpty(String volume) throws IOException { - String dbVolumeRootName = OzoneConsts.KSM_VOLUME_PREFIX + volume - + OzoneConsts.KSM_BUCKET_PREFIX; + String dbVolumeRootName = OzoneConsts.OM_VOLUME_PREFIX + volume + + OzoneConsts.OM_BUCKET_PREFIX; byte[] dbVolumeRootKey = DFSUtil.string2Bytes(dbVolumeRootName); ImmutablePair volumeRoot = store.peekAround(0, dbVolumeRootKey); @@ -282,18 +282,18 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { * {@inheritDoc} */ @Override - public List listBuckets(final String volumeName, - final String startBucket, final String bucketPrefix, - final int maxNumOfBuckets) throws IOException { - List result = new ArrayList<>(); + public List listBuckets(final String volumeName, + final String startBucket, final String bucketPrefix, + final int maxNumOfBuckets) throws IOException { + List result = new ArrayList<>(); if (Strings.isNullOrEmpty(volumeName)) { - throw new KSMException("Volume name is required.", + throw new OMException("Volume name is required.", ResultCodes.FAILED_VOLUME_NOT_FOUND); } byte[] volumeNameBytes = getVolumeKey(volumeName); if (store.get(volumeNameBytes) == null) { - throw new KSMException("Volume " + volumeName + " not found.", + throw new OMException("Volume " + volumeName + " not found.", ResultCodes.FAILED_VOLUME_NOT_FOUND); } @@ -325,7 +325,7 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { } for (Map.Entry entry : rangeResult) { - KsmBucketInfo info = KsmBucketInfo.getFromProtobuf( + OmBucketInfo info = OmBucketInfo.getFromProtobuf( BucketInfo.parseFrom(entry.getValue())); result.add(info); } @@ -333,22 +333,22 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { } @Override - public List listKeys(String volumeName, String bucketName, - String startKey, String keyPrefix, int maxKeys) throws IOException { - List result = new ArrayList<>(); + public List listKeys(String volumeName, String bucketName, + String startKey, String keyPrefix, int maxKeys) throws IOException { + List result = new ArrayList<>(); if (Strings.isNullOrEmpty(volumeName)) { - throw new KSMException("Volume name is required.", + throw new OMException("Volume name is required.", ResultCodes.FAILED_VOLUME_NOT_FOUND); } if (Strings.isNullOrEmpty(bucketName)) { - throw new KSMException("Bucket name is required.", + throw new OMException("Bucket name is required.", ResultCodes.FAILED_BUCKET_NOT_FOUND); } byte[] bucketNameBytes = getBucketKey(volumeName, bucketName); if (store.get(bucketNameBytes) == null) { - throw new KSMException("Bucket " + bucketName + " not found.", + throw new OMException("Bucket " + bucketName + " not found.", ResultCodes.FAILED_BUCKET_NOT_FOUND); } @@ -371,7 +371,7 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { } for (Map.Entry entry : rangeResult) { - KsmKeyInfo info = KsmKeyInfo.getFromProtobuf( + OmKeyInfo info = OmKeyInfo.getFromProtobuf( KeyInfo.parseFrom(entry.getValue())); result.add(info); } @@ -379,9 +379,9 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { } @Override - public List listVolumes(String userName, - String prefix, String startKey, int maxKeys) throws IOException { - List result = Lists.newArrayList(); + public List listVolumes(String userName, + String prefix, String startKey, int maxKeys) throws IOException { + List result = Lists.newArrayList(); VolumeList volumes; if (Strings.isNullOrEmpty(userName)) { volumes = getAllVolumes(); @@ -410,13 +410,13 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { if (volumeInfo == null) { // Could not get volume info by given volume name, // since the volume name is loaded from db, - // this probably means ksm db is corrupted or some entries are + // this probably means om db is corrupted or some entries are // accidentally removed. - throw new KSMException("Volume info not found for " + volumeName, + throw new OMException("Volume info not found for " + volumeName, ResultCodes.FAILED_VOLUME_NOT_FOUND); } VolumeInfo info = VolumeInfo.parseFrom(volumeInfo); - KsmVolumeArgs volumeArgs = KsmVolumeArgs.getFromProtobuf(info); + OmVolumeArgs volumeArgs = OmVolumeArgs.getFromProtobuf(info); result.add(volumeArgs); } } @@ -425,12 +425,12 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { } private VolumeList getVolumesByUser(String userName) - throws KSMException { + throws OMException { return getVolumesByUser(getUserKey(userName)); } private VolumeList getVolumesByUser(byte[] userNameKey) - throws KSMException { + throws OMException { VolumeList volumes = null; try { byte[] volumesInBytes = store.get(userNameKey); @@ -440,7 +440,7 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { } volumes = VolumeList.parseFrom(volumesInBytes); } catch (IOException e) { - throw new KSMException("Unable to get volumes info by the given user, " + throw new OMException("Unable to get volumes info by the given user, " + "metadata might be corrupted", e, ResultCodes.FAILED_METADATA_ERROR); } @@ -450,7 +450,7 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { private VolumeList getAllVolumes() throws IOException { // Scan all users in database KeyPrefixFilter filter = - new KeyPrefixFilter().addFilter(OzoneConsts.KSM_USER_PREFIX); + new KeyPrefixFilter().addFilter(OzoneConsts.OM_USER_PREFIX); // We are not expecting a huge number of users per cluster, // it should be fine to scan all users in db and return us a // list of volume names in string per user. @@ -474,10 +474,10 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { store.getRangeKVs(null, count, MetadataKeyFilters.getDeletingKeyFilter()); for (Map.Entry entry : rangeResult) { - KsmKeyInfo info = - KsmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(entry.getValue())); + OmKeyInfo info = + OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(entry.getValue())); // Get block keys as a list. - KsmKeyLocationInfoGroup latest = info.getLatestVersionLocations(); + OmKeyLocationInfoGroup latest = info.getLatestVersionLocations(); if (latest == null) { return Collections.emptyList(); } @@ -503,8 +503,8 @@ public class KSMMetadataManagerImpl implements KSMMetadataManager { store.getSequentialRangeKVs(null, Integer.MAX_VALUE, openKeyFilter); for (Map.Entry entry : rangeResult) { - KsmKeyInfo info = - KsmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(entry.getValue())); + OmKeyInfo info = + OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(entry.getValue())); long lastModify = info.getModificationTime(); if (now - lastModify < this.openKeyExpireThresholdMS) { // consider as may still be active, not hanging. diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/OpenKeyCleanupService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OpenKeyCleanupService.java similarity index 93% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/OpenKeyCleanupService.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OpenKeyCleanupService.java index 8e2540a6277..8d94f5ab776 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/OpenKeyCleanupService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OpenKeyCleanupService.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.hadoop.ozone.common.BlockGroup; import org.apache.hadoop.ozone.common.DeleteBlockGroupResult; @@ -35,7 +35,7 @@ import java.util.concurrent.TimeUnit; /** * This is the background service to delete hanging open keys. - * Scan the metadata of ksm periodically to get + * Scan the metadata of om periodically to get * the keys with prefix "#open#" and ask scm to * delete metadata accordingly, if scm returns * success for keys, then clean up those keys. @@ -80,7 +80,7 @@ public class OpenKeyCleanupService extends BackgroundService { List keyBlocksList = keyManager.getExpiredOpenKeys(); if (keyBlocksList.size() > 0) { int toDeleteSize = keyBlocksList.size(); - LOG.debug("Found {} to-delete open keys in KSM", toDeleteSize); + LOG.debug("Found {} to-delete open keys in OM", toDeleteSize); List results = scmClient.deleteKeyBlocks(keyBlocksList); int deletedSize = 0; @@ -88,7 +88,7 @@ public class OpenKeyCleanupService extends BackgroundService { if (result.isSuccess()) { try { keyManager.deleteExpiredOpenKey(result.getObjectKey()); - LOG.debug("Key {} deleted from KSM DB", result.getObjectKey()); + LOG.debug("Key {} deleted from OM DB", result.getObjectKey()); deletedSize += 1; } catch (IOException e) { LOG.warn("Failed to delete hanging-open key {}", @@ -105,7 +105,7 @@ public class OpenKeyCleanupService extends BackgroundService { "cleaned up {} entries", toDeleteSize, deletedSize); return results::size; } else { - LOG.debug("No hanging open key fond in KSM"); + LOG.debug("No hanging open key found in OM"); } } catch (IOException e) { LOG.error("Unable to get hanging open keys, retry in" diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java similarity index 74% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index 5fa313bbd35..71fa921cc56 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -15,7 +15,7 @@ * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -28,29 +28,28 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl; import org.apache.hadoop.ozone.common.Storage.StorageState; -import org.apache.hadoop.ozone.ksm.exceptions.KSMException; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; -import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession; -import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo; -import org.apache.hadoop.ozone.ksm.protocol.KeySpaceManagerProtocol; -import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolPB; -import org.apache.hadoop.ozone.ksm.exceptions.KSMException.ResultCodes; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; +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.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; +import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB; +import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .ServicePort; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo; + .OzoneManagerProtocolProtos.OzoneAclInfo; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.ozone.protocolPB - .KeySpaceManagerProtocolServerSideTranslatorPB; +import org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB; import org.apache.hadoop.hdds.scm.ScmInfo; import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; @@ -67,7 +66,7 @@ import org.apache.hadoop.util.StringUtils; import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForBlockClients; import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients; import static org.apache.hadoop.hdds.HddsUtils.isHddsEnabled; -import static org.apache.hadoop.ozone.KsmUtils.getKsmAddress; +import static org.apache.hadoop.ozone.OmUtils.getOmAddress; import static org.apache.hadoop.hdds.server.ServerUtils .updateRPCListenAddress; import org.slf4j.Logger; @@ -83,31 +82,31 @@ import java.util.List; import java.util.Map; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys - .OZONE_KSM_ADDRESS_KEY; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys - .OZONE_KSM_HANDLER_COUNT_DEFAULT; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys - .OZONE_KSM_HANDLER_COUNT_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys + .OZONE_OM_ADDRESS_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys + .OZONE_OM_HANDLER_COUNT_DEFAULT; +import static org.apache.hadoop.ozone.om.OMConfigKeys + .OZONE_OM_HANDLER_COUNT_KEY; import static org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.KeySpaceManagerService + .OzoneManagerProtocolProtos.OzoneManagerService .newReflectiveBlockingService; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos .NodeState.HEALTHY; import static org.apache.hadoop.util.ExitUtil.terminate; /** - * Ozone Keyspace manager is the metadata manager of ozone. + * Ozone Manager is the metadata manager of ozone. */ @InterfaceAudience.LimitedPrivate({"HDFS", "CBLOCK", "OZONE", "HBASE"}) -public final class KeySpaceManager extends ServiceRuntimeInfoImpl - implements KeySpaceManagerProtocol, KSMMXBean { +public final class OzoneManager extends ServiceRuntimeInfoImpl + implements OzoneManagerProtocol, OMMXBean { private static final Logger LOG = - LoggerFactory.getLogger(KeySpaceManager.class); + LoggerFactory.getLogger(OzoneManager.class); private static final String USAGE = - "Usage: \n ozone ksm [genericOptions] " + "[ " - + StartupOption.CREATEOBJECTSTORE.getName() + " ]\n " + "ozone ksm [ " + "Usage: \n ozone om [genericOptions] " + "[ " + + StartupOption.CREATEOBJECTSTORE.getName() + " ]\n " + "ozone om [ " + StartupOption.HELP.getName() + " ]\n"; /** Startup options. */ @@ -137,60 +136,60 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl } private final OzoneConfiguration configuration; - private final RPC.Server ksmRpcServer; - private final InetSocketAddress ksmRpcAddress; - private final KSMMetadataManager metadataManager; + private final RPC.Server omRpcServer; + private final InetSocketAddress omRpcAddress; + private final OMMetadataManager metadataManager; private final VolumeManager volumeManager; private final BucketManager bucketManager; private final KeyManager keyManager; - private final KSMMetrics metrics; - private final KeySpaceManagerHttpServer httpServer; - private final KSMStorage ksmStorage; + private final OMMetrics metrics; + private final OzoneManagerHttpServer httpServer; + private final OMStorage omStorage; private final ScmBlockLocationProtocol scmBlockClient; private final StorageContainerLocationProtocol scmContainerClient; - private ObjectName ksmInfoBeanName; + private ObjectName omInfoBeanName; - private KeySpaceManager(OzoneConfiguration conf) throws IOException { + private OzoneManager(OzoneConfiguration conf) throws IOException { Preconditions.checkNotNull(conf); configuration = conf; - ksmStorage = new KSMStorage(conf); + omStorage = new OMStorage(conf); scmBlockClient = getScmBlockClient(configuration); scmContainerClient = getScmContainerClient(configuration); - if (ksmStorage.getState() != StorageState.INITIALIZED) { - throw new KSMException("KSM not initialized.", - ResultCodes.KSM_NOT_INITIALIZED); + if (omStorage.getState() != StorageState.INITIALIZED) { + throw new OMException("OM not initialized.", + ResultCodes.OM_NOT_INITIALIZED); } - // verifies that the SCM info in the KSM Version file is correct. + // verifies that the SCM info in the OM Version file is correct. ScmInfo scmInfo = scmBlockClient.getScmInfo(); - if (!(scmInfo.getClusterId().equals(ksmStorage.getClusterID()) && scmInfo - .getScmId().equals(ksmStorage.getScmId()))) { - throw new KSMException("SCM version info mismatch.", + if (!(scmInfo.getClusterId().equals(omStorage.getClusterID()) && scmInfo + .getScmId().equals(omStorage.getScmId()))) { + throw new OMException("SCM version info mismatch.", ResultCodes.SCM_VERSION_MISMATCH_ERROR); } - final int handlerCount = conf.getInt(OZONE_KSM_HANDLER_COUNT_KEY, - OZONE_KSM_HANDLER_COUNT_DEFAULT); + final int handlerCount = conf.getInt(OZONE_OM_HANDLER_COUNT_KEY, + OZONE_OM_HANDLER_COUNT_DEFAULT); - RPC.setProtocolEngine(configuration, KeySpaceManagerProtocolPB.class, + RPC.setProtocolEngine(configuration, OzoneManagerProtocolPB.class, ProtobufRpcEngine.class); - BlockingService ksmService = newReflectiveBlockingService( - new KeySpaceManagerProtocolServerSideTranslatorPB(this)); - final InetSocketAddress ksmNodeRpcAddr = - getKsmAddress(configuration); - ksmRpcServer = startRpcServer(configuration, ksmNodeRpcAddr, - KeySpaceManagerProtocolPB.class, ksmService, + BlockingService omService = newReflectiveBlockingService( + new OzoneManagerProtocolServerSideTranslatorPB(this)); + final InetSocketAddress omNodeRpcAddr = + getOmAddress(configuration); + omRpcServer = startRpcServer(configuration, omNodeRpcAddr, + OzoneManagerProtocolPB.class, omService, handlerCount); - ksmRpcAddress = updateRPCListenAddress(configuration, - OZONE_KSM_ADDRESS_KEY, ksmNodeRpcAddr, ksmRpcServer); - metadataManager = new KSMMetadataManagerImpl(configuration); + omRpcAddress = updateRPCListenAddress(configuration, + OZONE_OM_ADDRESS_KEY, omNodeRpcAddr, omRpcServer); + metadataManager = new OmMetadataManagerImpl(configuration); volumeManager = new VolumeManagerImpl(metadataManager, configuration); bucketManager = new BucketManagerImpl(metadataManager); - metrics = KSMMetrics.create(); + metrics = OMMetrics.create(); keyManager = new KeyManagerImpl(scmBlockClient, metadataManager, configuration, - ksmStorage.getKsmId()); - httpServer = new KeySpaceManagerHttpServer(configuration, this); + omStorage.getOmId()); + httpServer = new OzoneManagerHttpServer(configuration, this); } /** @@ -250,8 +249,8 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl } @VisibleForTesting - public KSMStorage getKsmStorage() { - return ksmStorage; + public OMStorage getOmStorage() { + return omStorage; } /** * Starts an RPC server, if configured. @@ -286,16 +285,16 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * Get metadata manager. * @return metadata manager. */ - public KSMMetadataManager getMetadataManager() { + public OMMetadataManager getMetadataManager() { return metadataManager; } - public KSMMetrics getMetrics() { + public OMMetrics getMetrics() { return metrics; } /** - * Main entry point for starting KeySpaceManager. + * Main entry point for starting OzoneManager. * * @param argv arguments * @throws IOException if startup fails due to I/O error @@ -312,14 +311,14 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl hParser.printGenericCommandUsage(System.err); System.exit(1); } - StringUtils.startupShutdownMessage(KeySpaceManager.class, argv, LOG); - KeySpaceManager ksm = createKSM(hParser.getRemainingArgs(), conf); - if (ksm != null) { - ksm.start(); - ksm.join(); + StringUtils.startupShutdownMessage(OzoneManager.class, argv, LOG); + OzoneManager om = createOm(hParser.getRemainingArgs(), conf); + if (om != null) { + om.start(); + om.join(); } } catch (Throwable t) { - LOG.error("Failed to start the KeyspaceManager.", t); + LOG.error("Failed to start the OzoneManager.", t); terminate(1, t); } } @@ -329,17 +328,17 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl } /** - * Constructs KSM instance based on command line arguments. + * Constructs OM instance based on command line arguments. * @param argv Command line arguments * @param conf OzoneConfiguration - * @return KSM instance - * @throws IOException in case KSM instance creation fails. + * @return OM instance + * @throws IOException in case OM instance creation fails. */ - public static KeySpaceManager createKSM(String[] argv, - OzoneConfiguration conf) throws IOException { + public static OzoneManager createOm(String[] argv, + OzoneConfiguration conf) throws IOException { if (!isHddsEnabled(conf)) { - System.err.println("KSM cannot be started in secure mode or when " + + System.err.println("OM cannot be started in secure mode or when " + OZONE_ENABLED + " is set to false"); System.exit(1); } @@ -351,27 +350,27 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl } switch (startOpt) { case CREATEOBJECTSTORE: - terminate(ksmInit(conf) ? 0 : 1); + terminate(omInit(conf) ? 0 : 1); return null; case HELP: printUsage(System.err); terminate(0); return null; default: - return new KeySpaceManager(conf); + return new OzoneManager(conf); } } /** - * Initializes the KSM instance. + * Initializes the OM instance. * @param conf OzoneConfiguration - * @return true if KSM initialization succeeds , false otherwise + * @return true if OM initialization succeeds, false otherwise * @throws IOException in case ozone metadata directory path is not accessible */ - private static boolean ksmInit(OzoneConfiguration conf) throws IOException { - KSMStorage ksmStorage = new KSMStorage(conf); - StorageState state = ksmStorage.getState(); + private static boolean omInit(OzoneConfiguration conf) throws IOException { + OMStorage omStorage = new OMStorage(conf); + StorageState state = omStorage.getState(); if (state != StorageState.INITIALIZED) { try { ScmBlockLocationProtocol scmBlockClient = getScmBlockClient(conf); @@ -384,29 +383,29 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl if (scmId == null || scmId.isEmpty()) { throw new IOException("Invalid SCM ID"); } - ksmStorage.setClusterId(clusterId); - ksmStorage.setScmId(scmId); - ksmStorage.initialize(); + omStorage.setClusterId(clusterId); + omStorage.setScmId(scmId); + omStorage.initialize(); System.out.println( - "KSM initialization succeeded.Current cluster id for sd=" - + ksmStorage.getStorageDir() + ";cid=" + ksmStorage + "OM initialization succeeded.Current cluster id for sd=" + + omStorage.getStorageDir() + ";cid=" + omStorage .getClusterID()); return true; } catch (IOException ioe) { - LOG.error("Could not initialize KSM version file", ioe); + LOG.error("Could not initialize OM version file", ioe); return false; } } else { System.out.println( - "KSM already initialized.Reusing existing cluster id for sd=" - + ksmStorage.getStorageDir() + ";cid=" + ksmStorage + "OM already initialized.Reusing existing cluster id for sd=" + + omStorage.getStorageDir() + ";cid=" + omStorage .getClusterID()); return true; } } /** - * Parses the command line options for KSM initialization. + * Parses the command line options for OM initialization. * @param args command line arguments * @return StartupOption if options are valid, null otherwise */ @@ -437,12 +436,12 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * Start service. */ public void start() throws IOException { - LOG.info(buildRpcServerStartMessage("KeyspaceManager RPC server", - ksmRpcAddress)); - DefaultMetricsSystem.initialize("KeySpaceManager"); + LOG.info(buildRpcServerStartMessage("OzoneManager RPC server", + omRpcAddress)); + DefaultMetricsSystem.initialize("OzoneManager"); metadataManager.start(); keyManager.start(); - ksmRpcServer.start(); + omRpcServer.start(); httpServer.start(); registerMXBean(); setStartTime(); @@ -454,13 +453,13 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl public void stop() { try { metadataManager.stop(); - ksmRpcServer.stop(); + omRpcServer.stop(); keyManager.stop(); httpServer.stop(); metrics.unRegister(); unregisterMXBean(); } catch (Exception e) { - LOG.error("Key Space Manager stop failed.", e); + LOG.error("OzoneManager stop failed.", e); } } @@ -469,10 +468,10 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl */ public void join() { try { - ksmRpcServer.join(); + omRpcServer.join(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.info("Interrupted during KeyspaceManager join.", e); + LOG.info("Interrupted during OzoneManager join.", e); } } @@ -483,7 +482,7 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * @throws IOException */ @Override - public void createVolume(KsmVolumeArgs args) throws IOException { + public void createVolume(OmVolumeArgs args) throws IOException { try { metrics.incNumVolumeCreates(); volumeManager.createVolume(args); @@ -558,7 +557,7 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * @throws IOException */ @Override - public KsmVolumeArgs getVolumeInfo(String volume) throws IOException { + public OmVolumeArgs getVolumeInfo(String volume) throws IOException { try { metrics.incNumVolumeInfos(); return volumeManager.getVolumeInfo(volume); @@ -597,8 +596,8 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * @throws IOException */ @Override - public List listVolumeByUser(String userName, String prefix, - String prevKey, int maxKeys) throws IOException { + public List listVolumeByUser(String userName, String prefix, + String prevKey, int maxKeys) throws IOException { try { metrics.incNumVolumeLists(); return volumeManager.listVolumes(userName, prefix, prevKey, maxKeys); @@ -619,7 +618,7 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * @throws IOException */ @Override - public List listAllVolumes(String prefix, String prevKey, int + public List listAllVolumes(String prefix, String prevKey, int maxKeys) throws IOException { try { metrics.incNumVolumeLists(); @@ -637,7 +636,7 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * @throws IOException */ @Override - public void createBucket(KsmBucketInfo bucketInfo) throws IOException { + public void createBucket(OmBucketInfo bucketInfo) throws IOException { try { metrics.incNumBucketCreates(); bucketManager.createBucket(bucketInfo); @@ -651,8 +650,8 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * {@inheritDoc} */ @Override - public List listBuckets(String volumeName, - String startKey, String prefix, int maxNumOfBuckets) + public List listBuckets(String volumeName, + String startKey, String prefix, int maxNumOfBuckets) throws IOException { try { metrics.incNumBucketLists(); @@ -669,11 +668,11 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * * @param volume - Volume name. * @param bucket - Bucket name. - * @return KsmBucketInfo or exception is thrown. + * @return OmBucketInfo or exception is thrown. * @throws IOException */ @Override - public KsmBucketInfo getBucketInfo(String volume, String bucket) + public OmBucketInfo getBucketInfo(String volume, String bucket) throws IOException { try { metrics.incNumBucketInfos(); @@ -688,11 +687,11 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * Allocate a key. * * @param args - attributes of the key. - * @return KsmKeyInfo - the info about the allocated key. + * @return OmKeyInfo - the info about the allocated key. * @throws IOException */ @Override - public OpenKeySession openKey(KsmKeyArgs args) throws IOException { + public OpenKeySession openKey(OmKeyArgs args) throws IOException { try { metrics.incNumKeyAllocates(); return keyManager.openKey(args); @@ -703,7 +702,7 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl } @Override - public void commitKey(KsmKeyArgs args, int clientID) + public void commitKey(OmKeyArgs args, int clientID) throws IOException { try { metrics.incNumKeyCommits(); @@ -715,7 +714,7 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl } @Override - public KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID) + public OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID) throws IOException { try { metrics.incNumBlockAllocateCalls(); @@ -730,11 +729,11 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * Lookup a key. * * @param args - attributes of the key. - * @return KsmKeyInfo - the info about the requested key. + * @return OmKeyInfo - the info about the requested key. * @throws IOException */ @Override - public KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException { + public OmKeyInfo lookupKey(OmKeyArgs args) throws IOException { try { metrics.incNumKeyLookups(); return keyManager.lookupKey(args); @@ -745,7 +744,7 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl } @Override - public void renameKey(KsmKeyArgs args, String toKeyName) throws IOException { + public void renameKey(OmKeyArgs args, String toKeyName) throws IOException { try { metrics.incNumKeyRenames(); keyManager.renameKey(args, toKeyName); @@ -762,7 +761,7 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * @throws IOException */ @Override - public void deleteKey(KsmKeyArgs args) throws IOException { + public void deleteKey(OmKeyArgs args) throws IOException { try { metrics.incNumKeyDeletes(); keyManager.deleteKey(args); @@ -773,8 +772,8 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl } @Override - public List listKeys(String volumeName, String bucketName, - String startKey, String keyPrefix, int maxKeys) throws IOException { + public List listKeys(String volumeName, String bucketName, + String startKey, String keyPrefix, int maxKeys) throws IOException { try { metrics.incNumKeyLists(); return keyManager.listKeys(volumeName, bucketName, @@ -791,7 +790,7 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl * @throws IOException */ @Override - public void setBucketProperty(KsmBucketArgs args) + public void setBucketProperty(OmBucketArgs args) throws IOException { try { metrics.incNumBucketUpdates(); @@ -822,27 +821,27 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl private void registerMXBean() { Map jmxProperties = new HashMap(); jmxProperties.put("component", "ServerRuntime"); - this.ksmInfoBeanName = - MBeans.register("KeySpaceManager", - "KeySpaceManagerInfo", + this.omInfoBeanName = + MBeans.register("OzoneManager", + "OzoneManagerInfo", jmxProperties, this); } private void unregisterMXBean() { - if (this.ksmInfoBeanName != null) { - MBeans.unregister(this.ksmInfoBeanName); - this.ksmInfoBeanName = null; + if (this.omInfoBeanName != null) { + MBeans.unregister(this.omInfoBeanName); + this.omInfoBeanName = null; } } @Override public String getRpcPort() { - return "" + ksmRpcAddress.getPort(); + return "" + omRpcAddress.getPort(); } @VisibleForTesting - public KeySpaceManagerHttpServer getHttpServer() { + public OzoneManagerHttpServer getHttpServer() { return httpServer; } @@ -850,26 +849,26 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl public List getServiceList() throws IOException { // When we implement multi-home this call has to be handled properly. List services = new ArrayList<>(); - ServiceInfo.Builder ksmServiceInfoBuilder = ServiceInfo.newBuilder() - .setNodeType(HddsProtos.NodeType.KSM) - .setHostname(ksmRpcAddress.getHostName()) + ServiceInfo.Builder omServiceInfoBuilder = ServiceInfo.newBuilder() + .setNodeType(HddsProtos.NodeType.OM) + .setHostname(omRpcAddress.getHostName()) .addServicePort(ServicePort.newBuilder() .setType(ServicePort.Type.RPC) - .setValue(ksmRpcAddress.getPort()) + .setValue(omRpcAddress.getPort()) .build()); if (httpServer.getHttpAddress() != null) { - ksmServiceInfoBuilder.addServicePort(ServicePort.newBuilder() + omServiceInfoBuilder.addServicePort(ServicePort.newBuilder() .setType(ServicePort.Type.HTTP) .setValue(httpServer.getHttpAddress().getPort()) .build()); } if (httpServer.getHttpsAddress() != null) { - ksmServiceInfoBuilder.addServicePort(ServicePort.newBuilder() + omServiceInfoBuilder.addServicePort(ServicePort.newBuilder() .setType(ServicePort.Type.HTTPS) .setValue(httpServer.getHttpsAddress().getPort()) .build()); } - services.add(ksmServiceInfoBuilder.build()); + services.add(omServiceInfoBuilder.build()); // For client we have to return SCM with container protocol port, // not block protocol. diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManagerHttpServer.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerHttpServer.java similarity index 68% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManagerHttpServer.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerHttpServer.java index 478804b32c5..bd6ab6910aa 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManagerHttpServer.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerHttpServer.java @@ -15,7 +15,7 @@ * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ozone.OzoneConfigKeys; @@ -25,47 +25,47 @@ import org.apache.hadoop.hdds.server.BaseHttpServer; import java.io.IOException; /** - * HttpServer wrapper for the KeySpaceManager. + * HttpServer wrapper for the OzoneManager. */ -public class KeySpaceManagerHttpServer extends BaseHttpServer { +public class OzoneManagerHttpServer extends BaseHttpServer { - public KeySpaceManagerHttpServer(Configuration conf, KeySpaceManager ksm) + public OzoneManagerHttpServer(Configuration conf, OzoneManager om) throws IOException { - super(conf, "ksm"); + super(conf, "ozoneManager"); addServlet("serviceList", "/serviceList", ServiceListJSONServlet.class); - getWebAppContext().setAttribute(OzoneConsts.KSM_CONTEXT_ATTRIBUTE, ksm); + getWebAppContext().setAttribute(OzoneConsts.OM_CONTEXT_ATTRIBUTE, om); } @Override protected String getHttpAddressKey() { - return KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY; + return OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY; } @Override protected String getHttpBindHostKey() { - return KSMConfigKeys.OZONE_KSM_HTTP_BIND_HOST_KEY; + return OMConfigKeys.OZONE_OM_HTTP_BIND_HOST_KEY; } @Override protected String getHttpsAddressKey() { - return KSMConfigKeys.OZONE_KSM_HTTPS_ADDRESS_KEY; + return OMConfigKeys.OZONE_OM_HTTPS_ADDRESS_KEY; } @Override protected String getHttpsBindHostKey() { - return KSMConfigKeys.OZONE_KSM_HTTPS_BIND_HOST_KEY; + return OMConfigKeys.OZONE_OM_HTTPS_BIND_HOST_KEY; } @Override protected String getBindHostDefault() { - return KSMConfigKeys.OZONE_KSM_HTTP_BIND_HOST_DEFAULT; + return OMConfigKeys.OZONE_OM_HTTP_BIND_HOST_DEFAULT; } @Override protected int getHttpBindPortDefault() { - return KSMConfigKeys.OZONE_KSM_HTTP_BIND_PORT_DEFAULT; + return OMConfigKeys.OZONE_OM_HTTP_BIND_PORT_DEFAULT; } @Override protected int getHttpsBindPortDefault() { - return KSMConfigKeys.OZONE_KSM_HTTPS_BIND_PORT_DEFAULT; + return OMConfigKeys.OZONE_OM_HTTPS_BIND_PORT_DEFAULT; } @Override protected String getKeytabFile() { - return KSMConfigKeys.OZONE_KSM_KEYTAB_FILE; + return OMConfigKeys.OZONE_OM_KEYTAB_FILE; } @Override protected String getSpnegoPrincipal() { @@ -73,6 +73,6 @@ public class KeySpaceManagerHttpServer extends BaseHttpServer { } @Override protected String getEnabledKey() { - return KSMConfigKeys.OZONE_KSM_HTTP_ENABLED_KEY; + return OMConfigKeys.OZONE_OM_HTTP_ENABLED_KEY; } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/ServiceListJSONServlet.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ServiceListJSONServlet.java similarity index 89% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/ServiceListJSONServlet.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ServiceListJSONServlet.java index 34a80ce104d..47713e2010c 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/ServiceListJSONServlet.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ServiceListJSONServlet.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; @@ -37,7 +37,7 @@ import java.io.PrintWriter; * Provides REST access to Ozone Service List. *

* This servlet generally will be placed under the /serviceList URL of - * KeySpaceManager HttpServer. + * OzoneManager HttpServer. * * The return format is of JSON and in the form *

@@ -45,7 +45,7 @@ import java.io.PrintWriter; * { * "services" : [ * { - * "NodeType":"KSM", + * "NodeType":"OM", * "Hostname" "$hostname", * "ports" : { * "$PortType" : "$port", @@ -64,11 +64,11 @@ public class ServiceListJSONServlet extends HttpServlet { LoggerFactory.getLogger(ServiceListJSONServlet.class); private static final long serialVersionUID = 1L; - private KeySpaceManager ksm; + private transient OzoneManager om; public void init() throws ServletException { - this.ksm = (KeySpaceManager) getServletContext() - .getAttribute(OzoneConsts.KSM_CONTEXT_ATTRIBUTE); + this.om = (OzoneManager) getServletContext() + .getAttribute(OzoneConsts.OM_CONTEXT_ATTRIBUTE); } /** @@ -87,7 +87,7 @@ public class ServiceListJSONServlet extends HttpServlet { response.setContentType("application/json; charset=utf8"); PrintWriter writer = response.getWriter(); try { - writer.write(objectMapper.writeValueAsString(ksm.getServiceList())); + writer.write(objectMapper.writeValueAsString(om.getServiceList())); } finally { if (writer != null) { writer.close(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManager.java similarity index 84% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManager.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManager.java index 6ac78d6eaeb..8475dd9e4a7 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManager.java @@ -14,17 +14,17 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; -import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo; + .OzoneManagerProtocolProtos.OzoneAclInfo; import java.io.IOException; import java.util.List; /** - * KSM volume manager interface. + * OM volume manager interface. */ public interface VolumeManager { @@ -32,7 +32,7 @@ public interface VolumeManager { * Create a new volume. * @param args - Volume args to create a volume */ - void createVolume(KsmVolumeArgs args) throws IOException; + void createVolume(OmVolumeArgs args) throws IOException; /** * Changes the owner of a volume. @@ -58,7 +58,7 @@ public interface VolumeManager { * @return VolumeArgs or exception is thrown. * @throws IOException */ - KsmVolumeArgs getVolumeInfo(String volume) throws IOException; + OmVolumeArgs getVolumeInfo(String volume) throws IOException; /** * Deletes an existing empty volume. @@ -92,9 +92,9 @@ public interface VolumeManager { * this key is excluded from the result. * @param maxKeys * the maximum number of volumes to return. - * @return a list of {@link KsmVolumeArgs} + * @return a list of {@link OmVolumeArgs} * @throws IOException */ - List listVolumes(String userName, String prefix, - String startKey, int maxKeys) throws IOException; + List listVolumes(String userName, String prefix, + String startKey, int maxKeys) throws IOException; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java similarity index 82% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManagerImpl.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java index cc2f78aa34d..e50145debdb 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java @@ -14,18 +14,18 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import com.google.common.base.Preconditions; -import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.ozone.ksm.exceptions.KSMException; +import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.OzoneAclInfo; + .OzoneManagerProtocolProtos.OzoneAclInfo; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.VolumeList; + .OzoneManagerProtocolProtos.VolumeList; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.VolumeInfo; + .OzoneManagerProtocolProtos.VolumeInfo; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.util.Time; import org.apache.hadoop.utils.BatchOperation; @@ -37,21 +37,20 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys - .OZONE_KSM_USER_MAX_VOLUME_DEFAULT; -import static org.apache.hadoop.ozone.ksm.KSMConfigKeys - .OZONE_KSM_USER_MAX_VOLUME; -import static org.apache.hadoop.ozone.ksm.exceptions - .KSMException.ResultCodes; +import static org.apache.hadoop.ozone.om.OMConfigKeys + .OZONE_OM_USER_MAX_VOLUME_DEFAULT; +import static org.apache.hadoop.ozone.om.OMConfigKeys + .OZONE_OM_USER_MAX_VOLUME; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes; /** - * KSM volume management code. + * OM volume management code. */ public class VolumeManagerImpl implements VolumeManager { private static final Logger LOG = LoggerFactory.getLogger(VolumeManagerImpl.class); - private final KSMMetadataManager metadataManager; + private final OMMetadataManager metadataManager; private final int maxUserVolumeCount; /** @@ -59,11 +58,11 @@ public class VolumeManagerImpl implements VolumeManager { * @param conf - Ozone configuration. * @throws IOException */ - public VolumeManagerImpl(KSMMetadataManager metadataManager, + public VolumeManagerImpl(OMMetadataManager metadataManager, OzoneConfiguration conf) throws IOException { this.metadataManager = metadataManager; - this.maxUserVolumeCount = conf.getInt(OZONE_KSM_USER_MAX_VOLUME, - OZONE_KSM_USER_MAX_VOLUME_DEFAULT); + this.maxUserVolumeCount = conf.getInt(OZONE_OM_USER_MAX_VOLUME, + OZONE_OM_USER_MAX_VOLUME_DEFAULT); } // Helpers to add and delete volume from user list @@ -81,7 +80,7 @@ public class VolumeManagerImpl implements VolumeManager { // Check the volume count if (prevVolList.size() >= maxUserVolumeCount) { LOG.debug("Too many volumes for user:{}", owner); - throw new KSMException(ResultCodes.FAILED_TOO_MANY_USER_VOLUMES); + throw new OMException(ResultCodes.FAILED_TOO_MANY_USER_VOLUMES); } // Add the new volume to the list @@ -103,7 +102,7 @@ public class VolumeManagerImpl implements VolumeManager { prevVolList.addAll(vlist.getVolumeNamesList()); } else { LOG.debug("volume:{} not found for user:{}"); - throw new KSMException(ResultCodes.FAILED_USER_NOT_FOUND); + throw new OMException(ResultCodes.FAILED_USER_NOT_FOUND); } // Remove the volume from the list @@ -119,10 +118,10 @@ public class VolumeManagerImpl implements VolumeManager { /** * Creates a volume. - * @param args - KsmVolumeArgs. + * @param args - OmVolumeArgs. */ @Override - public void createVolume(KsmVolumeArgs args) throws IOException { + public void createVolume(OmVolumeArgs args) throws IOException { Preconditions.checkNotNull(args); metadataManager.writeLock().lock(); try { @@ -132,7 +131,7 @@ public class VolumeManagerImpl implements VolumeManager { // Check of the volume already exists if (volumeInfo != null) { LOG.debug("volume:{} already exists", args.getVolume()); - throw new KSMException(ResultCodes.FAILED_VOLUME_ALREADY_EXISTS); + throw new OMException(ResultCodes.FAILED_VOLUME_ALREADY_EXISTS); } BatchOperation batch = new BatchOperation(); @@ -161,7 +160,7 @@ public class VolumeManagerImpl implements VolumeManager { LOG.debug("created volume:{} user:{}", args.getVolume(), args.getOwnerName()); } catch (IOException ex) { - if (!(ex instanceof KSMException)) { + if (!(ex instanceof OMException)) { LOG.error("Volume creation failed for user:{} volume:{}", args.getOwnerName(), args.getVolume(), ex); } @@ -189,19 +188,19 @@ public class VolumeManagerImpl implements VolumeManager { if (volInfo == null) { LOG.debug("Changing volume ownership failed for user:{} volume:{}", owner, volume); - throw new KSMException(ResultCodes.FAILED_VOLUME_NOT_FOUND); + throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND); } VolumeInfo volumeInfo = VolumeInfo.parseFrom(volInfo); - KsmVolumeArgs volumeArgs = KsmVolumeArgs.getFromProtobuf(volumeInfo); + OmVolumeArgs volumeArgs = OmVolumeArgs.getFromProtobuf(volumeInfo); Preconditions.checkState(volume.equals(volumeInfo.getVolume())); BatchOperation batch = new BatchOperation(); delVolumeFromOwnerList(volume, volumeArgs.getOwnerName(), batch); addVolumeToOwnerList(volume, owner, batch); - KsmVolumeArgs newVolumeArgs = - KsmVolumeArgs.newBuilder().setVolume(volumeArgs.getVolume()) + OmVolumeArgs newVolumeArgs = + OmVolumeArgs.newBuilder().setVolume(volumeArgs.getVolume()) .setAdminName(volumeArgs.getAdminName()) .setOwnerName(owner) .setQuotaInBytes(volumeArgs.getQuotaInBytes()) @@ -213,7 +212,7 @@ public class VolumeManagerImpl implements VolumeManager { metadataManager.writeBatch(batch); } catch (IOException ex) { - if (!(ex instanceof KSMException)) { + if (!(ex instanceof OMException)) { LOG.error("Changing volume ownership failed for user:{} volume:{}", owner, volume, ex); } @@ -238,15 +237,15 @@ public class VolumeManagerImpl implements VolumeManager { byte[] volInfo = metadataManager.get(dbVolumeKey); if (volInfo == null) { LOG.debug("volume:{} does not exist", volume); - throw new KSMException(ResultCodes.FAILED_VOLUME_NOT_FOUND); + throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND); } VolumeInfo volumeInfo = VolumeInfo.parseFrom(volInfo); - KsmVolumeArgs volumeArgs = KsmVolumeArgs.getFromProtobuf(volumeInfo); + OmVolumeArgs volumeArgs = OmVolumeArgs.getFromProtobuf(volumeInfo); Preconditions.checkState(volume.equals(volumeInfo.getVolume())); - KsmVolumeArgs newVolumeArgs = - KsmVolumeArgs.newBuilder() + OmVolumeArgs newVolumeArgs = + OmVolumeArgs.newBuilder() .setVolume(volumeArgs.getVolume()) .setAdminName(volumeArgs.getAdminName()) .setOwnerName(volumeArgs.getOwnerName()) @@ -256,7 +255,7 @@ public class VolumeManagerImpl implements VolumeManager { VolumeInfo newVolumeInfo = newVolumeArgs.getProtobuf(); metadataManager.put(dbVolumeKey, newVolumeInfo.toByteArray()); } catch (IOException ex) { - if (!(ex instanceof KSMException)) { + if (!(ex instanceof OMException)) { LOG.error("Changing volume quota failed for volume:{} quota:{}", volume, quota, ex); } @@ -272,7 +271,7 @@ public class VolumeManagerImpl implements VolumeManager { * @return VolumeArgs or exception is thrown. * @throws IOException */ - public KsmVolumeArgs getVolumeInfo(String volume) throws IOException { + public OmVolumeArgs getVolumeInfo(String volume) throws IOException { Preconditions.checkNotNull(volume); metadataManager.readLock().lock(); try { @@ -280,15 +279,15 @@ public class VolumeManagerImpl implements VolumeManager { byte[] volInfo = metadataManager.get(dbVolumeKey); if (volInfo == null) { LOG.debug("volume:{} does not exist", volume); - throw new KSMException(ResultCodes.FAILED_VOLUME_NOT_FOUND); + throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND); } VolumeInfo volumeInfo = VolumeInfo.parseFrom(volInfo); - KsmVolumeArgs volumeArgs = KsmVolumeArgs.getFromProtobuf(volumeInfo); + OmVolumeArgs volumeArgs = OmVolumeArgs.getFromProtobuf(volumeInfo); Preconditions.checkState(volume.equals(volumeInfo.getVolume())); return volumeArgs; } catch (IOException ex) { - if (!(ex instanceof KSMException)) { + if (!(ex instanceof OMException)) { LOG.warn("Info volume failed for volume:{}", volume, ex); } throw ex; @@ -313,12 +312,12 @@ public class VolumeManagerImpl implements VolumeManager { byte[] volInfo = metadataManager.get(dbVolumeKey); if (volInfo == null) { LOG.debug("volume:{} does not exist", volume); - throw new KSMException(ResultCodes.FAILED_VOLUME_NOT_FOUND); + throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND); } if (!metadataManager.isVolumeEmpty(volume)) { LOG.debug("volume:{} is not empty", volume); - throw new KSMException(ResultCodes.FAILED_VOLUME_NOT_EMPTY); + throw new OMException(ResultCodes.FAILED_VOLUME_NOT_EMPTY); } VolumeInfo volumeInfo = VolumeInfo.parseFrom(volInfo); @@ -329,7 +328,7 @@ public class VolumeManagerImpl implements VolumeManager { batch.delete(dbVolumeKey); metadataManager.writeBatch(batch); } catch (IOException ex) { - if (!(ex instanceof KSMException)) { + if (!(ex instanceof OMException)) { LOG.error("Delete volume failed for volume:{}", volume, ex); } throw ex; @@ -356,15 +355,15 @@ public class VolumeManagerImpl implements VolumeManager { byte[] volInfo = metadataManager.get(dbVolumeKey); if (volInfo == null) { LOG.debug("volume:{} does not exist", volume); - throw new KSMException(ResultCodes.FAILED_VOLUME_NOT_FOUND); + throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND); } VolumeInfo volumeInfo = VolumeInfo.parseFrom(volInfo); - KsmVolumeArgs volumeArgs = KsmVolumeArgs.getFromProtobuf(volumeInfo); + OmVolumeArgs volumeArgs = OmVolumeArgs.getFromProtobuf(volumeInfo); Preconditions.checkState(volume.equals(volumeInfo.getVolume())); return volumeArgs.getAclMap().hasAccess(userAcl); } catch (IOException ex) { - if (!(ex instanceof KSMException)) { + if (!(ex instanceof OMException)) { LOG.error("Check volume access failed for volume:{} user:{} rights:{}", volume, userAcl.getName(), userAcl.getRights(), ex); } @@ -378,8 +377,8 @@ public class VolumeManagerImpl implements VolumeManager { * {@inheritDoc} */ @Override - public List listVolumes(String userName, - String prefix, String startKey, int maxKeys) throws IOException { + public List listVolumes(String userName, + String prefix, String startKey, int maxKeys) throws IOException { metadataManager.readLock().lock(); try { return metadataManager.listVolumes( diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/exceptions/KSMException.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java similarity index 85% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/exceptions/KSMException.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java index b902eab7ce7..55cef97ed54 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/exceptions/KSMException.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java @@ -15,21 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm.exceptions; +package org.apache.hadoop.ozone.om.exceptions; import java.io.IOException; /** - * Exception thrown by KSM. + * Exception thrown by Ozone Manager. */ -public class KSMException extends IOException { - private final KSMException.ResultCodes result; +public class OMException extends IOException { + private final OMException.ResultCodes result; /** * Constructs an {@code IOException} with {@code null} * as its error detail message. */ - public KSMException(KSMException.ResultCodes result) { + public OMException(OMException.ResultCodes result) { this.result = result; } @@ -40,7 +40,7 @@ public class KSMException extends IOException { * the * {@link #getMessage()} method) */ - public KSMException(String message, KSMException.ResultCodes result) { + public OMException(String message, OMException.ResultCodes result) { super(message); this.result = result; } @@ -61,8 +61,8 @@ public class KSMException extends IOException { * cause is nonexistent or unknown.) * @since 1.6 */ - public KSMException(String message, Throwable cause, - KSMException.ResultCodes result) { + public OMException(String message, Throwable cause, + OMException.ResultCodes result) { super(message, cause); this.result = result; } @@ -79,7 +79,7 @@ public class KSMException extends IOException { * cause is nonexistent or unknown.) * @since 1.6 */ - public KSMException(Throwable cause, KSMException.ResultCodes result) { + public OMException(Throwable cause, OMException.ResultCodes result) { super(cause); this.result = result; } @@ -88,7 +88,7 @@ public class KSMException extends IOException { * Returns resultCode. * @return ResultCode */ - public KSMException.ResultCodes getResult() { + public OMException.ResultCodes getResult() { return result; } @@ -112,7 +112,7 @@ public class KSMException extends IOException { FAILED_INVALID_KEY_NAME, FAILED_METADATA_ERROR, FAILED_INTERNAL_ERROR, - KSM_NOT_INITIALIZED, + OM_NOT_INITIALIZED, SCM_VERSION_MISMATCH_ERROR } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/exceptions/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/exceptions/package-info.java similarity index 91% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/exceptions/package-info.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/exceptions/package-info.java index 09fd87f22c9..50915454338 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/exceptions/package-info.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/exceptions/package-info.java @@ -15,5 +15,5 @@ * the License. */ -package org.apache.hadoop.ozone.ksm.exceptions; -// Exception thrown by KSM. +package org.apache.hadoop.ozone.om.exceptions; +// Exception thrown by OM. diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/package-info.java similarity index 89% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/package-info.java index 09d9f326357..7904d5da083 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/package-info.java @@ -15,7 +15,7 @@ * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; /* - This package contains the keyspace manager classes. + This package contains the Ozone Manager classes. */ \ No newline at end of file diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java similarity index 76% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java index 38e7797ff03..40a88b698a6 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java @@ -19,89 +19,89 @@ package org.apache.hadoop.ozone.protocolPB; import com.google.common.collect.Lists; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo; -import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs; -import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession; -import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo; -import org.apache.hadoop.ozone.ksm.protocol.KeySpaceManagerProtocol; -import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolPB; -import org.apache.hadoop.ozone.ksm.exceptions.KSMException; +import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; +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.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OpenKeySession; +import org.apache.hadoop.ozone.om.helpers.ServiceInfo; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; +import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB; +import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.AllocateBlockRequest; + .OzoneManagerProtocolProtos.AllocateBlockRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.AllocateBlockResponse; + .OzoneManagerProtocolProtos.AllocateBlockResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CommitKeyRequest; + .OzoneManagerProtocolProtos.CommitKeyRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CommitKeyResponse; + .OzoneManagerProtocolProtos.CommitKeyResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CreateBucketRequest; + .OzoneManagerProtocolProtos.CreateBucketRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CreateBucketResponse; + .OzoneManagerProtocolProtos.CreateBucketResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.InfoBucketRequest; + .OzoneManagerProtocolProtos.InfoBucketRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.InfoBucketResponse; + .OzoneManagerProtocolProtos.InfoBucketResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.SetBucketPropertyRequest; + .OzoneManagerProtocolProtos.SetBucketPropertyRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.SetBucketPropertyResponse; + .OzoneManagerProtocolProtos.SetBucketPropertyResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.DeleteBucketRequest; + .OzoneManagerProtocolProtos.DeleteBucketRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.DeleteBucketResponse; + .OzoneManagerProtocolProtos.DeleteBucketResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CreateVolumeRequest; + .OzoneManagerProtocolProtos.CreateVolumeRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CreateVolumeResponse; + .OzoneManagerProtocolProtos.CreateVolumeResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.LocateKeyRequest; + .OzoneManagerProtocolProtos.LocateKeyRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.LocateKeyResponse; + .OzoneManagerProtocolProtos.LocateKeyResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.RenameKeyRequest; + .OzoneManagerProtocolProtos.RenameKeyRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.RenameKeyResponse; + .OzoneManagerProtocolProtos.RenameKeyResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.KeyArgs; + .OzoneManagerProtocolProtos.KeyArgs; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.SetVolumePropertyRequest; + .OzoneManagerProtocolProtos.SetVolumePropertyRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.SetVolumePropertyResponse; + .OzoneManagerProtocolProtos.SetVolumePropertyResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CheckVolumeAccessRequest; + .OzoneManagerProtocolProtos.CheckVolumeAccessRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.CheckVolumeAccessResponse; + .OzoneManagerProtocolProtos.CheckVolumeAccessResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.InfoVolumeRequest; + .OzoneManagerProtocolProtos.InfoVolumeRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.InfoVolumeResponse; + .OzoneManagerProtocolProtos.InfoVolumeResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.DeleteVolumeRequest; + .OzoneManagerProtocolProtos.DeleteVolumeRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.DeleteVolumeResponse; + .OzoneManagerProtocolProtos.DeleteVolumeResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListVolumeRequest; + .OzoneManagerProtocolProtos.ListVolumeRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListVolumeResponse; + .OzoneManagerProtocolProtos.ListVolumeResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListBucketsRequest; + .OzoneManagerProtocolProtos.ListBucketsRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListBucketsResponse; + .OzoneManagerProtocolProtos.ListBucketsResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListKeysRequest; + .OzoneManagerProtocolProtos.ListKeysRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ListKeysResponse; + .OzoneManagerProtocolProtos.ListKeysResponse; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.Status; + .OzoneManagerProtocolProtos.Status; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ServiceListRequest; + .OzoneManagerProtocolProtos.ServiceListRequest; import org.apache.hadoop.ozone.protocol.proto - .KeySpaceManagerProtocolProtos.ServiceListResponse; + .OzoneManagerProtocolProtos.ServiceListResponse; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -112,30 +112,30 @@ import java.util.stream.Collectors; /** * This class is the server-side translator that forwards requests received on - * {@link org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolPB} - * to the KeySpaceManagerService server implementation. + * {@link OzoneManagerProtocolPB} + * to the OzoneManagerService server implementation. */ -public class KeySpaceManagerProtocolServerSideTranslatorPB implements - KeySpaceManagerProtocolPB { +public class OzoneManagerProtocolServerSideTranslatorPB implements + OzoneManagerProtocolPB { private static final Logger LOG = LoggerFactory - .getLogger(KeySpaceManagerProtocolServerSideTranslatorPB.class); - private final KeySpaceManagerProtocol impl; + .getLogger(OzoneManagerProtocolServerSideTranslatorPB.class); + private final OzoneManagerProtocol impl; /** * Constructs an instance of the server handler. * - * @param impl KeySpaceManagerProtocolPB + * @param impl OzoneManagerProtocolPB */ - public KeySpaceManagerProtocolServerSideTranslatorPB( - KeySpaceManagerProtocol impl) { + public OzoneManagerProtocolServerSideTranslatorPB( + OzoneManagerProtocol impl) { this.impl = impl; } // Convert and exception to corresponding status code private Status exceptionToResponseStatus(IOException ex) { - if (ex instanceof KSMException) { - KSMException ksmException = (KSMException)ex; - switch (ksmException.getResult()) { + if (ex instanceof OMException) { + OMException omException = (OMException)ex; + switch (omException.getResult()) { case FAILED_VOLUME_ALREADY_EXISTS: return Status.VOLUME_ALREADY_EXISTS; case FAILED_TOO_MANY_USER_VOLUMES: @@ -158,6 +158,18 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements return Status.KEY_NOT_FOUND; case FAILED_INVALID_KEY_NAME: return Status.INVALID_KEY_NAME; + case FAILED_KEY_ALLOCATION: + return Status.KEY_ALLOCATION_ERROR; + case FAILED_KEY_DELETION: + return Status.KEY_DELETION_ERROR; + case FAILED_KEY_RENAME: + return Status.KEY_RENAME_ERROR; + case FAILED_METADATA_ERROR: + return Status.METADATA_ERROR; + case OM_NOT_INITIALIZED: + return Status.OM_NOT_INITIALIZED; + case SCM_VERSION_MISMATCH_ERROR: + return Status.SCM_VERSION_MISMATCH_ERROR; default: return Status.INTERNAL_ERROR; } @@ -176,7 +188,7 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements CreateVolumeResponse.Builder resp = CreateVolumeResponse.newBuilder(); resp.setStatus(Status.OK); try { - impl.createVolume(KsmVolumeArgs.getFromProtobuf(request.getVolumeInfo())); + impl.createVolume(OmVolumeArgs.getFromProtobuf(request.getVolumeInfo())); } catch (IOException e) { resp.setStatus(exceptionToResponseStatus(e)); } @@ -235,7 +247,7 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements resp.setStatus(Status.OK); String volume = request.getVolumeName(); try { - KsmVolumeArgs ret = impl.getVolumeInfo(volume); + OmVolumeArgs ret = impl.getVolumeInfo(volume); resp.setVolumeInfo(ret.getProtobuf()); } catch (IOException e) { resp.setStatus(exceptionToResponseStatus(e)); @@ -262,7 +274,7 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements RpcController controller, ListVolumeRequest request) throws ServiceException { ListVolumeResponse.Builder resp = ListVolumeResponse.newBuilder(); - List result = Lists.newArrayList(); + List result = Lists.newArrayList(); try { if (request.getScope() == ListVolumeRequest.Scope.VOLUMES_BY_USER) { @@ -294,7 +306,7 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements CreateBucketResponse.Builder resp = CreateBucketResponse.newBuilder(); try { - impl.createBucket(KsmBucketInfo.getFromProtobuf( + impl.createBucket(OmBucketInfo.getFromProtobuf( request.getBucketInfo())); resp.setStatus(Status.OK); } catch (IOException e) { @@ -310,10 +322,10 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements InfoBucketResponse.Builder resp = InfoBucketResponse.newBuilder(); try { - KsmBucketInfo ksmBucketInfo = impl.getBucketInfo( + OmBucketInfo omBucketInfo = impl.getBucketInfo( request.getVolumeName(), request.getBucketName()); resp.setStatus(Status.OK); - resp.setBucketInfo(ksmBucketInfo.getProtobuf()); + resp.setBucketInfo(omBucketInfo.getProtobuf()); } catch(IOException e) { resp.setStatus(exceptionToResponseStatus(e)); } @@ -332,7 +344,7 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements keyArgs.hasType()? keyArgs.getType() : null; HddsProtos.ReplicationFactor factor = keyArgs.hasFactor()? keyArgs.getFactor() : null; - KsmKeyArgs ksmKeyArgs = new KsmKeyArgs.Builder() + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() .setVolumeName(keyArgs.getVolumeName()) .setBucketName(keyArgs.getBucketName()) .setKeyName(keyArgs.getKeyName()) @@ -341,11 +353,11 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements .setFactor(factor) .build(); if (keyArgs.hasDataSize()) { - ksmKeyArgs.setDataSize(keyArgs.getDataSize()); + omKeyArgs.setDataSize(keyArgs.getDataSize()); } else { - ksmKeyArgs.setDataSize(0); + omKeyArgs.setDataSize(0); } - OpenKeySession openKey = impl.openKey(ksmKeyArgs); + OpenKeySession openKey = impl.openKey(omKeyArgs); resp.setKeyInfo(openKey.getKeyInfo().getProtobuf()); resp.setID(openKey.getId()); resp.setOpenVersion(openKey.getOpenVersion()); @@ -364,12 +376,12 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements LocateKeyResponse.newBuilder(); try { KeyArgs keyArgs = request.getKeyArgs(); - KsmKeyArgs ksmKeyArgs = new KsmKeyArgs.Builder() + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() .setVolumeName(keyArgs.getVolumeName()) .setBucketName(keyArgs.getBucketName()) .setKeyName(keyArgs.getKeyName()) .build(); - KsmKeyInfo keyInfo = impl.lookupKey(ksmKeyArgs); + OmKeyInfo keyInfo = impl.lookupKey(omKeyArgs); resp.setKeyInfo(keyInfo.getProtobuf()); resp.setStatus(Status.OK); } catch (IOException e) { @@ -385,12 +397,12 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements RenameKeyResponse.Builder resp = RenameKeyResponse.newBuilder(); try { KeyArgs keyArgs = request.getKeyArgs(); - KsmKeyArgs ksmKeyArgs = new KsmKeyArgs.Builder() + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() .setVolumeName(keyArgs.getVolumeName()) .setBucketName(keyArgs.getBucketName()) .setKeyName(keyArgs.getKeyName()) .build(); - impl.renameKey(ksmKeyArgs, request.getToKeyName()); + impl.renameKey(omKeyArgs, request.getToKeyName()); resp.setStatus(Status.OK); } catch (IOException e){ resp.setStatus(exceptionToResponseStatus(e)); @@ -405,7 +417,7 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements SetBucketPropertyResponse.Builder resp = SetBucketPropertyResponse.newBuilder(); try { - impl.setBucketProperty(KsmBucketArgs.getFromProtobuf( + impl.setBucketProperty(OmBucketArgs.getFromProtobuf( request.getBucketArgs())); resp.setStatus(Status.OK); } catch(IOException e) { @@ -421,12 +433,12 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements LocateKeyResponse.newBuilder(); try { KeyArgs keyArgs = request.getKeyArgs(); - KsmKeyArgs ksmKeyArgs = new KsmKeyArgs.Builder() + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() .setVolumeName(keyArgs.getVolumeName()) .setBucketName(keyArgs.getBucketName()) .setKeyName(keyArgs.getKeyName()) .build(); - impl.deleteKey(ksmKeyArgs); + impl.deleteKey(omKeyArgs); resp.setStatus(Status.OK); } catch (IOException e) { resp.setStatus(exceptionToResponseStatus(e)); @@ -455,12 +467,12 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements ListBucketsResponse.Builder resp = ListBucketsResponse.newBuilder(); try { - List buckets = impl.listBuckets( + List buckets = impl.listBuckets( request.getVolumeName(), request.getStartKey(), request.getPrefix(), request.getCount()); - for(KsmBucketInfo bucket : buckets) { + for(OmBucketInfo bucket : buckets) { resp.addBucketInfo(bucket.getProtobuf()); } resp.setStatus(Status.OK); @@ -476,13 +488,13 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements ListKeysResponse.Builder resp = ListKeysResponse.newBuilder(); try { - List keys = impl.listKeys( + List keys = impl.listKeys( request.getVolumeName(), request.getBucketName(), request.getStartKey(), request.getPrefix(), request.getCount()); - for(KsmKeyInfo key : keys) { + for(OmKeyInfo key : keys) { resp.addKeyInfo(key.getProtobuf()); } resp.setStatus(Status.OK); @@ -503,7 +515,7 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements keyArgs.hasType()? keyArgs.getType() : null; HddsProtos.ReplicationFactor factor = keyArgs.hasFactor()? keyArgs.getFactor() : null; - KsmKeyArgs ksmKeyArgs = new KsmKeyArgs.Builder() + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() .setVolumeName(keyArgs.getVolumeName()) .setBucketName(keyArgs.getBucketName()) .setKeyName(keyArgs.getKeyName()) @@ -512,7 +524,7 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements .setFactor(factor) .build(); int id = request.getClientID(); - impl.commitKey(ksmKeyArgs, id); + impl.commitKey(omKeyArgs, id); resp.setStatus(Status.OK); } catch (IOException e) { resp.setStatus(exceptionToResponseStatus(e)); @@ -527,13 +539,13 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements AllocateBlockResponse.newBuilder(); try { KeyArgs keyArgs = request.getKeyArgs(); - KsmKeyArgs ksmKeyArgs = new KsmKeyArgs.Builder() + OmKeyArgs omKeyArgs = new OmKeyArgs.Builder() .setVolumeName(keyArgs.getVolumeName()) .setBucketName(keyArgs.getBucketName()) .setKeyName(keyArgs.getKeyName()) .build(); int id = request.getClientID(); - KsmKeyLocationInfo newLocation = impl.allocateBlock(ksmKeyArgs, id); + OmKeyLocationInfo newLocation = impl.allocateBlock(omKeyArgs, id); resp.setKeyLocation(newLocation.getProtobuf()); resp.setStatus(Status.OK); } catch (IOException e) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/package-info.java index e9c24306ee6..9bc393dd18b 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/package-info.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/package-info.java @@ -18,5 +18,5 @@ package org.apache.hadoop.ozone.protocolPB; /** - * KSM protocol buffer translators. + * OM protocol buffer translators. */ \ No newline at end of file diff --git a/hadoop-ozone/ozone-manager/src/main/webapps/ksm/index.html b/hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/index.html similarity index 88% rename from hadoop-ozone/ozone-manager/src/main/webapps/ksm/index.html rename to hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/index.html index 7f18028f2b4..ba54cb2cf43 100644 --- a/hadoop-ozone/ozone-manager/src/main/webapps/ksm/index.html +++ b/hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/index.html @@ -22,9 +22,9 @@ - + - HDFS Key Space Manager + Ozone Manager @@ -34,7 +34,7 @@ - +

@@ -64,7 +64,7 @@ - + diff --git a/hadoop-ozone/ozone-manager/src/main/webapps/ksm/main.css b/hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/main.css similarity index 100% rename from hadoop-ozone/ozone-manager/src/main/webapps/ksm/main.css rename to hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/main.css diff --git a/hadoop-ozone/ozone-manager/src/main/webapps/ksm/main.html b/hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/main.html similarity index 100% rename from hadoop-ozone/ozone-manager/src/main/webapps/ksm/main.html rename to hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/main.html diff --git a/hadoop-ozone/ozone-manager/src/main/webapps/ksm/ksm-metrics.html b/hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/om-metrics.html similarity index 98% rename from hadoop-ozone/ozone-manager/src/main/webapps/ksm/ksm-metrics.html rename to hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/om-metrics.html index e63fb000cdb..15fba2fcb3d 100644 --- a/hadoop-ozone/ozone-manager/src/main/webapps/ksm/ksm-metrics.html +++ b/hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/om-metrics.html @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. --> -

KSM Metrics

+

OzoneManager Metrics

{{type}}

diff --git a/hadoop-ozone/ozone-manager/src/main/webapps/ksm/ksm.js b/hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/ozoneManager.js similarity index 90% rename from hadoop-ozone/ozone-manager/src/main/webapps/ksm/ksm.js rename to hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/ozoneManager.js index ab6f73bfc94..ca035548cd4 100644 --- a/hadoop-ozone/ozone-manager/src/main/webapps/ksm/ksm.js +++ b/hadoop-ozone/ozone-manager/src/main/webapps/ozoneManager/ozoneManager.js @@ -22,15 +22,15 @@ return key == 'name' || key == 'modelerType' || key.match(/tag.*/); }; - angular.module('ksm', ['ozone', 'nvd3']); - angular.module('ksm').config(function ($routeProvider) { + angular.module('ozoneManager', ['ozone', 'nvd3']); + angular.module('ozoneManager').config(function ($routeProvider) { $routeProvider - .when("/metrics/ksm", { - template: "" + .when("/metrics/ozoneManager", { + template: "" }); }); - angular.module('ksm').component('ksmMetrics', { - templateUrl: 'ksm-metrics.html', + angular.module('ozoneManager').component('omMetrics', { + templateUrl: 'om-metrics.html', controller: function ($http) { var ctrl = this; @@ -63,7 +63,7 @@ }; - $http.get("jmx?qry=Hadoop:service=KeySpaceManager,name=KSMMetrics") + $http.get("jmx?qry=Hadoop:service=OzoneManager,name=OMMetrics") .then(function (result) { var groupedMetrics = {others: [], nums: {}}; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java similarity index 77% rename from hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java rename to hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java index 0b43bf9b70a..1ecac7fdacb 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java @@ -14,16 +14,15 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.DFSUtil; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs; -import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmBucketArgs; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.OzoneConsts; -import org.apache.hadoop.ozone.ksm.exceptions.KSMException; -import org.apache.hadoop.ozone.ksm.exceptions - .KSMException.ResultCodes; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes; import org.apache.hadoop.ozone.OzoneAcl; import org.junit.Assert; import org.junit.Rule; @@ -47,16 +46,16 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import static org.mockito.Mockito.any; /** - * Tests BucketManagerImpl, mocks KSMMetadataManager for testing. + * Tests BucketManagerImpl, mocks OMMetadataManager for testing. */ @RunWith(MockitoJUnitRunner.class) public class TestBucketManagerImpl { @Rule public ExpectedException thrown = ExpectedException.none(); - private KSMMetadataManager getMetadataManagerMock(String... volumesToCreate) + private OMMetadataManager getMetadataManagerMock(String... volumesToCreate) throws IOException { - KSMMetadataManager metadataManager = Mockito.mock(KSMMetadataManager.class); + OMMetadataManager metadataManager = Mockito.mock(OMMetadataManager.class); Map metadataDB = new HashMap<>(); ReadWriteLock lock = new ReentrantReadWriteLock(); @@ -65,14 +64,14 @@ public class TestBucketManagerImpl { Mockito.when(metadataManager.getVolumeKey(any(String.class))).thenAnswer( (InvocationOnMock invocation) -> DFSUtil.string2Bytes( - OzoneConsts.KSM_VOLUME_PREFIX + invocation.getArguments()[0])); + OzoneConsts.OM_VOLUME_PREFIX + invocation.getArguments()[0])); Mockito.when(metadataManager .getBucketKey(any(String.class), any(String.class))).thenAnswer( (InvocationOnMock invocation) -> DFSUtil.string2Bytes( - OzoneConsts.KSM_VOLUME_PREFIX + OzoneConsts.OM_VOLUME_PREFIX + invocation.getArguments()[0] - + OzoneConsts.KSM_BUCKET_PREFIX + + OzoneConsts.OM_BUCKET_PREFIX + invocation.getArguments()[1])); Mockito.doAnswer( @@ -80,11 +79,11 @@ public class TestBucketManagerImpl { @Override public Boolean answer(InvocationOnMock invocation) throws Throwable { - String keyRootName = OzoneConsts.KSM_KEY_PREFIX + String keyRootName = OzoneConsts.OM_KEY_PREFIX + invocation.getArguments()[0] - + OzoneConsts.KSM_KEY_PREFIX + + OzoneConsts.OM_KEY_PREFIX + invocation.getArguments()[1] - + OzoneConsts.KSM_KEY_PREFIX; + + OzoneConsts.OM_KEY_PREFIX; Iterator keyIterator = metadataDB.keySet().iterator(); while(keyIterator.hasNext()) { if(keyIterator.next().startsWith(keyRootName)) { @@ -124,7 +123,7 @@ public class TestBucketManagerImpl { for(String volumeName : volumesToCreate) { byte[] dummyVolumeInfo = DFSUtil.string2Bytes(volumeName); - metadataDB.put(OzoneConsts.KSM_VOLUME_PREFIX + volumeName, + metadataDB.put(OzoneConsts.OM_VOLUME_PREFIX + volumeName, dummyVolumeInfo); } return metadataManager; @@ -133,26 +132,26 @@ public class TestBucketManagerImpl { @Test public void testCreateBucketWithoutVolume() throws IOException { thrown.expectMessage("Volume doesn't exist"); - KSMMetadataManager metaMgr = getMetadataManagerMock(); + OMMetadataManager metaMgr = getMetadataManagerMock(); try { BucketManager bucketManager = new BucketManagerImpl(metaMgr); - KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder() + OmBucketInfo bucketInfo = OmBucketInfo.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .build(); bucketManager.createBucket(bucketInfo); - } catch(KSMException ksmEx) { + } catch(OMException omEx) { Assert.assertEquals(ResultCodes.FAILED_VOLUME_NOT_FOUND, - ksmEx.getResult()); - throw ksmEx; + omEx.getResult()); + throw omEx; } } @Test public void testCreateBucket() throws IOException { - KSMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); + OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); BucketManager bucketManager = new BucketManagerImpl(metaMgr); - KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder() + OmBucketInfo bucketInfo = OmBucketInfo.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .build(); @@ -163,19 +162,19 @@ public class TestBucketManagerImpl { @Test public void testCreateAlreadyExistingBucket() throws IOException { thrown.expectMessage("Bucket already exist"); - KSMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); + OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); try { BucketManager bucketManager = new BucketManagerImpl(metaMgr); - KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder() + OmBucketInfo bucketInfo = OmBucketInfo.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .build(); bucketManager.createBucket(bucketInfo); bucketManager.createBucket(bucketInfo); - } catch(KSMException ksmEx) { + } catch(OMException omEx) { Assert.assertEquals(ResultCodes.FAILED_BUCKET_ALREADY_EXISTS, - ksmEx.getResult()); - throw ksmEx; + omEx.getResult()); + throw omEx; } } @@ -183,28 +182,28 @@ public class TestBucketManagerImpl { public void testGetBucketInfoForInvalidBucket() throws IOException { thrown.expectMessage("Bucket not found"); try { - KSMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); + OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); BucketManager bucketManager = new BucketManagerImpl(metaMgr); bucketManager.getBucketInfo("sampleVol", "bucketOne"); - } catch(KSMException ksmEx) { + } catch(OMException omEx) { Assert.assertEquals(ResultCodes.FAILED_BUCKET_NOT_FOUND, - ksmEx.getResult()); - throw ksmEx; + omEx.getResult()); + throw omEx; } } @Test public void testGetBucketInfo() throws IOException { - KSMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); + OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); BucketManager bucketManager = new BucketManagerImpl(metaMgr); - KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder() + OmBucketInfo bucketInfo = OmBucketInfo.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .setStorageType(StorageType.DISK) .setIsVersionEnabled(false) .build(); bucketManager.createBucket(bucketInfo); - KsmBucketInfo result = bucketManager.getBucketInfo( + OmBucketInfo result = bucketManager.getBucketInfo( "sampleVol", "bucketOne"); Assert.assertEquals("sampleVol", result.getVolumeName()); Assert.assertEquals("bucketOne", result.getBucketName()); @@ -215,13 +214,13 @@ public class TestBucketManagerImpl { @Test public void testSetBucketPropertyAddACL() throws IOException { - KSMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); + OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); List acls = new LinkedList<>(); OzoneAcl ozoneAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "root", OzoneAcl.OzoneACLRights.READ); acls.add(ozoneAcl); BucketManager bucketManager = new BucketManagerImpl(metaMgr); - KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder() + OmBucketInfo bucketInfo = OmBucketInfo.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .setAcls(acls) @@ -229,7 +228,7 @@ public class TestBucketManagerImpl { .setIsVersionEnabled(false) .build(); bucketManager.createBucket(bucketInfo); - KsmBucketInfo result = bucketManager.getBucketInfo( + OmBucketInfo result = bucketManager.getBucketInfo( "sampleVol", "bucketOne"); Assert.assertEquals("sampleVol", result.getVolumeName()); Assert.assertEquals("bucketOne", result.getBucketName()); @@ -238,13 +237,13 @@ public class TestBucketManagerImpl { OzoneAcl newAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "ozone", OzoneAcl.OzoneACLRights.READ); addAcls.add(newAcl); - KsmBucketArgs bucketArgs = KsmBucketArgs.newBuilder() + OmBucketArgs bucketArgs = OmBucketArgs.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .setAddAcls(addAcls) .build(); bucketManager.setBucketProperty(bucketArgs); - KsmBucketInfo updatedResult = bucketManager.getBucketInfo( + OmBucketInfo updatedResult = bucketManager.getBucketInfo( "sampleVol", "bucketOne"); Assert.assertEquals(2, updatedResult.getAcls().size()); Assert.assertTrue(updatedResult.getAcls().contains(newAcl)); @@ -252,7 +251,7 @@ public class TestBucketManagerImpl { @Test public void testSetBucketPropertyRemoveACL() throws IOException { - KSMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); + OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); List acls = new LinkedList<>(); OzoneAcl aclOne = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "root", OzoneAcl.OzoneACLRights.READ); @@ -261,7 +260,7 @@ public class TestBucketManagerImpl { acls.add(aclOne); acls.add(aclTwo); BucketManager bucketManager = new BucketManagerImpl(metaMgr); - KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder() + OmBucketInfo bucketInfo = OmBucketInfo.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .setAcls(acls) @@ -269,18 +268,18 @@ public class TestBucketManagerImpl { .setIsVersionEnabled(false) .build(); bucketManager.createBucket(bucketInfo); - KsmBucketInfo result = bucketManager.getBucketInfo( + OmBucketInfo result = bucketManager.getBucketInfo( "sampleVol", "bucketOne"); Assert.assertEquals(2, result.getAcls().size()); List removeAcls = new LinkedList<>(); removeAcls.add(aclTwo); - KsmBucketArgs bucketArgs = KsmBucketArgs.newBuilder() + OmBucketArgs bucketArgs = OmBucketArgs.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .setRemoveAcls(removeAcls) .build(); bucketManager.setBucketProperty(bucketArgs); - KsmBucketInfo updatedResult = bucketManager.getBucketInfo( + OmBucketInfo updatedResult = bucketManager.getBucketInfo( "sampleVol", "bucketOne"); Assert.assertEquals(1, updatedResult.getAcls().size()); Assert.assertFalse(updatedResult.getAcls().contains(aclTwo)); @@ -288,25 +287,25 @@ public class TestBucketManagerImpl { @Test public void testSetBucketPropertyChangeStorageType() throws IOException { - KSMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); + OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); BucketManager bucketManager = new BucketManagerImpl(metaMgr); - KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder() + OmBucketInfo bucketInfo = OmBucketInfo.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .setStorageType(StorageType.DISK) .build(); bucketManager.createBucket(bucketInfo); - KsmBucketInfo result = bucketManager.getBucketInfo( + OmBucketInfo result = bucketManager.getBucketInfo( "sampleVol", "bucketOne"); Assert.assertEquals(StorageType.DISK, result.getStorageType()); - KsmBucketArgs bucketArgs = KsmBucketArgs.newBuilder() + OmBucketArgs bucketArgs = OmBucketArgs.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .setStorageType(StorageType.SSD) .build(); bucketManager.setBucketProperty(bucketArgs); - KsmBucketInfo updatedResult = bucketManager.getBucketInfo( + OmBucketInfo updatedResult = bucketManager.getBucketInfo( "sampleVol", "bucketOne"); Assert.assertEquals(StorageType.SSD, updatedResult.getStorageType()); @@ -314,24 +313,24 @@ public class TestBucketManagerImpl { @Test public void testSetBucketPropertyChangeVersioning() throws IOException { - KSMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); + OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); BucketManager bucketManager = new BucketManagerImpl(metaMgr); - KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder() + OmBucketInfo bucketInfo = OmBucketInfo.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .setIsVersionEnabled(false) .build(); bucketManager.createBucket(bucketInfo); - KsmBucketInfo result = bucketManager.getBucketInfo( + OmBucketInfo result = bucketManager.getBucketInfo( "sampleVol", "bucketOne"); Assert.assertFalse(result.getIsVersionEnabled()); - KsmBucketArgs bucketArgs = KsmBucketArgs.newBuilder() + OmBucketArgs bucketArgs = OmBucketArgs.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .setIsVersionEnabled(true) .build(); bucketManager.setBucketProperty(bucketArgs); - KsmBucketInfo updatedResult = bucketManager.getBucketInfo( + OmBucketInfo updatedResult = bucketManager.getBucketInfo( "sampleVol", "bucketOne"); Assert.assertTrue(updatedResult.getIsVersionEnabled()); } @@ -339,10 +338,10 @@ public class TestBucketManagerImpl { @Test public void testDeleteBucket() throws IOException { thrown.expectMessage("Bucket not found"); - KSMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); + OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); BucketManager bucketManager = new BucketManagerImpl(metaMgr); for(int i = 0; i < 5; i++) { - KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder() + OmBucketInfo bucketInfo = OmBucketInfo.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucket_" + i) .build(); @@ -362,19 +361,19 @@ public class TestBucketManagerImpl { } try { bucketManager.getBucketInfo("sampleVol", "bucket_1"); - } catch(KSMException ksmEx) { + } catch(OMException omEx) { Assert.assertEquals(ResultCodes.FAILED_BUCKET_NOT_FOUND, - ksmEx.getResult()); - throw ksmEx; + omEx.getResult()); + throw omEx; } } @Test public void testDeleteNonEmptyBucket() throws IOException { thrown.expectMessage("Bucket is not empty"); - KSMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); + OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol"); BucketManager bucketManager = new BucketManagerImpl(metaMgr); - KsmBucketInfo bucketInfo = KsmBucketInfo.newBuilder() + OmBucketInfo bucketInfo = OmBucketInfo.newBuilder() .setVolumeName("sampleVol") .setBucketName("bucketOne") .build(); @@ -386,10 +385,10 @@ public class TestBucketManagerImpl { DFSUtil.string2Bytes("value_two")); try { bucketManager.deleteBucket("sampleVol", "bucketOne"); - } catch(KSMException ksmEx) { + } catch(OMException omEx) { Assert.assertEquals(ResultCodes.FAILED_BUCKET_NOT_EMPTY, - ksmEx.getResult()); - throw ksmEx; + omEx.getResult()); + throw omEx; } } } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/TestChunkStreams.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java similarity index 99% rename from hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/TestChunkStreams.java rename to hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java index e6158bddd0c..7ce916a9cc3 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/TestChunkStreams.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java @@ -14,7 +14,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerHttpServer.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHttpServer.java similarity index 91% rename from hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerHttpServer.java rename to hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHttpServer.java index b263df56a1e..3e11a13b141 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerHttpServer.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHttpServer.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; @@ -44,12 +44,12 @@ import java.util.Arrays; import java.util.Collection; /** - * Test http server os KSM with various HTTP option. + * Test http server of OM with various HTTP option. */ @RunWith(value = Parameterized.class) -public class TestKeySpaceManagerHttpServer { +public class TestOzoneManagerHttpServer { private static final String BASEDIR = GenericTestUtils - .getTempPath(TestKeySpaceManagerHttpServer.class.getSimpleName()); + .getTempPath(TestOzoneManagerHttpServer.class.getSimpleName()); private static String keystoresDir; private static String sslConfDir; private static Configuration conf; @@ -65,7 +65,7 @@ public class TestKeySpaceManagerHttpServer { private final HttpConfig.Policy policy; - public TestKeySpaceManagerHttpServer(Policy policy) { + public TestOzoneManagerHttpServer(Policy policy) { super(); this.policy = policy; } @@ -77,7 +77,7 @@ public class TestKeySpaceManagerHttpServer { conf = new Configuration(); keystoresDir = new File(BASEDIR).getAbsolutePath(); sslConfDir = KeyStoreTestUtil.getClasspathDir( - TestKeySpaceManagerHttpServer.class); + TestOzoneManagerHttpServer.class); KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false); connectionFactory = URLConnectionFactory.newDefaultURLConnectionFactory(conf); @@ -97,9 +97,9 @@ public class TestKeySpaceManagerHttpServer { conf.set(ScmConfigKeys.OZONE_SCM_HTTPS_ADDRESS_KEY, "localhost:0"); InetSocketAddress addr = InetSocketAddress.createUnresolved("localhost", 0); - KeySpaceManagerHttpServer server = null; + OzoneManagerHttpServer server = null; try { - server = new KeySpaceManagerHttpServer(conf, null); + server = new OzoneManagerHttpServer(conf, null); server.start(); Assert.assertTrue(implies(policy.isHttpEnabled(), diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/package-info.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/package-info.java similarity index 94% rename from hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/package-info.java rename to hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/package-info.java index 089ff4be31a..12fcf7c2728 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/ksm/package-info.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/package-info.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.ozone.ksm; +package org.apache.hadoop.ozone.om; /** - * KSM tests + * OM tests */ diff --git a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/contract/OzoneContract.java b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/contract/OzoneContract.java index 8417e463f90..b63e182a1e7 100644 --- a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/contract/OzoneContract.java +++ b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/contract/OzoneContract.java @@ -33,7 +33,7 @@ import org.apache.hadoop.ozone.web.handlers.UserArgs; import org.apache.hadoop.ozone.web.handlers.VolumeArgs; import org.apache.hadoop.ozone.web.interfaces.StorageHandler; import org.apache.hadoop.ozone.web.utils.OzoneUtils; -import org.apache.hadoop.ozone.ksm.KSMConfigKeys; +import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.junit.Assert; @@ -109,7 +109,7 @@ class OzoneContract extends AbstractFSContract { String uri = String.format("%s://%s.%s/", OzoneConsts.OZONE_URI_SCHEME, bucketName, volumeName); getConf().set("fs.defaultFS", uri); - copyClusterConfigs(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY); + copyClusterConfigs(OMConfigKeys.OZONE_OM_ADDRESS_KEY); copyClusterConfigs(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY); return FileSystem.get(getConf()); } diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java index 26776c5c35b..3884eddfc01 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java @@ -32,11 +32,11 @@ import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtilClient; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.OzoneAclInfo; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.BucketInfo; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyInfo; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.VolumeInfo; -import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.VolumeList; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.util.Tool; @@ -60,10 +60,10 @@ import java.util.HashSet; import java.util.Set; import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_SUFFIX; -import static org.apache.hadoop.ozone.OzoneConsts.KSM_DB_NAME; -import static org.apache.hadoop.ozone.OzoneConsts.KSM_USER_PREFIX; -import static org.apache.hadoop.ozone.OzoneConsts.KSM_BUCKET_PREFIX; -import static org.apache.hadoop.ozone.OzoneConsts.KSM_VOLUME_PREFIX; +import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME; +import static org.apache.hadoop.ozone.OzoneConsts.OM_USER_PREFIX; +import static org.apache.hadoop.ozone.OzoneConsts.OM_BUCKET_PREFIX; +import static org.apache.hadoop.ozone.OzoneConsts.OM_VOLUME_PREFIX; import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB; /** @@ -120,7 +120,7 @@ public class SQLCLI extends Configured implements Tool { "INSERT INTO openContainer (containerName, containerUsed) " + "VALUES (\"%s\", \"%s\")"; - // for ksm.db + // for om.db private static final String CREATE_VOLUME_LIST = "CREATE TABLE volumeList (" + "userName TEXT NOT NULL," + @@ -278,9 +278,9 @@ public class SQLCLI extends Configured implements Tool { } else if (dbName.toString().equals(OPEN_CONTAINERS_DB)) { LOG.info("Converting open container DB"); convertOpenContainerDB(dbPath, outPath); - } else if (dbName.toString().equals(KSM_DB_NAME)) { - LOG.info("Converting ksm DB"); - convertKSMDB(dbPath, outPath); + } else if (dbName.toString().equals(OM_DB_NAME)) { + LOG.info("Converting om DB"); + convertOMDB(dbPath, outPath); } else { LOG.error("Unrecognized db name {}", dbName); } @@ -301,7 +301,7 @@ public class SQLCLI extends Configured implements Tool { } /** - * Convert ksm.db to sqlite db file. With following schema. + * Convert om.db to sqlite db file. With following schema. * (* for primary key) * * 1. for key type USER, it contains a username and a list volumes @@ -341,8 +341,8 @@ public class SQLCLI extends Configured implements Tool { * @param outPath * @throws Exception */ - private void convertKSMDB(Path dbPath, Path outPath) throws Exception { - LOG.info("Create tables for sql ksm db."); + private void convertOMDB(Path dbPath, Path outPath) throws Exception { + LOG.info("Create tables for sql om db."); File dbFile = dbPath.toFile(); try (MetadataStore dbStore = MetadataStoreBuilder.newBuilder() .setConf(conf).setDbFile(dbFile).build(); @@ -357,7 +357,7 @@ public class SQLCLI extends Configured implements Tool { String keyString = DFSUtilClient.bytes2String(key); KeyType type = getKeyType(keyString); try { - insertKSMDB(conn, type, keyString, value); + insertOMDB(conn, type, keyString, value); } catch (IOException | SQLException ex) { LOG.error("Exception inserting key {} type {}", keyString, type, ex); } @@ -366,8 +366,8 @@ public class SQLCLI extends Configured implements Tool { } } - private void insertKSMDB(Connection conn, KeyType type, String keyName, - byte[] value) throws IOException, SQLException { + private void insertOMDB(Connection conn, KeyType type, String keyName, + byte[] value) throws IOException, SQLException { switch (type) { case USER: VolumeList volumeList = VolumeList.parseFrom(value); @@ -412,16 +412,16 @@ public class SQLCLI extends Configured implements Tool { executeSQL(conn, insertKeyInfo); break; default: - throw new IOException("Unknown key from ksm.db"); + throw new IOException("Unknown key from om.db"); } } private KeyType getKeyType(String key) { - if (key.startsWith(KSM_USER_PREFIX)) { + if (key.startsWith(OM_USER_PREFIX)) { return KeyType.USER; - } else if (key.startsWith(KSM_VOLUME_PREFIX)) { - return key.replaceFirst(KSM_VOLUME_PREFIX, "") - .contains(KSM_BUCKET_PREFIX) ? KeyType.BUCKET : KeyType.VOLUME; + } else if (key.startsWith(OM_VOLUME_PREFIX)) { + return key.replaceFirst(OM_VOLUME_PREFIX, "") + .contains(OM_BUCKET_PREFIX) ? KeyType.BUCKET : KeyType.VOLUME; }else { return KeyType.KEY; } From 0247cb6318507afe06816e337a19f396afc53efa Mon Sep 17 00:00:00 2001 From: Wei-Chiu Chuang Date: Fri, 6 Jul 2018 14:59:49 -0700 Subject: [PATCH 38/47] HDFS-13121. NPE when request file descriptors when SC read. Contributed by Zsolt Venczel. --- .../hdfs/client/impl/BlockReaderFactory.java | 5 ++ .../shortcircuit/TestShortCircuitCache.java | 89 +++++++++++++++++++ 2 files changed, 94 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java index 1003b957c64..ce4318531a3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java @@ -598,6 +598,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { sock.recvFileInputStreams(fis, buf, 0, buf.length); ShortCircuitReplica replica = null; try { + if (fis[0] == null || fis[1] == null) { + throw new IOException("the datanode " + datanode + " failed to " + + "pass a file descriptor (might have reached open file limit)."); + } + ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()); if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java index 4e2cedef560..ac29c3c33f7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java @@ -42,6 +42,10 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.ClientContext; +import org.apache.hadoop.hdfs.DFSClient; +import org.apache.hadoop.hdfs.DFSUtilClient; +import org.apache.hadoop.hdfs.PeerCache; import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory; import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil; import org.apache.hadoop.hdfs.DFSInputStream; @@ -50,10 +54,12 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.ExtendedBlockId; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.apache.hadoop.hdfs.client.impl.DfsClientConf; import org.apache.hadoop.hdfs.net.DomainPeer; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader; import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector; import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry; @@ -66,9 +72,11 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId; import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ipc.RetriableException; +import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.TemporarySocketDirectory; import org.apache.hadoop.security.token.SecretManager.InvalidToken; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Time; @@ -819,4 +827,85 @@ public class TestShortCircuitCache { .fetch(Mockito.eq(extendedBlockId), Mockito.any()); } } + + @Test + public void testRequestFileDescriptorsWhenULimit() throws Exception { + TemporarySocketDirectory sockDir = new TemporarySocketDirectory(); + Configuration conf = createShortCircuitConf( + "testRequestFileDescriptorsWhenULimit", sockDir); + + final short replicas = 1; + final int fileSize = 3; + final String testFile = "/testfile"; + + try (MiniDFSCluster cluster = + new MiniDFSCluster.Builder(conf).numDataNodes(replicas).build()) { + + cluster.waitActive(); + + DistributedFileSystem fs = cluster.getFileSystem(); + DFSTestUtil.createFile(fs, new Path(testFile), fileSize, replicas, 0L); + + LocatedBlock blk = new DFSClient(DFSUtilClient.getNNAddress(conf), conf) + .getLocatedBlocks(testFile, 0, fileSize).get(0); + + ClientContext clientContext = Mockito.mock(ClientContext.class); + Mockito.when(clientContext.getPeerCache()).thenAnswer( + (Answer) peerCacheCall -> { + PeerCache peerCache = new PeerCache(10, Long.MAX_VALUE); + DomainPeer peer = Mockito.spy(getDomainPeerToDn(conf)); + peerCache.put(blk.getLocations()[0], peer); + + Mockito.when(peer.getDomainSocket()).thenAnswer( + (Answer) domainSocketCall -> { + DomainSocket domainSocket = Mockito.mock(DomainSocket.class); + Mockito.when(domainSocket + .recvFileInputStreams( + Mockito.any(FileInputStream[].class), + Mockito.any(byte[].class), + Mockito.anyInt(), + Mockito.anyInt()) + ).thenAnswer( + // we are mocking the FileOutputStream array with nulls + (Answer) recvFileInputStreamsCall -> null + ); + return domainSocket; + } + ); + + return peerCache; + }); + + Mockito.when(clientContext.getShortCircuitCache()).thenAnswer( + (Answer) shortCircuitCacheCall -> { + ShortCircuitCache cache = Mockito.mock(ShortCircuitCache.class); + Mockito.when(cache.allocShmSlot( + Mockito.any(DatanodeInfo.class), + Mockito.any(DomainPeer.class), + Mockito.any(MutableBoolean.class), + Mockito.any(ExtendedBlockId.class), + Mockito.anyString())) + .thenAnswer((Answer) call -> null); + + return cache; + } + ); + + DatanodeInfo[] nodes = blk.getLocations(); + + try { + Assert.assertNull(new BlockReaderFactory(new DfsClientConf(conf)) + .setInetSocketAddress(NetUtils.createSocketAddr(nodes[0] + .getXferAddr())) + .setClientCacheContext(clientContext) + .setDatanodeInfo(blk.getLocations()[0]) + .setBlock(blk.getBlock()) + .setBlockToken(new Token()) + .createShortCircuitReplicaInfo()); + } catch (NullPointerException ex) { + Assert.fail("Should not throw NPE when the native library is unable " + + "to create new files!"); + } + } + } } From ba683204498c97654be4727ab9e128c433a45498 Mon Sep 17 00:00:00 2001 From: Rohith Sharma K S Date: Fri, 6 Jul 2018 15:19:01 -0700 Subject: [PATCH 39/47] YARN-8302. ATS v2 should handle HBase connection issue properly. Contributed by Billie Rinaldi. --- .../hadoop/yarn/conf/YarnConfiguration.java | 7 + .../storage/TestTimelineReaderHBaseDown.java | 220 ++++++++++++++++++ .../storage/HBaseTimelineReaderImpl.java | 93 ++++++++ 3 files changed, 320 insertions(+) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineReaderHBaseDown.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index 5842d643579..9156c2dff05 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -3659,6 +3659,13 @@ public class YarnConfiguration extends Configuration { DEFAULT_TIMELINE_SERVICE_READER_WEBAPP_HTTPS_ADDRESS = DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS; + @Private + public static final String + TIMELINE_SERVICE_READER_STORAGE_MONITOR_INTERVAL_MS = + TIMELINE_SERVICE_READER_PREFIX + "storage-monitor.interval-ms"; + public static final long + DEFAULT_TIMELINE_SERVICE_STORAGE_MONITOR_INTERVAL_MS = 60 * 1000; + /** * Marked collector properties as Private since it run as auxillary service. */ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineReaderHBaseDown.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineReaderHBaseDown.java new file mode 100644 index 00000000000..786f529a7aa --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineReaderHBaseDown.java @@ -0,0 +1,220 @@ +/** + * 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.yarn.server.timelineservice.storage; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.service.Service; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Set; +import java.util.concurrent.TimeoutException; + +import static org.apache.hadoop.yarn.conf.YarnConfiguration.TIMELINE_SERVICE_READER_STORAGE_MONITOR_INTERVAL_MS; +import static org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl.DATA_TO_RETRIEVE; +import static org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl.MONITOR_FILTERS; + +public class TestTimelineReaderHBaseDown { + + @Test(timeout=300000) + public void testTimelineReaderHBaseUp() throws Exception { + HBaseTestingUtility util = new HBaseTestingUtility(); + configure(util); + try { + util.startMiniCluster(); + DataGeneratorForTest.createSchema(util.getConfiguration()); + DataGeneratorForTest.loadApps(util, System.currentTimeMillis()); + + TimelineReaderServer server = getTimelineReaderServer(); + server.init(util.getConfiguration()); + HBaseTimelineReaderImpl htr = getHBaseTimelineReaderImpl(server); + server.start(); + checkQuery(htr); + } finally { + util.shutdownMiniCluster(); + } + } + + @Test(timeout=300000) + public void testTimelineReaderInitWhenHBaseIsDown() throws + TimeoutException, InterruptedException { + HBaseTestingUtility util = new HBaseTestingUtility(); + configure(util); + TimelineReaderServer server = getTimelineReaderServer(); + + // init timeline reader when hbase is not running + server.init(util.getConfiguration()); + HBaseTimelineReaderImpl htr = getHBaseTimelineReaderImpl(server); + server.start(); + waitForHBaseDown(htr); + } + + @Test(timeout=300000) + public void testTimelineReaderDetectsHBaseDown() throws Exception { + HBaseTestingUtility util = new HBaseTestingUtility(); + configure(util); + + try { + // start minicluster + util.startMiniCluster(); + DataGeneratorForTest.createSchema(util.getConfiguration()); + DataGeneratorForTest.loadApps(util, System.currentTimeMillis()); + + // init timeline reader + TimelineReaderServer server = getTimelineReaderServer(); + server.init(util.getConfiguration()); + HBaseTimelineReaderImpl htr = getHBaseTimelineReaderImpl(server); + + // stop hbase after timeline reader init + util.shutdownMiniHBaseCluster(); + + // start server and check that it detects hbase is down + server.start(); + waitForHBaseDown(htr); + } finally { + util.shutdownMiniCluster(); + } + } + + @Test(timeout=300000) + public void testTimelineReaderDetectsZooKeeperDown() throws Exception { + HBaseTestingUtility util = new HBaseTestingUtility(); + configure(util); + + try { + // start minicluster + util.startMiniCluster(); + DataGeneratorForTest.createSchema(util.getConfiguration()); + DataGeneratorForTest.loadApps(util, System.currentTimeMillis()); + + // init timeline reader + TimelineReaderServer server = getTimelineReaderServer(); + server.init(util.getConfiguration()); + HBaseTimelineReaderImpl htr = getHBaseTimelineReaderImpl(server); + + // stop hbase and zookeeper after timeline reader init + util.shutdownMiniCluster(); + + // start server and check that it detects hbase is down + server.start(); + waitForHBaseDown(htr); + } finally { + util.shutdownMiniCluster(); + } + } + + @Test(timeout=300000) + public void testTimelineReaderRecoversAfterHBaseReturns() throws Exception { + HBaseTestingUtility util = new HBaseTestingUtility(); + configure(util); + + try { + // start minicluster + util.startMiniCluster(); + DataGeneratorForTest.createSchema(util.getConfiguration()); + DataGeneratorForTest.loadApps(util, System.currentTimeMillis()); + + // init timeline reader + TimelineReaderServer server = getTimelineReaderServer(); + server.init(util.getConfiguration()); + HBaseTimelineReaderImpl htr = getHBaseTimelineReaderImpl(server); + + // stop hbase after timeline reader init + util.shutdownMiniHBaseCluster(); + + // start server and check that it detects hbase is down + server.start(); + waitForHBaseDown(htr); + + util.startMiniHBaseCluster(1, 1); + GenericTestUtils.waitFor(() -> !htr.isHBaseDown(), 1000, 150000); + } finally { + util.shutdownMiniCluster(); + } + } + + private static void waitForHBaseDown(HBaseTimelineReaderImpl htr) throws + TimeoutException, InterruptedException { + GenericTestUtils.waitFor(() -> htr.isHBaseDown(), 1000, 150000); + try { + checkQuery(htr); + Assert.fail("Query should fail when HBase is down"); + } catch (IOException e) { + Assert.assertEquals("HBase is down", e.getMessage()); + } + } + + private static void checkQuery(HBaseTimelineReaderImpl htr) throws + IOException { + TimelineReaderContext context = + new TimelineReaderContext(YarnConfiguration.DEFAULT_RM_CLUSTER_ID, + null, null, null, null, TimelineEntityType + .YARN_FLOW_ACTIVITY.toString(), null, null); + Set entities = htr.getEntities(context, MONITOR_FILTERS, + DATA_TO_RETRIEVE); + } + + private static void configure(HBaseTestingUtility util) { + Configuration config = util.getConfiguration(); + config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + config.set(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS, + "localhost:0"); + config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1"); + config.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS, + "org.apache.hadoop.yarn.server.timelineservice.storage." + + "HBaseTimelineReaderImpl"); + config.setInt("hfile.format.version", 3); + config.setLong(TIMELINE_SERVICE_READER_STORAGE_MONITOR_INTERVAL_MS, 5000); + } + + private static TimelineReaderServer getTimelineReaderServer() { + return new TimelineReaderServer() { + @Override + protected void addFilters(Configuration conf) { + // The parent code uses hadoop-common jar from this version of + // Hadoop, but the tests are using hadoop-common jar from + // ${hbase-compatible-hadoop.version}. This version uses Jetty 9 + // while ${hbase-compatible-hadoop.version} uses Jetty 6, and there + // are many differences, including classnames and packages. + // We do nothing here, so that we don't cause a NoSuchMethodError or + // NoClassDefFoundError. + // Once ${hbase-compatible-hadoop.version} is changed to Hadoop 3, + // we should be able to remove this @Override. + } + }; + } + + private static HBaseTimelineReaderImpl getHBaseTimelineReaderImpl( + TimelineReaderServer server) { + for (Service s: server.getServices()) { + if (s instanceof HBaseTimelineReaderImpl) { + return (HBaseTimelineReaderImpl) s; + } + } + throw new IllegalStateException("Couldn't find HBaseTimelineReaderImpl"); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java index 1ebfab29cad..fadfd14edb6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java @@ -20,12 +20,18 @@ package org.apache.hadoop.yarn.server.timelineservice.storage; import java.io.IOException; import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; @@ -47,6 +53,12 @@ public class HBaseTimelineReaderImpl private Configuration hbaseConf = null; private Connection conn; + private Configuration monitorHBaseConf = null; + private Connection monitorConn; + private ScheduledExecutorService monitorExecutorService; + private TimelineReaderContext monitorContext; + private long monitorInterval; + private AtomicBoolean hbaseDown = new AtomicBoolean(); public HBaseTimelineReaderImpl() { super(HBaseTimelineReaderImpl.class.getName()); @@ -55,22 +67,72 @@ public class HBaseTimelineReaderImpl @Override public void serviceInit(Configuration conf) throws Exception { super.serviceInit(conf); + + String clusterId = conf.get( + YarnConfiguration.RM_CLUSTER_ID, + YarnConfiguration.DEFAULT_RM_CLUSTER_ID); + monitorContext = + new TimelineReaderContext(clusterId, null, null, null, null, + TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null, null); + monitorInterval = conf.getLong( + YarnConfiguration.TIMELINE_SERVICE_READER_STORAGE_MONITOR_INTERVAL_MS, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_STORAGE_MONITOR_INTERVAL_MS); + + monitorHBaseConf = HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(conf); + monitorHBaseConf.setInt("hbase.client.retries.number", 3); + monitorHBaseConf.setLong("hbase.client.pause", 1000); + monitorHBaseConf.setLong("hbase.rpc.timeout", monitorInterval); + monitorHBaseConf.setLong("hbase.client.scanner.timeout.period", + monitorInterval); + monitorHBaseConf.setInt("zookeeper.recovery.retry", 1); + monitorConn = ConnectionFactory.createConnection(monitorHBaseConf); + + monitorExecutorService = Executors.newScheduledThreadPool(1); + hbaseConf = HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(conf); conn = ConnectionFactory.createConnection(hbaseConf); } + @Override + protected void serviceStart() throws Exception { + super.serviceStart(); + LOG.info("Scheduling HBase liveness monitor at interval {}", + monitorInterval); + monitorExecutorService.scheduleAtFixedRate(new HBaseMonitor(), 0, + monitorInterval, TimeUnit.MILLISECONDS); + } + @Override protected void serviceStop() throws Exception { if (conn != null) { LOG.info("closing the hbase Connection"); conn.close(); } + if (monitorExecutorService != null) { + monitorExecutorService.shutdownNow(); + if (!monitorExecutorService.awaitTermination(30, TimeUnit.SECONDS)) { + LOG.warn("failed to stop the monitir task in time. " + + "will still proceed to close the monitor."); + } + } + monitorConn.close(); super.serviceStop(); } + private void checkHBaseDown() throws IOException { + if (hbaseDown.get()) { + throw new IOException("HBase is down"); + } + } + + public boolean isHBaseDown() { + return hbaseDown.get(); + } + @Override public TimelineEntity getEntity(TimelineReaderContext context, TimelineDataToRetrieve dataToRetrieve) throws IOException { + checkHBaseDown(); TimelineEntityReader reader = TimelineEntityReaderFactory.createSingleEntityReader(context, dataToRetrieve); @@ -81,6 +143,7 @@ public class HBaseTimelineReaderImpl public Set getEntities(TimelineReaderContext context, TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve) throws IOException { + checkHBaseDown(); TimelineEntityReader reader = TimelineEntityReaderFactory.createMultipleEntitiesReader(context, filters, dataToRetrieve); @@ -90,7 +153,37 @@ public class HBaseTimelineReaderImpl @Override public Set getEntityTypes(TimelineReaderContext context) throws IOException { + checkHBaseDown(); EntityTypeReader reader = new EntityTypeReader(context); return reader.readEntityTypes(hbaseConf, conn); } + + protected static final TimelineEntityFilters MONITOR_FILTERS = + new TimelineEntityFilters.Builder().entityLimit(1L).build(); + protected static final TimelineDataToRetrieve DATA_TO_RETRIEVE = + new TimelineDataToRetrieve(null, null, null, null, null, null); + + private class HBaseMonitor implements Runnable { + @Override + public void run() { + try { + LOG.info("Running HBase liveness monitor"); + TimelineEntityReader reader = + TimelineEntityReaderFactory.createMultipleEntitiesReader( + monitorContext, MONITOR_FILTERS, DATA_TO_RETRIEVE); + reader.readEntities(monitorHBaseConf, monitorConn); + + // on success, reset hbase down flag + if (hbaseDown.getAndSet(false)) { + if(LOG.isDebugEnabled()) { + LOG.debug("HBase request succeeded, assuming HBase up"); + } + } + } catch (Exception e) { + LOG.warn("Got failure attempting to read from timeline storage, " + + "assuming HBase down", e); + hbaseDown.getAndSet(true); + } + } + } } From 936e0df0d344f13eea97fe624b154e8356cdea7c Mon Sep 17 00:00:00 2001 From: Xiao Chen Date: Fri, 6 Jul 2018 20:45:27 -0700 Subject: [PATCH 40/47] HDFS-13721. NPE in DataNode due to uninitialized DiskBalancer. --- .../hadoop/hdfs/server/datanode/DataNode.java | 19 +++++++++++++------ .../server/diskbalancer/TestDiskBalancer.java | 5 +++++ 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index 4baafb92847..7df92f6083c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -3132,7 +3132,7 @@ public class DataNode extends ReconfigurableBase @Override // DataNodeMXBean public String getDiskBalancerStatus() { try { - return this.diskBalancer.queryWorkStatus().toJsonString(); + return getDiskBalancer().queryWorkStatus().toJsonString(); } catch (IOException ex) { LOG.debug("Reading diskbalancer Status failed. ex:{}", ex); return ""; @@ -3510,7 +3510,7 @@ public class DataNode extends ReconfigurableBase DiskBalancerException.Result.DATANODE_STATUS_NOT_REGULAR); } - this.diskBalancer.submitPlan(planID, planVersion, planFile, planData, + getDiskBalancer().submitPlan(planID, planVersion, planFile, planData, skipDateCheck); } @@ -3522,7 +3522,7 @@ public class DataNode extends ReconfigurableBase public void cancelDiskBalancePlan(String planID) throws IOException { checkSuperuserPrivilege(); - this.diskBalancer.cancelPlan(planID); + getDiskBalancer().cancelPlan(planID); } /** @@ -3533,7 +3533,7 @@ public class DataNode extends ReconfigurableBase @Override public DiskBalancerWorkStatus queryDiskBalancerPlan() throws IOException { checkSuperuserPrivilege(); - return this.diskBalancer.queryWorkStatus(); + return getDiskBalancer().queryWorkStatus(); } /** @@ -3550,9 +3550,9 @@ public class DataNode extends ReconfigurableBase Preconditions.checkNotNull(key); switch (key) { case DiskBalancerConstants.DISKBALANCER_VOLUME_NAME: - return this.diskBalancer.getVolumeNames(); + return getDiskBalancer().getVolumeNames(); case DiskBalancerConstants.DISKBALANCER_BANDWIDTH : - return Long.toString(this.diskBalancer.getBandwidth()); + return Long.toString(getDiskBalancer().getBandwidth()); default: LOG.error("Disk Balancer - Unknown key in get balancer setting. Key: {}", key); @@ -3606,4 +3606,11 @@ public class DataNode extends ReconfigurableBase } return volumeInfoList; } + + private DiskBalancer getDiskBalancer() throws IOException { + if (this.diskBalancer == null) { + throw new IOException("DiskBalancer is not initialized"); + } + return this.diskBalancer; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java index 55cc57ed03f..e7896944f64 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java @@ -109,6 +109,11 @@ public class TestDiskBalancer { .getFsVolumeReferences()) { assertEquals(ref.size(), dbDnNode.getVolumeCount()); } + + // Shutdown the DN first, to verify that calling diskbalancer APIs on + // uninitialized DN doesn't NPE + dnNode.shutdown(); + assertEquals("", dnNode.getDiskBalancerStatus()); } finally { cluster.shutdown(); } From 2f51cd60ef082cd0360fe46e9d2a4ec9b8ed979a Mon Sep 17 00:00:00 2001 From: Anu Engineer Date: Sun, 8 Jul 2018 11:11:21 -0700 Subject: [PATCH 41/47] HDDS-217. Move all SCMEvents to a package. Contributed by Anu Engineer. --- .../container/CloseContainerEventHandler.java | 13 ++- .../hadoop/hdds/scm/events/SCMEvents.java | 80 +++++++++++++++++++ .../hadoop/hdds/scm/events/package-info.java | 23 ++++++ .../hadoop/hdds/scm/node/SCMNodeManager.java | 5 +- .../SCMDatanodeHeartbeatDispatcher.java | 11 ++- .../scm/server/StorageContainerManager.java | 7 +- .../TestCloseContainerEventHandler.java | 27 ++++--- .../hadoop/hdds/scm/node/TestNodeManager.java | 10 +-- .../TestSCMDatanodeHeartbeatDispatcher.java | 20 +++-- 9 files changed, 147 insertions(+), 49 deletions(-) create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/package-info.java diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java index 7b24538a6bc..f1053d53432 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java @@ -24,15 +24,14 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.server.events.EventHandler; import org.apache.hadoop.hdds.server.events.EventPublisher; -import org.apache.hadoop.hdds.server.events.TypedEvent; import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * In case of a node failure, volume failure, volume out of spapce, node - * out of space etc, CLOSE_CONTAINER_EVENT will be triggered. - * CloseContainerEventHandler is the handler for CLOSE_CONTAINER_EVENT. + * out of space etc, CLOSE_CONTAINER will be triggered. + * CloseContainerEventHandler is the handler for CLOSE_CONTAINER. * When a close container event is fired, a close command for the container * should be sent to all the datanodes in the pipeline and containerStateManager * needs to update the container state to Closing. @@ -42,8 +41,6 @@ public class CloseContainerEventHandler implements EventHandler { public static final Logger LOG = LoggerFactory.getLogger(CloseContainerEventHandler.class); - public static final TypedEvent CLOSE_CONTAINER_EVENT = - new TypedEvent<>(ContainerID.class); private final Mapping containerManager; @@ -59,7 +56,8 @@ public class CloseContainerEventHandler implements EventHandler { ContainerWithPipeline containerWithPipeline = null; ContainerInfo info; try { - containerWithPipeline = containerManager.getContainerWithPipeline(containerID.getId()); + containerWithPipeline = + containerManager.getContainerWithPipeline(containerID.getId()); info = containerWithPipeline.getContainerInfo(); if (info == null) { LOG.info("Failed to update the container state. Container with id : {} " @@ -73,7 +71,8 @@ public class CloseContainerEventHandler implements EventHandler { } if (info.getState() == HddsProtos.LifeCycleState.OPEN) { - for (DatanodeDetails datanode : containerWithPipeline.getPipeline().getMachines()) { + for (DatanodeDetails datanode : + containerWithPipeline.getPipeline().getMachines()) { containerManager.getNodeManager().addDatanodeCommand(datanode.getUuid(), new CloseContainerCommand(containerID.getId(), info.getReplicationType())); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java new file mode 100644 index 00000000000..2c9c4310b94 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java @@ -0,0 +1,80 @@ +/* + * 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.hdds.scm.events; + +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.ContainerReportFromDatanode; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.NodeReportFromDatanode; + +import org.apache.hadoop.hdds.server.events.Event; +import org.apache.hadoop.hdds.server.events.TypedEvent; +import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode; + +/** + * Class that acts as the namespace for all SCM Events. + */ +public final class SCMEvents { + + /** + * NodeReports are sent out by Datanodes. This report is + * received by SCMDatanodeHeartbeatDispatcher and NodeReport Event is + * generated. + */ + public static final TypedEvent NODE_REPORT = + new TypedEvent<>(NodeReportFromDatanode.class, "Node_Report"); + /** + * ContainerReports are send out by Datanodes. This report + * is received by SCMDatanodeHeartbeatDispatcher and Container_Report Event + * i generated. + */ + public static final TypedEvent CONTAINER_REPORT = + new TypedEvent<>(ContainerReportFromDatanode.class, "Container_Report"); + + /** + * When ever a command for the Datanode needs to be issued by any component + * inside SCM, a Datanode_Command event is generated. NodeManager listens + * to these events and dispatches them to Datanode for further processing. + */ + public static final Event DATANODE_COMMAND = + new TypedEvent<>(CommandForDatanode.class, "Datanode_Command"); + + /** + * A Close Container Event can be triggered under many condition. + * Some of them are: + * 1. A Container is full, then we stop writing further information to + * that container. DN's let SCM know that current state and sends a + * informational message that allows SCM to close the container. + * + * 2. If a pipeline is open; for example Ratis; if a single node fails, + * we will proactively close these containers. + * + * Once a command is dispatched to DN, we will also listen to updates from + * the datanode which lets us know that this command completed or timed out. + */ + public static final TypedEvent CLOSE_CONTAINER = + new TypedEvent<>(ContainerID.class, "Close_Container"); + + /** + * Private Ctor. Never Constructed. + */ + private SCMEvents() { + } + +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/package-info.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/package-info.java new file mode 100644 index 00000000000..46181a3eb5f --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + * + */ +/** + * Events Package contains all the Events used by SCM internally to + * communicate between different sub-systems that make up SCM. + */ +package org.apache.hadoop.hdds.scm.events; \ No newline at end of file diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index 15ac3f283e9..664a80f31d5 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -25,10 +25,8 @@ import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.scm.VersionInfo; import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric; import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat; -import org.apache.hadoop.hdds.server.events.Event; import org.apache.hadoop.hdds.server.events.EventHandler; import org.apache.hadoop.hdds.server.events.EventPublisher; -import org.apache.hadoop.hdds.server.events.TypedEvent; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; @@ -118,8 +116,7 @@ public class SCMNodeManager // Node pool manager. private final StorageContainerManager scmManager; - public static final Event DATANODE_COMMAND = - new TypedEvent<>(CommandForDatanode.class, "DATANODE_COMMAND"); + /** * Constructs SCM machine Manager. diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java index f221584bddf..a6354affaae 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java @@ -25,12 +25,14 @@ import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto; import org.apache.hadoop.hdds.server.events.EventPublisher; -import org.apache.hadoop.hdds.server.events.TypedEvent; import com.google.protobuf.GeneratedMessage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.hdds.scm.events.SCMEvents.CONTAINER_REPORT; +import static org.apache.hadoop.hdds.scm.events.SCMEvents.NODE_REPORT; + /** * This class is responsible for dispatching heartbeat from datanode to * appropriate EventHandler at SCM. @@ -42,11 +44,6 @@ public final class SCMDatanodeHeartbeatDispatcher { private EventPublisher eventPublisher; - public static final TypedEvent NODE_REPORT = - new TypedEvent<>(NodeReportFromDatanode.class); - - public static final TypedEvent CONTAINER_REPORT = - new TypedEvent(ContainerReportFromDatanode.class); public SCMDatanodeHeartbeatDispatcher(EventPublisher eventPublisher) { this.eventPublisher = eventPublisher; @@ -63,12 +60,14 @@ public final class SCMDatanodeHeartbeatDispatcher { DatanodeDetails.getFromProtoBuf(heartbeat.getDatanodeDetails()); // should we dispatch heartbeat through eventPublisher? if (heartbeat.hasNodeReport()) { + LOG.debug("Dispatching Node Report."); eventPublisher.fireEvent(NODE_REPORT, new NodeReportFromDatanode(datanodeDetails, heartbeat.getNodeReport())); } if (heartbeat.hasContainerReport()) { + LOG.debug("Dispatching Container Report."); eventPublisher.fireEvent(CONTAINER_REPORT, new ContainerReportFromDatanode(datanodeDetails, heartbeat.getContainerReport())); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index 568a86ab4fc..49d3a405428 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -70,6 +70,8 @@ import java.util.concurrent.TimeUnit; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_DEFAULT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_MB; + +import static org.apache.hadoop.hdds.scm.events.SCMEvents.DATANODE_COMMAND; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED; import static org.apache.hadoop.util.ExitUtil.terminate; @@ -164,9 +166,10 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl } EventQueue eventQueue = new EventQueue(); - SCMNodeManager nm = new SCMNodeManager(conf, scmStorage.getClusterID(), this); + SCMNodeManager nm = + new SCMNodeManager(conf, scmStorage.getClusterID(), this); scmNodeManager = nm; - eventQueue.addHandler(SCMNodeManager.DATANODE_COMMAND, nm); + eventQueue.addHandler(DATANODE_COMMAND, nm); scmContainerManager = new ContainerMapping(conf, getScmNodeManager(), cacheSize); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java index 721dbf6b582..0d46ffae56c 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java @@ -17,11 +17,13 @@ package org.apache.hadoop.hdds.scm.container; +import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdds.scm.container.common.helpers + .ContainerWithPipeline; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.container.common.SCMTestUtils; @@ -33,12 +35,12 @@ import org.junit.Test; import java.io.File; import java.io.IOException; -import java.util.Random; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATE; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATED; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB; +import static org.apache.hadoop.hdds.scm.events.SCMEvents.CLOSE_CONTAINER; /** * Tests the closeContainerEventHandler class. @@ -65,7 +67,7 @@ public class TestCloseContainerEventHandler { nodeManager = new MockNodeManager(true, 10); mapping = new ContainerMapping(configuration, nodeManager, 128); eventQueue = new EventQueue(); - eventQueue.addHandler(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, + eventQueue.addHandler(CLOSE_CONTAINER, new CloseContainerEventHandler(mapping)); } @@ -81,8 +83,8 @@ public class TestCloseContainerEventHandler { public void testIfCloseContainerEventHadnlerInvoked() { GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer .captureLogs(CloseContainerEventHandler.LOG); - eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, - new ContainerID(Math.abs(new Random().nextLong()))); + eventQueue.fireEvent(CLOSE_CONTAINER, + new ContainerID(Math.abs(RandomUtils.nextInt()))); eventQueue.processAll(1000); Assert.assertTrue(logCapturer.getOutput() .contains("Close container Event triggered for container")); @@ -90,10 +92,10 @@ public class TestCloseContainerEventHandler { @Test public void testCloseContainerEventWithInvalidContainer() { - long id = Math.abs(new Random().nextLong()); + long id = Math.abs(RandomUtils.nextInt()); GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer .captureLogs(CloseContainerEventHandler.LOG); - eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, + eventQueue.fireEvent(CLOSE_CONTAINER, new ContainerID(id)); eventQueue.processAll(1000); Assert.assertTrue(logCapturer.getOutput() @@ -112,7 +114,7 @@ public class TestCloseContainerEventHandler { containerWithPipeline.getContainerInfo().getContainerID()); DatanodeDetails datanode = containerWithPipeline.getPipeline().getLeader(); int closeCount = nodeManager.getCommandCount(datanode); - eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, id); + eventQueue.fireEvent(CLOSE_CONTAINER, id); eventQueue.processAll(1000); // At this point of time, the allocated container is not in open // state, so firing close container event should not queue CLOSE @@ -125,11 +127,12 @@ public class TestCloseContainerEventHandler { //Execute these state transitions so that we can close the container. mapping.updateContainerState(id.getId(), CREATE); mapping.updateContainerState(id.getId(), CREATED); - eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, + eventQueue.fireEvent(CLOSE_CONTAINER, new ContainerID( containerWithPipeline.getContainerInfo().getContainerID())); eventQueue.processAll(1000); - Assert.assertEquals(closeCount + 1, nodeManager.getCommandCount(datanode)); + Assert.assertEquals(closeCount + 1, + nodeManager.getCommandCount(datanode)); Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING, mapping.getStateManager().getContainer(id).getState()); } @@ -145,7 +148,7 @@ public class TestCloseContainerEventHandler { ContainerID id = new ContainerID( containerWithPipeline.getContainerInfo().getContainerID()); int[] closeCount = new int[3]; - eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, id); + eventQueue.fireEvent(CLOSE_CONTAINER, id); eventQueue.processAll(1000); int i = 0; for (DatanodeDetails details : containerWithPipeline.getPipeline() @@ -166,7 +169,7 @@ public class TestCloseContainerEventHandler { //Execute these state transitions so that we can close the container. mapping.updateContainerState(id.getId(), CREATE); mapping.updateContainerState(id.getId(), CREATED); - eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, id); + eventQueue.fireEvent(CLOSE_CONTAINER, id); eventQueue.processAll(1000); i = 0; // Make sure close is queued for each datanode on the pipeline diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java index 0a4e33d444c..d72309e9792 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java @@ -68,6 +68,7 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState .HEALTHY; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE; +import static org.apache.hadoop.hdds.scm.events.SCMEvents.DATANODE_COMMAND; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.core.StringStartsWith.startsWith; import static org.junit.Assert.assertEquals; @@ -1068,11 +1069,6 @@ public class TestNodeManager { foundRemaining = nodeManager.getStats().getRemaining().get(); assertEquals(0, foundRemaining); - // Send a new report to bring the dead node back to healthy - String storagePath = testDir.getAbsolutePath() + "/" + dnId; - List reports = TestUtils - .createStorageReport(capacity, expectedScmUsed, expectedRemaining, - storagePath, null, dnId, 1); nodeManager.processHeartbeat(datanodeDetails); // Wait up to 5 seconds so that the dead node becomes healthy @@ -1111,11 +1107,11 @@ public class TestNodeManager { EventQueue eq = new EventQueue(); try (SCMNodeManager nodemanager = createNodeManager(conf)) { - eq.addHandler(SCMNodeManager.DATANODE_COMMAND, nodemanager); + eq.addHandler(DATANODE_COMMAND, nodemanager); nodemanager .register(datanodeDetails, TestUtils.createNodeReport(reports)); - eq.fireEvent(SCMNodeManager.DATANODE_COMMAND, + eq.fireEvent(DATANODE_COMMAND, new CommandForDatanode(datanodeDetails.getUuid(), new CloseContainerCommand(1L, ReplicationType.STAND_ALONE))); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMDatanodeHeartbeatDispatcher.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMDatanodeHeartbeatDispatcher.java index 326a34b7920..a77ed0452dd 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMDatanodeHeartbeatDispatcher.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMDatanodeHeartbeatDispatcher.java @@ -20,8 +20,6 @@ package org.apache.hadoop.hdds.scm.server; import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.ContainerReportsProto; @@ -40,6 +38,9 @@ import org.apache.hadoop.hdds.server.events.EventPublisher; import org.junit.Assert; import org.junit.Test; +import static org.apache.hadoop.hdds.scm.events.SCMEvents.CONTAINER_REPORT; +import static org.apache.hadoop.hdds.scm.events.SCMEvents.NODE_REPORT; + /** * This class tests the behavior of SCMDatanodeHeartbeatDispatcher. */ @@ -49,8 +50,6 @@ public class TestSCMDatanodeHeartbeatDispatcher { @Test public void testNodeReportDispatcher() throws IOException { - Configuration conf = new OzoneConfiguration(); - AtomicInteger eventReceived = new AtomicInteger(); NodeReportProto nodeReport = NodeReportProto.getDefaultInstance(); @@ -60,10 +59,10 @@ public class TestSCMDatanodeHeartbeatDispatcher { @Override public > void fireEvent( EVENT_TYPE event, PAYLOAD payload) { - Assert.assertEquals(event, - SCMDatanodeHeartbeatDispatcher.NODE_REPORT); + Assert.assertEquals(event, NODE_REPORT); eventReceived.incrementAndGet(); - Assert.assertEquals(nodeReport, ((NodeReportFromDatanode)payload).getReport()); + Assert.assertEquals(nodeReport, + ((NodeReportFromDatanode)payload).getReport()); } }); @@ -84,7 +83,6 @@ public class TestSCMDatanodeHeartbeatDispatcher { @Test public void testContainerReportDispatcher() throws IOException { - Configuration conf = new OzoneConfiguration(); AtomicInteger eventReceived = new AtomicInteger(); @@ -96,9 +94,9 @@ public class TestSCMDatanodeHeartbeatDispatcher { @Override public > void fireEvent( EVENT_TYPE event, PAYLOAD payload) { - Assert.assertEquals(event, - SCMDatanodeHeartbeatDispatcher.CONTAINER_REPORT); - Assert.assertEquals(containerReport, ((ContainerReportFromDatanode)payload).getReport()); + Assert.assertEquals(event, CONTAINER_REPORT); + Assert.assertEquals(containerReport, + ((ContainerReportFromDatanode)payload).getReport()); eventReceived.incrementAndGet(); } }); From 790c563511161c901b7b667e787baca8725f9249 Mon Sep 17 00:00:00 2001 From: Anu Engineer Date: Sun, 8 Jul 2018 11:27:54 -0700 Subject: [PATCH 42/47] HDDS-235. Fix TestOzoneAuditLogger#verifyDefaultLogLevel. Contributed by Xiaoyu Yao. --- .../ozone/audit/TestOzoneAuditLogger.java | 46 +++++++++++++------ 1 file changed, 31 insertions(+), 15 deletions(-) diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java index d3cc9e4ce8a..57a7d9e1fc5 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java @@ -100,7 +100,7 @@ public class TestOzoneAuditLogger { AUDIT.logReadFailure(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.ERROR); AUDIT.logReadFailure(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.ERROR, new Exception("test")); - verifyLog(null); + verifyNoLog(); } /** @@ -110,22 +110,38 @@ public class TestOzoneAuditLogger { public void notLogDebugEvents() throws IOException { AUDIT.logWriteSuccess(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap(), Level.DEBUG); AUDIT.logReadSuccess(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.DEBUG); - verifyLog(null); + verifyNoLog(); } - public void verifyLog(String expected) throws IOException { - File file = new File("audit.log"); - List lines = FileUtils.readLines(file, (String)null); - if(expected == null){ - // When no log entry is expected, the log file must be empty - assertTrue(lines.size() == 0); - } else { - // When log entry is expected, the log file will contain one line and - // that must be equal to the expected string - assertTrue(expected.equalsIgnoreCase(lines.get(0))); - //empty the file - lines.remove(0); - FileUtils.writeLines(file, lines, false); + private void verifyLog(String expected) throws IOException { + File file = new File("audit.log"); + List lines = FileUtils.readLines(file, (String)null); + final int retry = 5; + int i = 0; + while (lines.isEmpty() && i < retry) { + lines = FileUtils.readLines(file, (String)null); + try { + Thread.sleep( 500 * (i + 1)); + } catch(InterruptedException ie) { + Thread.currentThread().interrupt(); + break; + } + i++; } + + // When log entry is expected, the log file will contain one line and + // that must be equal to the expected string + assertTrue(lines.size() != 0); + assertTrue(expected.equalsIgnoreCase(lines.get(0))); + //empty the file + lines.remove(0); + FileUtils.writeLines(file, lines, false); + } + + private void verifyNoLog() throws IOException { + File file = new File("audit.log"); + List lines = FileUtils.readLines(file, (String)null); + // When no log entry is expected, the log file must be empty + assertTrue(lines.size() == 0); } } From 7a68ac607c52c8a28dcd75a367ae77331787a3b4 Mon Sep 17 00:00:00 2001 From: Yiqun Lin Date: Mon, 9 Jul 2018 14:27:34 +0800 Subject: [PATCH 43/47] HDFS-13674. Improve documentation on Metrics. Contributed by Chao Sun. --- .../src/site/markdown/Metrics.md | 39 ++++++++++++------- 1 file changed, 24 insertions(+), 15 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md index 676ab0b508e..2c7bd4d0c94 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md @@ -159,14 +159,17 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a | `TransactionsAvgTime` | Average time of Journal transactions in milliseconds | | `SyncsNumOps` | Total number of Journal syncs | | `SyncsAvgTime` | Average time of Journal syncs in milliseconds | +| `SyncsTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of Journal sync time in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `TransactionsBatchedInSync` | Total number of Journal transactions batched in sync | +| `TransactionsBatchedInSync`*num*`s(50/75/90/95/99)thPercentileCount` | The 50/75/90/95/99th percentile of number of batched Journal transactions (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `StorageBlockReportNumOps` | Total number of processing block reports from individual storages in DataNode | | `StorageBlockReportAvgTime` | Average time of processing block reports in milliseconds | +| `StorageBlockReport`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of block report processing time in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `CacheReportNumOps` | Total number of processing cache reports from DataNode | | `CacheReportAvgTime` | Average time of processing cache reports in milliseconds | +| `CacheReport`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of cached report processing time in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `SafeModeTime` | The interval between FSNameSystem starts and the last time safemode leaves in milliseconds.  (sometimes not equal to the time in SafeMode, see [HDFS-5156](https://issues.apache.org/jira/browse/HDFS-5156)) | | `FsImageLoadTime` | Time loading FS Image at startup in milliseconds | -| `FsImageLoadTime` | Time loading FS Image at startup in milliseconds | | `GetEditNumOps` | Total number of edits downloads from SecondaryNameNode | | `GetEditAvgTime` | Average edits download time in milliseconds | | `GetImageNumOps` | Total number of fsimage downloads from SecondaryNameNode | @@ -177,22 +180,23 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a | `NNStartedTimeInMillis`| NameNode start time in milliseconds | | `GenerateEDEKTimeNumOps` | Total number of generating EDEK | | `GenerateEDEKTimeAvgTime` | Average time of generating EDEK in milliseconds | +| `GenerateEDEKTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in generating EDEK in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `WarmUpEDEKTimeNumOps` | Total number of warming up EDEK | | `WarmUpEDEKTimeAvgTime` | Average time of warming up EDEK in milliseconds | -| `ResourceCheckTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of NameNode resource check latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | -| `StorageBlockReport`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of storage block report latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `WarmUpEDEKTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in warming up EDEK in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `ResourceCheckTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of of NameNode resource check latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `EditLogTailTimeNumOps` | Total number of times the standby NameNode tailed the edit log | | `EditLogTailTimeAvgTime` | Average time (in milliseconds) spent by standby NameNode in tailing edit log | -| `EditLogTailTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in tailing edit logs by standby NameNode, in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `EditLogTailTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in tailing edit logs by standby NameNode in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `EditLogFetchTimeNumOps` | Total number of times the standby NameNode fetched remote edit streams from journal nodes | | `EditLogFetchTimeAvgTime` | Average time (in milliseconds) spent by standby NameNode in fetching remote edit streams from journal nodes | -| `EditLogFetchTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in fetching edit streams from journal nodes by standby NameNode, in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `EditLogFetchTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in fetching edit streams from journal nodes by standby NameNode in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `NumEditLogLoadedNumOps` | Total number of times edits were loaded by standby NameNode | | `NumEditLogLoadedAvgCount` | Average number of edits loaded by standby NameNode in each edit log tailing | -| `NumEditLogLoaded`*num*`s(50/75/90/95/99)thPercentileCount` | The 50/75/90/95/99th percentile of number of edits loaded by standby NameNode in each edit log tailing. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `NumEditLogLoaded`*num*`s(50/75/90/95/99)thPercentileCount` | The 50/75/90/95/99th percentile of number of edits loaded by standby NameNode in each edit log tailing (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `EditLogTailIntervalNumOps` | Total number of intervals between edit log tailings by standby NameNode | | `EditLogTailIntervalAvgTime` | Average time of intervals between edit log tailings by standby NameNode in milliseconds | -| `EditLogTailInterval`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time between edit log tailings by standby NameNode, in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `EditLogTailInterval`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time between edit log tailings by standby NameNode in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | FSNamesystem ------------ @@ -338,13 +342,13 @@ Each metrics record contains tags such as SessionId and Hostname as additional i | `RamDiskBlocksEvictedWithoutRead` | Total number of blocks evicted in memory without ever being read from memory | | `RamDiskBlocksEvictionWindowMsNumOps` | Number of blocks evicted in memory| | `RamDiskBlocksEvictionWindowMsAvgTime` | Average time of blocks in memory before being evicted in milliseconds | -| `RamDiskBlocksEvictionWindows`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and eviction in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `RamDiskBlocksEvictionWindows`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and eviction in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `RamDiskBlocksLazyPersisted` | Total number of blocks written to disk by lazy writer | | `RamDiskBlocksDeletedBeforeLazyPersisted` | Total number of blocks deleted by application before being persisted to disk | | `RamDiskBytesLazyPersisted` | Total number of bytes written to disk by lazy writer | | `RamDiskBlocksLazyPersistWindowMsNumOps` | Number of blocks written to disk by lazy writer | | `RamDiskBlocksLazyPersistWindowMsAvgTime` | Average time of blocks written to disk by lazy writer in milliseconds | -| `RamDiskBlocksLazyPersistWindows`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and disk persist in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `RamDiskBlocksLazyPersistWindows`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and disk persist in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `FsyncCount` | Total number of fsync | | `VolumeFailures` | Total number of volume failures occurred | | `ReadBlockOpNumOps` | Total number of read operations | @@ -371,14 +375,19 @@ Each metrics record contains tags such as SessionId and Hostname as additional i | `CacheReportsAvgTime` | Average time of cache report operations in milliseconds | | `PacketAckRoundTripTimeNanosNumOps` | Total number of ack round trip | | `PacketAckRoundTripTimeNanosAvgTime` | Average time from ack send to receive minus the downstream ack time in nanoseconds | +| `PacketAckRoundTripTimeNanos`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile latency from ack send to receive minus the downstream ack time in nanoseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `FlushNanosNumOps` | Total number of flushes | | `FlushNanosAvgTime` | Average flush time in nanoseconds | +| `FlushNanos`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile flush time in nanoseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `FsyncNanosNumOps` | Total number of fsync | | `FsyncNanosAvgTime` | Average fsync time in nanoseconds | +| `FsyncNanos`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile fsync time in nanoseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `SendDataPacketBlockedOnNetworkNanosNumOps` | Total number of sending packets | | `SendDataPacketBlockedOnNetworkNanosAvgTime` | Average waiting time of sending packets in nanoseconds | +| `SendDataPacketBlockedOnNetworkNanos`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile waiting time of sending packets in nanoseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `SendDataPacketTransferNanosNumOps` | Total number of sending packets | | `SendDataPacketTransferNanosAvgTime` | Average transfer time of sending packets in nanoseconds | +| `SendDataPacketTransferNanos`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile transfer time of sending packets in nanoseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `TotalWriteTime`| Total number of milliseconds spent on write operation | | `TotalReadTime` | Total number of milliseconds spent on read operation | | `RemoteBytesRead` | Number of bytes read by remote clients | @@ -410,23 +419,23 @@ contains tags such as Hostname as additional information along with metrics. | `TotalMetadataOperations` | Total number (monotonically increasing) of metadata operations. Metadata operations include stat, list, mkdir, delete, move, open and posix_fadvise. | | `MetadataOperationRateNumOps` | The number of metadata operations within an interval time of metric | | `MetadataOperationRateAvgTime` | Mean time of metadata operations in milliseconds | -| `MetadataOperationLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of metadata operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `MetadataOperationLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of metadata operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `TotalDataFileIos` | Total number (monotonically increasing) of data file io operations | | `DataFileIoRateNumOps` | The number of data file io operations within an interval time of metric | | `DataFileIoRateAvgTime` | Mean time of data file io operations in milliseconds | -| `DataFileIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of data file io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `DataFileIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of data file io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `FlushIoRateNumOps` | The number of file flush io operations within an interval time of metric | | `FlushIoRateAvgTime` | Mean time of file flush io operations in milliseconds | -| `FlushIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file flush io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `FlushIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file flush io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `SyncIoRateNumOps` | The number of file sync io operations within an interval time of metric | | `SyncIoRateAvgTime` | Mean time of file sync io operations in milliseconds | -| `SyncIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file sync io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `SyncIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file sync io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `ReadIoRateNumOps` | The number of file read io operations within an interval time of metric | | `ReadIoRateAvgTime` | Mean time of file read io operations in milliseconds | -| `ReadIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file read io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `ReadIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file read io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `WriteIoRateNumOps` | The number of file write io operations within an interval time of metric | | `WriteIoRateAvgTime` | Mean time of file write io operations in milliseconds | -| `WriteIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file write io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | +| `WriteIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file write io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. | | `TotalFileIoErrors` | Total number (monotonically increasing) of file io error operations | | `FileIoErrorRateNumOps` | The number of file io error operations within an interval time of metric | | `FileIoErrorRateAvgTime` | It measures the mean time in milliseconds from the start of an operation to hitting a failure | From 43f7fe8aae0eca89cce4d67bfc4965fe8ce63e38 Mon Sep 17 00:00:00 2001 From: Yiqun Lin Date: Mon, 9 Jul 2018 15:06:07 +0800 Subject: [PATCH 44/47] HDFS-13710. RBF: setQuota and getQuotaUsage should check the dfs.federation.router.quota.enable. Contributed by yanghuafeng. --- .../hdfs/server/federation/router/Quota.java | 8 ++ .../federation/router/RouterRpcServer.java | 1 - .../router/TestDisableRouterQuota.java | 94 +++++++++++++++++++ 3 files changed, 102 insertions(+), 1 deletion(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestDisableRouterQuota.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java index 413a4e1d099..75d3e0498dd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java @@ -67,6 +67,9 @@ public class Quota { public void setQuota(String path, long namespaceQuota, long storagespaceQuota, StorageType type) throws IOException { rpcServer.checkOperation(OperationCategory.WRITE); + if (!router.isQuotaEnabled()) { + throw new IOException("The quota system is disabled in Router."); + } // Set quota for current path and its children mount table path. final List locations = getQuotaRemoteLocations(path); @@ -91,6 +94,11 @@ public class Quota { * @throws IOException */ public QuotaUsage getQuotaUsage(String path) throws IOException { + rpcServer.checkOperation(OperationCategory.READ); + if (!router.isQuotaEnabled()) { + throw new IOException("The quota system is disabled in Router."); + } + final List quotaLocs = getValidQuotaLocations(path); RemoteMethod method = new RemoteMethod("getQuotaUsage", new Class[] {String.class}, new RemoteParam()); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java index 716ebee9bc4..7031af71fa8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java @@ -1996,7 +1996,6 @@ public class RouterRpcServer extends AbstractService @Override // ClientProtocol public QuotaUsage getQuotaUsage(String path) throws IOException { - checkOperation(OperationCategory.READ); return this.quotaCall.getQuotaUsage(path); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestDisableRouterQuota.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestDisableRouterQuota.java new file mode 100644 index 00000000000..2632f5936fb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestDisableRouterQuota.java @@ -0,0 +1,94 @@ +/** + * 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.hdfs.server.federation.router; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; + +import java.io.IOException; + +import org.apache.hadoop.test.GenericTestUtils; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.fail; + +/** + * Test the behavior when disabling the Router quota. + */ +public class TestDisableRouterQuota { + + private static Router router; + + @BeforeClass + public static void setUp() throws Exception { + // Build and start a router + router = new Router(); + Configuration routerConf = new RouterConfigBuilder() + .quota(false) //set false to verify the quota disabled in Router + .rpc() + .build(); + router.init(routerConf); + router.setRouterId("TestRouterId"); + router.start(); + } + + @AfterClass + public static void tearDown() throws IOException { + if (router != null) { + router.stop(); + router.close(); + } + } + + @Before + public void checkDisableQuota() { + assertFalse(router.isQuotaEnabled()); + } + + @Test + public void testSetQuota() throws Exception { + long nsQuota = 1024; + long ssQuota = 1024; + + try { + Quota quotaModule = router.getRpcServer().getQuotaModule(); + quotaModule.setQuota("/test", nsQuota, ssQuota, null); + fail("The setQuota call should fail."); + } catch (IOException ioe) { + GenericTestUtils.assertExceptionContains( + "The quota system is disabled in Router.", ioe); + } + } + + @Test + public void testGetQuotaUsage() throws Exception { + try { + Quota quotaModule = router.getRpcServer().getQuotaModule(); + quotaModule.getQuotaUsage("/test"); + fail("The getQuotaUsage call should fail."); + } catch (IOException ioe) { + GenericTestUtils.assertExceptionContains( + "The quota system is disabled in Router.", ioe); + } + } + +} From eecb5baaaaa54599aeae758abd4007e55e5b531f Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Mon, 9 Jul 2018 15:17:21 +0200 Subject: [PATCH 45/47] HDFS-13719. Docs around dfs.image.transfer.timeout are misleading. Contributed by Kitti Nansi. --- .../hadoop-hdfs/src/main/resources/hdfs-default.xml | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 6dd2d927962..384cedf5107 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -1289,11 +1289,10 @@ dfs.image.transfer.timeout 60000 - Socket timeout for image transfer in milliseconds. This timeout and the related - dfs.image.transfer.bandwidthPerSec parameter should be configured such - that normal image transfer can complete successfully. - This timeout prevents client hangs when the sender fails during - image transfer. This is socket timeout during image transfer. + Socket timeout for the HttpURLConnection instance used in the image + transfer. This is measured in milliseconds. + This timeout prevents client hangs if the connection is idle + for this configured timeout, during image transfer. @@ -1304,9 +1303,7 @@ Maximum bandwidth used for regular image transfers (instead of bootstrapping the standby namenode), in bytes per second. This can help keep normal namenode operations responsive during - checkpointing. The maximum bandwidth and timeout in - dfs.image.transfer.timeout should be set such that normal image - transfers can complete successfully. + checkpointing. A default value of 0 indicates that throttling is disabled. The maximum bandwidth used for bootstrapping standby namenode is configured with dfs.image.transfer-bootstrap-standby.bandwidthPerSec. From 0838fe833738e04f5e6f6408e97866d77bebbf30 Mon Sep 17 00:00:00 2001 From: Robert Kanter Date: Mon, 9 Jul 2018 10:37:20 -0700 Subject: [PATCH 46/47] Only mount non-empty directories for cgroups (miklos.szegedi@cloudera.com via rkanter) --- .../impl/container-executor.c | 30 ++++++++++++++++++- .../test/test-container-executor.c | 20 +++++++++++++ 2 files changed, 49 insertions(+), 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c index baf0e8b4703..effeeeece3b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c @@ -2379,6 +2379,28 @@ void chown_dir_contents(const char *dir_path, uid_t uid, gid_t gid) { free(path_tmp); } +int is_empty(char *target_dir) { + DIR *dir = NULL; + struct dirent *entry = NULL; + dir = opendir(target_dir); + if (!dir) { + fprintf(LOGFILE, "Could not open directory %s - %s\n", target_dir, + strerror(errno)); + return 0; + } + while ((entry = readdir(dir)) != NULL) { + if (strcmp(entry->d_name, ".") == 0) { + continue; + } + if (strcmp(entry->d_name, "..") == 0) { + continue; + } + fprintf(LOGFILE, "Directory is not empty %s\n", target_dir); + return 0; + } + return 1; +} + /** * Mount a cgroup controller at the requested mount point and create * a hierarchy for the Hadoop NodeManager to manage. @@ -2413,7 +2435,13 @@ int mount_cgroup(const char *pair, const char *hierarchy) { result = -1; } else { if (strstr(mount_path, "..") != NULL) { - fprintf(LOGFILE, "Unsupported cgroup mount path detected.\n"); + fprintf(LOGFILE, "Unsupported cgroup mount path detected. %s\n", + mount_path); + result = INVALID_COMMAND_PROVIDED; + goto cleanup; + } + if (!is_empty(mount_path)) { + fprintf(LOGFILE, "cgroup mount path is not empty. %s\n", mount_path); result = INVALID_COMMAND_PROVIDED; goto cleanup; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c index 3d328833fe6..a199d84ece7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c @@ -1203,6 +1203,23 @@ void test_trim_function() { free(trimmed); } +void test_is_empty() { + printf("\nTesting is_empty function\n"); + if (is_empty("/")) { + printf("FAIL: / should not be empty\n"); + exit(1); + } + if (is_empty("/tmp/2938rf2983hcqnw8ud/noexist")) { + printf("FAIL: /tmp/2938rf2983hcqnw8ud/noexist should not exist\n"); + exit(1); + } + mkdir("/tmp/2938rf2983hcqnw8ud/emptydir", S_IRWXU); + if (!is_empty("/tmp/2938rf2983hcqnw8ud/emptydir")) { + printf("FAIL: /tmp/2938rf2983hcqnw8ud/emptydir be empty\n"); + exit(1); + } +} + // This test is expected to be executed either by a regular // user or by root. If executed by a regular user it doesn't // test all the functions that would depend on changing the @@ -1264,6 +1281,9 @@ int main(int argc, char **argv) { printf("\nStarting tests\n"); + printf("\ntest_is_empty()\n"); + test_is_empty(); + printf("\nTesting recursive_unlink_children()\n"); test_recursive_unlink_children(); From 83cd84b70bac7b613eb4b2901d5ffe40098692eb Mon Sep 17 00:00:00 2001 From: Wangda Tan Date: Mon, 9 Jul 2018 11:30:08 -0700 Subject: [PATCH 47/47] YARN-8506. Make GetApplicationsRequestPBImpl thread safe. (wangda) Change-Id: If304567abb77a01b686d82c769bdf50728484163 --- .../impl/pb/GetApplicationsRequestPBImpl.java | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsRequestPBImpl.java index a6abb99b439..4c5fee0e7fd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsRequestPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationsRequestPBImpl.java @@ -65,7 +65,7 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { viaProto = true; } - public GetApplicationsRequestProto getProto() { + public synchronized GetApplicationsRequestProto getProto() { mergeLocalToProto(); proto = viaProto ? proto : builder.build(); viaProto = true; @@ -175,13 +175,13 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public Set getApplicationTypes() { + public synchronized Set getApplicationTypes() { initApplicationTypes(); return this.applicationTypes; } @Override - public void setApplicationTypes(Set applicationTypes) { + public synchronized void setApplicationTypes(Set applicationTypes) { maybeInitBuilder(); if (applicationTypes == null) builder.clearApplicationTypes(); @@ -198,13 +198,13 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public Set getApplicationTags() { + public synchronized Set getApplicationTags() { initApplicationTags(); return this.applicationTags; } @Override - public void setApplicationTags(Set tags) { + public synchronized void setApplicationTags(Set tags) { maybeInitBuilder(); if (tags == null || tags.isEmpty()) { builder.clearApplicationTags(); @@ -219,7 +219,7 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public EnumSet getApplicationStates() { + public synchronized EnumSet getApplicationStates() { initApplicationStates(); return this.applicationStates; } @@ -233,12 +233,12 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public ApplicationsRequestScope getScope() { + public synchronized ApplicationsRequestScope getScope() { initScope(); return this.scope; } - public void setScope(ApplicationsRequestScope scope) { + public synchronized void setScope(ApplicationsRequestScope scope) { maybeInitBuilder(); if (scope == null) { builder.clearScope(); @@ -247,7 +247,7 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public void setApplicationStates(EnumSet applicationStates) { + public synchronized void setApplicationStates(EnumSet applicationStates) { maybeInitBuilder(); if (applicationStates == null) { builder.clearApplicationStates(); @@ -256,7 +256,7 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public void setApplicationStates(Set applicationStates) { + public synchronized void setApplicationStates(Set applicationStates) { EnumSet appStates = null; for (YarnApplicationState state : YarnApplicationState.values()) { if (applicationStates.contains( @@ -272,12 +272,12 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public Set getUsers() { + public synchronized Set getUsers() { initUsers(); return this.users; } - public void setUsers(Set users) { + public synchronized void setUsers(Set users) { maybeInitBuilder(); if (users == null) { builder.clearUsers(); @@ -286,13 +286,13 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public Set getQueues() { + public synchronized Set getQueues() { initQueues(); return this.queues; } @Override - public void setQueues(Set queues) { + public synchronized void setQueues(Set queues) { maybeInitBuilder(); if (queues == null) { builder.clearQueues(); @@ -301,7 +301,7 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public long getLimit() { + public synchronized long getLimit() { if (this.limit == Long.MAX_VALUE) { GetApplicationsRequestProtoOrBuilder p = viaProto ? proto : builder; this.limit = p.hasLimit() ? p.getLimit() : Long.MAX_VALUE; @@ -310,13 +310,13 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public void setLimit(long limit) { + public synchronized void setLimit(long limit) { maybeInitBuilder(); this.limit = limit; } @Override - public Range getStartRange() { + public synchronized Range getStartRange() { if (this.start == null) { GetApplicationsRequestProtoOrBuilder p = viaProto ? proto: builder; if (p.hasStartBegin() || p.hasStartEnd()) { @@ -329,12 +329,12 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public void setStartRange(Range range) { + public synchronized void setStartRange(Range range) { this.start = range; } @Override - public void setStartRange(long begin, long end) + public synchronized void setStartRange(long begin, long end) throws IllegalArgumentException { if (begin > end) { throw new IllegalArgumentException("begin > end in range (begin, " + @@ -344,7 +344,7 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public Range getFinishRange() { + public synchronized Range getFinishRange() { if (this.finish == null) { GetApplicationsRequestProtoOrBuilder p = viaProto ? proto: builder; if (p.hasFinishBegin() || p.hasFinishEnd()) { @@ -357,12 +357,12 @@ public class GetApplicationsRequestPBImpl extends GetApplicationsRequest { } @Override - public void setFinishRange(Range range) { + public synchronized void setFinishRange(Range range) { this.finish = range; } @Override - public void setFinishRange(long begin, long end) { + public synchronized void setFinishRange(long begin, long end) { if (begin > end) { throw new IllegalArgumentException("begin > end in range (begin, " + "end): (" + begin + ", " + end + ")");