From 3503461f3bb4a762d6c35a0f71ee78018aeec2d3 Mon Sep 17 00:00:00 2001 From: Simbarashe Dzinamarira Date: Mon, 28 Nov 2022 16:47:01 -0800 Subject: [PATCH 01/45] HDFS-16847: RBF: Prevents StateStoreFileSystemImpl from committing tmp file after encountering an IOException. (#5145) --- .../driver/impl/StateStoreFileBaseImpl.java | 18 +++++--------- .../store/driver/impl/StateStoreFileImpl.java | 4 +++- .../driver/impl/StateStoreFileSystemImpl.java | 19 +++++---------- .../driver/TestStateStoreDriverBase.java | 19 +++++++++++++++ .../driver/TestStateStoreFileSystem.java | 24 +++++++++++++++++++ 5 files changed, 58 insertions(+), 26 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java index 871919594f57d..c93d919aea0a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java @@ -85,7 +85,8 @@ protected abstract BufferedReader getReader( * @param path Path of the record to write. * @return Writer for the record. */ - protected abstract BufferedWriter getWriter( + @VisibleForTesting + public abstract BufferedWriter getWriter( String path); /** @@ -348,25 +349,18 @@ public boolean putAll( for (Entry entry : toWrite.entrySet()) { String recordPath = entry.getKey(); String recordPathTemp = recordPath + "." + now() + TMP_MARK; - BufferedWriter writer = getWriter(recordPathTemp); - try { + boolean recordWrittenSuccessfully = true; + try (BufferedWriter writer = getWriter(recordPathTemp)) { T record = entry.getValue(); String line = serializeString(record); writer.write(line); } catch (IOException e) { LOG.error("Cannot write {}", recordPathTemp, e); + recordWrittenSuccessfully = false; success = false; - } finally { - if (writer != null) { - try { - writer.close(); - } catch (IOException e) { - LOG.error("Cannot close the writer for {}", recordPathTemp, e); - } - } } // Commit - if (!rename(recordPathTemp, recordPath)) { + if (recordWrittenSuccessfully && !rename(recordPathTemp, recordPath)) { LOG.error("Failed committing record into {}", recordPath); success = false; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java index 9d2b1ab2fb73a..6ca2663716162 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java @@ -31,6 +31,7 @@ import java.util.List; import org.apache.commons.lang3.ArrayUtils; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys; import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord; import org.slf4j.Logger; @@ -125,7 +126,8 @@ protected BufferedReader getReader(String filename) { } @Override - protected BufferedWriter getWriter(String filename) { + @VisibleForTesting + public BufferedWriter getWriter(String filename) { BufferedWriter writer = null; try { LOG.debug("Writing file: {}", filename); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java index e6bf159e2f597..ee34d8a4cabbb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java @@ -28,13 +28,14 @@ import java.util.Collections; import java.util.List; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys; import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver; import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord; @@ -82,17 +83,8 @@ protected boolean mkdir(String path) { @Override protected boolean rename(String src, String dst) { try { - if (fs instanceof DistributedFileSystem) { - DistributedFileSystem dfs = (DistributedFileSystem)fs; - dfs.rename(new Path(src), new Path(dst), Options.Rename.OVERWRITE); - return true; - } else { - // Replace should be atomic but not available - if (fs.exists(new Path(dst))) { - fs.delete(new Path(dst), true); - } - return fs.rename(new Path(src), new Path(dst)); - } + FileUtil.rename(fs, new Path(src), new Path(dst), Options.Rename.OVERWRITE); + return true; } catch (Exception e) { LOG.error("Cannot rename {} to {}", src, dst, e); return false; @@ -148,7 +140,8 @@ protected BufferedReader getReader(String pathName) { } @Override - protected BufferedWriter getWriter(String pathName) { + @VisibleForTesting + public BufferedWriter getWriter(String pathName) { BufferedWriter writer = null; Path path = new Path(pathName); try { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java index b8bb7c4d2d115..5ad01dce8e729 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java @@ -234,6 +234,25 @@ public void testInsert( assertEquals(11, records2.size()); } + public void testInsertWithErrorDuringWrite( + StateStoreDriver driver, Class recordClass) + throws IllegalArgumentException, IllegalAccessException, IOException { + + assertTrue(driver.removeAll(recordClass)); + QueryResult queryResult0 = driver.get(recordClass); + List records0 = queryResult0.getRecords(); + assertTrue(records0.isEmpty()); + + // Insert single + BaseRecord record = generateFakeRecord(recordClass); + driver.put(record, true, false); + + // Verify that no record was inserted. + QueryResult queryResult1 = driver.get(recordClass); + List records1 = queryResult1.getRecords(); + assertEquals(0, records1.size()); + } + public void testFetchErrors(StateStoreDriver driver, Class clazz) throws IllegalAccessException, IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.java index 8c4b188cc47e3..dbd4b9bdae2ea 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.java @@ -17,16 +17,26 @@ */ package org.apache.hadoop.hdfs.server.federation.store.driver; +import java.io.BufferedWriter; import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils; +import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileBaseImpl; import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileSystemImpl; +import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.mockito.stubbing.Answer; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; + /** * Test the FileSystem (e.g., HDFS) implementation of the State Store driver. @@ -91,4 +101,18 @@ public void testMetrics() throws IllegalArgumentException, IllegalAccessException, IOException { testMetrics(getStateStoreDriver()); } + + @Test + public void testInsertWithErrorDuringWrite() + throws IllegalArgumentException, IllegalAccessException, IOException { + StateStoreFileBaseImpl driver = spy((StateStoreFileBaseImpl)getStateStoreDriver()); + doAnswer((Answer) a -> { + BufferedWriter writer = (BufferedWriter) a.callRealMethod(); + BufferedWriter spyWriter = spy(writer); + doThrow(IOException.class).when(spyWriter).write(any(String.class)); + return spyWriter; + }).when(driver).getWriter(any()); + + testInsertWithErrorDuringWrite(driver, MembershipState.class); + } } \ No newline at end of file From e263d731d21b172993efa29ac6e87f55931321d1 Mon Sep 17 00:00:00 2001 From: Simbarashe Dzinamarira Date: Mon, 28 Nov 2022 16:49:10 -0800 Subject: [PATCH 02/45] HDFS-16845: Adds configuration flag to allow clients to use router observer reads without using the ObserverReadProxyProvider. (#5142) --- .../hadoop/hdfs/NameNodeProxiesClient.java | 6 ++ .../hdfs/client/HdfsClientConfigKeys.java | 2 + .../federation/MiniRouterDFSCluster.java | 7 +- .../router/TestObserverWithRouter.java | 74 ++++++++++++++++++- .../src/main/resources/hdfs-default.xml | 7 ++ 5 files changed, 91 insertions(+), 5 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java index aa9577330cfae..2e5532381978d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java @@ -349,6 +349,12 @@ public static ClientProtocol createProxyWithAlignmentContext( boolean withRetries, AtomicBoolean fallbackToSimpleAuth, AlignmentContext alignmentContext) throws IOException { + if (alignmentContext == null && + conf.getBoolean(HdfsClientConfigKeys.DFS_RBF_OBSERVER_READ_ENABLE, + HdfsClientConfigKeys.DFS_RBF_OBSERVER_READ_ENABLE_DEFAULT)) { + alignmentContext = new ClientGSIContext(); + } + RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class, ProtobufRpcEngine2.class); 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 e3e01fde3a51c..2b511bfc2ebee 100755 --- 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 @@ -78,6 +78,8 @@ public interface HdfsClientConfigKeys { int DFS_NAMENODE_HTTPS_PORT_DEFAULT = 9871; String DFS_NAMENODE_HTTPS_ADDRESS_KEY = "dfs.namenode.https-address"; String DFS_HA_NAMENODES_KEY_PREFIX = "dfs.ha.namenodes"; + String DFS_RBF_OBSERVER_READ_ENABLE = "dfs.client.rbf.observer.read.enable"; + boolean DFS_RBF_OBSERVER_READ_ENABLE_DEFAULT = false; int DFS_NAMENODE_RPC_PORT_DEFAULT = 8020; String DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY = "dfs.namenode.kerberos.principal"; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java index bdf4697d2aa92..2c70395870496 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java @@ -234,7 +234,12 @@ public FileSystem getFileSystem() throws IOException { return DistributedFileSystem.get(conf); } - public FileSystem getFileSystemWithObserverReadsEnabled() throws IOException { + public FileSystem getFileSystem(Configuration configuration) throws IOException { + configuration.addResource(conf); + return DistributedFileSystem.get(configuration); + } + + public FileSystem getFileSystemWithObserverReadProxyProvider() throws IOException { Configuration observerReadConf = new Configuration(conf); observerReadConf.set(DFS_NAMESERVICES, observerReadConf.get(DFS_NAMESERVICES)+ ",router-service"); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java index 23095186d0133..e38b0b2a35af4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java @@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster; import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext; import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; @@ -122,11 +123,17 @@ public void startUpCluster(int numberOfObserver, Configuration confOverrides) th cluster.waitActiveNamespaces(); routerContext = cluster.getRandomRouter(); - fileSystem = routerContext.getFileSystemWithObserverReadsEnabled(); + } + + private static Configuration getConfToEnableObserverReads() { + Configuration conf = new Configuration(); + conf.setBoolean(HdfsClientConfigKeys.DFS_RBF_OBSERVER_READ_ENABLE, true); + return conf; } @Test public void testObserverRead() throws Exception { + fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads()); internalTestObserverRead(); } @@ -137,7 +144,6 @@ public void testObserverRead() throws Exception { */ @Test public void testReadWithoutObserverClientConfigurations() throws Exception { - fileSystem.close(); fileSystem = routerContext.getFileSystem(); assertThrows(AssertionError.class, this::internalTestObserverRead); } @@ -173,6 +179,7 @@ public void testObserverReadWithoutFederatedStatePropagation() throws Exception Configuration confOverrides = new Configuration(false); confOverrides.setInt(RBFConfigKeys.DFS_ROUTER_OBSERVER_FEDERATED_STATE_PROPAGATION_MAXSIZE, 0); startUpCluster(2, confOverrides); + fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads()); List namenodes = routerContext .getRouter().getNamenodeResolver() .getNamenodesForNameserviceId(cluster.getNameservices().get(0), true); @@ -202,6 +209,7 @@ public void testDisablingObserverReadUsingNameserviceOverride() throws Exception Configuration confOverrides = new Configuration(false); confOverrides.set(RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_OVERRIDES, "ns0"); startUpCluster(2, confOverrides); + fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads()); Path path = new Path("/testFile"); fileSystem.create(path).close(); @@ -219,6 +227,7 @@ public void testDisablingObserverReadUsingNameserviceOverride() throws Exception @Test public void testReadWhenObserverIsDown() throws Exception { + fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads()); Path path = new Path("/testFile1"); // Send Create call to active fileSystem.create(path).close(); @@ -246,6 +255,7 @@ public void testReadWhenObserverIsDown() throws Exception { @Test public void testMultipleObserver() throws Exception { + fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads()); Path path = new Path("/testFile1"); // Send Create call to active fileSystem.create(path).close(); @@ -384,6 +394,7 @@ public void testMultipleObserverRouter() throws Exception { @Test public void testUnavailableObserverNN() throws Exception { + fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads()); stopObserver(2); Path path = new Path("/testFile"); @@ -417,10 +428,9 @@ public void testUnavailableObserverNN() throws Exception { assertTrue("There must be unavailable namenodes", hasUnavailable); } - - @Test public void testRouterMsync() throws Exception { + fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads()); Path path = new Path("/testFile"); // Send Create call to active @@ -439,4 +449,60 @@ public void testRouterMsync() throws Exception { assertEquals("Four calls should be sent to active", 4, rpcCountForActive); } + + @Test + public void testSingleRead() throws Exception { + fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads()); + List namenodes = routerContext + .getRouter().getNamenodeResolver() + .getNamenodesForNameserviceId(cluster.getNameservices().get(0), true); + assertEquals("First namenode should be observer", namenodes.get(0).getState(), + FederationNamenodeServiceState.OBSERVER); + Path path = new Path("/"); + + long rpcCountForActive; + long rpcCountForObserver; + + // Send read request + fileSystem.listFiles(path, false); + fileSystem.close(); + + rpcCountForActive = routerContext.getRouter().getRpcServer() + .getRPCMetrics().getActiveProxyOps(); + // getListingCall sent to active. + assertEquals("Only one call should be sent to active", 1, rpcCountForActive); + + rpcCountForObserver = routerContext.getRouter().getRpcServer() + .getRPCMetrics().getObserverProxyOps(); + // getList call should be sent to observer + assertEquals("No calls should be sent to observer", 0, rpcCountForObserver); + } + + @Test + public void testSingleReadUsingObserverReadProxyProvider() throws Exception { + fileSystem = routerContext.getFileSystemWithObserverReadProxyProvider(); + List namenodes = routerContext + .getRouter().getNamenodeResolver() + .getNamenodesForNameserviceId(cluster.getNameservices().get(0), true); + assertEquals("First namenode should be observer", namenodes.get(0).getState(), + FederationNamenodeServiceState.OBSERVER); + Path path = new Path("/"); + + long rpcCountForActive; + long rpcCountForObserver; + + // Send read request + fileSystem.listFiles(path, false); + fileSystem.close(); + + rpcCountForActive = routerContext.getRouter().getRpcServer() + .getRPCMetrics().getActiveProxyOps(); + // Two msync calls to the active namenodes. + assertEquals("Two calls should be sent to active", 2, rpcCountForActive); + + rpcCountForObserver = routerContext.getRouter().getRpcServer() + .getRPCMetrics().getObserverProxyOps(); + // getList call should be sent to observer + assertEquals("One call should be sent to observer", 1, rpcCountForObserver); + } } \ No newline at end of file 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 e4cb5b9ffe16d..2a0a4945faa6e 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -6442,4 +6442,11 @@ If the namespace is DEFAULT, it's best to change this conf to other value. + + dfs.client.rbf.observer.read.enable + false + + Enables observer reads for clients. This should only be enabled when clients are using routers. + + From d53f699de2cc167bcc07422820dd614a9caec860 Mon Sep 17 00:00:00 2001 From: caozhiqiang Date: Tue, 29 Nov 2022 09:51:21 +0800 Subject: [PATCH 03/45] HDFS-16846. EC: Only EC blocks should be effected by max-streams-hard-limit configuration (#5143) Signed-off-by: Takanobu Asanuma --- .../blockmanagement/DatanodeDescriptor.java | 47 ++++++++++++--- .../blockmanagement/DatanodeManager.java | 43 +++++++++----- .../blockmanagement/ErasureCodingWork.java | 2 +- .../hdfs/TestDecommissionWithStriped.java | 6 +- .../blockmanagement/TestDatanodeManager.java | 59 ++++++++++++------- 5 files changed, 107 insertions(+), 50 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java index a2b7afedfdd31..c77d54591a9f7 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java @@ -197,8 +197,10 @@ public Type getType() { /** A queue of blocks to be replicated by this datanode */ private final BlockQueue replicateBlocks = new BlockQueue<>(); - /** A queue of blocks to be erasure coded by this datanode */ - private final BlockQueue erasurecodeBlocks = + /** A queue of ec blocks to be replicated by this datanode. */ + private final BlockQueue ecBlocksToBeReplicated = new BlockQueue<>(); + /** A queue of ec blocks to be erasure coded by this datanode. */ + private final BlockQueue ecBlocksToBeErasureCoded = new BlockQueue<>(); /** A queue of blocks to be recovered by this datanode */ private final BlockQueue recoverBlocks = new BlockQueue<>(); @@ -358,7 +360,8 @@ public void clearBlockQueues() { } this.recoverBlocks.clear(); this.replicateBlocks.clear(); - this.erasurecodeBlocks.clear(); + this.ecBlocksToBeReplicated.clear(); + this.ecBlocksToBeErasureCoded.clear(); // pendingCached, cached, and pendingUncached are protected by the // FSN lock. this.pendingCached.clear(); @@ -678,6 +681,15 @@ public void addBlockToBeReplicated(Block block, replicateBlocks.offer(new BlockTargetPair(block, targets)); } + /** + * Store ec block to be replicated work. + */ + @VisibleForTesting + public void addECBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) { + assert (block != null && targets != null && targets.length > 0); + ecBlocksToBeReplicated.offer(new BlockTargetPair(block, targets)); + } + /** * Store block erasure coding work. */ @@ -687,9 +699,9 @@ void addBlockToBeErasureCoded(ExtendedBlock block, assert (block != null && sources != null && sources.length > 0); BlockECReconstructionInfo task = new BlockECReconstructionInfo(block, sources, targets, liveBlockIndices, excludeReconstrutedIndices, ecPolicy); - erasurecodeBlocks.offer(task); + ecBlocksToBeErasureCoded.offer(task); BlockManager.LOG.debug("Adding block reconstruction task " + task + "to " - + getName() + ", current queue size is " + erasurecodeBlocks.size()); + + getName() + ", current queue size is " + ecBlocksToBeErasureCoded.size()); } /** @@ -720,7 +732,8 @@ void addBlocksToBeInvalidated(List blocklist) { * The number of work items that are pending to be replicated. */ int getNumberOfBlocksToBeReplicated() { - return pendingReplicationWithoutTargets + replicateBlocks.size(); + return pendingReplicationWithoutTargets + replicateBlocks.size() + + ecBlocksToBeReplicated.size(); } /** @@ -728,7 +741,15 @@ int getNumberOfBlocksToBeReplicated() { */ @VisibleForTesting public int getNumberOfBlocksToBeErasureCoded() { - return erasurecodeBlocks.size(); + return ecBlocksToBeErasureCoded.size(); + } + + /** + * The number of ec work items that are pending to be replicated. + */ + @VisibleForTesting + public int getNumberOfECBlocksToBeReplicated() { + return ecBlocksToBeReplicated.size(); } @VisibleForTesting @@ -740,9 +761,13 @@ List getReplicationCommand(int maxTransfers) { return replicateBlocks.poll(maxTransfers); } + List getECReplicatedCommand(int maxTransfers) { + return ecBlocksToBeReplicated.poll(maxTransfers); + } + public List getErasureCodeCommand( int maxTransfers) { - return erasurecodeBlocks.poll(maxTransfers); + return ecBlocksToBeErasureCoded.poll(maxTransfers); } public BlockInfo[] getLeaseRecoveryCommand(int maxTransfers) { @@ -994,7 +1019,11 @@ public String dumpDatanode() { if (repl > 0) { sb.append(" ").append(repl).append(" blocks to be replicated;"); } - int ec = erasurecodeBlocks.size(); + int ecRepl = ecBlocksToBeReplicated.size(); + if (ecRepl > 0) { + sb.append(" ").append(ecRepl).append(" ec blocks to be replicated;"); + } + int ec = ecBlocksToBeErasureCoded.size(); if(ec > 0) { sb.append(" ").append(ec).append(" blocks to be erasure coded;"); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index b2c5cb0b557f2..88f3ac4e7c4cb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -1825,28 +1825,41 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, // Allocate _approximately_ maxTransfers pending tasks to DataNode. // NN chooses pending tasks based on the ratio between the lengths of // replication and erasure-coded block queues. - int totalReplicateBlocks = nodeinfo.getNumberOfReplicateBlocks(); - int totalECBlocks = nodeinfo.getNumberOfBlocksToBeErasureCoded(); - int totalBlocks = totalReplicateBlocks + totalECBlocks; + int replicationBlocks = nodeinfo.getNumberOfReplicateBlocks(); + int ecBlocksToBeReplicated = nodeinfo.getNumberOfECBlocksToBeReplicated(); + int ecBlocksToBeErasureCoded = nodeinfo.getNumberOfBlocksToBeErasureCoded(); + int totalBlocks = replicationBlocks + ecBlocksToBeReplicated + ecBlocksToBeErasureCoded; if (totalBlocks > 0) { - int maxTransfers; + int maxTransfers = blockManager.getMaxReplicationStreams() - xmitsInProgress; + int maxECReplicatedTransfers; if (nodeinfo.isDecommissionInProgress()) { - maxTransfers = blockManager.getReplicationStreamsHardLimit() + maxECReplicatedTransfers = blockManager.getReplicationStreamsHardLimit() - xmitsInProgress; } else { - maxTransfers = blockManager.getMaxReplicationStreams() - - xmitsInProgress; + maxECReplicatedTransfers = maxTransfers; } int numReplicationTasks = (int) Math.ceil( - (double) (totalReplicateBlocks * maxTransfers) / totalBlocks); - int numECTasks = (int) Math.ceil( - (double) (totalECBlocks * maxTransfers) / totalBlocks); - LOG.debug("Pending replication tasks: {} erasure-coded tasks: {}.", - numReplicationTasks, numECTasks); + (double) (replicationBlocks * maxTransfers) / totalBlocks); + int numEcReplicatedTasks = (int) Math.ceil( + (double) (ecBlocksToBeReplicated * maxECReplicatedTransfers) / totalBlocks); + int numECReconstructedTasks = (int) Math.ceil( + (double) (ecBlocksToBeErasureCoded * maxTransfers) / totalBlocks); + LOG.debug("Pending replication tasks: {} ec to be replicated tasks: {} " + + "ec reconstruction tasks: {}.", + numReplicationTasks, numEcReplicatedTasks, numECReconstructedTasks); // check pending replication tasks - List pendingList = nodeinfo.getReplicationCommand( + List pendingReplicationList = nodeinfo.getReplicationCommand( numReplicationTasks); - if (pendingList != null && !pendingList.isEmpty()) { + List pendingECReplicatedList = nodeinfo.getECReplicatedCommand( + numEcReplicatedTasks); + List pendingList = new ArrayList(); + if(pendingReplicationList != null && !pendingReplicationList.isEmpty()) { + pendingList.addAll(pendingReplicationList); + } + if(pendingECReplicatedList != null && !pendingECReplicatedList.isEmpty()) { + pendingList.addAll(pendingECReplicatedList); + } + if (!pendingList.isEmpty()) { // If the block is deleted, the block size will become // BlockCommand.NO_ACK (LONG.MAX_VALUE) . This kind of block we don't // need @@ -1868,7 +1881,7 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, } // check pending erasure coding tasks List pendingECList = nodeinfo - .getErasureCodeCommand(numECTasks); + .getErasureCodeCommand(numECReconstructedTasks); if (pendingECList != null && !pendingECList.isEmpty()) { cmds.add(new BlockECReconstructionCommand( DNA_ERASURE_CODING_RECONSTRUCTION, pendingECList)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java index e5303a28d714e..147f4c3fd624a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java @@ -164,7 +164,7 @@ private void createReplicationWork(int sourceIndex, stripedBlk.getDataBlockNum(), blockIndex); final Block targetBlk = new Block(stripedBlk.getBlockId() + blockIndex, internBlkLen, stripedBlk.getGenerationStamp()); - source.addBlockToBeReplicated(targetBlk, + source.addECBlockToBeReplicated(targetBlk, new DatanodeStorageInfo[] {target}); LOG.debug("Add replication task from source {} to " + "target {} for EC block {}", source, target, targetBlk); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommissionWithStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommissionWithStriped.java index c68cb1707c2fe..206f75eae7035 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommissionWithStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommissionWithStriped.java @@ -759,7 +759,7 @@ public void testDecommissionWithFailedReplicating() throws Exception { DatanodeInfo extraDn = getDatanodeOutOfTheBlock(blk); DatanodeDescriptor target = bm.getDatanodeManager() .getDatanode(extraDn.getDatanodeUuid()); - dn0.addBlockToBeReplicated(targetBlk, + dn0.addECBlockToBeReplicated(targetBlk, new DatanodeStorageInfo[] {target.getStorageInfos()[0]}); // dn0 replicates in success @@ -883,7 +883,7 @@ public void testDecommissionWithMissingBlock() throws Exception { .getDatanode(extraDn.getDatanodeUuid()); DatanodeDescriptor dnStartIndexDecommission = bm.getDatanodeManager() .getDatanode(dnLocs[decommNodeIndex].getDatanodeUuid()); - dnStartIndexDecommission.addBlockToBeReplicated(targetBlk, + dnStartIndexDecommission.addECBlockToBeReplicated(targetBlk, new DatanodeStorageInfo[] {target.getStorageInfos()[0]}); // Wait for replication success. @@ -972,7 +972,7 @@ public void testCountNodes() throws Exception{ DatanodeInfo extraDn = getDatanodeOutOfTheBlock(blk); DatanodeDescriptor target = bm.getDatanodeManager() .getDatanode(extraDn.getDatanodeUuid()); - dn0.addBlockToBeReplicated(targetBlk, + dn0.addECBlockToBeReplicated(targetBlk, new DatanodeStorageInfo[] {target.getStorageInfos()[0]}); // dn0 replicates in success diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java index 35ff36a856ba8..015a0385a735d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java @@ -967,20 +967,22 @@ public void testRemoveIncludedNode() throws IOException { * Verify the correctness of pending recovery process. * * @param numReplicationBlocks the number of replication blocks in the queue. - * @param numECBlocks number of EC blocks in the queue. + * @param numEcBlocksToBeReplicated the number of EC blocks to be replicated in the queue. + * @param numBlocksToBeErasureCoded number of EC blocks to be erasure coded in the queue. * @param maxTransfers the maxTransfer value. * @param maxTransfersHardLimit the maxTransfer hard limit value. - * @param numReplicationTasks the number of replication tasks polled from - * the queue. - * @param numECTasks the number of EC tasks polled from the queue. + * @param numReplicationTasks the number of replication tasks polled from the queue. + * @param numECTasksToBeReplicated the number of EC tasks to be replicated polled from the queue. + * @param numECTasksToBeErasureCoded the number of EC tasks to be erasure coded polled from + * the queue. * @param isDecommissioning if the node is in the decommissioning process. * * @throws IOException */ private void verifyPendingRecoveryTasks( - int numReplicationBlocks, int numECBlocks, - int maxTransfers, int maxTransfersHardLimit, - int numReplicationTasks, int numECTasks, boolean isDecommissioning) + int numReplicationBlocks, int numEcBlocksToBeReplicated, int numBlocksToBeErasureCoded, + int maxTransfers, int maxTransfersHardLimit, int numReplicationTasks, + int numECTasksToBeReplicated, int numECTasksToBeErasureCoded, boolean isDecommissioning) throws IOException { FSNamesystem fsn = Mockito.mock(FSNamesystem.class); Mockito.when(fsn.hasWriteLock()).thenReturn(true); @@ -1009,13 +1011,25 @@ private void verifyPendingRecoveryTasks( .thenReturn(tasks); } - if (numECBlocks > 0) { + if (numEcBlocksToBeReplicated > 0) { + Mockito.when(nodeInfo.getNumberOfECBlocksToBeReplicated()) + .thenReturn(numEcBlocksToBeReplicated); + + List ecReplicatedTasks = + Collections.nCopies( + Math.min(numECTasksToBeReplicated, numEcBlocksToBeReplicated), + new BlockTargetPair(null, null)); + Mockito.when(nodeInfo.getECReplicatedCommand(numECTasksToBeReplicated)) + .thenReturn(ecReplicatedTasks); + } + + if (numBlocksToBeErasureCoded > 0) { Mockito.when(nodeInfo.getNumberOfBlocksToBeErasureCoded()) - .thenReturn(numECBlocks); + .thenReturn(numBlocksToBeErasureCoded); List tasks = - Collections.nCopies(numECTasks, null); - Mockito.when(nodeInfo.getErasureCodeCommand(numECTasks)) + Collections.nCopies(numECTasksToBeErasureCoded, null); + Mockito.when(nodeInfo.getErasureCodeCommand(numECTasksToBeErasureCoded)) .thenReturn(tasks); } @@ -1026,42 +1040,43 @@ private void verifyPendingRecoveryTasks( SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT); long expectedNumCmds = Arrays.stream( - new int[]{numReplicationTasks, numECTasks}) + new int[]{numReplicationTasks + numECTasksToBeReplicated, numECTasksToBeErasureCoded}) .filter(x -> x > 0) .count(); assertEquals(expectedNumCmds, cmds.length); int idx = 0; - if (numReplicationTasks > 0) { + if (numReplicationTasks > 0 || numECTasksToBeReplicated > 0) { assertTrue(cmds[idx] instanceof BlockCommand); BlockCommand cmd = (BlockCommand) cmds[0]; - assertEquals(numReplicationTasks, cmd.getBlocks().length); - assertEquals(numReplicationTasks, cmd.getTargets().length); + assertEquals(numReplicationTasks + numECTasksToBeReplicated, cmd.getBlocks().length); + assertEquals(numReplicationTasks + numECTasksToBeReplicated, cmd.getTargets().length); idx++; } - if (numECTasks > 0) { + if (numECTasksToBeErasureCoded > 0) { assertTrue(cmds[idx] instanceof BlockECReconstructionCommand); BlockECReconstructionCommand cmd = (BlockECReconstructionCommand) cmds[idx]; - assertEquals(numECTasks, cmd.getECTasks().size()); + assertEquals(numECTasksToBeErasureCoded, cmd.getECTasks().size()); } Mockito.verify(nodeInfo).getReplicationCommand(numReplicationTasks); - Mockito.verify(nodeInfo).getErasureCodeCommand(numECTasks); + Mockito.verify(nodeInfo).getECReplicatedCommand(numECTasksToBeReplicated); + Mockito.verify(nodeInfo).getErasureCodeCommand(numECTasksToBeErasureCoded); } @Test public void testPendingRecoveryTasks() throws IOException { // Tasks are slitted according to the ratio between queue lengths. - verifyPendingRecoveryTasks(20, 20, 20, 30, 10, 10, false); - verifyPendingRecoveryTasks(40, 10, 20, 30, 16, 4, false); + verifyPendingRecoveryTasks(20, 0, 20, 20, 30, 10, 0, 10, false); + verifyPendingRecoveryTasks(40, 0, 10, 20, 30, 16, 0, 4, false); // Approximately load tasks if the ratio between queue length is large. - verifyPendingRecoveryTasks(400, 1, 20, 30, 20, 1, false); + verifyPendingRecoveryTasks(400, 0, 1, 20, 30, 20, 0, 1, false); // Tasks use dfs.namenode.replication.max-streams-hard-limit for decommissioning node - verifyPendingRecoveryTasks(30, 30, 20, 30, 15, 15, true); + verifyPendingRecoveryTasks(20, 10, 10, 20, 40, 10, 10, 5, true); } @Test From 12d42fdbaf47ce66465985a67b720c3866e9d717 Mon Sep 17 00:00:00 2001 From: HarshitGupta11 <50410275+HarshitGupta11@users.noreply.github.com> Date: Tue, 29 Nov 2022 20:21:22 +0530 Subject: [PATCH 04/45] HADOOP-18530. ChecksumFileSystem::readVectored might return byte buffers not positioned at 0 (#5168) Contributed by Harshit Gupta --- .../main/java/org/apache/hadoop/fs/VectoredReadUtils.java | 7 +++++++ .../java/org/apache/hadoop/fs/TestVectoredReadUtils.java | 6 ++++++ 2 files changed, 13 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java index 50cab7dc4ccf8..cf1b1ef969863 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java @@ -307,9 +307,16 @@ public static ByteBuffer sliceTo(ByteBuffer readData, long readOffset, FileRange request) { int offsetChange = (int) (request.getOffset() - readOffset); int requestLength = request.getLength(); + // Create a new buffer that is backed by the original contents + // The buffer will have position 0 and the same limit as the original one readData = readData.slice(); + // Change the offset and the limit of the buffer as the reader wants to see + // only relevant data readData.position(offsetChange); readData.limit(offsetChange + requestLength); + // Create a new buffer after the limit change so that only that portion of the data is + // returned to the reader. + readData = readData.slice(); return readData; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java index fdfa8f6eb6f62..e964d23f4b750 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java @@ -61,6 +61,9 @@ public void testSliceTo() { .describedAs("Slicing on the same offset shouldn't " + "create a new buffer") .isEqualTo(slice); + Assertions.assertThat(slice.position()) + .describedAs("Slicing should return buffers starting from position 0") + .isEqualTo(0); // try slicing a range final int offset = 100; @@ -77,6 +80,9 @@ public void testSliceTo() { .describedAs("Slicing should use the same underlying " + "data") .isEqualTo(slice.array()); + Assertions.assertThat(slice.position()) + .describedAs("Slicing should return buffers starting from position 0") + .isEqualTo(0); // test the contents of the slice intBuffer = slice.asIntBuffer(); for(int i=0; i < sliceLength / Integer.BYTES; ++i) { From a1f28d58411677e5277c11020cf691976c9655bc Mon Sep 17 00:00:00 2001 From: Owen O'Malley Date: Tue, 29 Nov 2022 22:12:35 +0000 Subject: [PATCH 05/45] HDFS-16851: RBF: Add a utility to dump the StateStore. (#5155) --- .../federation/store/StateStoreService.java | 9 +++ .../hdfs/tools/federation/RouterAdmin.java | 74 ++++++++++++++++++- .../src/site/markdown/HDFSRouterFederation.md | 11 +++ .../federation/router/TestRouterAdminCLI.java | 73 +++++++++++++++++- .../store/records/MockStateStoreDriver.java | 19 +++-- .../store/records/TestRouterState.java | 1 + 6 files changed, 177 insertions(+), 10 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java index 201c7a325f105..77939799e72b4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java @@ -272,6 +272,15 @@ public > T getRegisteredRecordStore( return null; } + /** + * Get the list of all RecordStores. + * @return a list of each RecordStore. + */ + @SuppressWarnings("unchecked") + public > List getRecordStores() { + return new ArrayList<>((Collection) recordStores.values()); + } + /** * List of records supported by this State Store. * diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java index d7fcf862fb6e9..3ecb4c2caba70 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.tools.federation; import java.io.IOException; +import java.io.PrintStream; import java.net.InetSocketAddress; import java.util.Arrays; import java.util.Collection; @@ -26,6 +27,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.TreeMap; import java.util.regex.Pattern; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -46,6 +48,10 @@ import org.apache.hadoop.hdfs.server.federation.router.RouterClient; import org.apache.hadoop.hdfs.server.federation.router.RouterQuotaUsage; import org.apache.hadoop.hdfs.server.federation.router.RouterStateManager; +import org.apache.hadoop.hdfs.server.federation.store.CachedRecordStore; +import org.apache.hadoop.hdfs.server.federation.store.RecordStore; +import org.apache.hadoop.hdfs.server.federation.store.StateStoreService; +import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils; import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest; import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse; import org.apache.hadoop.hdfs.server.federation.store.protocol.DisableNameserviceRequest; @@ -70,7 +76,9 @@ import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse; import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest; import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse; +import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord; import org.apache.hadoop.hdfs.server.federation.store.records.MountTable; +import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord; import org.apache.hadoop.ipc.ProtobufRpcEngine2; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RefreshResponse; @@ -97,6 +105,7 @@ public class RouterAdmin extends Configured implements Tool { private static final Logger LOG = LoggerFactory.getLogger(RouterAdmin.class); + private static final String DUMP_COMMAND = "-dumpState"; private RouterClient client; @@ -133,7 +142,7 @@ private String getUsage(String cmd) { String[] commands = {"-add", "-update", "-rm", "-ls", "-getDestination", "-setQuota", "-setStorageTypeQuota", "-clrQuota", "-clrStorageTypeQuota", - "-safemode", "-nameservice", "-getDisabledNameservices", + DUMP_COMMAND, "-safemode", "-nameservice", "-getDisabledNameservices", "-refresh", "-refreshRouterArgs", "-refreshSuperUserGroupsConfiguration", "-refreshCallQueue"}; StringBuilder usage = new StringBuilder(); @@ -187,6 +196,8 @@ private String getUsage(String cmd) { return "\t[-refreshSuperUserGroupsConfiguration]"; } else if (cmd.equals("-refreshCallQueue")) { return "\t[-refreshCallQueue]"; + } else if (cmd.equals(DUMP_COMMAND)) { + return "\t[" + DUMP_COMMAND + "]"; } return getUsage(null); } @@ -224,7 +235,8 @@ private void validateMax(String[] arg) { if (arg.length > 1) { throw new IllegalArgumentException("No arguments allowed"); } - } else if (arg[0].equals("-refreshCallQueue")) { + } else if (arg[0].equals("-refreshCallQueue") || + arg[0].equals(DUMP_COMMAND)) { if (arg.length > 1) { throw new IllegalArgumentException("No arguments allowed"); } @@ -286,6 +298,15 @@ private boolean validateMin(String[] argv) { return true; } + /** + * Does this command run in the local process? + * @param cmd the string of the command + * @return is this a local command? + */ + boolean isLocalCommand(String cmd) { + return DUMP_COMMAND.equals(cmd); + } + @Override public int run(String[] argv) throws Exception { if (argv.length < 1) { @@ -303,6 +324,10 @@ public int run(String[] argv) throws Exception { System.err.println("Not enough parameters specificed for cmd " + cmd); printUsage(cmd); return exitCode; + } else if (isLocalCommand(argv[0])) { + if (DUMP_COMMAND.equals(argv[0])) { + return dumpStateStore(getConf(), System.out) ? 0 : -1; + } } String address = null; // Initialize RouterClient @@ -1301,6 +1326,49 @@ private int refreshCallQueue() throws IOException { return returnCode; } + /** + * Dumps the contents of the StateStore to stdout. + * @return true if it was successful + */ + public static boolean dumpStateStore(Configuration conf, + PrintStream output) throws IOException { + StateStoreService service = new StateStoreService(); + conf.setBoolean(RBFConfigKeys.DFS_ROUTER_METRICS_ENABLE, false); + service.init(conf); + service.loadDriver(); + if (!service.isDriverReady()) { + System.err.println("Can't initialize driver"); + return false; + } + // Get the stores sorted by name + Map> stores = new TreeMap<>(); + for(RecordStore store: service.getRecordStores()) { + String recordName = StateStoreUtils.getRecordName(store.getRecordClass()); + stores.put(recordName, store); + } + for (Entry> pair: stores.entrySet()) { + String recordName = pair.getKey(); + RecordStore store = pair.getValue(); + output.println("---- " + recordName + " ----"); + if (store instanceof CachedRecordStore) { + for (Object record: ((CachedRecordStore) store).getCachedRecords()) { + if (record instanceof BaseRecord && record instanceof PBRecord) { + BaseRecord baseRecord = (BaseRecord) record; + // Generate the pseudo-json format of the protobuf record + String recordString = ((PBRecord) record).getProto().toString(); + // Indent each line + recordString = " " + recordString.replaceAll("\n", "\n "); + output.println(String.format(" %s:", baseRecord.getPrimaryKey())); + output.println(recordString); + } + } + output.println(); + } + } + service.stop(); + return true; + } + /** * Normalize a path for that filesystem. * @@ -1341,4 +1409,4 @@ public FsPermission getMode() { return mode; } } -} \ No newline at end of file +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/site/markdown/HDFSRouterFederation.md b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/site/markdown/HDFSRouterFederation.md index 5a9c2fd42855c..098c73a3b71dd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/site/markdown/HDFSRouterFederation.md +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/site/markdown/HDFSRouterFederation.md @@ -328,6 +328,17 @@ To trigger a runtime-refresh of the resource specified by \ on \ [arg1..argn] +### Router state dump + +To diagnose the current state of the routers, you can use the +dumpState command. It generates a text dump of the records in the +State Store. Since it uses the configuration to find and read the +state store, it is often easiest to use the machine where the routers +run. The command runs locally, so the routers do not have to be up to +use this command. + + [hdfs]$ $HADOOP_HOME/bin/hdfs dfsrouteradmin -dumpState + Client configuration -------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java index 677f3b5e947e3..761fad2fb7ae4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -42,16 +42,20 @@ import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster; import org.apache.hadoop.hdfs.server.federation.metrics.RBFMetrics; import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState; import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager; import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver; import org.apache.hadoop.hdfs.server.federation.resolver.MultipleDestinationMountTableResolver; import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder; import org.apache.hadoop.hdfs.server.federation.store.StateStoreService; +import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver; import org.apache.hadoop.hdfs.server.federation.store.impl.DisabledNameserviceStoreImpl; import org.apache.hadoop.hdfs.server.federation.store.impl.MountTableStoreImpl; import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest; import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse; +import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState; +import org.apache.hadoop.hdfs.server.federation.store.records.MockStateStoreDriver; import org.apache.hadoop.hdfs.server.federation.store.records.MountTable; import org.apache.hadoop.hdfs.tools.federation.RouterAdmin; import org.apache.hadoop.security.UserGroupInformation; @@ -852,6 +856,7 @@ public void testInvalidArgumentMessage() throws Exception { + " ]\n" + "\t[-clrQuota ]\n" + "\t[-clrStorageTypeQuota ]\n" + + "\t[-dumpState]\n" + "\t[-safemode enter | leave | get]\n" + "\t[-nameservice enable | disable ]\n" + "\t[-getDisabledNameservices]\n" @@ -1759,6 +1764,72 @@ public void testRefreshCallQueue() throws Exception { assertTrue(err.toString().contains("No arguments allowed")); } + @Test + public void testDumpState() throws Exception { + MockStateStoreDriver driver = new MockStateStoreDriver(); + driver.clearAll(); + // Add two records for block1 + driver.put(MembershipState.newInstance("routerId", "ns1", + "ns1-ha1", "cluster1", "block1", "rpc1", + "service1", "lifeline1", "https", "nn01", + FederationNamenodeServiceState.ACTIVE, false), false, false); + driver.put(MembershipState.newInstance("routerId", "ns1", + "ns1-ha2", "cluster1", "block1", "rpc2", + "service2", "lifeline2", "https", "nn02", + FederationNamenodeServiceState.STANDBY, false), false, false); + Configuration conf = new Configuration(); + conf.setClass(RBFConfigKeys.FEDERATION_STORE_DRIVER_CLASS, + MockStateStoreDriver.class, + StateStoreDriver.class); + ByteArrayOutputStream buffer = new ByteArrayOutputStream(); + try (PrintStream stream = new PrintStream(buffer)) { + RouterAdmin.dumpStateStore(conf, stream); + } + final String expected = + "---- DisabledNameservice ----\n" + + "\n" + + "---- MembershipState ----\n" + + " ns1-ha1-ns1-routerId:\n" + + " dateCreated: XXX\n" + + " dateModified: XXX\n" + + " routerId: \"routerId\"\n" + + " nameserviceId: \"ns1\"\n" + + " namenodeId: \"ns1-ha1\"\n" + + " clusterId: \"cluster1\"\n" + + " blockPoolId: \"block1\"\n" + + " webAddress: \"nn01\"\n" + + " rpcAddress: \"rpc1\"\n" + + " serviceAddress: \"service1\"\n" + + " lifelineAddress: \"lifeline1\"\n" + + " state: \"ACTIVE\"\n" + + " isSafeMode: false\n" + + " webScheme: \"https\"\n" + + " \n" + + " ns1-ha2-ns1-routerId:\n" + + " dateCreated: XXX\n" + + " dateModified: XXX\n" + + " routerId: \"routerId\"\n" + + " nameserviceId: \"ns1\"\n" + + " namenodeId: \"ns1-ha2\"\n" + + " clusterId: \"cluster1\"\n" + + " blockPoolId: \"block1\"\n" + + " webAddress: \"nn02\"\n" + + " rpcAddress: \"rpc2\"\n" + + " serviceAddress: \"service2\"\n" + + " lifelineAddress: \"lifeline2\"\n" + + " state: \"STANDBY\"\n" + + " isSafeMode: false\n" + + " webScheme: \"https\"\n" + + " \n" + + "\n" + + "---- MountTable ----\n" + + "\n" + + "---- RouterState ----"; + // Replace the time values with XXX + assertEquals(expected, + buffer.toString().trim().replaceAll("[0-9]{4,}+", "XXX")); + } + private void addMountTable(String src, String nsId, String dst) throws Exception { String[] argv = new String[] {"-add", src, nsId, dst}; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/MockStateStoreDriver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/MockStateStoreDriver.java index 57185a0a6008e..9f600cb6f3fde 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/MockStateStoreDriver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/MockStateStoreDriver.java @@ -35,7 +35,7 @@ public class MockStateStoreDriver extends StateStoreBaseImpl { private boolean giveErrors = false; private boolean initialized = false; - private final Map> valueMap = new HashMap<>(); + private static final Map> VALUE_MAP = new HashMap<>(); @Override public boolean initDriver() { @@ -56,7 +56,7 @@ public boolean isDriverReady() { @Override public void close() throws Exception { - valueMap.clear(); + VALUE_MAP.clear(); initialized = false; } @@ -82,7 +82,7 @@ private void checkErrors() throws IOException { @SuppressWarnings("unchecked") public QueryResult get(Class clazz) throws IOException { checkErrors(); - Map map = valueMap.get(StateStoreUtils.getRecordName(clazz)); + Map map = VALUE_MAP.get(StateStoreUtils.getRecordName(clazz)); List results = map != null ? new ArrayList<>((Collection) map.values()) : new ArrayList<>(); return new QueryResult<>(results, System.currentTimeMillis()); @@ -96,7 +96,7 @@ public boolean putAll(List records, checkErrors(); for (T record : records) { Map map = - valueMap.computeIfAbsent(StateStoreUtils.getRecordName(record.getClass()), + VALUE_MAP.computeIfAbsent(StateStoreUtils.getRecordName(record.getClass()), k -> new HashMap<>()); String key = record.getPrimaryKey(); BaseRecord oldRecord = map.get(key); @@ -110,10 +110,17 @@ public boolean putAll(List records, return true; } + /** + * Clear all records from the store. + */ + public void clearAll() { + VALUE_MAP.clear(); + } + @Override public boolean removeAll(Class clazz) throws IOException { checkErrors(); - return valueMap.remove(StateStoreUtils.getRecordName(clazz)) != null; + return VALUE_MAP.remove(StateStoreUtils.getRecordName(clazz)) != null; } @Override @@ -124,7 +131,7 @@ public int remove(Class clazz, checkErrors(); int result = 0; Map map = - valueMap.get(StateStoreUtils.getRecordName(clazz)); + VALUE_MAP.get(StateStoreUtils.getRecordName(clazz)); if (map != null) { for (Iterator itr = map.values().iterator(); itr.hasNext();) { BaseRecord record = itr.next(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestRouterState.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestRouterState.java index 4289999429b32..8226178fe7691 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestRouterState.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestRouterState.java @@ -101,6 +101,7 @@ public void testStateStoreResilience() throws Exception { conf.setBoolean(RBFConfigKeys.DFS_ROUTER_METRICS_ENABLE, false); service.init(conf); MockStateStoreDriver driver = (MockStateStoreDriver) service.getDriver(); + driver.clearAll(); // Add two records for block1 driver.put(MembershipState.newInstance("routerId", "ns1", "ns1-ha1", "cluster1", "block1", "rpc1", From 42a358612537069af2bcf8f8975a7afa1e8507fa Mon Sep 17 00:00:00 2001 From: Kidd5368 <57645247+Kidd53685368@users.noreply.github.com> Date: Wed, 30 Nov 2022 09:43:15 +0800 Subject: [PATCH 06/45] HDFS-16839 It should consider EC reconstruction work when we determine if a node is busy (#5128) Co-authored-by: Takanobu Asanuma Reviewed-by: Tao Li --- .../server/blockmanagement/BlockManager.java | 6 ++- .../blockmanagement/TestBlockManager.java | 52 +++++++++++++++++++ 2 files changed, 56 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index ded489308326f..9836baa004058 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -2599,7 +2599,8 @@ DatanodeDescriptor[] chooseSourceDatanodes(BlockInfo block, if (priority != LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY && (!node.isDecommissionInProgress() && !node.isEnteringMaintenance()) - && node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams) { + && node.getNumberOfBlocksToBeReplicated() + + node.getNumberOfBlocksToBeErasureCoded() >= maxReplicationStreams) { if (isStriped && (state == StoredReplicaState.LIVE || state == StoredReplicaState.DECOMMISSIONING)) { liveBusyBlockIndices.add(blockIndex); @@ -2609,7 +2610,8 @@ DatanodeDescriptor[] chooseSourceDatanodes(BlockInfo block, continue; // already reached replication limit } - if (node.getNumberOfBlocksToBeReplicated() >= replicationStreamsHardLimit) { + if (node.getNumberOfBlocksToBeReplicated() + + node.getNumberOfBlocksToBeErasureCoded() >= replicationStreamsHardLimit) { if (isStriped && (state == StoredReplicaState.LIVE || state == StoredReplicaState.DECOMMISSIONING)) { liveBusyBlockIndices.add(blockIndex); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java index 8ebcbfe2e347f..04f474f67ef90 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java @@ -957,6 +957,58 @@ public void testSkipReconstructionWithManyBusyNodes2() { assertNull(work); } + @Test + public void testSkipReconstructionWithManyBusyNodes3() { + NameNode.initMetrics(new Configuration(), HdfsServerConstants.NamenodeRole.NAMENODE); + long blockId = -9223372036854775776L; // Real ec block id + // RS-3-2 EC policy + ErasureCodingPolicy ecPolicy = + SystemErasureCodingPolicies.getPolicies().get(1); + + // Create an EC block group: 3 data blocks + 2 parity blocks. + Block aBlockGroup = new Block(blockId, ecPolicy.getCellSize() * ecPolicy.getNumDataUnits(), 0); + BlockInfoStriped aBlockInfoStriped = new BlockInfoStriped(aBlockGroup, ecPolicy); + + // Create 4 storageInfo, which means 1 block is missing. + DatanodeStorageInfo ds1 = DFSTestUtil.createDatanodeStorageInfo( + "storage1", "1.1.1.1", "rack1", "host1"); + DatanodeStorageInfo ds2 = DFSTestUtil.createDatanodeStorageInfo( + "storage2", "2.2.2.2", "rack2", "host2"); + DatanodeStorageInfo ds3 = DFSTestUtil.createDatanodeStorageInfo( + "storage3", "3.3.3.3", "rack3", "host3"); + DatanodeStorageInfo ds4 = DFSTestUtil.createDatanodeStorageInfo( + "storage4", "4.4.4.4", "rack4", "host4"); + + // Link block with storage. + aBlockInfoStriped.addStorage(ds1, aBlockGroup); + aBlockInfoStriped.addStorage(ds2, new Block(blockId + 1, 0, 0)); + aBlockInfoStriped.addStorage(ds3, new Block(blockId + 2, 0, 0)); + aBlockInfoStriped.addStorage(ds4, new Block(blockId + 3, 0, 0)); + + addEcBlockToBM(blockId, ecPolicy); + aBlockInfoStriped.setBlockCollectionId(mockINodeId); + + // Reconstruction should be scheduled. + BlockReconstructionWork work = bm.scheduleReconstruction(aBlockInfoStriped, 3); + assertNotNull(work); + + ExtendedBlock dummyBlock = new ExtendedBlock("bpid", 1, 1, 1); + DatanodeDescriptor dummyDD = ds1.getDatanodeDescriptor(); + DatanodeDescriptor[] dummyDDArray = new DatanodeDescriptor[]{dummyDD}; + DatanodeStorageInfo[] dummyDSArray = new DatanodeStorageInfo[]{ds1}; + // Simulate the 2 nodes reach maxReplicationStreams. + for(int i = 0; i < bm.maxReplicationStreams; i++){ //Add some dummy EC reconstruction task. + ds3.getDatanodeDescriptor().addBlockToBeErasureCoded(dummyBlock, dummyDDArray, + dummyDSArray, new byte[0], new byte[0], ecPolicy); + ds4.getDatanodeDescriptor().addBlockToBeErasureCoded(dummyBlock, dummyDDArray, + dummyDSArray, new byte[0], new byte[0], ecPolicy); + } + + // Reconstruction should be skipped since the number of non-busy nodes are not enough. + work = bm.scheduleReconstruction(aBlockInfoStriped, 3); + assertNull(work); + } + @Test public void testFavorDecomUntilHardLimit() throws Exception { bm.maxReplicationStreams = 0; From 513caffa4f2dd3d9a4f95832fdb4d3f9fe5e9595 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 30 Nov 2022 18:35:31 +0530 Subject: [PATCH 07/45] HADOOP-18457. ABFS: Support account level throttling (#5034) This allows abfs request throttling to be shared across all abfs connections talking to containers belonging to the same abfs storage account -as that is the level at which IO throttling is applied. The option is enabled/disabled in the configuration option "fs.azure.account.throttling.enabled"; The default is "true" Contributed by Anmol Asrani --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 24 +++ .../fs/azurebfs/AzureBlobFileSystem.java | 2 - .../azurebfs/constants/ConfigurationKeys.java | 3 + .../constants/FileSystemConfigurations.java | 3 + .../fs/azurebfs/services/AbfsClient.java | 14 ++ .../AbfsClientThrottlingAnalyzer.java | 128 +++++++----- .../AbfsClientThrottlingIntercept.java | 107 +++++++--- .../services/AbfsNoOpThrottlingIntercept.java | 37 ++++ .../services/AbfsOperationMetrics.java | 139 +++++++++++++ .../azurebfs/services/AbfsRestOperation.java | 11 +- .../services/AbfsThrottlingIntercept.java | 49 +++++ .../AbfsThrottlingInterceptFactory.java | 102 ++++++++++ .../azurebfs/services/TimerFunctionality.java | 26 +++ .../hadoop-azure/src/site/markdown/abfs.md | 9 + .../constants/TestConfigurationKeys.java | 3 + .../fs/azurebfs/services/TestAbfsClient.java | 5 + .../TestAbfsClientThrottlingAnalyzer.java | 30 ++- .../services/TestExponentialRetryPolicy.java | 192 ++++++++++++++++++ 18 files changed, 794 insertions(+), 90 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingInterceptFactory.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimerFunctionality.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index ecfeb41a34697..80f803d80dab0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -117,6 +117,10 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_OPTIMIZE_FOOTER_READ) private boolean optimizeFooterRead; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED, + DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED) + private boolean accountThrottlingEnabled; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_READ_BUFFER_SIZE, MinValue = MIN_BUFFER_SIZE, MaxValue = MAX_BUFFER_SIZE, @@ -260,6 +264,14 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_ENABLE_AUTOTHROTTLING) private boolean enableAutoThrottling; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_OPERATION_IDLE_TIMEOUT, + DefaultValue = DEFAULT_ACCOUNT_OPERATION_IDLE_TIMEOUT_MS) + private int accountOperationIdleTimeout; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ANALYSIS_PERIOD, + DefaultValue = DEFAULT_ANALYSIS_PERIOD_MS) + private int analysisPeriod; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ABFS_IO_RATE_LIMIT, MinValue = 0, DefaultValue = RATE_LIMIT_DEFAULT) @@ -694,6 +706,10 @@ public String getAppendBlobDirs() { return this.azureAppendBlobDirs; } + public boolean accountThrottlingEnabled() { + return accountThrottlingEnabled; + } + public String getAzureInfiniteLeaseDirs() { return this.azureInfiniteLeaseDirs; } @@ -736,6 +752,14 @@ public boolean isAutoThrottlingEnabled() { return this.enableAutoThrottling; } + public int getAccountOperationIdleTimeout() { + return accountOperationIdleTimeout; + } + + public int getAnalysisPeriod() { + return analysisPeriod; + } + public int getRateLimit() { return rateLimit; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index d0bdd9818db24..21501d28f4238 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -55,7 +55,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.azurebfs.commit.ResilientCommitByRename; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept; import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.classification.InterfaceStability; @@ -225,7 +224,6 @@ public void initialize(URI uri, Configuration configuration) } } - AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); rateLimiting = RateLimitingFactory.create(abfsConfiguration.getRateLimit()); LOG.debug("Initializing AzureBlobFileSystem for {} complete", uri); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 0353f3e01ffb1..a59f76b6d0fe0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -38,6 +38,7 @@ public final class ConfigurationKeys { public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key"; public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX = "fs\\.azure\\.account\\.key\\.(.*)"; public static final String FS_AZURE_SECURE_MODE = "fs.azure.secure.mode"; + public static final String FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED = "fs.azure.account.throttling.enabled"; // Retry strategy defined by the user public static final String AZURE_MIN_BACKOFF_INTERVAL = "fs.azure.io.retry.min.backoff.interval"; @@ -116,6 +117,8 @@ public final class ConfigurationKeys { public static final String AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = "fs.azure.createRemoteFileSystemDuringInitialization"; public static final String AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION = "fs.azure.skipUserGroupMetadataDuringInitialization"; public static final String FS_AZURE_ENABLE_AUTOTHROTTLING = "fs.azure.enable.autothrottling"; + public static final String FS_AZURE_ACCOUNT_OPERATION_IDLE_TIMEOUT = "fs.azure.account.operation.idle.timeout"; + public static final String FS_AZURE_ANALYSIS_PERIOD = "fs.azure.analysis.period"; public static final String FS_AZURE_ALWAYS_USE_HTTPS = "fs.azure.always.use.https"; public static final String FS_AZURE_ATOMIC_RENAME_KEY = "fs.azure.atomic.rename.key"; /** This config ensures that during create overwrite an existing file will be diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 097285bb48fbc..0ea2c929800a3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -94,6 +94,9 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_ENABLE_FLUSH = true; public static final boolean DEFAULT_DISABLE_OUTPUTSTREAM_FLUSH = true; public static final boolean DEFAULT_ENABLE_AUTOTHROTTLING = true; + public static final boolean DEFAULT_FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED = true; + public static final int DEFAULT_ACCOUNT_OPERATION_IDLE_TIMEOUT_MS = 60_000; + public static final int DEFAULT_ANALYSIS_PERIOD_MS = 10_000; public static final DelegatingSSLSocketFactory.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE = DelegatingSSLSocketFactory.SSLChannelMode.Default; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index a614dbb15b3bf..25562660ae231 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -101,6 +101,7 @@ public class AbfsClient implements Closeable { private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; + private final AbfsThrottlingIntercept intercept; private final ListeningScheduledExecutorService executorService; @@ -120,6 +121,7 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden this.retryPolicy = abfsClientContext.getExponentialRetryPolicy(); this.accountName = abfsConfiguration.getAccountName().substring(0, abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT)); this.authType = abfsConfiguration.getAuthType(accountName); + this.intercept = AbfsThrottlingInterceptFactory.getInstance(accountName, abfsConfiguration); String encryptionKey = this.abfsConfiguration .getClientProvidedEncryptionKey(); @@ -216,6 +218,10 @@ SharedKeyCredentials getSharedKeyCredentials() { return sharedKeyCredentials; } + AbfsThrottlingIntercept getIntercept() { + return intercept; + } + List createDefaultHeaders() { final List requestHeaders = new ArrayList(); requestHeaders.add(new AbfsHttpHeader(X_MS_VERSION, xMsVersion)); @@ -1277,6 +1283,14 @@ protected AbfsCounters getAbfsCounters() { return abfsCounters; } + /** + * Getter for abfsConfiguration from AbfsClient. + * @return AbfsConfiguration instance + */ + protected AbfsConfiguration getAbfsConfiguration() { + return abfsConfiguration; + } + public int getNumLeaseThreads() { return abfsConfiguration.getNumLeaseThreads(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java index 6dfd352954d34..f1eb3a2a77476 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java @@ -20,20 +20,23 @@ import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.util.Preconditions; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.util.Time.now; + class AbfsClientThrottlingAnalyzer { private static final Logger LOG = LoggerFactory.getLogger( AbfsClientThrottlingAnalyzer.class); - private static final int DEFAULT_ANALYSIS_PERIOD_MS = 10 * 1000; private static final int MIN_ANALYSIS_PERIOD_MS = 1000; private static final int MAX_ANALYSIS_PERIOD_MS = 30000; private static final double MIN_ACCEPTABLE_ERROR_PERCENTAGE = .1; @@ -50,42 +53,38 @@ class AbfsClientThrottlingAnalyzer { private String name = null; private Timer timer = null; private AtomicReference blobMetrics = null; + private AtomicLong lastExecutionTime = null; + private final AtomicBoolean isOperationOnAccountIdle = new AtomicBoolean(false); + private AbfsConfiguration abfsConfiguration = null; + private boolean accountLevelThrottlingEnabled = true; private AbfsClientThrottlingAnalyzer() { // hide default constructor } - /** - * Creates an instance of the AbfsClientThrottlingAnalyzer class with - * the specified name. - * - * @param name a name used to identify this instance. - * @throws IllegalArgumentException if name is null or empty. - */ - AbfsClientThrottlingAnalyzer(String name) throws IllegalArgumentException { - this(name, DEFAULT_ANALYSIS_PERIOD_MS); - } - /** * Creates an instance of the AbfsClientThrottlingAnalyzer class with * the specified name and period. * * @param name A name used to identify this instance. - * @param period The frequency, in milliseconds, at which metrics are - * analyzed. + * @param abfsConfiguration The configuration set. * @throws IllegalArgumentException If name is null or empty. * If period is less than 1000 or greater than 30000 milliseconds. */ - AbfsClientThrottlingAnalyzer(String name, int period) + AbfsClientThrottlingAnalyzer(String name, AbfsConfiguration abfsConfiguration) throws IllegalArgumentException { Preconditions.checkArgument( StringUtils.isNotEmpty(name), "The argument 'name' cannot be null or empty."); + int period = abfsConfiguration.getAnalysisPeriod(); Preconditions.checkArgument( period >= MIN_ANALYSIS_PERIOD_MS && period <= MAX_ANALYSIS_PERIOD_MS, "The argument 'period' must be between 1000 and 30000."); this.name = name; - this.analysisPeriodMs = period; + this.abfsConfiguration = abfsConfiguration; + this.accountLevelThrottlingEnabled = abfsConfiguration.accountThrottlingEnabled(); + this.analysisPeriodMs = abfsConfiguration.getAnalysisPeriod(); + this.lastExecutionTime = new AtomicLong(now()); this.blobMetrics = new AtomicReference( new AbfsOperationMetrics(System.currentTimeMillis())); this.timer = new Timer( @@ -95,6 +94,47 @@ private AbfsClientThrottlingAnalyzer() { analysisPeriodMs); } + /** + * Resumes the timer if it was stopped. + */ + private void resumeTimer() { + blobMetrics = new AtomicReference( + new AbfsOperationMetrics(System.currentTimeMillis())); + timer.schedule(new TimerTaskImpl(), + analysisPeriodMs, + analysisPeriodMs); + isOperationOnAccountIdle.set(false); + } + + /** + * Synchronized method to suspend or resume timer. + * @param timerFunctionality resume or suspend. + * @param timerTask The timertask object. + * @return true or false. + */ + private synchronized boolean timerOrchestrator(TimerFunctionality timerFunctionality, + TimerTask timerTask) { + switch (timerFunctionality) { + case RESUME: + if (isOperationOnAccountIdle.get()) { + resumeTimer(); + } + break; + case SUSPEND: + if (accountLevelThrottlingEnabled && (System.currentTimeMillis() + - lastExecutionTime.get() >= getOperationIdleTimeout())) { + isOperationOnAccountIdle.set(true); + timerTask.cancel(); + timer.purge(); + return true; + } + break; + default: + break; + } + return false; + } + /** * Updates metrics with results from the current storage operation. * @@ -104,12 +144,13 @@ private AbfsClientThrottlingAnalyzer() { public void addBytesTransferred(long count, boolean isFailedOperation) { AbfsOperationMetrics metrics = blobMetrics.get(); if (isFailedOperation) { - metrics.bytesFailed.addAndGet(count); - metrics.operationsFailed.incrementAndGet(); + metrics.addBytesFailed(count); + metrics.incrementOperationsFailed(); } else { - metrics.bytesSuccessful.addAndGet(count); - metrics.operationsSuccessful.incrementAndGet(); + metrics.addBytesSuccessful(count); + metrics.incrementOperationsSuccessful(); } + blobMetrics.set(metrics); } /** @@ -117,6 +158,8 @@ public void addBytesTransferred(long count, boolean isFailedOperation) { * @return true if Thread sleeps(Throttling occurs) else false. */ public boolean suspendIfNecessary() { + lastExecutionTime.set(now()); + timerOrchestrator(TimerFunctionality.RESUME, null); int duration = sleepDuration; if (duration > 0) { try { @@ -134,19 +177,27 @@ int getSleepDuration() { return sleepDuration; } + int getOperationIdleTimeout() { + return abfsConfiguration.getAccountOperationIdleTimeout(); + } + + AtomicBoolean getIsOperationOnAccountIdle() { + return isOperationOnAccountIdle; + } + private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics, int sleepDuration) { final double percentageConversionFactor = 100; - double bytesFailed = metrics.bytesFailed.get(); - double bytesSuccessful = metrics.bytesSuccessful.get(); - double operationsFailed = metrics.operationsFailed.get(); - double operationsSuccessful = metrics.operationsSuccessful.get(); + double bytesFailed = metrics.getBytesFailed().get(); + double bytesSuccessful = metrics.getBytesSuccessful().get(); + double operationsFailed = metrics.getOperationsFailed().get(); + double operationsSuccessful = metrics.getOperationsSuccessful().get(); double errorPercentage = (bytesFailed <= 0) ? 0 : (percentageConversionFactor * bytesFailed / (bytesFailed + bytesSuccessful)); - long periodMs = metrics.endTime - metrics.startTime; + long periodMs = metrics.getEndTime() - metrics.getStartTime(); double newSleepDuration; @@ -238,10 +289,13 @@ public void run() { } long now = System.currentTimeMillis(); - if (now - blobMetrics.get().startTime >= analysisPeriodMs) { + if (timerOrchestrator(TimerFunctionality.SUSPEND, this)) { + return; + } + if (now - blobMetrics.get().getStartTime() >= analysisPeriodMs) { AbfsOperationMetrics oldMetrics = blobMetrics.getAndSet( new AbfsOperationMetrics(now)); - oldMetrics.endTime = now; + oldMetrics.setEndTime(now); sleepDuration = analyzeMetricsAndUpdateSleepDuration(oldMetrics, sleepDuration); } @@ -252,24 +306,4 @@ public void run() { } } } - - /** - * Stores Abfs operation metrics during each analysis period. - */ - static class AbfsOperationMetrics { - private AtomicLong bytesFailed; - private AtomicLong bytesSuccessful; - private AtomicLong operationsFailed; - private AtomicLong operationsSuccessful; - private long endTime; - private long startTime; - - AbfsOperationMetrics(long startTime) { - this.startTime = startTime; - this.bytesFailed = new AtomicLong(); - this.bytesSuccessful = new AtomicLong(); - this.operationsFailed = new AtomicLong(); - this.operationsSuccessful = new AtomicLong(); - } - } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java index 7303e833418db..52a46bc746934 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java @@ -19,10 +19,12 @@ package org.apache.hadoop.fs.azurebfs.services; import java.net.HttpURLConnection; +import java.util.concurrent.locks.ReentrantLock; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbfsStatistic; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; @@ -38,35 +40,89 @@ * and sleeps just enough to minimize errors, allowing optimal ingress and/or * egress throughput. */ -public final class AbfsClientThrottlingIntercept { +public final class AbfsClientThrottlingIntercept implements AbfsThrottlingIntercept { private static final Logger LOG = LoggerFactory.getLogger( AbfsClientThrottlingIntercept.class); private static final String RANGE_PREFIX = "bytes="; - private static AbfsClientThrottlingIntercept singleton = null; - private AbfsClientThrottlingAnalyzer readThrottler = null; - private AbfsClientThrottlingAnalyzer writeThrottler = null; - private static boolean isAutoThrottlingEnabled = false; + private static AbfsClientThrottlingIntercept singleton; // singleton, initialized in static initialization block + private static final ReentrantLock LOCK = new ReentrantLock(); + private final AbfsClientThrottlingAnalyzer readThrottler; + private final AbfsClientThrottlingAnalyzer writeThrottler; + private final String accountName; // Hide default constructor - private AbfsClientThrottlingIntercept() { - readThrottler = new AbfsClientThrottlingAnalyzer("read"); - writeThrottler = new AbfsClientThrottlingAnalyzer("write"); + public AbfsClientThrottlingIntercept(String accountName, AbfsConfiguration abfsConfiguration) { + this.accountName = accountName; + this.readThrottler = setAnalyzer("read " + accountName, abfsConfiguration); + this.writeThrottler = setAnalyzer("write " + accountName, abfsConfiguration); + LOG.debug("Client-side throttling is enabled for the ABFS file system for the account : {}", accountName); } - public static synchronized void initializeSingleton(boolean enableAutoThrottling) { - if (!enableAutoThrottling) { - return; - } + // Hide default constructor + private AbfsClientThrottlingIntercept(AbfsConfiguration abfsConfiguration) { + //Account name is kept as empty as same instance is shared across all accounts + this.accountName = ""; + this.readThrottler = setAnalyzer("read", abfsConfiguration); + this.writeThrottler = setAnalyzer("write", abfsConfiguration); + LOG.debug("Client-side throttling is enabled for the ABFS file system using singleton intercept"); + } + + /** + * Sets the analyzer for the intercept. + * @param name Name of the analyzer. + * @param abfsConfiguration The configuration. + * @return AbfsClientThrottlingAnalyzer instance. + */ + private AbfsClientThrottlingAnalyzer setAnalyzer(String name, AbfsConfiguration abfsConfiguration) { + return new AbfsClientThrottlingAnalyzer(name, abfsConfiguration); + } + + /** + * Returns the analyzer for read operations. + * @return AbfsClientThrottlingAnalyzer for read. + */ + AbfsClientThrottlingAnalyzer getReadThrottler() { + return readThrottler; + } + + /** + * Returns the analyzer for write operations. + * @return AbfsClientThrottlingAnalyzer for write. + */ + AbfsClientThrottlingAnalyzer getWriteThrottler() { + return writeThrottler; + } + + /** + * Creates a singleton object of the AbfsClientThrottlingIntercept. + * which is shared across all filesystem instances. + * @param abfsConfiguration configuration set. + * @return singleton object of intercept. + */ + static AbfsClientThrottlingIntercept initializeSingleton(AbfsConfiguration abfsConfiguration) { if (singleton == null) { - singleton = new AbfsClientThrottlingIntercept(); - isAutoThrottlingEnabled = true; - LOG.debug("Client-side throttling is enabled for the ABFS file system."); + LOCK.lock(); + try { + if (singleton == null) { + singleton = new AbfsClientThrottlingIntercept(abfsConfiguration); + LOG.debug("Client-side throttling is enabled for the ABFS file system."); + } + } finally { + LOCK.unlock(); + } } + return singleton; } - static void updateMetrics(AbfsRestOperationType operationType, - AbfsHttpOperation abfsHttpOperation) { - if (!isAutoThrottlingEnabled || abfsHttpOperation == null) { + /** + * Updates the metrics for successful and failed read and write operations. + * @param operationType Only applicable for read and write operations. + * @param abfsHttpOperation Used for status code and data transferred. + */ + @Override + public void updateMetrics(AbfsRestOperationType operationType, + AbfsHttpOperation abfsHttpOperation) { + if (abfsHttpOperation == null) { return; } @@ -82,7 +138,7 @@ static void updateMetrics(AbfsRestOperationType operationType, case Append: contentLength = abfsHttpOperation.getBytesSent(); if (contentLength > 0) { - singleton.writeThrottler.addBytesTransferred(contentLength, + writeThrottler.addBytesTransferred(contentLength, isFailedOperation); } break; @@ -90,7 +146,7 @@ static void updateMetrics(AbfsRestOperationType operationType, String range = abfsHttpOperation.getConnection().getRequestProperty(HttpHeaderConfigurations.RANGE); contentLength = getContentLengthIfKnown(range); if (contentLength > 0) { - singleton.readThrottler.addBytesTransferred(contentLength, + readThrottler.addBytesTransferred(contentLength, isFailedOperation); } break; @@ -104,21 +160,18 @@ static void updateMetrics(AbfsRestOperationType operationType, * uses this to suspend the request, if necessary, to minimize errors and * maximize throughput. */ - static void sendingRequest(AbfsRestOperationType operationType, + @Override + public void sendingRequest(AbfsRestOperationType operationType, AbfsCounters abfsCounters) { - if (!isAutoThrottlingEnabled) { - return; - } - switch (operationType) { case ReadFile: - if (singleton.readThrottler.suspendIfNecessary() + if (readThrottler.suspendIfNecessary() && abfsCounters != null) { abfsCounters.incrementCounter(AbfsStatistic.READ_THROTTLES, 1); } break; case Append: - if (singleton.writeThrottler.suspendIfNecessary() + if (writeThrottler.suspendIfNecessary() && abfsCounters != null) { abfsCounters.incrementCounter(AbfsStatistic.WRITE_THROTTLES, 1); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.java new file mode 100644 index 0000000000000..6b84e583c337a --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.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.fs.azurebfs.services; + +final class AbfsNoOpThrottlingIntercept implements AbfsThrottlingIntercept { + + public static final AbfsNoOpThrottlingIntercept INSTANCE = new AbfsNoOpThrottlingIntercept(); + + private AbfsNoOpThrottlingIntercept() { + } + + @Override + public void updateMetrics(final AbfsRestOperationType operationType, + final AbfsHttpOperation abfsHttpOperation) { + } + + @Override + public void sendingRequest(final AbfsRestOperationType operationType, + final AbfsCounters abfsCounters) { + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java new file mode 100644 index 0000000000000..2e53367d39fd2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.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.fs.azurebfs.services; + +import java.util.concurrent.atomic.AtomicLong; + +/** + * Stores Abfs operation metrics during each analysis period. + */ +class AbfsOperationMetrics { + + /** + * No of bytes which could not be transferred due to a failed operation. + */ + private final AtomicLong bytesFailed; + + /** + * No of bytes successfully transferred during a successful operation. + */ + private final AtomicLong bytesSuccessful; + + /** + * Total no of failed operations. + */ + private final AtomicLong operationsFailed; + + /** + * Total no of successful operations. + */ + private final AtomicLong operationsSuccessful; + + /** + * Time when collection of metrics ended. + */ + private long endTime; + + /** + * Time when the collection of metrics started. + */ + private final long startTime; + + AbfsOperationMetrics(long startTime) { + this.startTime = startTime; + this.bytesFailed = new AtomicLong(); + this.bytesSuccessful = new AtomicLong(); + this.operationsFailed = new AtomicLong(); + this.operationsSuccessful = new AtomicLong(); + } + + /** + * + * @return bytes failed to transfer. + */ + AtomicLong getBytesFailed() { + return bytesFailed; + } + + /** + * + * @return bytes successfully transferred. + */ + AtomicLong getBytesSuccessful() { + return bytesSuccessful; + } + + /** + * + * @return no of operations failed. + */ + AtomicLong getOperationsFailed() { + return operationsFailed; + } + + /** + * + * @return no of successful operations. + */ + AtomicLong getOperationsSuccessful() { + return operationsSuccessful; + } + + /** + * + * @return end time of metric collection. + */ + long getEndTime() { + return endTime; + } + + /** + * + * @param endTime sets the end time. + */ + void setEndTime(final long endTime) { + this.endTime = endTime; + } + + /** + * + * @return start time of metric collection. + */ + long getStartTime() { + return startTime; + } + + void addBytesFailed(long bytes) { + this.getBytesFailed().addAndGet(bytes); + } + + void addBytesSuccessful(long bytes) { + this.getBytesSuccessful().addAndGet(bytes); + } + + void incrementOperationsFailed() { + this.getOperationsFailed().incrementAndGet(); + } + + void incrementOperationsSuccessful() { + this.getOperationsSuccessful().incrementAndGet(); + } + +} + diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java index 74b267d563eb2..00da9b6601338 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java @@ -45,6 +45,8 @@ public class AbfsRestOperation { private final AbfsRestOperationType operationType; // Blob FS client, which has the credentials, retry policy, and logs. private final AbfsClient client; + // Return intercept instance + private final AbfsThrottlingIntercept intercept; // the HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE) private final String method; // full URL including query parameters @@ -145,6 +147,7 @@ String getSasToken() { || AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method)); this.sasToken = sasToken; this.abfsCounters = client.getAbfsCounters(); + this.intercept = client.getIntercept(); } /** @@ -241,7 +244,8 @@ private void completeExecute(TracingContext tracingContext) */ private boolean executeHttpOperation(final int retryCount, TracingContext tracingContext) throws AzureBlobFileSystemException { - AbfsHttpOperation httpOperation = null; + AbfsHttpOperation httpOperation; + try { // initialize the HTTP request and open the connection httpOperation = new AbfsHttpOperation(url, method, requestHeaders); @@ -278,8 +282,7 @@ private boolean executeHttpOperation(final int retryCount, // dump the headers AbfsIoUtils.dumpHeadersToDebugLog("Request Headers", httpOperation.getConnection().getRequestProperties()); - AbfsClientThrottlingIntercept.sendingRequest(operationType, abfsCounters); - + intercept.sendingRequest(operationType, abfsCounters); if (hasRequestBody) { // HttpUrlConnection requires httpOperation.sendRequest(buffer, bufferOffset, bufferLength); @@ -317,7 +320,7 @@ private boolean executeHttpOperation(final int retryCount, return false; } finally { - AbfsClientThrottlingIntercept.updateMetrics(operationType, httpOperation); + intercept.updateMetrics(operationType, httpOperation); } LOG.debug("HttpRequest: {}: {}", operationType, httpOperation); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.java new file mode 100644 index 0000000000000..57b5095bb3219 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.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.fs.azurebfs.services; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * An interface for Abfs Throttling Interface. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface AbfsThrottlingIntercept { + + /** + * Updates the metrics for successful and failed read and write operations. + * @param operationType Only applicable for read and write operations. + * @param abfsHttpOperation Used for status code and data transferred. + */ + void updateMetrics(AbfsRestOperationType operationType, + AbfsHttpOperation abfsHttpOperation); + + /** + * Called before the request is sent. Client-side throttling + * uses this to suspend the request, if necessary, to minimize errors and + * maximize throughput. + * @param operationType Only applicable for read and write operations. + * @param abfsCounters Used for counters. + */ + void sendingRequest(AbfsRestOperationType operationType, + AbfsCounters abfsCounters); + +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingInterceptFactory.java new file mode 100644 index 0000000000000..279b7a318caf0 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingInterceptFactory.java @@ -0,0 +1,102 @@ +/** + * 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.fs.azurebfs.services; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.util.WeakReferenceMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Class to get an instance of throttling intercept class per account. + */ +final class AbfsThrottlingInterceptFactory { + + private AbfsThrottlingInterceptFactory() { + } + + private static AbfsConfiguration abfsConfig; + + /** + * List of references notified of loss. + */ + private static List lostReferences = new ArrayList<>(); + + private static final Logger LOG = LoggerFactory.getLogger( + AbfsThrottlingInterceptFactory.class); + + /** + * Map which stores instance of ThrottlingIntercept class per account. + */ + private static WeakReferenceMap + interceptMap = new WeakReferenceMap<>( + AbfsThrottlingInterceptFactory::factory, + AbfsThrottlingInterceptFactory::referenceLost); + + /** + * Returns instance of throttling intercept. + * @param accountName Account name. + * @return instance of throttling intercept. + */ + private static AbfsClientThrottlingIntercept factory(final String accountName) { + return new AbfsClientThrottlingIntercept(accountName, abfsConfig); + } + + /** + * Reference lost callback. + * @param accountName key lost. + */ + private static void referenceLost(String accountName) { + lostReferences.add(accountName); + } + + /** + * Returns an instance of AbfsThrottlingIntercept. + * + * @param accountName The account for which we need instance of throttling intercept. + @param abfsConfiguration The object of abfsconfiguration class. + * @return Instance of AbfsThrottlingIntercept. + */ + static synchronized AbfsThrottlingIntercept getInstance(String accountName, + AbfsConfiguration abfsConfiguration) { + abfsConfig = abfsConfiguration; + AbfsThrottlingIntercept intercept; + if (!abfsConfiguration.isAutoThrottlingEnabled()) { + return AbfsNoOpThrottlingIntercept.INSTANCE; + } + // If singleton is enabled use a static instance of the intercept class for all accounts + if (!abfsConfiguration.accountThrottlingEnabled()) { + intercept = AbfsClientThrottlingIntercept.initializeSingleton( + abfsConfiguration); + } else { + // Return the instance from the map + intercept = interceptMap.get(accountName); + if (intercept == null) { + intercept = new AbfsClientThrottlingIntercept(accountName, + abfsConfiguration); + interceptMap.put(accountName, intercept); + } + } + return intercept; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimerFunctionality.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimerFunctionality.java new file mode 100644 index 0000000000000..bf7da69ec4982 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimerFunctionality.java @@ -0,0 +1,26 @@ +/** + * 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.fs.azurebfs.services; + +public enum TimerFunctionality { + RESUME, + + SUSPEND +} + diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index 35d360556047e..31498df17904a 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -767,6 +767,12 @@ Hflush() being the only documented API that can provide persistent data transfer, Flush() also attempting to persist buffered data will lead to performance issues. + +### Account level throttling Options + +`fs.azure.account.operation.idle.timeout`: This value specifies the time after which the timer for the analyzer (read or +write) should be paused until no new request is made again. The default value for the same is 60 seconds. + ### HNS Check Options Config `fs.azure.account.hns.enabled` provides an option to specify whether the storage account is HNS enabled or not. In case the config is not provided, @@ -877,6 +883,9 @@ when there are too many writes from the same process. tuned with this config considering each queued request holds a buffer. Set the value 3 or 4 times the value set for s.azure.write.max.concurrent.requests. +`fs.azure.analysis.period`: The time after which sleep duration is recomputed after analyzing metrics. The default value +for the same is 10 seconds. + ### Security Options `fs.azure.always.use.https`: Enforces to use HTTPS instead of HTTP when the flag is made true. Irrespective of the flag, `AbfsClient` will use HTTPS if the secure diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java index 565eb38c4f70a..9e40f22d231b0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java @@ -24,6 +24,9 @@ public final class TestConfigurationKeys { public static final String FS_AZURE_ACCOUNT_NAME = "fs.azure.account.name"; public static final String FS_AZURE_ABFS_ACCOUNT_NAME = "fs.azure.abfs.account.name"; + public static final String FS_AZURE_ABFS_ACCOUNT1_NAME = "fs.azure.abfs.account1.name"; + public static final String FS_AZURE_ENABLE_AUTOTHROTTLING = "fs.azure.enable.autothrottling"; + public static final String FS_AZURE_ANALYSIS_PERIOD = "fs.azure.analysis.period"; public static final String FS_AZURE_ACCOUNT_KEY = "fs.azure.account.key"; public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs"; public static final String FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT = "fs.azure.test.namespace.enabled"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 0a1dca7e7d8d7..08eb3adc92697 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -306,6 +306,11 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, when(client.getAccessToken()).thenCallRealMethod(); when(client.getSharedKeyCredentials()).thenCallRealMethod(); when(client.createDefaultHeaders()).thenCallRealMethod(); + when(client.getAbfsConfiguration()).thenReturn(abfsConfig); + when(client.getIntercept()).thenReturn( + AbfsThrottlingInterceptFactory.getInstance( + abfsConfig.getAccountName().substring(0, + abfsConfig.getAccountName().indexOf(DOT)), abfsConfig)); // override baseurl client = TestAbfsClient.setAbfsClientField(client, "abfsConfiguration", diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClientThrottlingAnalyzer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClientThrottlingAnalyzer.java index 3f680e499300d..22649cd190d83 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClientThrottlingAnalyzer.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClientThrottlingAnalyzer.java @@ -18,9 +18,15 @@ package org.apache.hadoop.fs.azurebfs.services; +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.junit.Test; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ANALYSIS_PERIOD; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -33,6 +39,15 @@ public class TestAbfsClientThrottlingAnalyzer { + ANALYSIS_PERIOD / 10; private static final long MEGABYTE = 1024 * 1024; private static final int MAX_ACCEPTABLE_PERCENT_DIFFERENCE = 20; + private AbfsConfiguration abfsConfiguration; + + public TestAbfsClientThrottlingAnalyzer() throws IOException, IllegalAccessException { + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + configuration.setInt(FS_AZURE_ANALYSIS_PERIOD, 1000); + this.abfsConfiguration = new AbfsConfiguration(configuration, + "dummy"); + } private void sleep(long milliseconds) { try { @@ -82,8 +97,7 @@ private void validateLessThanOrEqual(long maxExpected, long actual) { @Test public void testNoMetricUpdatesThenNoWaiting() { AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( - "test", - ANALYSIS_PERIOD); + "test", abfsConfiguration); validate(0, analyzer.getSleepDuration()); sleep(ANALYSIS_PERIOD_PLUS_10_PERCENT); validate(0, analyzer.getSleepDuration()); @@ -96,8 +110,7 @@ public void testNoMetricUpdatesThenNoWaiting() { @Test public void testOnlySuccessThenNoWaiting() { AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( - "test", - ANALYSIS_PERIOD); + "test", abfsConfiguration); analyzer.addBytesTransferred(8 * MEGABYTE, false); validate(0, analyzer.getSleepDuration()); sleep(ANALYSIS_PERIOD_PLUS_10_PERCENT); @@ -112,8 +125,7 @@ public void testOnlySuccessThenNoWaiting() { @Test public void testOnlyErrorsAndWaiting() { AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( - "test", - ANALYSIS_PERIOD); + "test", abfsConfiguration); validate(0, analyzer.getSleepDuration()); analyzer.addBytesTransferred(4 * MEGABYTE, true); sleep(ANALYSIS_PERIOD_PLUS_10_PERCENT); @@ -132,8 +144,7 @@ public void testOnlyErrorsAndWaiting() { @Test public void testSuccessAndErrorsAndWaiting() { AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( - "test", - ANALYSIS_PERIOD); + "test", abfsConfiguration); validate(0, analyzer.getSleepDuration()); analyzer.addBytesTransferred(8 * MEGABYTE, false); analyzer.addBytesTransferred(2 * MEGABYTE, true); @@ -157,8 +168,7 @@ public void testSuccessAndErrorsAndWaiting() { @Test public void testManySuccessAndErrorsAndWaiting() { AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( - "test", - ANALYSIS_PERIOD); + "test", abfsConfiguration); validate(0, analyzer.getSleepDuration()); final int numberOfRequests = 20; for (int i = 0; i < numberOfRequests; i++) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java index 0f8dc55aa14a4..a1fc4e138d6cd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java @@ -18,13 +18,35 @@ package org.apache.hadoop.fs.azurebfs.services; +import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR; + import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_BACKOFF_INTERVAL; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_IO_RETRIES; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_AUTOTHROTTLING; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT1_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; + +import static org.junit.Assume.assumeTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.assertj.core.api.Assertions; +import org.junit.Assume; +import org.mockito.Mockito; + +import java.net.URI; import java.util.Random; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; import org.junit.Assert; import org.junit.Test; @@ -41,6 +63,9 @@ public class TestExponentialRetryPolicy extends AbstractAbfsIntegrationTest { private final int noRetryCount = 0; private final int retryCount = new Random().nextInt(maxRetryCount); private final int retryCountBeyondMax = maxRetryCount + 1; + private static final String TEST_PATH = "/testfile"; + private static final double MULTIPLYING_FACTOR = 1.5; + private static final int ANALYSIS_PERIOD = 10000; public TestExponentialRetryPolicy() throws Exception { @@ -67,6 +92,173 @@ public void testDefaultMaxIORetryCount() throws Exception { testMaxIOConfig(abfsConfig); } + @Test + public void testThrottlingIntercept() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + configuration.setBoolean(FS_AZURE_ENABLE_AUTOTHROTTLING, false); + + // On disabling throttling AbfsNoOpThrottlingIntercept object is returned + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + "dummy.dfs.core.windows.net"); + AbfsThrottlingIntercept intercept; + AbfsClient abfsClient = TestAbfsClient.createTestClientFromCurrentContext(fs.getAbfsStore().getClient(), abfsConfiguration); + intercept = abfsClient.getIntercept(); + Assertions.assertThat(intercept) + .describedAs("AbfsNoOpThrottlingIntercept instance expected") + .isInstanceOf(AbfsNoOpThrottlingIntercept.class); + + configuration.setBoolean(FS_AZURE_ENABLE_AUTOTHROTTLING, true); + configuration.setBoolean(FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED, true); + // On disabling throttling AbfsClientThrottlingIntercept object is returned + AbfsConfiguration abfsConfiguration1 = new AbfsConfiguration(configuration, + "dummy1.dfs.core.windows.net"); + AbfsClient abfsClient1 = TestAbfsClient.createTestClientFromCurrentContext(fs.getAbfsStore().getClient(), abfsConfiguration1); + intercept = abfsClient1.getIntercept(); + Assertions.assertThat(intercept) + .describedAs("AbfsClientThrottlingIntercept instance expected") + .isInstanceOf(AbfsClientThrottlingIntercept.class); + } + + @Test + public void testCreateMultipleAccountThrottling() throws Exception { + Configuration config = new Configuration(getRawConfiguration()); + String accountName = config.get(FS_AZURE_ACCOUNT_NAME); + if (accountName == null) { + // check if accountName is set using different config key + accountName = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + } + assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT1_NAME, + accountName != null && !accountName.isEmpty()); + + Configuration rawConfig1 = new Configuration(); + rawConfig1.addResource(TEST_CONFIGURATION_FILE_NAME); + + AbfsRestOperation successOp = mock(AbfsRestOperation.class); + AbfsHttpOperation http500Op = mock(AbfsHttpOperation.class); + when(http500Op.getStatusCode()).thenReturn(HTTP_INTERNAL_ERROR); + when(successOp.getResult()).thenReturn(http500Op); + + AbfsConfiguration configuration = Mockito.mock(AbfsConfiguration.class); + when(configuration.getAnalysisPeriod()).thenReturn(ANALYSIS_PERIOD); + when(configuration.isAutoThrottlingEnabled()).thenReturn(true); + when(configuration.accountThrottlingEnabled()).thenReturn(false); + + AbfsThrottlingIntercept instance1 = AbfsThrottlingInterceptFactory.getInstance(accountName, configuration); + String accountName1 = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + + assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT1_NAME, + accountName1 != null && !accountName1.isEmpty()); + + AbfsThrottlingIntercept instance2 = AbfsThrottlingInterceptFactory.getInstance(accountName1, configuration); + //if singleton is enabled, for different accounts both the instances should return same value + Assertions.assertThat(instance1) + .describedAs( + "if singleton is enabled, for different accounts both the instances should return same value") + .isEqualTo(instance2); + + when(configuration.accountThrottlingEnabled()).thenReturn(true); + AbfsThrottlingIntercept instance3 = AbfsThrottlingInterceptFactory.getInstance(accountName, configuration); + AbfsThrottlingIntercept instance4 = AbfsThrottlingInterceptFactory.getInstance(accountName1, configuration); + AbfsThrottlingIntercept instance5 = AbfsThrottlingInterceptFactory.getInstance(accountName, configuration); + //if singleton is not enabled, for different accounts instances should return different value + Assertions.assertThat(instance3) + .describedAs( + "iff singleton is not enabled, for different accounts instances should return different value") + .isNotEqualTo(instance4); + + //if singleton is not enabled, for same accounts instances should return same value + Assertions.assertThat(instance3) + .describedAs( + "if singleton is not enabled, for same accounts instances should return same value") + .isEqualTo(instance5); + } + + @Test + public void testOperationOnAccountIdle() throws Exception { + //Get the filesystem. + AzureBlobFileSystem fs = getFileSystem(); + AbfsClient client = fs.getAbfsStore().getClient(); + AbfsConfiguration configuration1 = client.getAbfsConfiguration(); + Assume.assumeTrue(configuration1.isAutoThrottlingEnabled()); + Assume.assumeTrue(configuration1.accountThrottlingEnabled()); + + AbfsClientThrottlingIntercept accountIntercept + = (AbfsClientThrottlingIntercept) client.getIntercept(); + final byte[] b = new byte[2 * MIN_BUFFER_SIZE]; + new Random().nextBytes(b); + + Path testPath = path(TEST_PATH); + + //Do an operation on the filesystem. + try (FSDataOutputStream stream = fs.create(testPath)) { + stream.write(b); + } + + //Don't perform any operation on the account. + int sleepTime = (int) ((getAbfsConfig().getAccountOperationIdleTimeout()) * MULTIPLYING_FACTOR); + Thread.sleep(sleepTime); + + try (FSDataInputStream streamRead = fs.open(testPath)) { + streamRead.read(b); + } + + //Perform operations on another account. + AzureBlobFileSystem fs1 = new AzureBlobFileSystem(); + Configuration config = new Configuration(getRawConfiguration()); + String accountName1 = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT1_NAME, + accountName1 != null && !accountName1.isEmpty()); + final String abfsUrl1 = this.getFileSystemName() + "12" + "@" + accountName1; + URI defaultUri1 = null; + defaultUri1 = new URI("abfss", abfsUrl1, null, null, null); + fs1.initialize(defaultUri1, getRawConfiguration()); + AbfsClient client1 = fs1.getAbfsStore().getClient(); + AbfsClientThrottlingIntercept accountIntercept1 + = (AbfsClientThrottlingIntercept) client1.getIntercept(); + try (FSDataOutputStream stream1 = fs1.create(testPath)) { + stream1.write(b); + } + + //Verify the write analyzer for first account is idle but the read analyzer is not idle. + Assertions.assertThat(accountIntercept.getWriteThrottler() + .getIsOperationOnAccountIdle() + .get()) + .describedAs("Write analyzer for first account should be idle the first time") + .isTrue(); + + Assertions.assertThat( + accountIntercept.getReadThrottler() + .getIsOperationOnAccountIdle() + .get()) + .describedAs("Read analyzer for first account should not be idle") + .isFalse(); + + //Verify the write analyzer for second account is not idle. + Assertions.assertThat( + accountIntercept1.getWriteThrottler() + .getIsOperationOnAccountIdle() + .get()) + .describedAs("Write analyzer for second account should not be idle") + .isFalse(); + + //Again perform an operation on the first account. + try (FSDataOutputStream stream2 = fs.create(testPath)) { + stream2.write(b); + } + + //Verify the write analyzer on first account is not idle. + Assertions.assertThat( + + accountIntercept.getWriteThrottler() + .getIsOperationOnAccountIdle() + .get()) + .describedAs( + "Write analyzer for first account should not be idle second time") + .isFalse(); + } + @Test public void testAbfsConfigConstructor() throws Exception { // Ensure we choose expected values that are not defaults From d94d9fff940eaedd7c506ec10abb2e0209532623 Mon Sep 17 00:00:00 2001 From: litao Date: Wed, 30 Nov 2022 23:44:21 +0800 Subject: [PATCH 08/45] HDFS-16550. Allow JN edit cache size to be set as a fraction of heap memory (#4209) --- .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 5 +++- .../qjournal/server/JournaledEditsCache.java | 28 ++++++++++++++----- .../src/main/resources/hdfs-default.xml | 18 +++++++++++- .../markdown/HDFSHighAvailabilityWithQJM.md | 10 +++++++ .../src/site/markdown/ObserverNameNode.md | 18 ++++++++++++ .../server/TestJournaledEditsCache.java | 21 ++++++++++++++ 6 files changed, 91 insertions(+), 9 deletions(-) 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 f766c48d7c5fd..0e10bc61c99a1 100755 --- 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 @@ -1424,7 +1424,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final long DFS_JOURNALNODE_SYNC_INTERVAL_DEFAULT = 2*60*1000L; public static final String DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY = "dfs.journalnode.edit-cache-size.bytes"; - public static final int DFS_JOURNALNODE_EDIT_CACHE_SIZE_DEFAULT = 1024 * 1024; + + public static final String DFS_JOURNALNODE_EDIT_CACHE_SIZE_FRACTION_KEY = + "dfs.journalnode.edit-cache-size.fraction"; + public static final float DFS_JOURNALNODE_EDIT_CACHE_SIZE_FRACTION_DEFAULT = 0.5f; // Journal-node related configs for the client side. public static final String DFS_QJOURNAL_QUEUE_SIZE_LIMIT_KEY = "dfs.qjournal.queued-edits.limit.mb"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournaledEditsCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournaledEditsCache.java index 65f54609ef3f5..339b7fa7b680c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournaledEditsCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournaledEditsCache.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp; import org.apache.hadoop.util.AutoCloseableLock; +import org.apache.hadoop.util.Preconditions; /** * An in-memory cache of edits in their serialized form. This is used to serve @@ -121,12 +122,18 @@ class JournaledEditsCache { // ** End lock-protected fields ** JournaledEditsCache(Configuration conf) { + float fraction = conf.getFloat(DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_FRACTION_KEY, + DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_FRACTION_DEFAULT); + Preconditions.checkArgument((fraction > 0 && fraction < 1.0f), + String.format("Cache config %s is set at %f, it should be a positive float value, " + + "less than 1.0. The recommended value is less than 0.9.", + DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_FRACTION_KEY, fraction)); capacity = conf.getInt(DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY, - DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_DEFAULT); + (int) (Runtime.getRuntime().maxMemory() * fraction)); if (capacity > 0.9 * Runtime.getRuntime().maxMemory()) { Journal.LOG.warn(String.format("Cache capacity is set at %d bytes but " + "maximum JVM memory is only %d bytes. It is recommended that you " + - "decrease the cache size or increase the heap size.", + "decrease the cache size/fraction or increase the heap size.", capacity, Runtime.getRuntime().maxMemory())); } Journal.LOG.info("Enabling the journaled edits cache with a capacity " + @@ -277,11 +284,12 @@ void storeEdits(byte[] inputData, long newStartTxn, long newEndTxn, initialize(INVALID_TXN_ID); Journal.LOG.warn(String.format("A single batch of edits was too " + "large to fit into the cache: startTxn = %d, endTxn = %d, " + - "input length = %d. The capacity of the cache (%s) must be " + + "input length = %d. The cache size (%s) or cache fraction (%s) must be " + "increased for it to work properly (current capacity %d)." + "Cache is now empty.", newStartTxn, newEndTxn, inputData.length, - DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY, capacity)); + DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY, + DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_FRACTION_KEY, capacity)); return; } if (dataMap.isEmpty()) { @@ -388,10 +396,11 @@ private CacheMissException getCacheMissException(long requestedTxnId) { } else { return new CacheMissException(lowestTxnId - requestedTxnId, "Oldest txn ID available in the cache is %d, but requested txns " + - "starting at %d. The cache size (%s) may need to be increased " + - "to hold more transactions (currently %d bytes containing %d " + + "starting at %d. The cache size (%s) or cache fraction (%s) may need to be " + + "increased to hold more transactions (currently %d bytes containing %d " + "transactions)", lowestTxnId, requestedTxnId, - DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY, capacity, + DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY, + DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_FRACTION_KEY, capacity, highestTxnId - lowestTxnId + 1); } } @@ -414,4 +423,9 @@ long getCacheMissAmount() { } + @VisibleForTesting + int getCapacity() { + return capacity; + } + } 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 2a0a4945faa6e..bf7f99aa1fa6a 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -4945,7 +4945,7 @@ dfs.journalnode.edit-cache-size.bytes - 1048576 + The size, in bytes, of the in-memory cache of edits to keep on the JournalNode. This cache is used to serve edits for tailing via the RPC-based @@ -4955,6 +4955,22 @@ + + dfs.journalnode.edit-cache-size.fraction + 0.5f + + This ratio refers to the proportion of the maximum memory of the JVM. + Used to calculate the size of the edits cache that is kept in the JournalNode's memory. + This config is an alternative to the dfs.journalnode.edit-cache-size.bytes. + And it is used to serve edits for tailing via the RPC-based mechanism, and is only + enabled when dfs.ha.tail-edits.in-progress is true. Transactions range in size but + are around 200 bytes on average, so the default of 1MB can store around 5000 transactions. + So we can configure a reasonable value based on the maximum memory. The recommended value + is less than 0.9. If we set dfs.journalnode.edit-cache-size.bytes, this parameter will + not take effect. + + + dfs.journalnode.kerberos.internal.spnego.principal diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md index 5591f4f22453b..b6b408db8b4ab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md @@ -502,6 +502,16 @@ lag time will be much longer. The relevant configurations are: the oldest data in the cache was at transaction ID 20, a value of 10 would be added to the average. +* **dfs.journalnode.edit-cache-size.fraction** - This fraction refers to the proportion of + the maximum memory of the JVM. Used to calculate the size of the edits cache that is + kept in the JournalNode's memory. This config is an alternative to the + dfs.journalnode.edit-cache-size.bytes. And it is used to serve edits for tailing via + the RPC-based mechanism, and is only enabled when dfs.ha.tail-edits.in-progress is true. + Transactions range in size but are around 200 bytes on average, so the default of 1MB + can store around 5000 transactions. So we can configure a reasonable value based on + the maximum memory. The recommended value is less than 0.9. If we set + dfs.journalnode.edit-cache-size.bytes, this parameter will not take effect. + This feature is primarily useful in conjunction with the Standby/Observer Read feature. Using this feature, read requests can be serviced from non-active NameNodes; thus tailing in-progress edits provides these nodes with the ability to serve requests with data which is much more fresh. See the diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md index 00aeb5bd2e082..74026ec86250f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md @@ -194,6 +194,24 @@ few configurations to your **hdfs-site.xml**: 1048576 +* **dfs.journalnode.edit-cache-size.fraction** - the fraction refers to + the proportion of the maximum memory of the JVM. + + Used to calculate the size of the edits cache that + is kept in the JournalNode's memory. + This config is an alternative to the dfs.journalnode.edit-cache-size.bytes. + And it is used to serve edits for tailing via the RPC-based mechanism, and is only + enabled when dfs.ha.tail-edits.in-progress is true. Transactions range in size but + are around 200 bytes on average, so the default of 1MB can store around 5000 transactions. + So we can configure a reasonable value based on the maximum memory. The recommended value + is less than 0.9. If we set dfs.journalnode.edit-cache-size.bytes, this parameter will + not take effect. + + + dfs.journalnode.edit-cache-size.fraction + 0.5f + + * **dfs.namenode.accesstime.precision** -- whether to enable access time for HDFS file. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournaledEditsCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournaledEditsCache.java index 2a178a1547ec4..82b8b587694b9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournaledEditsCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournaledEditsCache.java @@ -221,6 +221,27 @@ public void testCacheMalformedInput() throws Exception { cache.retrieveEdits(-1, 10, new ArrayList<>()); } + @Test + public void testCacheSizeConfigs() { + // Assert the default configs. + Configuration config = new Configuration(); + cache = new JournaledEditsCache(config); + assertEquals((int) (Runtime.getRuntime().maxMemory() * 0.5f), cache.getCapacity()); + + // Set dfs.journalnode.edit-cache-size.bytes. + Configuration config1 = new Configuration(); + config1.setInt(DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY, 1); + config1.setFloat(DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_FRACTION_KEY, 0.1f); + cache = new JournaledEditsCache(config1); + assertEquals(1, cache.getCapacity()); + + // Don't set dfs.journalnode.edit-cache-size.bytes. + Configuration config2 = new Configuration(); + config2.setFloat(DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_FRACTION_KEY, 0.1f); + cache = new JournaledEditsCache(config2); + assertEquals((int) (Runtime.getRuntime().maxMemory() * 0.1f), cache.getCapacity()); + } + private void storeEdits(int startTxn, int endTxn) throws Exception { cache.storeEdits(createTxnData(startTxn, endTxn - startTxn + 1), startTxn, endTxn, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION); From f531c6b69f70f37a3a8836710bd3777691ad5bfb Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Thu, 1 Dec 2022 15:11:58 +0100 Subject: [PATCH 09/45] YARN-10946. AbstractCSQueue: Create separate class for constructing Queue API objects. Contributed by Peter Szucs --- .../scheduler/capacity/AbstractCSQueue.java | 80 +----------- .../capacity/CSQueueInfoProvider.java | 117 ++++++++++++++++++ 2 files changed, 118 insertions(+), 79 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueInfoProvider.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/scheduler/capacity/AbstractCSQueue.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/AbstractCSQueue.java index 1a5a1ce0fd461..7a3ec4f6f4e5e 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/AbstractCSQueue.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/AbstractCSQueue.java @@ -29,10 +29,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.QueueACL; -import org.apache.hadoop.yarn.api.records.QueueConfigurations; import org.apache.hadoop.yarn.api.records.QueueInfo; import org.apache.hadoop.yarn.api.records.QueueState; -import org.apache.hadoop.yarn.api.records.QueueStatistics; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -584,83 +582,7 @@ protected QueueInfo getQueueInfo() { // from schedulerApplicationAttempt, to avoid deadlock, sacrifice // consistency here. // TODO, improve this - QueueInfo queueInfo = recordFactory.newRecordInstance(QueueInfo.class); - queueInfo.setQueueName(queuePath.getLeafName()); - queueInfo.setQueuePath(queuePath.getFullPath()); - queueInfo.setAccessibleNodeLabels(queueNodeLabelsSettings.getAccessibleNodeLabels()); - queueInfo.setCapacity(queueCapacities.getCapacity()); - queueInfo.setMaximumCapacity(queueCapacities.getMaximumCapacity()); - queueInfo.setQueueState(getState()); - queueInfo.setDefaultNodeLabelExpression(queueNodeLabelsSettings.getDefaultLabelExpression()); - queueInfo.setCurrentCapacity(getUsedCapacity()); - queueInfo.setQueueStatistics(getQueueStatistics()); - queueInfo.setPreemptionDisabled(preemptionSettings.isPreemptionDisabled()); - queueInfo.setIntraQueuePreemptionDisabled( - getIntraQueuePreemptionDisabled()); - queueInfo.setQueueConfigurations(getQueueConfigurations()); - queueInfo.setWeight(queueCapacities.getWeight()); - queueInfo.setMaxParallelApps(queueAppLifetimeSettings.getMaxParallelApps()); - return queueInfo; - } - - public QueueStatistics getQueueStatistics() { - // Deliberately doesn't use lock here, because this method will be invoked - // from schedulerApplicationAttempt, to avoid deadlock, sacrifice - // consistency here. - // TODO, improve this - QueueStatistics stats = recordFactory.newRecordInstance( - QueueStatistics.class); - stats.setNumAppsSubmitted(getMetrics().getAppsSubmitted()); - stats.setNumAppsRunning(getMetrics().getAppsRunning()); - stats.setNumAppsPending(getMetrics().getAppsPending()); - stats.setNumAppsCompleted(getMetrics().getAppsCompleted()); - stats.setNumAppsKilled(getMetrics().getAppsKilled()); - stats.setNumAppsFailed(getMetrics().getAppsFailed()); - stats.setNumActiveUsers(getMetrics().getActiveUsers()); - stats.setAvailableMemoryMB(getMetrics().getAvailableMB()); - stats.setAllocatedMemoryMB(getMetrics().getAllocatedMB()); - stats.setPendingMemoryMB(getMetrics().getPendingMB()); - stats.setReservedMemoryMB(getMetrics().getReservedMB()); - stats.setAvailableVCores(getMetrics().getAvailableVirtualCores()); - stats.setAllocatedVCores(getMetrics().getAllocatedVirtualCores()); - stats.setPendingVCores(getMetrics().getPendingVirtualCores()); - stats.setReservedVCores(getMetrics().getReservedVirtualCores()); - stats.setPendingContainers(getMetrics().getPendingContainers()); - stats.setAllocatedContainers(getMetrics().getAllocatedContainers()); - stats.setReservedContainers(getMetrics().getReservedContainers()); - return stats; - } - - public Map getQueueConfigurations() { - Map queueConfigurations = new HashMap<>(); - Set nodeLabels = getNodeLabelsForQueue(); - QueueResourceQuotas queueResourceQuotas = usageTracker.getQueueResourceQuotas(); - for (String nodeLabel : nodeLabels) { - QueueConfigurations queueConfiguration = - recordFactory.newRecordInstance(QueueConfigurations.class); - float capacity = queueCapacities.getCapacity(nodeLabel); - float absoluteCapacity = queueCapacities.getAbsoluteCapacity(nodeLabel); - float maxCapacity = queueCapacities.getMaximumCapacity(nodeLabel); - float absMaxCapacity = - queueCapacities.getAbsoluteMaximumCapacity(nodeLabel); - float maxAMPercentage = - queueCapacities.getMaxAMResourcePercentage(nodeLabel); - queueConfiguration.setCapacity(capacity); - queueConfiguration.setAbsoluteCapacity(absoluteCapacity); - queueConfiguration.setMaxCapacity(maxCapacity); - queueConfiguration.setAbsoluteMaxCapacity(absMaxCapacity); - queueConfiguration.setMaxAMPercentage(maxAMPercentage); - queueConfiguration.setConfiguredMinCapacity( - queueResourceQuotas.getConfiguredMinResource(nodeLabel)); - queueConfiguration.setConfiguredMaxCapacity( - queueResourceQuotas.getConfiguredMaxResource(nodeLabel)); - queueConfiguration.setEffectiveMinCapacity( - queueResourceQuotas.getEffectiveMinResource(nodeLabel)); - queueConfiguration.setEffectiveMaxCapacity( - queueResourceQuotas.getEffectiveMaxResource(nodeLabel)); - queueConfigurations.put(nodeLabel, queueConfiguration); - } - return queueConfigurations; + return CSQueueInfoProvider.getQueueInfo(this); } @Private 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/CSQueueInfoProvider.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/CSQueueInfoProvider.java new file mode 100644 index 0000000000000..8daca2bc26be6 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueInfoProvider.java @@ -0,0 +1,117 @@ +/** + * 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.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.QueueConfigurations; +import org.apache.hadoop.yarn.api.records.QueueInfo; +import org.apache.hadoop.yarn.api.records.QueueStatistics; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public final class CSQueueInfoProvider { + + private static final RecordFactory RECORD_FACTORY = + RecordFactoryProvider.getRecordFactory(null); + + private CSQueueInfoProvider() { + } + + public static QueueInfo getQueueInfo(AbstractCSQueue csQueue) { + QueueInfo queueInfo = RECORD_FACTORY.newRecordInstance(QueueInfo.class); + queueInfo.setQueueName(csQueue.getQueuePathObject().getLeafName()); + queueInfo.setQueuePath(csQueue.getQueuePathObject().getFullPath()); + queueInfo.setAccessibleNodeLabels(csQueue.getAccessibleNodeLabels()); + queueInfo.setCapacity(csQueue.getCapacity()); + queueInfo.setMaximumCapacity(csQueue.getMaximumCapacity()); + queueInfo.setQueueState(csQueue.getState()); + queueInfo.setDefaultNodeLabelExpression(csQueue.getDefaultNodeLabelExpression()); + queueInfo.setCurrentCapacity(csQueue.getUsedCapacity()); + queueInfo.setQueueStatistics(getQueueStatistics(csQueue)); + queueInfo.setPreemptionDisabled(csQueue.getPreemptionDisabled()); + queueInfo.setIntraQueuePreemptionDisabled( + csQueue.getIntraQueuePreemptionDisabled()); + queueInfo.setQueueConfigurations(getQueueConfigurations(csQueue)); + queueInfo.setWeight(csQueue.getQueueCapacities().getWeight()); + queueInfo.setMaxParallelApps(csQueue.getMaxParallelApps()); + return queueInfo; + } + + private static QueueStatistics getQueueStatistics(AbstractCSQueue csQueue) { + QueueStatistics stats = RECORD_FACTORY.newRecordInstance( + QueueStatistics.class); + CSQueueMetrics queueMetrics = csQueue.getMetrics(); + stats.setNumAppsSubmitted(queueMetrics.getAppsSubmitted()); + stats.setNumAppsRunning(queueMetrics.getAppsRunning()); + stats.setNumAppsPending(queueMetrics.getAppsPending()); + stats.setNumAppsCompleted(queueMetrics.getAppsCompleted()); + stats.setNumAppsKilled(queueMetrics.getAppsKilled()); + stats.setNumAppsFailed(queueMetrics.getAppsFailed()); + stats.setNumActiveUsers(queueMetrics.getActiveUsers()); + stats.setAvailableMemoryMB(queueMetrics.getAvailableMB()); + stats.setAllocatedMemoryMB(queueMetrics.getAllocatedMB()); + stats.setPendingMemoryMB(queueMetrics.getPendingMB()); + stats.setReservedMemoryMB(queueMetrics.getReservedMB()); + stats.setAvailableVCores(queueMetrics.getAvailableVirtualCores()); + stats.setAllocatedVCores(queueMetrics.getAllocatedVirtualCores()); + stats.setPendingVCores(queueMetrics.getPendingVirtualCores()); + stats.setReservedVCores(queueMetrics.getReservedVirtualCores()); + stats.setPendingContainers(queueMetrics.getPendingContainers()); + stats.setAllocatedContainers(queueMetrics.getAllocatedContainers()); + stats.setReservedContainers(queueMetrics.getReservedContainers()); + return stats; + } + + private static Map getQueueConfigurations(AbstractCSQueue csQueue) { + Map queueConfigurations = new HashMap<>(); + Set nodeLabels = csQueue.getNodeLabelsForQueue(); + QueueResourceQuotas queueResourceQuotas = csQueue.getQueueResourceQuotas(); + for (String nodeLabel : nodeLabels) { + QueueConfigurations queueConfiguration = + RECORD_FACTORY.newRecordInstance(QueueConfigurations.class); + QueueCapacities queueCapacities = csQueue.getQueueCapacities(); + float capacity = queueCapacities.getCapacity(nodeLabel); + float absoluteCapacity = queueCapacities.getAbsoluteCapacity(nodeLabel); + float maxCapacity = queueCapacities.getMaximumCapacity(nodeLabel); + float absMaxCapacity = + queueCapacities.getAbsoluteMaximumCapacity(nodeLabel); + float maxAMPercentage = + queueCapacities.getMaxAMResourcePercentage(nodeLabel); + queueConfiguration.setCapacity(capacity); + queueConfiguration.setAbsoluteCapacity(absoluteCapacity); + queueConfiguration.setMaxCapacity(maxCapacity); + queueConfiguration.setAbsoluteMaxCapacity(absMaxCapacity); + queueConfiguration.setMaxAMPercentage(maxAMPercentage); + queueConfiguration.setConfiguredMinCapacity( + queueResourceQuotas.getConfiguredMinResource(nodeLabel)); + queueConfiguration.setConfiguredMaxCapacity( + queueResourceQuotas.getConfiguredMaxResource(nodeLabel)); + queueConfiguration.setEffectiveMinCapacity( + queueResourceQuotas.getEffectiveMinResource(nodeLabel)); + queueConfiguration.setEffectiveMaxCapacity( + queueResourceQuotas.getEffectiveMaxResource(nodeLabel)); + queueConfigurations.put(nodeLabel, queueConfiguration); + } + return queueConfigurations; + } +} From e570e75569b6f1533db082f109d84030c6ba4fcd Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Fri, 2 Dec 2022 05:20:21 +0800 Subject: [PATCH 10/45] YARN-11158. Support (Create/Renew/Cancel) DelegationToken API's for Federation. (#5104) --- .../yarn/server/router/RouterMetrics.java | 97 ++++++++++- .../yarn/server/router/RouterServerUtil.java | 24 +++ .../clientrm/FederationClientInterceptor.java | 97 ++++++++++- .../yarn/server/router/TestRouterMetrics.java | 80 ++++++++- .../TestFederationClientInterceptor.java | 156 +++++++++++++++++- .../TestableFederationClientInterceptor.java | 28 ++++ 6 files changed, 468 insertions(+), 14 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/RouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java index b03aeda38b44d..31d838d1b3ef7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java @@ -127,6 +127,12 @@ public final class RouterMetrics { private MutableGaugeInt numGetRMNodeLabelsFailedRetrieved; @Metric("# of checkUserAccessToQueue failed to be retrieved") private MutableGaugeInt numCheckUserAccessToQueueFailedRetrieved; + @Metric("# of getDelegationToken failed to be retrieved") + private MutableGaugeInt numGetDelegationTokenFailedRetrieved; + @Metric("# of renewDelegationToken failed to be retrieved") + private MutableGaugeInt numRenewDelegationTokenFailedRetrieved; + @Metric("# of renewDelegationToken failed to be retrieved") + private MutableGaugeInt numCancelDelegationTokenFailedRetrieved; // Aggregate metrics are shared, and don't have to be looked up per call @Metric("Total number of successful Submitted apps and latency(ms)") @@ -215,6 +221,12 @@ public final class RouterMetrics { private MutableRate totalSucceededGetRMNodeLabelsRetrieved; @Metric("Total number of successful Retrieved CheckUserAccessToQueue and latency(ms)") private MutableRate totalSucceededCheckUserAccessToQueueRetrieved; + @Metric("Total number of successful Retrieved GetDelegationToken and latency(ms)") + private MutableRate totalSucceededGetDelegationTokenRetrieved; + @Metric("Total number of successful Retrieved RenewDelegationToken and latency(ms)") + private MutableRate totalSucceededRenewDelegationTokenRetrieved; + @Metric("Total number of successful Retrieved CancelDelegationToken and latency(ms)") + private MutableRate totalSucceededCancelDelegationTokenRetrieved; /** * Provide quantile counters for all latencies. @@ -262,6 +274,9 @@ public final class RouterMetrics { private MutableQuantiles getRefreshQueuesLatency; private MutableQuantiles getRMNodeLabelsLatency; private MutableQuantiles checkUserAccessToQueueLatency; + private MutableQuantiles getDelegationTokenLatency; + private MutableQuantiles renewDelegationTokenLatency; + private MutableQuantiles cancelDelegationTokenLatency; private static volatile RouterMetrics instance = null; private static MetricsRegistry registry; @@ -423,6 +438,15 @@ private RouterMetrics() { checkUserAccessToQueueLatency = registry.newQuantiles("checkUserAccessToQueueLatency", "latency of get apptimeouts timeouts", "ops", "latency", 10); + + getDelegationTokenLatency = registry.newQuantiles("getDelegationTokenLatency", + "latency of get delegation token timeouts", "ops", "latency", 10); + + renewDelegationTokenLatency = registry.newQuantiles("renewDelegationTokenLatency", + "latency of renew delegation token timeouts", "ops", "latency", 10); + + cancelDelegationTokenLatency = registry.newQuantiles("cancelDelegationTokenLatency", + "latency of cancel delegation token timeouts", "ops", "latency", 10); } public static RouterMetrics getMetrics() { @@ -655,10 +679,25 @@ public long getNumSucceededGetRMNodeLabelsRetrieved() { } @VisibleForTesting - public long getNumSucceededCheckUserAccessToQueueRetrievedRetrieved() { + public long getNumSucceededCheckUserAccessToQueueRetrieved() { return totalSucceededCheckUserAccessToQueueRetrieved.lastStat().numSamples(); } + @VisibleForTesting + public long getNumSucceededGetDelegationTokenRetrieved() { + return totalSucceededGetDelegationTokenRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededRenewDelegationTokenRetrieved() { + return totalSucceededRenewDelegationTokenRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededCancelDelegationTokenRetrieved() { + return totalSucceededCancelDelegationTokenRetrieved.lastStat().numSamples(); + } + @VisibleForTesting public double getLatencySucceededAppsCreated() { return totalSucceededAppsCreated.lastStat().mean(); @@ -874,6 +913,21 @@ public double getLatencySucceededCheckUserAccessToQueueRetrieved() { return totalSucceededCheckUserAccessToQueueRetrieved.lastStat().mean(); } + @VisibleForTesting + public double getLatencySucceededGetDelegationTokenRetrieved() { + return totalSucceededGetDelegationTokenRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededRenewDelegationTokenRetrieved() { + return totalSucceededRenewDelegationTokenRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededCancelDelegationTokenRetrieved() { + return totalSucceededCancelDelegationTokenRetrieved.lastStat().mean(); + } + @VisibleForTesting public int getAppsFailedCreated() { return numAppsFailedCreated.value(); @@ -1068,6 +1122,18 @@ public int getCheckUserAccessToQueueFailedRetrieved() { return numCheckUserAccessToQueueFailedRetrieved.value(); } + public int getDelegationTokenFailedRetrieved() { + return numGetDelegationTokenFailedRetrieved.value(); + } + + public int getRenewDelegationTokenFailedRetrieved() { + return numRenewDelegationTokenFailedRetrieved.value(); + } + + public int getCancelDelegationTokenFailedRetrieved() { + return numCancelDelegationTokenFailedRetrieved.value(); + } + public void succeededAppsCreated(long duration) { totalSucceededAppsCreated.add(duration); getNewApplicationLatency.add(duration); @@ -1283,6 +1349,21 @@ public void succeededCheckUserAccessToQueueRetrieved(long duration) { checkUserAccessToQueueLatency.add(duration); } + public void succeededGetDelegationTokenRetrieved(long duration) { + totalSucceededGetDelegationTokenRetrieved.add(duration); + getDelegationTokenLatency.add(duration); + } + + public void succeededRenewDelegationTokenRetrieved(long duration) { + totalSucceededRenewDelegationTokenRetrieved.add(duration); + renewDelegationTokenLatency.add(duration); + } + + public void succeededCancelDelegationTokenRetrieved(long duration) { + totalSucceededCancelDelegationTokenRetrieved.add(duration); + cancelDelegationTokenLatency.add(duration); + } + public void incrAppsFailedCreated() { numAppsFailedCreated.incr(); } @@ -1454,4 +1535,16 @@ public void incrGetRMNodeLabelsFailedRetrieved() { public void incrCheckUserAccessToQueueFailedRetrieved() { numCheckUserAccessToQueueFailedRetrieved.incr(); } -} \ No newline at end of file + + public void incrGetDelegationTokenFailedRetrieved() { + numGetDelegationTokenFailedRetrieved.incr(); + } + + public void incrRenewDelegationTokenFailedRetrieved() { + numRenewDelegationTokenFailedRetrieved.incr(); + } + + public void incrCancelDelegationTokenFailedRetrieved() { + numCancelDelegationTokenFailedRetrieved.incr(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterServerUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterServerUtil.java index 93818229dd126..8c880f25ddb76 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterServerUtil.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterServerUtil.java @@ -24,10 +24,12 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,6 +38,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.EnumSet; import java.io.IOException; /** @@ -470,6 +473,27 @@ public static void validateContainerId(String containerId) } } + public static boolean isAllowedDelegationTokenOp() throws IOException { + if (UserGroupInformation.isSecurityEnabled()) { + return EnumSet.of(UserGroupInformation.AuthenticationMethod.KERBEROS, + UserGroupInformation.AuthenticationMethod.KERBEROS_SSL, + UserGroupInformation.AuthenticationMethod.CERTIFICATE) + .contains(UserGroupInformation.getCurrentUser() + .getRealAuthenticationMethod()); + } else { + return true; + } + } + + public static String getRenewerForToken(Token token) + throws IOException { + UserGroupInformation user = UserGroupInformation.getCurrentUser(); + UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); + // we can always renew our own tokens + return loginUser.getUserName().equals(user.getUserName()) + ? token.decodeIdentifier().getRenewer().toString() : user.getShortUserName(); + } + public static UserGroupInformation setupUser(final String userName) { UserGroupInformation user = null; try { 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/FederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java index cf457c70771a9..a50ea5bc42352 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java @@ -20,6 +20,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.io.Text; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.IOException; import java.lang.reflect.Method; @@ -40,7 +41,6 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.security.UserGroupInformation; @@ -118,9 +118,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.ReservationId; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.yarn.server.utils.BuilderUtils; + import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil; import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils; import org.apache.hadoop.yarn.server.federation.policies.RouterPolicyFacade; @@ -136,6 +140,7 @@ import org.apache.hadoop.yarn.server.router.RouterServerUtil; import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.MonotonicClock; +import org.apache.hadoop.yarn.util.Records; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1392,19 +1397,103 @@ public GetContainersResponse getContainers(GetContainersRequest request) @Override public GetDelegationTokenResponse getDelegationToken( GetDelegationTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + if (request == null || request.getRenewer() == null) { + routerMetrics.incrGetDelegationTokenFailedRetrieved(); + RouterServerUtil.logAndThrowException( + "Missing getDelegationToken request or Renewer.", null); + } + + try { + // Verify that the connection is kerberos authenticated + if (!RouterServerUtil.isAllowedDelegationTokenOp()) { + routerMetrics.incrGetDelegationTokenFailedRetrieved(); + throw new IOException( + "Delegation Token can be issued only with kerberos authentication."); + } + + long startTime = clock.getTime(); + UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + Text owner = new Text(ugi.getUserName()); + Text realUser = null; + if (ugi.getRealUser() != null) { + realUser = new Text(ugi.getRealUser().getUserName()); + } + + RMDelegationTokenIdentifier tokenIdentifier = + new RMDelegationTokenIdentifier(owner, new Text(request.getRenewer()), realUser); + Token realRMDToken = + new Token<>(tokenIdentifier, this.getTokenSecretManager()); + + org.apache.hadoop.yarn.api.records.Token routerRMDTToken = + BuilderUtils.newDelegationToken(realRMDToken.getIdentifier(), + realRMDToken.getKind().toString(), + realRMDToken.getPassword(), realRMDToken.getService().toString()); + + long stopTime = clock.getTime(); + routerMetrics.succeededGetDelegationTokenRetrieved((stopTime - startTime)); + return GetDelegationTokenResponse.newInstance(routerRMDTToken); + } catch(IOException e) { + routerMetrics.incrGetDelegationTokenFailedRetrieved(); + throw new YarnException(e); + } } @Override public RenewDelegationTokenResponse renewDelegationToken( RenewDelegationTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + try { + + if (!RouterServerUtil.isAllowedDelegationTokenOp()) { + routerMetrics.incrRenewDelegationTokenFailedRetrieved(); + throw new IOException( + "Delegation Token can be renewed only with kerberos authentication"); + } + + long startTime = clock.getTime(); + org.apache.hadoop.yarn.api.records.Token protoToken = request.getDelegationToken(); + Token token = new Token<>( + protoToken.getIdentifier().array(), protoToken.getPassword().array(), + new Text(protoToken.getKind()), new Text(protoToken.getService())); + String user = RouterServerUtil.getRenewerForToken(token); + long nextExpTime = this.getTokenSecretManager().renewToken(token, user); + RenewDelegationTokenResponse renewResponse = + Records.newRecord(RenewDelegationTokenResponse.class); + renewResponse.setNextExpirationTime(nextExpTime); + long stopTime = clock.getTime(); + routerMetrics.succeededRenewDelegationTokenRetrieved((stopTime - startTime)); + return renewResponse; + + } catch (IOException e) { + routerMetrics.incrRenewDelegationTokenFailedRetrieved(); + throw new YarnException(e); + } } @Override public CancelDelegationTokenResponse cancelDelegationToken( CancelDelegationTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + try { + if (!RouterServerUtil.isAllowedDelegationTokenOp()) { + routerMetrics.incrCancelDelegationTokenFailedRetrieved(); + throw new IOException( + "Delegation Token can be cancelled only with kerberos authentication"); + } + + long startTime = clock.getTime(); + org.apache.hadoop.yarn.api.records.Token protoToken = request.getDelegationToken(); + Token token = new Token<>( + protoToken.getIdentifier().array(), protoToken.getPassword().array(), + new Text(protoToken.getKind()), new Text(protoToken.getService())); + String user = UserGroupInformation.getCurrentUser().getUserName(); + this.getTokenSecretManager().cancelToken(token, user); + long stopTime = clock.getTime(); + routerMetrics.succeededCancelDelegationTokenRetrieved((stopTime - startTime)); + return Records.newRecord(CancelDelegationTokenResponse.class); + } catch (IOException e) { + routerMetrics.incrCancelDelegationTokenFailedRetrieved(); + throw new YarnException(e); + } } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java index 828e5c69f35ef..9d5aeab5c6d80 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java @@ -519,10 +519,20 @@ public void getRMNodeLabelsFailed() { metrics.incrGetRMNodeLabelsFailedRetrieved(); } - public void getCheckUserAccessToQueueRetrieved() { - LOG.info("Mocked: failed checkUserAccessToQueueRetrieved call"); + public void getCheckUserAccessToQueueFailed() { + LOG.info("Mocked: failed checkUserAccessToQueue call"); metrics.incrCheckUserAccessToQueueFailedRetrieved(); } + + public void getDelegationTokenFailed() { + LOG.info("Mocked: failed getDelegationToken call"); + metrics.incrGetDelegationTokenFailedRetrieved(); + } + + public void getRenewDelegationTokenFailed() { + LOG.info("Mocked: failed renewDelegationToken call"); + metrics.incrRenewDelegationTokenFailedRetrieved(); + } } // Records successes for all calls @@ -743,6 +753,16 @@ public void getCheckUserAccessToQueueRetrieved(long duration) { LOG.info("Mocked: successful CheckUserAccessToQueue call with duration {}", duration); metrics.succeededCheckUserAccessToQueueRetrieved(duration); } + + public void getGetDelegationTokenRetrieved(long duration) { + LOG.info("Mocked: successful GetDelegationToken call with duration {}", duration); + metrics.succeededGetDelegationTokenRetrieved(duration); + } + + public void getRenewDelegationTokenRetrieved(long duration) { + LOG.info("Mocked: successful RenewDelegationToken call with duration {}", duration); + metrics.succeededRenewDelegationTokenRetrieved(duration); + } } @Test @@ -1510,16 +1530,16 @@ public void testGetRMNodeLabelsRetrievedFailed() { } @Test - public void testCheckUserAccessToQueueRetrievedRetrieved() { - long totalGoodBefore = metrics.getNumSucceededCheckUserAccessToQueueRetrievedRetrieved(); + public void testCheckUserAccessToQueueRetrieved() { + long totalGoodBefore = metrics.getNumSucceededCheckUserAccessToQueueRetrieved(); goodSubCluster.getCheckUserAccessToQueueRetrieved(150); Assert.assertEquals(totalGoodBefore + 1, - metrics.getNumSucceededCheckUserAccessToQueueRetrievedRetrieved()); + metrics.getNumSucceededCheckUserAccessToQueueRetrieved()); Assert.assertEquals(150, metrics.getLatencySucceededCheckUserAccessToQueueRetrieved(), ASSERT_DOUBLE_DELTA); goodSubCluster.getCheckUserAccessToQueueRetrieved(300); Assert.assertEquals(totalGoodBefore + 2, - metrics.getNumSucceededCheckUserAccessToQueueRetrievedRetrieved()); + metrics.getNumSucceededCheckUserAccessToQueueRetrieved()); Assert.assertEquals(225, metrics.getLatencySucceededCheckUserAccessToQueueRetrieved(), ASSERT_DOUBLE_DELTA); } @@ -1527,8 +1547,54 @@ public void testCheckUserAccessToQueueRetrievedRetrieved() { @Test public void testCheckUserAccessToQueueRetrievedFailed() { long totalBadBefore = metrics.getCheckUserAccessToQueueFailedRetrieved(); - badSubCluster.getCheckUserAccessToQueueRetrieved(); + badSubCluster.getCheckUserAccessToQueueFailed(); Assert.assertEquals(totalBadBefore + 1, metrics.getCheckUserAccessToQueueFailedRetrieved()); } + + @Test + public void testGetDelegationTokenRetrieved() { + long totalGoodBefore = metrics.getNumSucceededGetDelegationTokenRetrieved(); + goodSubCluster.getGetDelegationTokenRetrieved(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededGetDelegationTokenRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededGetDelegationTokenRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getGetDelegationTokenRetrieved(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededGetDelegationTokenRetrieved()); + Assert.assertEquals(225, + metrics.getLatencySucceededGetDelegationTokenRetrieved(), ASSERT_DOUBLE_DELTA); + } + + @Test + public void testGetDelegationTokenRetrievedFailed() { + long totalBadBefore = metrics.getDelegationTokenFailedRetrieved(); + badSubCluster.getDelegationTokenFailed(); + Assert.assertEquals(totalBadBefore + 1, + metrics.getDelegationTokenFailedRetrieved()); + } + + @Test + public void testRenewDelegationTokenRetrieved() { + long totalGoodBefore = metrics.getNumSucceededRenewDelegationTokenRetrieved(); + goodSubCluster.getRenewDelegationTokenRetrieved(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededRenewDelegationTokenRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededRenewDelegationTokenRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getRenewDelegationTokenRetrieved(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededRenewDelegationTokenRetrieved()); + Assert.assertEquals(225, + metrics.getLatencySucceededRenewDelegationTokenRetrieved(), ASSERT_DOUBLE_DELTA); + } + + @Test + public void testRenewDelegationTokenRetrievedFailed() { + long totalBadBefore = metrics.getRenewDelegationTokenFailedRetrieved(); + badSubCluster.getRenewDelegationTokenFailed(); + Assert.assertEquals(totalBadBefore + 1, + metrics.getRenewDelegationTokenFailedRetrieved()); + } } \ No newline at end of file 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/TestFederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java index 38f571c288b5d..2488fc73b07b3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.router.clientrm; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import java.io.IOException; import java.util.ArrayList; @@ -32,6 +33,7 @@ import java.util.Arrays; import java.util.Collection; +import org.apache.hadoop.io.Text; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.Time; @@ -100,6 +102,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse; import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest; import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; @@ -123,10 +131,13 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition; import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter; import org.apache.hadoop.yarn.api.records.ReservationRequests; +import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.policies.manager.UniformBroadcastPolicyManager; import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore; +import org.apache.hadoop.yarn.server.federation.store.records.RouterRMDTSecretManagerState; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade; @@ -138,6 +149,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; +import org.apache.hadoop.yarn.server.router.security.RouterDelegationTokenSecretManager; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Times; import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.Assert; @@ -170,7 +184,7 @@ public class TestFederationClientInterceptor extends BaseRouterClientRMTest { private final static long DEFAULT_DURATION = 10 * 60 * 1000; @Override - public void setUp() { + public void setUp() throws IOException { super.setUpConfig(); interceptor = new TestableFederationClientInterceptor(); @@ -181,6 +195,11 @@ public void setUp() { interceptor.setConf(this.getConf()); interceptor.init(user); + RouterDelegationTokenSecretManager tokenSecretManager = + interceptor.createRouterRMDelegationTokenSecretManager(this.getConf()); + + tokenSecretManager.startThreads(); + interceptor.setTokenSecretManager(tokenSecretManager); subClusters = new ArrayList<>(); @@ -230,6 +249,7 @@ protected YarnConfiguration createConfiguration() { conf.setInt("yarn.scheduler.maximum-allocation-mb", 100 * 1024); conf.setInt("yarn.scheduler.maximum-allocation-vcores", 100); + conf.setBoolean("hadoop.security.authentication", true); return conf; } @@ -1550,4 +1570,138 @@ public void testGetNumMaxThreads() { int minThreads2 = interceptor.getNumMaxThreads(this.getConf()); Assert.assertEquals(8, minThreads2); } + + @Test + public void testGetDelegationToken() throws IOException, YarnException { + + // We design such a unit test to check + // that the execution of the GetDelegationToken method is as expected. + // + // 1. Apply for a DelegationToken for renewer1, + // the Router returns the DelegationToken of the user, and the KIND of the token is + // RM_DELEGATION_TOKEN + // + // 2. We maintain the compatibility with RMDelegationTokenIdentifier, + // we can serialize the token into RMDelegationTokenIdentifier. + // + // 3. We can get the issueDate, and compare the data in the StateStore, + // the data should be consistent. + + // Step1. We apply for DelegationToken for renewer1 + // Both response & delegationToken cannot be empty + GetDelegationTokenRequest request = mock(GetDelegationTokenRequest.class); + when(request.getRenewer()).thenReturn("renewer1"); + GetDelegationTokenResponse response = interceptor.getDelegationToken(request); + Assert.assertNotNull(response); + Token delegationToken = response.getRMDelegationToken(); + Assert.assertNotNull(delegationToken); + Assert.assertEquals("RM_DELEGATION_TOKEN", delegationToken.getKind()); + + // Step2. Serialize the returned Token as RMDelegationTokenIdentifier. + org.apache.hadoop.security.token.Token token = + ConverterUtils.convertFromYarn(delegationToken, (Text) null); + RMDelegationTokenIdentifier rMDelegationTokenIdentifier = token.decodeIdentifier(); + Assert.assertNotNull(rMDelegationTokenIdentifier); + + // Step3. Verify the returned data of the token. + String renewer = rMDelegationTokenIdentifier.getRenewer().toString(); + long issueDate = rMDelegationTokenIdentifier.getIssueDate(); + long maxDate = rMDelegationTokenIdentifier.getMaxDate(); + Assert.assertEquals("renewer1", renewer); + + long tokenMaxLifetime = this.getConf().getLong( + YarnConfiguration.RM_DELEGATION_TOKEN_MAX_LIFETIME_KEY, + YarnConfiguration.RM_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT); + Assert.assertEquals(issueDate + tokenMaxLifetime, maxDate); + + RouterRMDTSecretManagerState managerState = stateStore.getRouterRMSecretManagerState(); + Assert.assertNotNull(managerState); + + Map delegationTokenState = managerState.getTokenState(); + Assert.assertNotNull(delegationTokenState); + Assert.assertTrue(delegationTokenState.containsKey(rMDelegationTokenIdentifier)); + + long tokenRenewInterval = this.getConf().getLong( + YarnConfiguration.RM_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, + YarnConfiguration.RM_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT); + long renewDate = delegationTokenState.get(rMDelegationTokenIdentifier); + Assert.assertEquals(issueDate + tokenRenewInterval, renewDate); + } + + @Test + public void testRenewDelegationToken() throws IOException, YarnException { + + // We design such a unit test to check + // that the execution of the GetDelegationToken method is as expected + // 1. Call GetDelegationToken to apply for delegationToken. + // 2. Call renewDelegationToken to refresh delegationToken. + // By looking at the code of AbstractDelegationTokenSecretManager#renewToken, + // we know that renewTime is calculated as Math.min(id.getMaxDate(), now + tokenRenewInterval) + // so renewTime will be less than or equal to maxDate. + // 3. We will compare whether the expirationTime returned to the + // client is consistent with the renewDate in the stateStore. + + // Step1. Call GetDelegationToken to apply for delegationToken. + GetDelegationTokenRequest request = mock(GetDelegationTokenRequest.class); + when(request.getRenewer()).thenReturn("renewer2"); + GetDelegationTokenResponse response = interceptor.getDelegationToken(request); + Assert.assertNotNull(response); + Token delegationToken = response.getRMDelegationToken(); + + org.apache.hadoop.security.token.Token token = + ConverterUtils.convertFromYarn(delegationToken, (Text) null); + RMDelegationTokenIdentifier rMDelegationTokenIdentifier = token.decodeIdentifier(); + String renewer = rMDelegationTokenIdentifier.getRenewer().toString(); + long maxDate = rMDelegationTokenIdentifier.getMaxDate(); + Assert.assertEquals("renewer2", renewer); + + // Step2. Call renewDelegationToken to refresh delegationToken. + RenewDelegationTokenRequest renewRequest = Records.newRecord(RenewDelegationTokenRequest.class); + renewRequest.setDelegationToken(delegationToken); + RenewDelegationTokenResponse renewResponse = interceptor.renewDelegationToken(renewRequest); + Assert.assertNotNull(renewResponse); + + long expDate = renewResponse.getNextExpirationTime(); + Assert.assertTrue(expDate <= maxDate); + + // Step3. Compare whether the expirationTime returned to + // the client is consistent with the renewDate in the stateStore + RouterRMDTSecretManagerState managerState = stateStore.getRouterRMSecretManagerState(); + Map delegationTokenState = managerState.getTokenState(); + Assert.assertNotNull(delegationTokenState); + Assert.assertTrue(delegationTokenState.containsKey(rMDelegationTokenIdentifier)); + long renewDate = delegationTokenState.get(rMDelegationTokenIdentifier); + Assert.assertEquals(expDate, renewDate); + } + + @Test + public void testCancelDelegationToken() throws IOException, YarnException { + + // We design such a unit test to check + // that the execution of the CancelDelegationToken method is as expected + // 1. Call GetDelegationToken to apply for delegationToken. + // 2. Call CancelDelegationToken to cancel delegationToken. + // 3. Query the data in the StateStore and confirm that the Delegation has been deleted. + + // Step1. Call GetDelegationToken to apply for delegationToken. + GetDelegationTokenRequest request = mock(GetDelegationTokenRequest.class); + when(request.getRenewer()).thenReturn("renewer3"); + GetDelegationTokenResponse response = interceptor.getDelegationToken(request); + Assert.assertNotNull(response); + Token delegationToken = response.getRMDelegationToken(); + + // Step2. Call CancelDelegationToken to cancel delegationToken. + CancelDelegationTokenRequest cancelTokenRequest = + CancelDelegationTokenRequest.newInstance(delegationToken); + CancelDelegationTokenResponse cancelTokenResponse = + interceptor.cancelDelegationToken(cancelTokenRequest); + Assert.assertNotNull(cancelTokenResponse); + + // Step3. Query the data in the StateStore and confirm that the Delegation has been deleted. + // At this point, the size of delegationTokenState should be 0. + RouterRMDTSecretManagerState managerState = stateStore.getRouterRMSecretManagerState(); + Map delegationTokenState = managerState.getTokenState(); + Assert.assertNotNull(delegationTokenState); + Assert.assertEquals(0, delegationTokenState.size()); + } } 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/TestableFederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestableFederationClientInterceptor.java index 8279899e387ad..c8c647a0d22e5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestableFederationClientInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestableFederationClientInterceptor.java @@ -28,8 +28,10 @@ import java.util.Map; import java.util.HashMap; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; @@ -38,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.NodeAttribute; import org.apache.hadoop.yarn.api.records.NodeAttributeType; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; @@ -51,6 +54,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager; +import org.apache.hadoop.yarn.server.router.security.RouterDelegationTokenSecretManager; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.junit.Assert; import org.slf4j.Logger; @@ -216,4 +220,28 @@ public void shutdown() { mockRMs.clear(); super.shutdown(); } + + public RouterDelegationTokenSecretManager createRouterRMDelegationTokenSecretManager( + Configuration conf) { + + long secretKeyInterval = conf.getLong( + YarnConfiguration.RM_DELEGATION_KEY_UPDATE_INTERVAL_KEY, + YarnConfiguration.RM_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT); + + long tokenMaxLifetime = conf.getLong( + YarnConfiguration.RM_DELEGATION_TOKEN_MAX_LIFETIME_KEY, + YarnConfiguration.RM_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT); + + long tokenRenewInterval = conf.getLong( + YarnConfiguration.RM_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, + YarnConfiguration.RM_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT); + + long removeScanInterval = conf.getTimeDuration( + YarnConfiguration.RM_DELEGATION_TOKEN_REMOVE_SCAN_INTERVAL_KEY, + YarnConfiguration.RM_DELEGATION_TOKEN_REMOVE_SCAN_INTERVAL_DEFAULT, + TimeUnit.MILLISECONDS); + + return new RouterDelegationTokenSecretManager(secretKeyInterval, + tokenMaxLifetime, tokenRenewInterval, removeScanInterval); + } } \ No newline at end of file From c47384f3aec066b1c30f6a93d15467b55a83c3aa Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sat, 3 Dec 2022 02:56:17 +0800 Subject: [PATCH 11/45] YARN-11381. Fix hadoop-yarn-common module Java Doc Errors. (#5179) --- .../hadoop/yarn/api/impl/pb/client/package-info.java | 6 +++--- .../hadoop/yarn/api/impl/pb/service/package-info.java | 6 +++--- .../org/apache/hadoop/yarn/api/pb/package-info.java | 6 +++--- .../yarn/api/protocolrecords/impl/pb/package-info.java | 6 +++--- .../hadoop/yarn/api/records/impl/pb/package-info.java | 6 +++--- .../apache/hadoop/yarn/api/resource/package-info.java | 6 +++--- .../org/apache/hadoop/yarn/event/package-info.java | 6 +++--- .../hadoop/yarn/factories/impl/pb/package-info.java | 6 +++--- .../java/org/apache/hadoop/yarn/ipc/package-info.java | 6 +++--- .../logaggregation/filecontroller/package-info.java | 6 +++--- .../hadoop/yarn/logaggregation/package-info.java | 6 +++--- .../org/apache/hadoop/yarn/metrics/package-info.java | 6 +++--- .../hadoop/yarn/nodelabels/store/op/package-info.java | 6 +++--- .../hadoop/yarn/nodelabels/store/package-info.java | 6 +++--- .../main/java/org/apache/hadoop/yarn/package-info.java | 6 +++--- .../hadoop/yarn/security/client/package-info.java | 6 +++--- .../org/apache/hadoop/yarn/security/package-info.java | 6 +++--- .../api/protocolrecords/impl/pb/package-info.java | 6 +++--- .../hadoop/yarn/server/metrics/package-info.java | 6 +++--- .../hadoop/yarn/server/security/package-info.java | 6 +++--- .../org/apache/hadoop/yarn/state/package-info.java | 6 +++--- .../java/org/apache/hadoop/yarn/util/package-info.java | 6 +++--- .../apache/hadoop/yarn/util/timeline/package-info.java | 6 +++--- .../apache/hadoop/yarn/webapp/dao/package-info.java | 10 +++++----- .../hadoop/yarn/webapp/example/package-info.java | 6 +++--- .../hadoop/yarn/webapp/hamlet2/package-info.java | 6 +++--- .../apache/hadoop/yarn/webapp/log/package-info.java | 6 +++--- .../org/apache/hadoop/yarn/webapp/package-info.java | 6 +++--- .../apache/hadoop/yarn/webapp/view/package-info.java | 6 +++--- 29 files changed, 89 insertions(+), 89 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/package-info.java index a4349b2295768..6ded96f0e7e54 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,6 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.api.impl.pb.client; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/package-info.java index 1d3d435385f30..730ac9c628b40 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,6 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.api.impl.pb.service; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java index 18da80f4869b1..167e078643354 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -18,6 +18,6 @@ /** * API related to protobuf objects that are not backed by PBImpl classes. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.api.pb; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/package-info.java index 4b29e4f740ea2..e783448d2b40e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.api.protocolrecords.impl.pb; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/package-info.java index 2571db8e8dc5e..81334c0591dc0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.api.records.impl.pb; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java index 660dc02d45c41..e1bd6dc17378e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -18,6 +18,6 @@ /** * API related to resources. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.api.resource; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/package-info.java index c410160f84133..a9129dc145612 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn.event; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Public; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/package-info.java index aae0b4896fc37..ea47367e88a5c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/impl/pb/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.factories.impl.pb; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/package-info.java index eec93feb53858..62d5e02797c1e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.LimitedPrivate({ "MapReduce", "YARN" }) +@LimitedPrivate({ "MapReduce", "YARN" }) package org.apache.hadoop.yarn.ipc; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/package-info.java index cad238a9a4205..1b53de6d1d65c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn.logaggregation.filecontroller; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Public; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/package-info.java index 90dce80e63ca9..c1f1379ee80eb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn.logaggregation; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Public; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/metrics/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/metrics/package-info.java index 5df20b1bf88d1..7497e43331579 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/metrics/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/metrics/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -18,6 +18,6 @@ /** * Provides common metrics (available, allocated) for custom resources. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.metrics; -import org.apache.hadoop.classification.InterfaceAudience; \ No newline at end of file +import org.apache.hadoop.classification.InterfaceAudience.Private; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/package-info.java index f6fb3d3ecaa74..3175b2c57888e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.nodelabels.store.op; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/package-info.java index 0444807071ab5..d1546cf2b68ac 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.nodelabels.store; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/package-info.java index 99ab44ce42d29..69aeda005df1a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Public; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/package-info.java index 7aa12fd75fd8a..e44f8a270c0ec 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn.security.client; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Public; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/package-info.java index f09a1f32011bd..38a174db369cd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn.security; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Public; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/package-info.java index e9c394bbd78c4..72e1d777658cc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/metrics/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/metrics/package-info.java index b7ef6fb924fee..c293580f1968e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/metrics/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/metrics/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -16,7 +16,7 @@ * limitations under the License. */ /** Yarn Common Metrics package. **/ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.server.metrics; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/package-info.java index 8aa55050c4d91..4fc7a7df40da3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.server.security; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/package-info.java index c62c5015e938e..4cdee806bbd6a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn.state; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Public; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/package-info.java index 8a22bfc1c5bed..0e9d95b9ff58e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn.util; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Public; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/package-info.java index 5c18a55518d43..98fe0783694cb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn.util.timeline; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Public; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/package-info.java index aec6762772992..9e2c2316c77e7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -19,9 +19,9 @@ /** * Data structures for scheduler configuration mutation info. */ -@InterfaceAudience.Private -@InterfaceStability.Unstable +@Private +@Unstable package org.apache.hadoop.yarn.webapp.dao; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/package-info.java index 0f0936b7cd3ef..e164a0219fbb7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"}) +@LimitedPrivate({"YARN", "MapReduce"}) package org.apache.hadoop.yarn.webapp.example; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/package-info.java index 64a8447e024e7..54aa474bf016a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -21,7 +21,7 @@ * The old package is using _ as a one-character identifier, * which is banned from JDK9. */ -@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"}) +@LimitedPrivate({"YARN", "MapReduce"}) package org.apache.hadoop.yarn.webapp.hamlet2; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/package-info.java index 1d64404cc3623..23c4328ad8d17 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"}) +@LimitedPrivate({"YARN", "MapReduce"}) package org.apache.hadoop.yarn.webapp.log; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/package-info.java index 342f28df283eb..4d179099eb7a7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"}) +@LimitedPrivate({"YARN", "MapReduce"}) package org.apache.hadoop.yarn.webapp; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/package-info.java index c1f92eb7075f9..a22c1b3601c34 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"}) +@LimitedPrivate({"YARN", "MapReduce"}) package org.apache.hadoop.yarn.webapp.view; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; From 2f146d3590a2413f469a1148f5e32299fa13b593 Mon Sep 17 00:00:00 2001 From: dingshun3016 Date: Mon, 5 Dec 2022 15:34:51 +0800 Subject: [PATCH 12/45] HDFS-16809. EC striped block is not sufficient when doing in maintenance. (#5050) --- .../server/blockmanagement/BlockManager.java | 2 +- .../blockmanagement/DatanodeAdminManager.java | 3 +- .../hdfs/TestMaintenanceWithStriped.java | 267 ++++++++++++++++++ 3 files changed, 269 insertions(+), 3 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMaintenanceWithStriped.java diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index 9836baa004058..3b0bbe847bfb1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -1117,7 +1117,7 @@ public short getMinReplicationToBeInMaintenance() { return minReplicationToBeInMaintenance; } - private short getMinMaintenanceStorageNum(BlockInfo block) { + short getMinMaintenanceStorageNum(BlockInfo block) { if (block.isStriped()) { return ((BlockInfoStriped) block).getRealDataBlockNum(); } else { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java index aec413f5903e3..af207a843fd27 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java @@ -321,8 +321,7 @@ protected boolean isSufficient(BlockInfo block, BlockCollection bc, } } } - if (isMaintenance - && numLive >= blockManager.getMinReplicationToBeInMaintenance()) { + if (isMaintenance && numLive >= blockManager.getMinMaintenanceStorageNum(block)) { return true; } return false; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMaintenanceWithStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMaintenanceWithStriped.java new file mode 100644 index 0000000000000..2e17b9681b790 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMaintenanceWithStriped.java @@ -0,0 +1,267 @@ +/** + * 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; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeoutException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileChecksum; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.hdfs.server.namenode.INodeFile; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; +import org.apache.hadoop.hdfs.util.HostsFileWriter; +import org.apache.hadoop.test.GenericTestUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class tests the in maintenance of datanode with striped blocks. + */ +public class TestMaintenanceWithStriped { + private static final Logger LOG = + LoggerFactory.getLogger(TestMaintenanceWithStriped.class); + + // heartbeat interval in seconds + private static final int HEARTBEAT_INTERVAL = 1; + // block report in msec + private static final int BLOCKREPORT_INTERVAL_MSEC = 1000; + // replication interval + private static final int NAMENODE_REPLICATION_INTERVAL = 1; + + private Configuration conf; + private MiniDFSCluster cluster; + private DistributedFileSystem dfs; + private final ErasureCodingPolicy ecPolicy = + StripedFileTestUtil.getDefaultECPolicy(); + private int numDNs; + private final int cellSize = ecPolicy.getCellSize(); + private final int dataBlocks = ecPolicy.getNumDataUnits(); + private final int parityBlocks = ecPolicy.getNumParityUnits(); + private final int blockSize = cellSize * 4; + private final int blockGroupSize = blockSize * dataBlocks; + private final Path ecDir = new Path("/" + this.getClass().getSimpleName()); + private HostsFileWriter hostsFileWriter; + private boolean useCombinedHostFileManager = true; + + private FSNamesystem fsn; + private BlockManager bm; + + protected Configuration createConfiguration() { + return new HdfsConfiguration(); + } + + @Before + public void setup() throws IOException { + // Set up the hosts/exclude files. + hostsFileWriter = new HostsFileWriter(); + conf = createConfiguration(); + if (useCombinedHostFileManager) { + conf.setClass(DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY, + CombinedHostFileManager.class, HostConfigManager.class); + } + hostsFileWriter.initialize(conf, "temp/admin"); + + + conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, + 2000); + conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1); + conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, + BLOCKREPORT_INTERVAL_MSEC); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, + 4); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, + NAMENODE_REPLICATION_INTERVAL); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setInt( + DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE_KEY, + cellSize - 1); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1); + conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY, + false); + + numDNs = dataBlocks + parityBlocks + 5; + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.waitActive(); + dfs = cluster.getFileSystem(0); + fsn = cluster.getNamesystem(); + bm = fsn.getBlockManager(); + + dfs.enableErasureCodingPolicy( + StripedFileTestUtil.getDefaultECPolicy().getName()); + dfs.mkdirs(ecDir); + dfs.setErasureCodingPolicy(ecDir, + StripedFileTestUtil.getDefaultECPolicy().getName()); + } + + @After + public void teardown() throws IOException { + hostsFileWriter.cleanup(); + if (cluster != null) { + cluster.shutdown(); + cluster = null; + } + } + + /** + * test DN maintenance with striped blocks. + * @throws Exception + */ + @Test(timeout = 120000) + public void testInMaintenance() throws Exception { + //1. create EC file + // d0 d1 d2 d3 d4 d5 d6 d7 d8 + final Path ecFile = new Path(ecDir, "testInMaintenance"); + int writeBytes = cellSize * dataBlocks; + writeStripedFile(dfs, ecFile, writeBytes); + Assert.assertEquals(0, bm.numOfUnderReplicatedBlocks()); + FileChecksum fileChecksum1 = dfs.getFileChecksum(ecFile, writeBytes); + + final INodeFile fileNode = cluster.getNamesystem().getFSDirectory() + .getINode4Write(ecFile.toString()).asFile(); + BlockInfo firstBlock = fileNode.getBlocks()[0]; + DatanodeStorageInfo[] dnStorageInfos = bm.getStorages(firstBlock); + + //2. maintenance node + // d4 d5 d6 d7 d8 + int maintenanceDNIndex = 4; + int numMaintenance= 5; + List maintenanceNodes = new ArrayList<>(); + + for (int i = maintenanceDNIndex; i < numMaintenance + maintenanceDNIndex; ++i) { + maintenanceNodes.add(dnStorageInfos[i].getDatanodeDescriptor()); + } + + maintenanceNode(0, maintenanceNodes, AdminStates.IN_MAINTENANCE, Long.MAX_VALUE); + + //3. wait for maintenance block to replicate + GenericTestUtils.waitFor( + () -> maintenanceNodes.size() == fsn.getNumInMaintenanceLiveDataNodes(), + 100, 60000); + + //4. check DN status, it should be reconstructed again + LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations( + ecFile.toString(), 0, writeBytes); + LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0)); + + BlockInfoStriped blockInfo = + (BlockInfoStriped)bm.getStoredBlock( + new Block(bg.getBlock().getBlockId())); + + // So far, there are 11 total internal blocks, 6 live (d0 d1 d2 d3 d4' d5') + // and 5 in maintenance (d4 d5 d6 d7 d8) internal blocks. + + assertEquals(6, bm.countNodes(blockInfo).liveReplicas()); + assertEquals(5, bm.countNodes(blockInfo).maintenanceNotForReadReplicas()); + + FileChecksum fileChecksum2 = dfs.getFileChecksum(ecFile, writeBytes); + Assert.assertEquals("Checksum mismatches!", fileChecksum1, fileChecksum2); + } + + + /* Get DFSClient to the namenode */ + private static DFSClient getDfsClient(NameNode nn, Configuration conf) + throws IOException { + return new DFSClient(nn.getNameNodeAddress(), conf); + } + + private byte[] writeStripedFile(DistributedFileSystem fs, Path ecFile, + int writeBytes) throws Exception { + byte[] bytes = StripedFileTestUtil.generateBytes(writeBytes); + DFSTestUtil.writeFile(fs, ecFile, new String(bytes)); + StripedFileTestUtil.waitBlockGroupsReported(fs, ecFile.toString()); + + StripedFileTestUtil.checkData(fs, ecFile, writeBytes, + new ArrayList(), null, blockGroupSize); + return bytes; + } + + /* + * maintenance the DN at index dnIndex or one random node if dnIndex is set + * to -1 and wait for the node to reach the given {@code waitForState}. + */ + private void maintenanceNode(int nnIndex, List maintenancedNodes, + AdminStates waitForState, long maintenanceExpirationInMS) + throws IOException, TimeoutException, InterruptedException { + DFSClient client = getDfsClient(cluster.getNameNode(nnIndex), conf); + DatanodeInfo[] info = client.datanodeReport(DatanodeReportType.LIVE); + + // write nodename into the exclude file. + Map maintenanceNodes = new HashMap<>(); + + for (DatanodeInfo dn : maintenancedNodes) { + boolean nodeExists = false; + for (DatanodeInfo dninfo : info) { + if (dninfo.getDatanodeUuid().equals(dn.getDatanodeUuid())) { + nodeExists = true; + break; + } + } + assertTrue("Datanode: " + dn + " is not LIVE", nodeExists); + maintenanceNodes.put(dn.getName(), maintenanceExpirationInMS); + LOG.info("Maintenance node: " + dn.getName()); + } + // write node names into the json host file. + hostsFileWriter.initOutOfServiceHosts(null, maintenanceNodes); + + refreshNodes(cluster.getNamesystem(nnIndex), conf); + for (DatanodeInfo dn : maintenancedNodes) { + DatanodeInfo ret = NameNodeAdapter + .getDatanode(cluster.getNamesystem(nnIndex), dn); + LOG.info("Waiting for node " + ret + " to change state to " + waitForState + + " current state: " + ret.getAdminState()); + GenericTestUtils.waitFor( + () -> ret.getAdminState() == waitForState, + 100, 60000); + LOG.info("node " + ret + " reached the state " + waitForState); + } + } + + private static void refreshNodes(final FSNamesystem ns, + final Configuration conf) throws IOException { + ns.getBlockManager().getDatanodeManager().refreshNodes(conf); + } + +} From 1cd582b97aa45e35a584a1316c990994e054dee5 Mon Sep 17 00:00:00 2001 From: ZanderXu Date: Mon, 5 Dec 2022 16:15:47 +0800 Subject: [PATCH 13/45] HDFS-16837. [RBF SBN] ClientGSIContext should merge RouterFederatedStates to get the max state id for each namespaces (#5123) --- .../apache/hadoop/hdfs/ClientGSIContext.java | 47 ++++++++++++++++++- .../src/main/proto/hdfs.proto | 12 +++++ .../router/RouterStateIdContext.java | 13 +++-- .../src/main/proto/FederationProtocol.proto | 13 ----- .../router/TestConnectionManager.java | 2 +- .../router/TestObserverWithRouter.java | 42 ++++++++++++++++- .../router/TestRouterFederatedState.java | 3 +- 7 files changed, 108 insertions(+), 24 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientGSIContext.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientGSIContext.java index bcbb4b96c2aeb..7b03e1f3518f2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientGSIContext.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientGSIContext.java @@ -20,13 +20,19 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RouterFederatedStateProto; import org.apache.hadoop.ipc.AlignmentContext; import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto; import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto; import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.atomic.LongAccumulator; import org.apache.hadoop.thirdparty.protobuf.ByteString; +import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException; /** * Global State Id context for the client. @@ -77,12 +83,46 @@ public void updateResponseState(RpcResponseHeaderProto.Builder header) { @Override public synchronized void receiveResponseState(RpcResponseHeaderProto header) { if (header.hasRouterFederatedState()) { - routerFederatedState = header.getRouterFederatedState(); + routerFederatedState = mergeRouterFederatedState( + this.routerFederatedState, header.getRouterFederatedState()); } else { lastSeenStateId.accumulate(header.getStateId()); } } + /** + * Utility function to parse routerFederatedState field in RPC headers. + */ + public static Map getRouterFederatedStateMap(ByteString byteString) { + if (byteString != null) { + try { + RouterFederatedStateProto federatedState = RouterFederatedStateProto.parseFrom(byteString); + return federatedState.getNamespaceStateIdsMap(); + } catch (InvalidProtocolBufferException e) { + // Ignore this exception and will return an empty map + } + } + return Collections.emptyMap(); + } + + /** + * Merge state1 and state2 to get the max value for each namespace. + * @param state1 input ByteString. + * @param state2 input ByteString. + * @return one ByteString object which contains the max value of each namespace. + */ + public static ByteString mergeRouterFederatedState(ByteString state1, ByteString state2) { + Map mapping1 = new HashMap<>(getRouterFederatedStateMap(state1)); + Map mapping2 = getRouterFederatedStateMap(state2); + mapping2.forEach((k, v) -> { + long localValue = mapping1.getOrDefault(k, 0L); + mapping1.put(k, Math.max(v, localValue)); + }); + RouterFederatedStateProto.Builder federatedBuilder = RouterFederatedStateProto.newBuilder(); + mapping1.forEach(federatedBuilder::putNamespaceStateIds); + return federatedBuilder.build().toByteString(); + } + /** * Client side implementation for providing state alignment info in requests. */ @@ -106,4 +146,9 @@ public long receiveRequestState(RpcRequestHeaderProto header, long threshold) // Do nothing. return 0; } + + @VisibleForTesting + public ByteString getRouterFederatedState() { + return this.routerFederatedState; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto index a4d36180c2c7a..e1e7f7d780dac 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto @@ -723,3 +723,15 @@ message BlockTokenSecretProto { repeated string storageIds = 8; optional bytes handshakeSecret = 9; } + +///////////////////////////////////////////////// +// Alignment state for namespaces. +///////////////////////////////////////////////// +/** + * Clients should receive this message in RPC responses and forward it + * in RPC requests without interpreting it. It should be encoded + * as an obscure byte array when being sent to clients. + */ +message RouterFederatedStateProto { + map namespaceStateIds = 1; // Last seen state IDs for multiple namespaces. +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStateIdContext.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStateIdContext.java index 9d2b75b0b552b..a15a0001e535a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStateIdContext.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStateIdContext.java @@ -28,8 +28,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos; import org.apache.hadoop.hdfs.protocol.ClientProtocol; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RouterFederatedStateProto; import org.apache.hadoop.hdfs.server.namenode.ha.ReadOnly; import org.apache.hadoop.ipc.AlignmentContext; import org.apache.hadoop.ipc.RetriableException; @@ -83,10 +83,9 @@ public void setResponseHeaderState(RpcResponseHeaderProto.Builder headerBuilder) if (namespaceIdMap.isEmpty()) { return; } - HdfsServerFederationProtos.RouterFederatedStateProto.Builder federatedStateBuilder = - HdfsServerFederationProtos.RouterFederatedStateProto.newBuilder(); - namespaceIdMap.forEach((k, v) -> federatedStateBuilder.putNamespaceStateIds(k, v.get())); - headerBuilder.setRouterFederatedState(federatedStateBuilder.build().toByteString()); + RouterFederatedStateProto.Builder builder = RouterFederatedStateProto.newBuilder(); + namespaceIdMap.forEach((k, v) -> builder.putNamespaceStateIds(k, v.get())); + headerBuilder.setRouterFederatedState(builder.build().toByteString()); } public LongAccumulator getNamespaceStateId(String nsId) { @@ -102,9 +101,9 @@ public void removeNamespaceStateId(String nsId) { */ public static Map getRouterFederatedStateMap(ByteString byteString) { if (byteString != null) { - HdfsServerFederationProtos.RouterFederatedStateProto federatedState; + RouterFederatedStateProto federatedState; try { - federatedState = HdfsServerFederationProtos.RouterFederatedStateProto.parseFrom(byteString); + federatedState = RouterFederatedStateProto.parseFrom(byteString); } catch (InvalidProtocolBufferException e) { throw new RuntimeException(e); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto index 7f61d80fe1aa8..c8636826c3cc9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto @@ -311,17 +311,4 @@ message GetDisabledNameservicesRequestProto { message GetDisabledNameservicesResponseProto { repeated string nameServiceIds = 1; -} - -///////////////////////////////////////////////// -// Alignment state for namespaces. -///////////////////////////////////////////////// - -/** - * Clients should receive this message in RPC responses and forward it - * in RPC requests without interpreting it. It should be encoded - * as an obscure byte array when being sent to clients. - */ -message RouterFederatedStateProto { - map namespaceStateIds = 1; // Last seen state IDs for multiple namespaces. } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestConnectionManager.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestConnectionManager.java index 42288bcf53a4a..920c9c4e519cc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestConnectionManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestConnectionManager.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hdfs.server.federation.router; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterFederatedStateProto; import org.apache.hadoop.hdfs.protocol.ClientProtocol; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RouterFederatedStateProto; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.Server; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java index e38b0b2a35af4..23e72546aac55 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java @@ -27,14 +27,18 @@ import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_MONITOR_NAMENODE; import java.io.IOException; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.ClientGSIContext; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RouterFederatedStateProto; import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster; import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext; import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; @@ -43,6 +47,8 @@ import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState; import org.apache.hadoop.hdfs.server.federation.resolver.MembershipNamenodeResolver; import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -505,4 +511,38 @@ public void testSingleReadUsingObserverReadProxyProvider() throws Exception { // getList call should be sent to observer assertEquals("One call should be sent to observer", 1, rpcCountForObserver); } -} \ No newline at end of file + + @Test + @Tag(SKIP_BEFORE_EACH_CLUSTER_STARTUP) + public void testClientReceiveResponseState() { + ClientGSIContext clientGSIContext = new ClientGSIContext(); + + Map mockMapping = new HashMap<>(); + mockMapping.put("ns0", 10L); + RouterFederatedStateProto.Builder builder = RouterFederatedStateProto.newBuilder(); + mockMapping.forEach(builder::putNamespaceStateIds); + RpcHeaderProtos.RpcResponseHeaderProto header = RpcHeaderProtos.RpcResponseHeaderProto + .newBuilder() + .setCallId(1) + .setStatus(RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto.SUCCESS) + .setRouterFederatedState(builder.build().toByteString()) + .build(); + clientGSIContext.receiveResponseState(header); + + Map mockLowerMapping = new HashMap<>(); + mockLowerMapping.put("ns0", 8L); + builder = RouterFederatedStateProto.newBuilder(); + mockLowerMapping.forEach(builder::putNamespaceStateIds); + header = RpcHeaderProtos.RpcResponseHeaderProto.newBuilder() + .setRouterFederatedState(builder.build().toByteString()) + .setCallId(2) + .setStatus(RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto.SUCCESS) + .build(); + clientGSIContext.receiveResponseState(header); + + Map latestFederateState = ClientGSIContext.getRouterFederatedStateMap( + clientGSIContext.getRouterFederatedState()); + Assertions.assertEquals(1, latestFederateState.size()); + Assertions.assertEquals(10L, latestFederateState.get("ns0")); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterFederatedState.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterFederatedState.java index 2bc8cfc21b230..be8fcf682bd1a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterFederatedState.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterFederatedState.java @@ -19,12 +19,13 @@ import java.util.HashMap; import java.util.Map; + +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RouterFederatedStateProto; import org.apache.hadoop.ipc.AlignmentContext; import org.apache.hadoop.ipc.ClientId; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RpcConstants; import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos; -import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterFederatedStateProto; import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException; import org.apache.hadoop.util.ProtoUtil; import org.junit.Test; From 4144a8a35f3da92de188090e249878a517d3e54f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 5 Dec 2022 16:10:23 +0000 Subject: [PATCH 14/45] HADOOP-18470. index.md update for 3.3.5 release --- .../src/site/markdown/ClusterSetup.md | 12 +- .../src/site/markdown/SingleCluster.md.vm | 11 +- hadoop-project/src/site/markdown/index.md.vm | 259 +++++------------- 3 files changed, 86 insertions(+), 196 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/ClusterSetup.md b/hadoop-common-project/hadoop-common/src/site/markdown/ClusterSetup.md index 4f76979ea6ae9..9095d6f98903d 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/ClusterSetup.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/ClusterSetup.md @@ -22,7 +22,17 @@ Purpose This document describes how to install and configure Hadoop clusters ranging from a few nodes to extremely large clusters with thousands of nodes. To play with Hadoop, you may first want to install it on a single machine (see [Single Node Setup](./SingleCluster.html)). -This document does not cover advanced topics such as [Security](./SecureMode.html) or High Availability. +This document does not cover advanced topics such as High Availability. + +*Important*: all production Hadoop clusters use Kerberos to authenticate callers +and secure access to HDFS data as well as restriction access to computation +services (YARN etc.). + +These instructions do not cover integration with any Kerberos services, +-everyone bringing up a production cluster should include connecting to their +organisation's Kerberos infrastructure as a key part of the deployment. + +See [Security](./SecureMode.html) for details on how to secure a cluster. Prerequisites ------------- diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm b/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm index 8d0a7d195a82f..3c8af8fd6e974 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm +++ b/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm @@ -26,6 +26,15 @@ Purpose This document describes how to set up and configure a single-node Hadoop installation so that you can quickly perform simple operations using Hadoop MapReduce and the Hadoop Distributed File System (HDFS). + +*Important*: all production Hadoop clusters use Kerberos to authenticate callers +and secure access to HDFS data as well as restriction access to computation +services (YARN etc.). + +These instructions do not cover integration with any Kerberos services, +-everyone bringing up a production cluster should include connecting to their +organisation's Kerberos infrastructure as a key part of the deployment. + Prerequisites ------------- @@ -33,8 +42,6 @@ $H3 Supported Platforms * GNU/Linux is supported as a development and production platform. Hadoop has been demonstrated on GNU/Linux clusters with 2000 nodes. -* Windows is also a supported platform but the followings steps are for Linux only. To set up Hadoop on Windows, see [wiki page](http://wiki.apache.org/hadoop/Hadoop2OnWindows). - $H3 Required Software Required software for Linux include: diff --git a/hadoop-project/src/site/markdown/index.md.vm b/hadoop-project/src/site/markdown/index.md.vm index edc38a5286111..05478ea50ac0e 100644 --- a/hadoop-project/src/site/markdown/index.md.vm +++ b/hadoop-project/src/site/markdown/index.md.vm @@ -15,226 +15,99 @@ Apache Hadoop ${project.version} ================================ -Apache Hadoop ${project.version} incorporates a number of significant -enhancements over the previous major release line (hadoop-2.x). +Apache Hadoop ${project.version} is an update to the Hadoop 3.3.x release branch. -This release is generally available (GA), meaning that it represents a point of -API stability and quality that we consider production-ready. - -Overview -======== +Overview of Changes +=================== Users are encouraged to read the full set of release notes. This page provides an overview of the major changes. -Minimum required Java version increased from Java 7 to Java 8 ------------------- +Vectored IO API +--------------- -All Hadoop JARs are now compiled targeting a runtime version of Java 8. -Users still using Java 7 or below must upgrade to Java 8. +The `PositionedReadable` interface has now added an operation for +Vectored (also known as Scatter/Gather IO): -Support for erasure coding in HDFS ------------------- +```java +void readVectored(List ranges, IntFunction allocate) +``` -Erasure coding is a method for durably storing data with significant space -savings compared to replication. Standard encodings like Reed-Solomon (10,4) -have a 1.4x space overhead, compared to the 3x overhead of standard HDFS -replication. +All the requested ranges will be retrieved into the supplied byte buffers -possibly asynchronously, +possibly in parallel, with results potentially coming in out-of-order. -Since erasure coding imposes additional overhead during reconstruction -and performs mostly remote reads, it has traditionally been used for -storing colder, less frequently accessed data. Users should consider -the network and CPU overheads of erasure coding when deploying this -feature. +1. The default implementation uses a series of `readFully()` calls, so delivers + equivalent performance. +2. The local filesystem uses java native IO calls for higher performance reads than `readFully()` +3. The S3A filesystem issues parallel HTTP GET requests in different threads. -More details are available in the -[HDFS Erasure Coding](./hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html) -documentation. +Benchmarking of (modified) ORC and Parquet clients through `file://` and `s3a://` +show tangible improvements in query times. -YARN Timeline Service v.2 -------------------- +Further Reading: [FsDataInputStream](./hadoop-project-dist/hadoop-common/filesystem/fsdatainputstream.html). -We are introducing an early preview (alpha 2) of a major revision of YARN -Timeline Service: v.2. YARN Timeline Service v.2 addresses two major -challenges: improving scalability and reliability of Timeline Service, and -enhancing usability by introducing flows and aggregation. +Manifest Committer for Azure ABFS and google GCS performance +------------------------------------------------------------ -YARN Timeline Service v.2 alpha 2 is provided so that users and developers -can test it and provide feedback and suggestions for making it a ready -replacement for Timeline Service v.1.x. It should be used only in a test -capacity. +A new "intermediate manifest committer" uses a manifest file +to commit the work of successful task attempts, rather than +renaming directories. +Job commit is matter of reading all the manifests, creating the +destination directories (parallelized) and renaming the files, +again in parallel. + +This is fast and correct on Azure Storage and Google GCS, +and should be used there instead of the classic v1/v2 file +output committers. + +It is also safe to use on HDFS, where it should be faster +than the v1 committer. It is however optimized for +cloud storage where list and rename operations are significantly +slower; the benefits may be less. More details are available in the -[YARN Timeline Service v.2](./hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html) +[manifest committer](./hadoop-mapreduce-client/hadoop-mapreduce-client-core/manifest_committer.html). documentation. -Shell script rewrite -------------------- +Transitive CVE fixes +-------------------- -The Hadoop shell scripts have been rewritten to fix many long-standing -bugs and include some new features. While an eye has been kept towards -compatibility, some changes may break existing installations. +A lot of dependencies have been upgraded to address recent CVEs. +Many of the CVEs were not actually exploitable through the Hadoop +so much of this work is just due diligence. +However applications which have all the library is on a class path may +be vulnerable, and the ugprades should also reduce the number of false +positives security scanners report. -Incompatible changes are documented in the release notes, with related -discussion on [HADOOP-9902](https://issues.apache.org/jira/browse/HADOOP-9902). +We have not been able to upgrade every single dependency to the latest +version there is. Some of those changes are just going to be incompatible. +If you have concerns about the state of a specific library, consult the apache JIRA +issue tracker to see what discussions have taken place about the library in question. -More details are available in the -[Unix Shell Guide](./hadoop-project-dist/hadoop-common/UnixShellGuide.html) -documentation. Power users will also be pleased by the -[Unix Shell API](./hadoop-project-dist/hadoop-common/UnixShellAPI.html) -documentation, which describes much of the new functionality, particularly -related to extensibility. - -Shaded client jars ------------------- - -The `hadoop-client` Maven artifact available in 2.x releases pulls -Hadoop's transitive dependencies onto a Hadoop application's classpath. -This can be problematic if the versions of these transitive dependencies -conflict with the versions used by the application. - -[HADOOP-11804](https://issues.apache.org/jira/browse/HADOOP-11804) adds -new `hadoop-client-api` and `hadoop-client-runtime` artifacts that -shade Hadoop's dependencies into a single jar. This avoids leaking -Hadoop's dependencies onto the application's classpath. - -Support for Opportunistic Containers and Distributed Scheduling. --------------------- +As an open source project, contributions in this area are always welcome, +especially in testing the active branches, testing applications downstream of +those branches and of whether updated dependencies trigger regressions. -A notion of `ExecutionType` has been introduced, whereby Applications can -now request for containers with an execution type of `Opportunistic`. -Containers of this type can be dispatched for execution at an NM even if -there are no resources available at the moment of scheduling. In such a -case, these containers will be queued at the NM, waiting for resources to -be available for it to start. Opportunistic containers are of lower priority -than the default `Guaranteed` containers and are therefore preempted, -if needed, to make room for Guaranteed containers. This should -improve cluster utilization. - -Opportunistic containers are by default allocated by the central RM, but -support has also been added to allow opportunistic containers to be -allocated by a distributed scheduler which is implemented as an -AMRMProtocol interceptor. - -Please see [documentation](./hadoop-yarn/hadoop-yarn-site/OpportunisticContainers.html) -for more details. - -MapReduce task-level native optimization --------------------- +HDFS: Router Based Federation +----------------------------- -MapReduce has added support for a native implementation of the map output -collector. For shuffle-intensive jobs, this can lead to a performance -improvement of 30% or more. +A lot of effort has been invested into stabilizing/improving the HDFS Router Based Federation feature. -See the release notes for -[MAPREDUCE-2841](https://issues.apache.org/jira/browse/MAPREDUCE-2841) -for more detail. +1. HDFS-13522, HDFS-16767 & Related Jiras: Allow Observer Reads in HDFS Router Based Federation. +2. HDFS-13248: RBF supports Client Locality -Support for more than 2 NameNodes. --------------------- -The initial implementation of HDFS NameNode high-availability provided -for a single active NameNode and a single Standby NameNode. By replicating -edits to a quorum of three JournalNodes, this architecture is able to -tolerate the failure of any one node in the system. - -However, some deployments require higher degrees of fault-tolerance. -This is enabled by this new feature, which allows users to run multiple -standby NameNodes. For instance, by configuring three NameNodes and -five JournalNodes, the cluster is able to tolerate the failure of two -nodes rather than just one. - -The [HDFS high-availability documentation](./hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithQJM.html) -has been updated with instructions on how to configure more than two -NameNodes. - -Default ports of multiple services have been changed. ------------------------- - -Previously, the default ports of multiple Hadoop services were in the -Linux ephemeral port range (32768-61000). This meant that at startup, -services would sometimes fail to bind to the port due to a conflict -with another application. - -These conflicting ports have been moved out of the ephemeral range, -affecting the NameNode, Secondary NameNode, DataNode, and KMS. Our -documentation has been updated appropriately, but see the release -notes for [HDFS-9427](https://issues.apache.org/jira/browse/HDFS-9427) and -[HADOOP-12811](https://issues.apache.org/jira/browse/HADOOP-12811) -for a list of port changes. - -Support for Microsoft Azure Data Lake and Aliyun Object Storage System filesystem connectors ---------------------- - -Hadoop now supports integration with Microsoft Azure Data Lake and -Aliyun Object Storage System as alternative Hadoop-compatible filesystems. - -Intra-datanode balancer -------------------- - -A single DataNode manages multiple disks. During normal write operation, -disks will be filled up evenly. However, adding or replacing disks can -lead to significant skew within a DataNode. This situation is not handled -by the existing HDFS balancer, which concerns itself with inter-, not intra-, -DN skew. - -This situation is handled by the new intra-DataNode balancing -functionality, which is invoked via the `hdfs diskbalancer` CLI. -See the disk balancer section in the -[HDFS Commands Guide](./hadoop-project-dist/hadoop-hdfs/HDFSCommands.html) -for more information. - -Reworked daemon and task heap management ---------------------- - -A series of changes have been made to heap management for Hadoop daemons -as well as MapReduce tasks. - -[HADOOP-10950](https://issues.apache.org/jira/browse/HADOOP-10950) introduces -new methods for configuring daemon heap sizes. -Notably, auto-tuning is now possible based on the memory size of the host, -and the `HADOOP_HEAPSIZE` variable has been deprecated. -See the full release notes of HADOOP-10950 for more detail. - -[MAPREDUCE-5785](https://issues.apache.org/jira/browse/MAPREDUCE-5785) -simplifies the configuration of map and reduce task -heap sizes, so the desired heap size no longer needs to be specified -in both the task configuration and as a Java option. -Existing configs that already specify both are not affected by this change. -See the full release notes of MAPREDUCE-5785 for more details. - -HDFS Router-Based Federation ---------------------- -HDFS Router-Based Federation adds a RPC routing layer that provides a federated -view of multiple HDFS namespaces. This is similar to the existing -[ViewFs](./hadoop-project-dist/hadoop-hdfs/ViewFs.html)) and -[HDFS Federation](./hadoop-project-dist/hadoop-hdfs/Federation.html) -functionality, except the mount table is managed on the server-side by the -routing layer rather than on the client. This simplifies access to a federated -cluster for existing HDFS clients. - -See [HDFS-10467](https://issues.apache.org/jira/browse/HDFS-10467) and the -HDFS Router-based Federation -[documentation](./hadoop-project-dist/hadoop-hdfs-rbf/HDFSRouterFederation.html) for -more details. - -API-based configuration of Capacity Scheduler queue configuration ----------------------- - -The OrgQueue extension to the capacity scheduler provides a programmatic way to -change configurations by providing a REST API that users can call to modify -queue configurations. This enables automation of queue configuration management -by administrators in the queue's `administer_queue` ACL. - -See [YARN-5734](https://issues.apache.org/jira/browse/YARN-5734) and the -[Capacity Scheduler documentation](./hadoop-yarn/hadoop-yarn-site/CapacityScheduler.html) for more information. - -YARN Resource Types ---------------- +HDFS: Dynamic Datanode Reconfiguration +-------------------------------------- + +HDFS-16400, HDFS-16399, HDFS-16396, HDFS-16397, HDFS-16413, HDFS-16457. -The YARN resource model has been generalized to support user-defined countable resource types beyond CPU and memory. For instance, the cluster administrator could define resources like GPUs, software licenses, or locally-attached storage. YARN tasks can then be scheduled based on the availability of these resources. +A number of Datanode configuration options can be changed without having to restart +the datanode. This makes it possible to tune deployment configurations without +cluster-wide Datanode Restarts. -See [YARN-3926](https://issues.apache.org/jira/browse/YARN-3926) and the [YARN resource model documentation](./hadoop-yarn/hadoop-yarn-site/ResourceModel.html) for more information. +See [DataNode.java](https://github.com/apache/hadoop/blob/branch-3.3.5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L346-L361) +for the list of dynamically reconfigurable attributes. Getting Started =============== From f1462e72c00f48094f5319f590cda86aeffa24b0 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 6 Dec 2022 09:58:51 +0000 Subject: [PATCH 15/45] HADOOP-18560. AvroFSInput opens a stream twice and discards the second one without closing (#5186) This is needed for branches with the hadoop-common changes of HADOOP-16202. Enhanced openFile() --- .../src/main/java/org/apache/hadoop/fs/AvroFSInput.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java index 7518dd2f7ef74..155381de949ef 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java @@ -60,7 +60,6 @@ public AvroFSInput(final FileContext fc, final Path p) throws IOException { FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL) .withFileStatus(status) .build()); - fc.open(p); } @Override From 94db31c9ffc075d5fdb37e03c775d07e762eaf89 Mon Sep 17 00:00:00 2001 From: Gautham B A Date: Tue, 6 Dec 2022 16:32:26 +0530 Subject: [PATCH 16/45] YARN-11386. Fix issue with classpath resolution (#5183) * This PR ensures that all the special notations such as are resolved before getting added to classpath. --- .../launcher/ContainerLaunch.java | 64 ++++++++++--------- .../launcher/TestContainerLaunch.java | 3 + 2 files changed, 38 insertions(+), 29 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/launcher/ContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java index 0acc4de4704a4..7d91e5d395f86 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java @@ -246,7 +246,7 @@ public Integer call() { launchContext.setCommands(newCmds); // The actual expansion of environment variables happens after calling - // sanitizeEnv. This allows variables specified in NM_ADMIN_USER_ENV + // addConfigsToEnv. This allows variables specified in NM_ADMIN_USER_ENV // to reference user or container-defined variables. Map environment = launchContext.getEnvironment(); // /////////////////////////// End of variable expansion @@ -340,13 +340,15 @@ public Integer call() { try (DataOutputStream containerScriptOutStream = lfs.create(nmPrivateContainerScriptPath, EnumSet.of(CREATE, OVERWRITE))) { + addConfigsToEnv(environment); + + expandAllEnvironmentVars(environment, containerLogDir); + // Sanitize the container's environment sanitizeEnv(environment, containerWorkDir, appDirs, userLocalDirs, containerLogDirs, localResources, nmPrivateClasspathJarDir, nmEnvVars); - expandAllEnvironmentVars(environment, containerLogDir); - // Add these if needed after expanding so we don't expand key values. if (keystore != null) { addKeystoreVars(environment, containerWorkDir); @@ -1641,13 +1643,35 @@ public void sanitizeEnv(Map environment, Path pwd, addToEnvMap(environment, nmVars, "JVM_PID", "$$"); } + // TODO: Remove Windows check and use this approach on all platforms after + // additional testing. See YARN-358. + if (Shell.WINDOWS) { + sanitizeWindowsEnv(environment, pwd, + resources, nmPrivateClasspathJarDir); + } + + // put AuxiliaryService data to environment + for (Map.Entry meta : containerManager + .getAuxServiceMetaData().entrySet()) { + AuxiliaryServiceHelper.setServiceDataIntoEnv( + meta.getKey(), meta.getValue(), environment); + nmVars.add(AuxiliaryServiceHelper.getPrefixServiceName(meta.getKey())); + } + } + + /** + * There are some configurations (such as {@value YarnConfiguration#NM_ADMIN_USER_ENV}) whose + * values need to be added to the environment variables. + * + * @param environment The environment variables map to add the configuration values to. + */ + public void addConfigsToEnv(Map environment) { // variables here will be forced in, even if the container has // specified them. Note: we do not track these in nmVars, to // allow them to be ordered properly if they reference variables // defined by the user. String defEnvStr = conf.get(YarnConfiguration.DEFAULT_NM_ADMIN_USER_ENV); - Apps.setEnvFromInputProperty(environment, - YarnConfiguration.NM_ADMIN_USER_ENV, defEnvStr, conf, + Apps.setEnvFromInputProperty(environment, YarnConfiguration.NM_ADMIN_USER_ENV, defEnvStr, conf, File.pathSeparator); if (!Shell.WINDOWS) { @@ -1658,39 +1682,21 @@ public void sanitizeEnv(Map environment, Path pwd, String userPath = environment.get(Environment.PATH.name()); environment.remove(Environment.PATH.name()); if (userPath == null || userPath.isEmpty()) { - Apps.addToEnvironment(environment, Environment.PATH.name(), - forcePath, File.pathSeparator); - Apps.addToEnvironment(environment, Environment.PATH.name(), - "$PATH", File.pathSeparator); + Apps.addToEnvironment(environment, Environment.PATH.name(), forcePath, + File.pathSeparator); + Apps.addToEnvironment(environment, Environment.PATH.name(), "$PATH", File.pathSeparator); } else { - Apps.addToEnvironment(environment, Environment.PATH.name(), - forcePath, File.pathSeparator); - Apps.addToEnvironment(environment, Environment.PATH.name(), - userPath, File.pathSeparator); + Apps.addToEnvironment(environment, Environment.PATH.name(), forcePath, + File.pathSeparator); + Apps.addToEnvironment(environment, Environment.PATH.name(), userPath, File.pathSeparator); } } } - - // TODO: Remove Windows check and use this approach on all platforms after - // additional testing. See YARN-358. - if (Shell.WINDOWS) { - - sanitizeWindowsEnv(environment, pwd, - resources, nmPrivateClasspathJarDir); - } - // put AuxiliaryService data to environment - for (Map.Entry meta : containerManager - .getAuxServiceMetaData().entrySet()) { - AuxiliaryServiceHelper.setServiceDataIntoEnv( - meta.getKey(), meta.getValue(), environment); - nmVars.add(AuxiliaryServiceHelper.getPrefixServiceName(meta.getKey())); - } } private void sanitizeWindowsEnv(Map environment, Path pwd, Map> resources, Path nmPrivateClasspathJarDir) throws IOException { - String inputClassPath = environment.get(Environment.CLASSPATH.name()); if (inputClassPath != null && !inputClassPath.isEmpty()) { 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/launcher/TestContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java index 6b0732b4e5c5a..ed22254906bbb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java @@ -808,6 +808,7 @@ public void handle(Event event) { resources.put(userjar, lpaths); Path nmp = new Path(testDir); + launch.addConfigsToEnv(userSetEnv); launch.sanitizeEnv(userSetEnv, pwd, appDirs, userLocalDirs, containerLogs, resources, nmp, nmEnvTrack); Assert.assertTrue(userSetEnv.containsKey("MALLOC_ARENA_MAX")); @@ -864,6 +865,7 @@ public void handle(Event event) { ContainerLaunch launch = new ContainerLaunch(distContext, conf, dispatcher, exec, null, container, dirsHandler, containerManager); + launch.addConfigsToEnv(userSetEnv); launch.sanitizeEnv(userSetEnv, pwd, appDirs, userLocalDirs, containerLogs, resources, nmp, nmEnvTrack); @@ -876,6 +878,7 @@ public void handle(Event event) { containerLaunchContext.setEnvironment(userSetEnv); when(container.getLaunchContext()).thenReturn(containerLaunchContext); + launch.addConfigsToEnv(userSetEnv); launch.sanitizeEnv(userSetEnv, pwd, appDirs, userLocalDirs, containerLogs, resources, nmp, nmEnvTrack); From a9376d9279b1f229a437898cf05bc83fb57f07d4 Mon Sep 17 00:00:00 2001 From: Akshat Bordia <31816865+akshatb1@users.noreply.github.com> Date: Tue, 6 Dec 2022 16:39:49 +0530 Subject: [PATCH 17/45] YARN-10978. Fix ApplicationClassLoader to Correctly Expand Glob for Windows Path (#3558) --- .../java/org/apache/hadoop/util/ApplicationClassLoader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java index 972bbff4cfd49..4e8a9c9b275b5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java @@ -108,7 +108,7 @@ static URL[] constructUrlsFromClasspath(String classpath) throws MalformedURLException { List urls = new ArrayList(); for (String element : classpath.split(File.pathSeparator)) { - if (element.endsWith("/*")) { + if (element.endsWith(File.separator + "*")) { List jars = FileUtil.getJarsInDirectory(element); if (!jars.isEmpty()) { for (Path jar: jars) { From 81c486372efab7c5ef6a8424f5a86432ba9a1782 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Wed, 7 Dec 2022 00:17:05 +0800 Subject: [PATCH 18/45] YARN-11373. [Federation] Support refreshQueues refreshNodes API's for Federation. (#5146) --- .../protocolrecords/RefreshNodesRequest.java | 25 +++++++ .../protocolrecords/RefreshQueuesRequest.java | 17 +++++ ...erver_resourcemanager_service_protos.proto | 2 + .../filecontroller/ifile/package-info.java | 6 +- .../filecontroller/tfile/package-info.java | 6 +- .../yarn/security/admin/package-info.java | 6 +- .../impl/pb/RefreshNodesRequestPBImpl.java | 22 +++++- .../impl/pb/RefreshQueuesRequestPBImpl.java | 33 ++++++++- .../yarn/server/router/RouterMetrics.java | 38 +++++++++- .../rmadmin/FederationRMAdminInterceptor.java | 73 ++++++++++++++++++- .../router/rmadmin/RMAdminProtocolMethod.java | 62 +++++++++++++++- .../TestFederationRMAdminInterceptor.java | 70 +++++++++++++++++- 12 files changed, 330 insertions(+), 30 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshNodesRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshNodesRequest.java index 732d98ebe447d..53e99cee6b5d2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshNodesRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshNodesRequest.java @@ -53,6 +53,17 @@ public static RefreshNodesRequest newInstance( return request; } + @Private + @Unstable + public static RefreshNodesRequest newInstance( + DecommissionType decommissionType, Integer timeout, String subClusterId) { + RefreshNodesRequest request = Records.newRecord(RefreshNodesRequest.class); + request.setDecommissionType(decommissionType); + request.setDecommissionTimeout(timeout); + request.setSubClusterId(subClusterId); + return request; + } + /** * Set the DecommissionType * @@ -80,4 +91,18 @@ public static RefreshNodesRequest newInstance( * @return decommissionTimeout */ public abstract Integer getDecommissionTimeout(); + + /** + * Get the subClusterId. + * + * @return subClusterId. + */ + public abstract String getSubClusterId(); + + /** + * Set the subClusterId. + * + * @param subClusterId subCluster Id. + */ + public abstract void setSubClusterId(String subClusterId); } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshQueuesRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshQueuesRequest.java index eff4b7f4d284b..ba332ad40cd38 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshQueuesRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshQueuesRequest.java @@ -20,6 +20,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.yarn.util.Records; @@ -33,4 +34,20 @@ public static RefreshQueuesRequest newInstance() { Records.newRecord(RefreshQueuesRequest.class); return request; } + + @Public + @Stable + public static RefreshQueuesRequest newInstance(String subClusterId) { + RefreshQueuesRequest request = Records.newRecord(RefreshQueuesRequest.class); + request.setSubClusterId(subClusterId); + return request; + } + + @Public + @Unstable + public abstract String getSubClusterId(); + + @Private + @Unstable + public abstract void setSubClusterId(String subClusterId); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto index 3f9913b9896d0..e1bf9edfccb45 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto @@ -32,6 +32,7 @@ package hadoop.yarn; import "yarn_protos.proto"; message RefreshQueuesRequestProto { + optional string sub_cluster_id = 1; } message RefreshQueuesResponseProto { } @@ -39,6 +40,7 @@ message RefreshQueuesResponseProto { message RefreshNodesRequestProto { optional DecommissionTypeProto decommissionType = 1 [default = NORMAL]; optional int32 decommissionTimeout = 2; + optional string sub_cluster_id = 3; } message RefreshNodesResponseProto { } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/package-info.java index 08ddecef5dbd0..9cbc99baad87e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn.logaggregation.filecontroller.ifile; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Public; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/package-info.java index b2e91ab48a989..e014350ec25db 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,6 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn.logaggregation.filecontroller.tfile; -import org.apache.hadoop.classification.InterfaceAudience; \ No newline at end of file +import org.apache.hadoop.classification.InterfaceAudience.Public; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/package-info.java index c66be222aea2a..99b857ac2ab87 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/admin/package-info.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Public +@Public package org.apache.hadoop.yarn.security.admin; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Public; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshNodesRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshNodesRequestPBImpl.java index 62a82912b59e3..a14aae74f6b7c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshNodesRequestPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshNodesRequestPBImpl.java @@ -31,9 +31,9 @@ @Private @Unstable public class RefreshNodesRequestPBImpl extends RefreshNodesRequest { - RefreshNodesRequestProto proto = RefreshNodesRequestProto.getDefaultInstance(); - RefreshNodesRequestProto.Builder builder = null; - boolean viaProto = false; + private RefreshNodesRequestProto proto = RefreshNodesRequestProto.getDefaultInstance(); + private RefreshNodesRequestProto.Builder builder = null; + private boolean viaProto = false; private DecommissionType decommissionType; public RefreshNodesRequestPBImpl() { @@ -123,6 +123,22 @@ public synchronized Integer getDecommissionTimeout() { return p.hasDecommissionTimeout()? p.getDecommissionTimeout() : null; } + @Override + public synchronized String getSubClusterId() { + RefreshNodesRequestProtoOrBuilder p = viaProto ? proto : builder; + return (p.hasSubClusterId()) ? p.getSubClusterId() : null; + } + + @Override + public synchronized void setSubClusterId(String subClusterId) { + maybeInitBuilder(); + if (subClusterId == null) { + builder.clearSubClusterId(); + return; + } + builder.setSubClusterId(subClusterId); + } + private DecommissionType convertFromProtoFormat(DecommissionTypeProto p) { return DecommissionType.valueOf(p.name()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshQueuesRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshQueuesRequestPBImpl.java index c21ec6d362c0e..2c174ad18fb37 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshQueuesRequestPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshQueuesRequestPBImpl.java @@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesRequestProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesRequestProtoOrBuilder; import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest; import org.apache.hadoop.thirdparty.protobuf.TextFormat; @@ -29,9 +30,9 @@ @Unstable public class RefreshQueuesRequestPBImpl extends RefreshQueuesRequest { - RefreshQueuesRequestProto proto = RefreshQueuesRequestProto.getDefaultInstance(); - RefreshQueuesRequestProto.Builder builder = null; - boolean viaProto = false; + private RefreshQueuesRequestProto proto = RefreshQueuesRequestProto.getDefaultInstance(); + private RefreshQueuesRequestProto.Builder builder = null; + private boolean viaProto = false; public RefreshQueuesRequestPBImpl() { builder = RefreshQueuesRequestProto.newBuilder(); @@ -55,8 +56,9 @@ public int hashCode() { @Override public boolean equals(Object other) { - if (other == null) + if (other == null) { return false; + } if (other.getClass().isAssignableFrom(this.getClass())) { return this.getProto().equals(this.getClass().cast(other).getProto()); } @@ -67,4 +69,27 @@ public boolean equals(Object other) { public String toString() { return TextFormat.shortDebugString(getProto()); } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = RefreshQueuesRequestProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public String getSubClusterId() { + RefreshQueuesRequestProtoOrBuilder p = viaProto ? proto : builder; + return (p.hasSubClusterId()) ? p.getSubClusterId() : null; + } + + @Override + public void setSubClusterId(String clusterId) { + maybeInitBuilder(); + if (clusterId == null) { + builder.clearSubClusterId(); + return; + } + builder.setSubClusterId(clusterId); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java index 31d838d1b3ef7..3268889c95cab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java @@ -127,6 +127,8 @@ public final class RouterMetrics { private MutableGaugeInt numGetRMNodeLabelsFailedRetrieved; @Metric("# of checkUserAccessToQueue failed to be retrieved") private MutableGaugeInt numCheckUserAccessToQueueFailedRetrieved; + @Metric("# of refreshNodes failed to be retrieved") + private MutableGaugeInt numRefreshNodesFailedRetrieved; @Metric("# of getDelegationToken failed to be retrieved") private MutableGaugeInt numGetDelegationTokenFailedRetrieved; @Metric("# of renewDelegationToken failed to be retrieved") @@ -221,6 +223,8 @@ public final class RouterMetrics { private MutableRate totalSucceededGetRMNodeLabelsRetrieved; @Metric("Total number of successful Retrieved CheckUserAccessToQueue and latency(ms)") private MutableRate totalSucceededCheckUserAccessToQueueRetrieved; + @Metric("Total number of successful Retrieved RefreshNodes and latency(ms)") + private MutableRate totalSucceededRefreshNodesRetrieved; @Metric("Total number of successful Retrieved GetDelegationToken and latency(ms)") private MutableRate totalSucceededGetDelegationTokenRetrieved; @Metric("Total number of successful Retrieved RenewDelegationToken and latency(ms)") @@ -271,9 +275,10 @@ public final class RouterMetrics { private MutableQuantiles getUpdateQueueLatency; private MutableQuantiles getAppTimeoutLatency; private MutableQuantiles getAppTimeoutsLatency; - private MutableQuantiles getRefreshQueuesLatency; + private MutableQuantiles refreshQueuesLatency; private MutableQuantiles getRMNodeLabelsLatency; private MutableQuantiles checkUserAccessToQueueLatency; + private MutableQuantiles refreshNodesLatency; private MutableQuantiles getDelegationTokenLatency; private MutableQuantiles renewDelegationTokenLatency; private MutableQuantiles cancelDelegationTokenLatency; @@ -430,7 +435,7 @@ private RouterMetrics() { getAppTimeoutsLatency = registry.newQuantiles("getAppTimeoutsLatency", "latency of get apptimeouts timeouts", "ops", "latency", 10); - getRefreshQueuesLatency = registry.newQuantiles("getRefreshQueuesLatency", + refreshQueuesLatency = registry.newQuantiles("refreshQueuesLatency", "latency of get refresh queues timeouts", "ops", "latency", 10); getRMNodeLabelsLatency = registry.newQuantiles("getRMNodeLabelsLatency", @@ -439,6 +444,9 @@ private RouterMetrics() { checkUserAccessToQueueLatency = registry.newQuantiles("checkUserAccessToQueueLatency", "latency of get apptimeouts timeouts", "ops", "latency", 10); + refreshNodesLatency = registry.newQuantiles("refreshNodesLatency", + "latency of get refresh nodes timeouts", "ops", "latency", 10); + getDelegationTokenLatency = registry.newQuantiles("getDelegationTokenLatency", "latency of get delegation token timeouts", "ops", "latency", 10); @@ -447,6 +455,7 @@ private RouterMetrics() { cancelDelegationTokenLatency = registry.newQuantiles("cancelDelegationTokenLatency", "latency of cancel delegation token timeouts", "ops", "latency", 10); + } public static RouterMetrics getMetrics() { @@ -673,6 +682,11 @@ public long getNumSucceededRefreshQueuesRetrieved() { return totalSucceededRefreshQueuesRetrieved.lastStat().numSamples(); } + @VisibleForTesting + public long getNumSucceededRefreshNodesRetrieved() { + return totalSucceededRefreshNodesRetrieved.lastStat().numSamples(); + } + @VisibleForTesting public long getNumSucceededGetRMNodeLabelsRetrieved() { return totalSucceededGetRMNodeLabelsRetrieved.lastStat().numSamples(); @@ -903,6 +917,11 @@ public double getLatencySucceededRefreshQueuesRetrieved() { return totalSucceededRefreshQueuesRetrieved.lastStat().mean(); } + @VisibleForTesting + public double getLatencySucceededRefreshNodesRetrieved() { + return totalSucceededRefreshNodesRetrieved.lastStat().mean(); + } + @VisibleForTesting public double getLatencySucceededGetRMNodeLabelsRetrieved() { return totalSucceededGetRMNodeLabelsRetrieved.lastStat().mean(); @@ -1122,6 +1141,10 @@ public int getCheckUserAccessToQueueFailedRetrieved() { return numCheckUserAccessToQueueFailedRetrieved.value(); } + public int getNumRefreshNodesFailedRetrieved() { + return numRefreshNodesFailedRetrieved.value(); + } + public int getDelegationTokenFailedRetrieved() { return numGetDelegationTokenFailedRetrieved.value(); } @@ -1336,7 +1359,12 @@ public void succeededGetAppTimeoutsRetrieved(long duration) { public void succeededRefreshQueuesRetrieved(long duration) { totalSucceededRefreshQueuesRetrieved.add(duration); - getRefreshQueuesLatency.add(duration); + refreshQueuesLatency.add(duration); + } + + public void succeededRefreshNodesRetrieved(long duration) { + totalSucceededRefreshNodesRetrieved.add(duration); + refreshNodesLatency.add(duration); } public void succeededGetRMNodeLabelsRetrieved(long duration) { @@ -1536,6 +1564,10 @@ public void incrCheckUserAccessToQueueFailedRetrieved() { numCheckUserAccessToQueueFailedRetrieved.incr(); } + public void incrRefreshNodesFailedRetrieved() { + numRefreshNodesFailedRetrieved.incr(); + } + public void incrGetDelegationTokenFailedRetrieved() { numGetDelegationTokenFailedRetrieved.incr(); } 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/FederationRMAdminInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java index 4564f8d8b8506..22ace295c4584 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java @@ -145,6 +145,23 @@ public void setNextInterceptor(RMAdminRequestInterceptor next) { + "is correct"); } + /** + * Refresh queue requests. + * + * The Router supports refreshing all SubCluster queues at once, + * and also supports refreshing queues by SubCluster. + * + * @param request RefreshQueuesRequest, If subClusterId is not empty, + * it means that we want to refresh the queue of the specified subClusterId. + * If subClusterId is empty, it means we want to refresh all queues. + * + * @return RefreshQueuesResponse, There is no specific information in the response, + * as long as it is not empty, it means that the request is successful. + * + * @throws StandbyException exception thrown by non-active server. + * @throws YarnException indicates exceptions from yarn servers. + * @throws IOException io error occurs. + */ @Override public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request) throws StandbyException, YarnException, IOException { @@ -161,8 +178,9 @@ public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request) RMAdminProtocolMethod remoteMethod = new RMAdminProtocolMethod( new Class[] {RefreshQueuesRequest.class}, new Object[] {request}); + String subClusterId = request.getSubClusterId(); Collection refreshQueueResps = - remoteMethod.invokeConcurrent(this, RefreshQueuesResponse.class); + remoteMethod.invokeConcurrent(this, RefreshQueuesResponse.class, subClusterId); // If we get the return result from refreshQueueResps, // it means that the call has been successful, @@ -172,19 +190,66 @@ public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request) routerMetrics.succeededRefreshQueuesRetrieved(stopTime - startTime); return RefreshQueuesResponse.newInstance(); } - } catch (Exception e) { + } catch (YarnException e) { routerMetrics.incrRefreshQueuesFailedRetrieved(); - RouterServerUtil.logAndThrowException("Unable to refreshQueue to exception.", e); + RouterServerUtil.logAndThrowException(e, "Unable to refreshQueue due to exception."); } routerMetrics.incrRefreshQueuesFailedRetrieved(); throw new YarnException("Unable to refreshQueue."); } + /** + * Refresh node requests. + * + * The Router supports refreshing all SubCluster nodes at once, + * and also supports refreshing node by SubCluster. + * + * @param request RefreshNodesRequest, If subClusterId is not empty, + * it means that we want to refresh the node of the specified subClusterId. + * If subClusterId is empty, it means we want to refresh all nodes. + * + * @return RefreshNodesResponse, There is no specific information in the response, + * as long as it is not empty, it means that the request is successful. + * + * @throws StandbyException exception thrown by non-active server. + * @throws YarnException indicates exceptions from yarn servers. + * @throws IOException io error occurs. + */ @Override public RefreshNodesResponse refreshNodes(RefreshNodesRequest request) throws StandbyException, YarnException, IOException { - throw new NotImplementedException(); + + // parameter verification. + // We will not check whether the DecommissionType is empty, + // because this parameter has a default value at the proto level. + if (request == null) { + routerMetrics.incrRefreshNodesFailedRetrieved(); + RouterServerUtil.logAndThrowException("Missing RefreshNodes request.", null); + } + + // call refreshNodes of activeSubClusters. + try { + long startTime = clock.getTime(); + RMAdminProtocolMethod remoteMethod = new RMAdminProtocolMethod( + new Class[] {RefreshNodesRequest.class}, new Object[] {request}); + + String subClusterId = request.getSubClusterId(); + Collection refreshNodesResps = + remoteMethod.invokeConcurrent(this, RefreshNodesResponse.class, subClusterId); + + if (CollectionUtils.isNotEmpty(refreshNodesResps)) { + long stopTime = clock.getTime(); + routerMetrics.succeededRefreshNodesRetrieved(stopTime - startTime); + return RefreshNodesResponse.newInstance(); + } + } catch (YarnException e) { + routerMetrics.incrRefreshNodesFailedRetrieved(); + RouterServerUtil.logAndThrowException(e, "Unable to refreshNodes due to exception."); + } + + routerMetrics.incrRefreshNodesFailedRetrieved(); + throw new YarnException("Unable to refreshNodes."); } @Override 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/RMAdminProtocolMethod.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RMAdminProtocolMethod.java index e1aa806ff86ae..1a5b038f19cb2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RMAdminProtocolMethod.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RMAdminProtocolMethod.java @@ -37,12 +37,12 @@ import java.util.TreeMap; import java.util.List; import java.util.ArrayList; +import java.util.Collections; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.ThreadPoolExecutor; - /** * Class to define admin method, params and arguments. */ @@ -61,11 +61,15 @@ public RMAdminProtocolMethod(Class[] pTypes, Object... pParams) } public Collection invokeConcurrent(FederationRMAdminInterceptor interceptor, - Class clazz) throws YarnException { + Class clazz, String subClusterId) throws YarnException { this.rmAdminInterceptor = interceptor; this.federationFacade = FederationStateStoreFacade.getInstance(); this.configuration = interceptor.getConf(); - return invokeConcurrent(clazz); + if (StringUtils.isNotBlank(subClusterId)) { + return invoke(clazz, subClusterId); + } else { + return invokeConcurrent(clazz); + } } @Override @@ -107,7 +111,10 @@ protected Collection invokeConcurrent(Class clazz) throws YarnExceptio Pair pair = future.get(); subClusterId = pair.getKey(); Object result = pair.getValue(); - results.put(subClusterId, clazz.cast(result)); + if (result != null) { + R rResult = clazz.cast(result); + results.put(subClusterId, rResult); + } } catch (InterruptedException | ExecutionException e) { Throwable cause = e.getCause(); LOG.error("Cannot execute {} on {}: {}", methodName, subClusterId, cause.getMessage()); @@ -129,4 +136,51 @@ protected Collection invokeConcurrent(Class clazz) throws YarnExceptio // return result return results.values(); } + + /** + * Call the method in the protocol according to the subClusterId. + * + * @param clazz return type + * @param subClusterId subCluster Id + * @param Generic R + * @return response collection. + * @throws YarnException yarn exception. + */ + protected Collection invoke(Class clazz, String subClusterId) throws YarnException { + + // Get the method name to call + String methodName = Thread.currentThread().getStackTrace()[3].getMethodName(); + this.setMethodName(methodName); + + // Get Active SubClusters + Map subClusterInfoMap = + federationFacade.getSubClusters(true); + + // According to subCluster of string type, convert to SubClusterId type + SubClusterId subClusterIdKey = SubClusterId.newInstance(subClusterId); + + // If the provided subCluster is not Active or does not exist, + // an exception will be returned directly. + if (!subClusterInfoMap.containsKey(subClusterIdKey)) { + throw new YarnException("subClusterId = " + subClusterId + " is not an active subCluster."); + } + + // Call the method in the protocol and convert it according to clazz. + try { + ResourceManagerAdministrationProtocol protocol = + rmAdminInterceptor.getAdminRMProxyForSubCluster(subClusterIdKey); + Class[] types = this.getTypes(); + Object[] params = this.getParams(); + Method method = ResourceManagerAdministrationProtocol.class.getMethod(methodName, types); + Object result = method.invoke(protocol, params); + if (result != null) { + return Collections.singletonList(clazz.cast(result)); + } + } catch (Exception e) { + throw new YarnException("invoke Failed, An exception occurred in subClusterId = " + + subClusterId, e); + } + throw new YarnException("invoke Failed, An exception occurred in subClusterId = " + + subClusterId); + } } 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/TestFederationRMAdminInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java index 3aa61a68a390d..e68e9dda3cab9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java @@ -19,8 +19,11 @@ package org.apache.hadoop.yarn.server.router.rmadmin; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.test.LambdaTestUtils; +import org.apache.hadoop.yarn.api.records.DecommissionType; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest; import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; @@ -31,7 +34,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -77,7 +79,7 @@ public void setUp() { subClusters = new ArrayList<>(); try { for (int i = 0; i < NUM_SUBCLUSTER; i++) { - SubClusterId sc = SubClusterId.newInstance(Integer.toString(i)); + SubClusterId sc = SubClusterId.newInstance("SC-" + i); stateStoreUtil.registerSubCluster(sc); subClusters.add(sc); } @@ -114,8 +116,70 @@ public void tearDown() { } @Test - public void testRefreshQueues() throws IOException, YarnException { + public void testRefreshQueues() throws Exception { + // We will test 2 cases: + // case 1, request is null. + // case 2, normal request. + // If the request is null, a Missing RefreshQueues request exception will be thrown. + + // null request. + LambdaTestUtils.intercept(YarnException.class, "Missing RefreshQueues request.", + () -> interceptor.refreshQueues(null)); + + // normal request. RefreshQueuesRequest request = RefreshQueuesRequest.newInstance(); interceptor.refreshQueues(request); } + + @Test + public void testSC1RefreshQueues() throws Exception { + // We will test 2 cases: + // case 1, test the existing subCluster (SC-1). + // case 2, test the non-exist subCluster. + + String existSubCluster = "SC-1"; + RefreshQueuesRequest request = RefreshQueuesRequest.newInstance(existSubCluster); + interceptor.refreshQueues(request); + + String notExistsSubCluster = "SC-NON"; + RefreshQueuesRequest request1 = RefreshQueuesRequest.newInstance(notExistsSubCluster); + LambdaTestUtils.intercept(YarnException.class, + "subClusterId = SC-NON is not an active subCluster.", + () -> interceptor.refreshQueues(request1)); + } + + @Test + public void testRefreshNodes() throws Exception { + // We will test 2 cases: + // case 1, request is null. + // case 2, normal request. + // If the request is null, a Missing RefreshNodes request exception will be thrown. + + // null request. + LambdaTestUtils.intercept(YarnException.class, + "Missing RefreshNodes request.", () -> interceptor.refreshNodes(null)); + + // normal request. + RefreshNodesRequest request = RefreshNodesRequest.newInstance(DecommissionType.NORMAL); + interceptor.refreshNodes(request); + } + + @Test + public void testSC1RefreshNodes() throws Exception { + + // We will test 2 cases: + // case 1, test the existing subCluster (SC-1). + // case 2, test the non-exist subCluster. + + RefreshNodesRequest request = + RefreshNodesRequest.newInstance(DecommissionType.NORMAL, 10, "SC-1"); + interceptor.refreshNodes(request); + + String notExistsSubCluster = "SC-NON"; + RefreshNodesRequest request1 = RefreshNodesRequest.newInstance( + DecommissionType.NORMAL, 10, notExistsSubCluster); + LambdaTestUtils.intercept(YarnException.class, + "subClusterId = SC-NON is not an active subCluster.", + () -> interceptor.refreshNodes(request1)); + } } From 5d220429e0312230b614a5b5640d5e8711ff4da6 Mon Sep 17 00:00:00 2001 From: Murali Krishna Date: Tue, 6 Dec 2022 22:27:46 +0530 Subject: [PATCH 19/45] HADOOP-18538. Upgrade kafka to 2.8.2 (#5164) Signed-off-by: Brahma Reddy Battula --- LICENSE-binary | 2 +- hadoop-project/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 988e38fa390d4..c4aa63df8805f 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -324,7 +324,7 @@ org.apache.htrace:htrace-core:3.1.0-incubating org.apache.htrace:htrace-core4:4.1.0-incubating org.apache.httpcomponents:httpclient:4.5.6 org.apache.httpcomponents:httpcore:4.4.10 -org.apache.kafka:kafka-clients:2.8.1 +org.apache.kafka:kafka-clients:2.8.2 org.apache.kerby:kerb-admin:2.0.2 org.apache.kerby:kerb-client:2.0.2 org.apache.kerby:kerb-common:2.0.2 diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 5c2fad157798e..17df3f1449779 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -50,7 +50,7 @@ 2.12.2 - 2.8.1 + 2.8.2 1.0.13 From f5e33a34cf26412a9f2a6e31192106ea55788805 Mon Sep 17 00:00:00 2001 From: Pranav Saxena <108325433+pranavsaxena-microsoft@users.noreply.github.com> Date: Wed, 7 Dec 2022 12:15:45 -0800 Subject: [PATCH 20/45] HADOOP-18546. ABFS. disable purging list of in progress reads in abfs stream close() (#5176) This addresses HADOOP-18521, "ABFS ReadBufferManager buffer sharing across concurrent HTTP requests" by not trying to cancel in progress reads. It supercedes HADOOP-18528, which disables the prefetching. If that patch is applied *after* this one, prefetching will be disabled. As well as changing the default value in the code, core-default.xml is updated to set fs.azure.enable.readahead = true As a result, if Configuration.get("fs.azure.enable.readahead") returns a non-null value, then it can be inferred that it was set in or core-default.xml (the fix is present) or in core-site.xml (someone asked for it). Contributed by Pranav Saxena. --- .../src/main/resources/core-default.xml | 5 +- .../constants/FileSystemConfigurations.java | 2 +- .../services/AbfsInputStreamContext.java | 2 +- .../azurebfs/services/ReadBufferManager.java | 6 +- .../ITestAbfsFileSystemContractSeek.java | 2 - .../services/TestAbfsInputStream.java | 71 ++++++++++++++++++- 6 files changed, 78 insertions(+), 10 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index 047c5482062be..e18a50c72e86e 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -2168,9 +2168,8 @@ The switch to turn S3A auditing on or off. fs.azure.enable.readahead - false - Disable readahead/prefetching in AbfsInputStream. - See HADOOP-18521 + true + Enabled readahead/prefetching in AbfsInputStream. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 0ea2c929800a3..9994d9f5207f3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -109,7 +109,7 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_ABFS_LATENCY_TRACK = false; public static final long DEFAULT_SAS_TOKEN_RENEW_PERIOD_FOR_STREAMS_IN_SECONDS = 120; - public static final boolean DEFAULT_ENABLE_READAHEAD = false; + public static final boolean DEFAULT_ENABLE_READAHEAD = true; public static final String DEFAULT_FS_AZURE_USER_AGENT_PREFIX = EMPTY_STRING; public static final String DEFAULT_VALUE_UNKNOWN = "UNKNOWN"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java index f6b330934cf5e..e258958b1a111 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java @@ -35,7 +35,7 @@ public class AbfsInputStreamContext extends AbfsStreamContext { private boolean tolerateOobAppends; - private boolean isReadAheadEnabled = false; + private boolean isReadAheadEnabled = true; private boolean alwaysReadBufferSize; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index 317aaf545a1ae..ac84f0b27cf12 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -544,7 +544,6 @@ public synchronized void purgeBuffersForStream(AbfsInputStream stream) { LOGGER.debug("Purging stale buffers for AbfsInputStream {} ", stream); readAheadQueue.removeIf(readBuffer -> readBuffer.getStream() == stream); purgeList(stream, completedReadList); - purgeList(stream, inProgressList); } /** @@ -642,4 +641,9 @@ void testMimicFullUseAndAddFailedBuffer(ReadBuffer buf) { freeList.clear(); completedReadList.add(buf); } + + @VisibleForTesting + int getNumBuffers() { + return NUM_BUFFERS; + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java index aaf47f7a9c871..f7fe5039799d7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java @@ -34,7 +34,6 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_READ_AHEAD_RANGE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_READ_BUFFER_SIZE; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_READAHEAD; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; @@ -69,7 +68,6 @@ protected Configuration createConfiguration() { protected AbstractFSContract createContract(final Configuration conf) { conf.setInt(AZURE_READ_AHEAD_RANGE, MIN_BUFFER_SIZE); conf.setInt(AZURE_READ_BUFFER_SIZE, MIN_BUFFER_SIZE); - conf.setBoolean(FS_AZURE_ENABLE_READAHEAD, true); return new AbfsFileSystemContract(conf, isSecure); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index 69795ee5bd857..0395c4183b9b7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -82,6 +82,12 @@ public class TestAbfsInputStream extends REDUCED_READ_BUFFER_AGE_THRESHOLD * 10; // 30 sec private static final int ALWAYS_READ_BUFFER_SIZE_TEST_FILE_SIZE = 16 * ONE_MB; + @Override + public void teardown() throws Exception { + super.teardown(); + ReadBufferManager.getBufferManager().testResetReadBufferManager(); + } + private AbfsRestOperation getMockRestOp() { AbfsRestOperation op = mock(AbfsRestOperation.class); AbfsHttpOperation httpOp = mock(AbfsHttpOperation.class); @@ -106,7 +112,6 @@ private AbfsClient getMockAbfsClient() { private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, String fileName) throws IOException { AbfsInputStreamContext inputStreamContext = new AbfsInputStreamContext(-1); - inputStreamContext.isReadAheadEnabled(true); // Create AbfsInputStream with the client instance AbfsInputStream inputStream = new AbfsInputStream( mockAbfsClient, @@ -132,7 +137,6 @@ public AbfsInputStream getAbfsInputStream(AbfsClient abfsClient, boolean alwaysReadBufferSize, int readAheadBlockSize) throws IOException { AbfsInputStreamContext inputStreamContext = new AbfsInputStreamContext(-1); - inputStreamContext.isReadAheadEnabled(true); // Create AbfsInputStream with the client instance AbfsInputStream inputStream = new AbfsInputStream( abfsClient, @@ -495,6 +499,69 @@ public void testSuccessfulReadAhead() throws Exception { checkEvictedStatus(inputStream, 0, true); } + /** + * This test expects InProgressList is not purged by the inputStream close. + */ + @Test + public void testStreamPurgeDuringReadAheadCallExecuting() throws Exception { + AbfsClient client = getMockAbfsClient(); + AbfsRestOperation successOp = getMockRestOp(); + final Long serverCommunicationMockLatency = 3_000L; + final Long readBufferTransferToInProgressProbableTime = 1_000L; + final Integer readBufferQueuedCount = 3; + + Mockito.doAnswer(invocationOnMock -> { + //sleeping thread to mock the network latency from client to backend. + Thread.sleep(serverCommunicationMockLatency); + return successOp; + }) + .when(client) + .read(any(String.class), any(Long.class), any(byte[].class), + any(Integer.class), any(Integer.class), any(String.class), + any(String.class), any(TracingContext.class)); + + final ReadBufferManager readBufferManager + = ReadBufferManager.getBufferManager(); + + final int readBufferTotal = readBufferManager.getNumBuffers(); + final int expectedFreeListBufferCount = readBufferTotal + - readBufferQueuedCount; + + try (AbfsInputStream inputStream = getAbfsInputStream(client, + "testSuccessfulReadAhead.txt")) { + // As this is try-with-resources block, the close() method of the created + // abfsInputStream object shall be called on the end of the block. + queueReadAheads(inputStream); + + //Sleeping to give ReadBufferWorker to pick the readBuffers for processing. + Thread.sleep(readBufferTransferToInProgressProbableTime); + + Assertions.assertThat(readBufferManager.getInProgressCopiedList()) + .describedAs(String.format("InProgressList should have %d elements", + readBufferQueuedCount)) + .hasSize(readBufferQueuedCount); + Assertions.assertThat(readBufferManager.getFreeListCopy()) + .describedAs(String.format("FreeList should have %d elements", + expectedFreeListBufferCount)) + .hasSize(expectedFreeListBufferCount); + Assertions.assertThat(readBufferManager.getCompletedReadListCopy()) + .describedAs("CompletedList should have 0 elements") + .hasSize(0); + } + + Assertions.assertThat(readBufferManager.getInProgressCopiedList()) + .describedAs(String.format("InProgressList should have %d elements", + readBufferQueuedCount)) + .hasSize(readBufferQueuedCount); + Assertions.assertThat(readBufferManager.getFreeListCopy()) + .describedAs(String.format("FreeList should have %d elements", + expectedFreeListBufferCount)) + .hasSize(expectedFreeListBufferCount); + Assertions.assertThat(readBufferManager.getCompletedReadListCopy()) + .describedAs("CompletedList should have 0 elements") + .hasSize(0); + } + /** * This test expects ReadAheadManager to throw exception if the read ahead * thread had failed within the last thresholdAgeMilliseconds. From 1269eb988ee5e805fcb45e5b56d24242ac2d3054 Mon Sep 17 00:00:00 2001 From: Oleksandr Shevchenko Date: Thu, 8 Dec 2022 17:07:59 +0200 Subject: [PATCH 21/45] HADOOP-18563. Misleading AWS SDK S3 timeout configuration comment (#5197) Contributed by Oleksandr Shevchenko --- .../src/main/java/org/apache/hadoop/fs/s3a/Constants.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index e5b0a9b5aa163..16472a75fd28f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -251,12 +251,12 @@ private Constants() { public static final boolean EXPERIMENTAL_AWS_INTERNAL_THROTTLING_DEFAULT = true; - // seconds until we give up trying to establish a connection to s3 + // milliseconds until we give up trying to establish a connection to s3 public static final String ESTABLISH_TIMEOUT = "fs.s3a.connection.establish.timeout"; public static final int DEFAULT_ESTABLISH_TIMEOUT = 50000; - // seconds until we give up on a connection to s3 + // milliseconds until we give up on a connection to s3 public static final String SOCKET_TIMEOUT = "fs.s3a.connection.timeout"; public static final int DEFAULT_SOCKET_TIMEOUT = 200000; From 7e7f1c66af7a210b6fff638e7d7fdfcb8a01c8a1 Mon Sep 17 00:00:00 2001 From: K0K0V0K <109747532+K0K0V0K@users.noreply.github.com> Date: Thu, 8 Dec 2022 18:52:19 +0100 Subject: [PATCH 22/45] YARN-11390. TestResourceTrackerService.testNodeRemovalNormally: Shutdown nodes should be 0 now expected: <1> but was: <0> (#5190) Reviewed-by: Peter Szucs Signed-off-by: Chris Nauroth --- .../TestResourceTrackerService.java | 62 ++++++++++++------- 1 file changed, 38 insertions(+), 24 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java index 33d5003d7e1a3..6a8806572cc9f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java @@ -56,13 +56,16 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.HashSet; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; import javax.xml.parsers.DocumentBuilderFactory; import javax.xml.transform.Transformer; import javax.xml.transform.TransformerFactory; @@ -2345,8 +2348,7 @@ public void testNodeRemovalUtil(boolean doGraceful) throws Exception { } //Test decommed/ing node that transitions to untracked,timer should remove - testNodeRemovalUtilDecomToUntracked(rmContext, conf, nm1, nm2, nm3, - maxThreadSleeptime, doGraceful); + testNodeRemovalUtilDecomToUntracked(rmContext, conf, nm1, nm2, nm3, doGraceful); rm.stop(); } @@ -2354,41 +2356,41 @@ public void testNodeRemovalUtil(boolean doGraceful) throws Exception { // max allowed length. private void testNodeRemovalUtilDecomToUntracked( RMContext rmContext, Configuration conf, - MockNM nm1, MockNM nm2, MockNM nm3, - long maxThreadSleeptime, boolean doGraceful) throws Exception { + MockNM nm1, MockNM nm2, MockNM nm3, boolean doGraceful + ) throws Exception { ClusterMetrics metrics = ClusterMetrics.getMetrics(); String ip = NetUtils.normalizeHostName("localhost"); - CountDownLatch latch = new CountDownLatch(1); writeToHostsFile("host1", ip, "host2"); writeToHostsFile(excludeHostFile, "host2"); refreshNodesOption(doGraceful, conf); nm1.nodeHeartbeat(true); //nm2.nodeHeartbeat(true); nm3.nodeHeartbeat(true); - latch.await(maxThreadSleeptime, TimeUnit.MILLISECONDS); - RMNode rmNode = doGraceful ? rmContext.getRMNodes().get(nm2.getNodeId()) : - rmContext.getInactiveRMNodes().get(nm2.getNodeId()); - Assert.assertNotEquals("Timer for this node was not canceled!", - rmNode, null); - Assert.assertTrue("Node should be DECOMMISSIONED or DECOMMISSIONING", - (rmNode.getState() == NodeState.DECOMMISSIONED) || - (rmNode.getState() == NodeState.DECOMMISSIONING)); + Supplier nodeSupplier = doGraceful + ? () -> rmContext.getRMNodes().get(nm2.getNodeId()) + : () -> rmContext.getInactiveRMNodes().get(nm2.getNodeId()); + pollingAssert(() -> nodeSupplier.get() != null, + "Timer for this node was not canceled!"); + final List expectedStates = Arrays.asList( + NodeState.DECOMMISSIONED, + NodeState.DECOMMISSIONING + ); + pollingAssert(() -> expectedStates.contains(nodeSupplier.get().getState()), + "Node should be in one of these states: " + expectedStates); + writeToHostsFile("host1", ip); writeToHostsFile(excludeHostFile, ""); refreshNodesOption(doGraceful, conf); nm2.nodeHeartbeat(true); - latch.await(maxThreadSleeptime, TimeUnit.MILLISECONDS); - rmNode = doGraceful ? rmContext.getRMNodes().get(nm2.getNodeId()) : - rmContext.getInactiveRMNodes().get(nm2.getNodeId()); - Assert.assertEquals("Node should have been forgotten!", - rmNode, null); - Assert.assertEquals("Shutdown nodes should be 0 now", - metrics.getNumDecommisionedNMs(), 0); - Assert.assertEquals("Shutdown nodes should be 0 now", - metrics.getNumShutdownNMs(), 0); - Assert.assertEquals("Active nodes should be 2", - metrics.getNumActiveNMs(), 2); + pollingAssert(() -> nodeSupplier.get() == null, + "Node should have been forgotten!"); + pollingAssert(metrics::getNumDecommisionedNMs, 0, + "metrics#getNumDecommisionedNMs should be 0 now"); + pollingAssert(metrics::getNumShutdownNMs, 0, + "metrics#getNumShutdownNMs should be 0 now"); + pollingAssert(metrics::getNumActiveNMs, 2, + "metrics#getNumActiveNMs should be 2 now"); } private void testNodeRemovalUtilLost(boolean doGraceful) throws Exception { @@ -2959,6 +2961,18 @@ protected ResourceTrackerService createResourceTrackerService() { mockRM.stop(); } + private void pollingAssert(Supplier supplier, String message) + throws InterruptedException, TimeoutException { + GenericTestUtils.waitFor(supplier, + 100, 10_000, message); + } + + private void pollingAssert(Supplier supplier, T expected, String message) + throws InterruptedException, TimeoutException { + GenericTestUtils.waitFor(() -> Objects.equals(supplier.get(), expected), + 100, 10_000, message); + } + /** * A no-op implementation of NodeAttributeStore for testing */ From 8024f542081fca8378abeec54b4481a651f66412 Mon Sep 17 00:00:00 2001 From: dingshun3016 Date: Fri, 9 Dec 2022 08:10:04 +0800 Subject: [PATCH 23/45] HDFS-16858. Dynamically adjust max slow disks to exclude. (#5180) Reviewed-by: Chris Nauroth Reviewed-by: slfan1989 <55643692+slfan1989@users.noreply.github.com> Signed-off-by: Tao Li --- .../apache/hadoop/hdfs/server/datanode/DataNode.java | 10 ++++++++++ .../server/datanode/metrics/DataNodeDiskMetrics.java | 10 +++++++++- .../server/datanode/TestDataNodeReconfiguration.java | 9 ++++++++- .../org/apache/hadoop/hdfs/tools/TestDFSAdmin.java | 2 +- 4 files changed, 28 insertions(+), 3 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 58334bf5c0750..c42abda72bc1d 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 @@ -72,6 +72,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT; @@ -353,6 +355,7 @@ public class DataNode extends ReconfigurableBase DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY, DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY, DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY, + DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY, FS_DU_INTERVAL_KEY, FS_GETSPACEUSED_JITTER_KEY, FS_GETSPACEUSED_CLASSNAME)); @@ -699,6 +702,7 @@ public String reconfigurePropertyImpl(String property, String newVal) case DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY: case DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY: case DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY: + case DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY: return reconfSlowDiskParameters(property, newVal); case FS_DU_INTERVAL_KEY: case FS_GETSPACEUSED_JITTER_KEY: @@ -877,6 +881,12 @@ private String reconfSlowDiskParameters(String property, String newVal) Long.parseLong(newVal)); result = Long.toString(threshold); diskMetrics.setLowThresholdMs(threshold); + } else if (property.equals(DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY)) { + checkNotNull(diskMetrics, "DataNode disk stats may be disabled."); + int maxSlowDisksToExclude = (newVal == null ? + DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_DEFAULT : Integer.parseInt(newVal)); + result = Integer.toString(maxSlowDisksToExclude); + diskMetrics.setMaxSlowDisksToExclude(maxSlowDisksToExclude); } LOG.info("RECONFIGURE* changed {} to {}", property, newVal); return result; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeDiskMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeDiskMetrics.java index 409084cfe8be8..a8ccd6d4ec4bd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeDiskMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeDiskMetrics.java @@ -80,7 +80,7 @@ public class DataNodeDiskMetrics { /** * The number of slow disks that needs to be excluded. */ - private int maxSlowDisksToExclude; + private volatile int maxSlowDisksToExclude; /** * List of slow disks that need to be excluded. */ @@ -274,6 +274,14 @@ public List getSlowDisksToExclude() { return slowDisksToExclude; } + public int getMaxSlowDisksToExclude() { + return maxSlowDisksToExclude; + } + + public void setMaxSlowDisksToExclude(int maxSlowDisksToExclude) { + this.maxSlowDisksToExclude = maxSlowDisksToExclude; + } + public void setLowThresholdMs(long thresholdMs) { Preconditions.checkArgument(thresholdMs > 0, DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY + " should be larger than 0"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java index 14e3f63691bd5..d9578ca02a949 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java @@ -45,6 +45,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; @@ -636,13 +637,15 @@ public void testSlowDiskParameters() throws ReconfigurationException, IOExceptio String[] slowDisksParameters2 = { DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY, - DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY}; + DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY, + DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY}; for (String parameter : slowDisksParameters2) { dn.reconfigureProperty(parameter, "99"); } // Assert diskMetrics. assertEquals(99, dn.getDiskMetrics().getMinOutlierDetectionDisks()); assertEquals(99, dn.getDiskMetrics().getLowThresholdMs()); + assertEquals(99, dn.getDiskMetrics().getMaxSlowDisksToExclude()); // Assert dnConf. assertTrue(dn.getDnConf().diskStatsEnabled); // Assert profilingEventHook. @@ -673,12 +676,16 @@ public void testSlowDiskParameters() throws ReconfigurationException, IOExceptio dn.reconfigureProperty(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, "1"); dn.reconfigureProperty(DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY, null); dn.reconfigureProperty(DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY, null); + dn.reconfigureProperty(DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY, null); assertEquals(String.format("expect %s is not configured", DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY), null, dn.getConf().get(DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY)); assertEquals(String.format("expect %s is not configured", DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY), null, dn.getConf().get(DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY)); + assertEquals(String.format("expect %s is not configured", + DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY), null, + dn.getConf().get(DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY)); assertEquals(DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_DEFAULT, dn.getDiskMetrics().getSlowDiskDetector().getMinOutlierDetectionNodes()); assertEquals(DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_DEFAULT, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java index 59491206dcbbe..9a87365eb2fb1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java @@ -345,7 +345,7 @@ public void testDataNodeGetReconfigurableProperties() throws IOException, Interr final List outs = Lists.newArrayList(); final List errs = Lists.newArrayList(); getReconfigurableProperties("datanode", address, outs, errs); - assertEquals(19, outs.size()); + assertEquals(20, outs.size()); assertEquals(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, outs.get(1)); } From ec0c622ea9b9b39b68ee7cc37ebbc95398e3a737 Mon Sep 17 00:00:00 2001 From: Anurag P <51438471+theradtad@users.noreply.github.com> Date: Fri, 9 Dec 2022 11:18:44 +0530 Subject: [PATCH 24/45] HDFS-16860 Upgrade moment.min.js to 2.29.4 (#5194) --- .../hadoop-hdfs/src/main/webapps/static/moment.min.js | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/moment.min.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/moment.min.js index d1204ccedd277..4b9e7a6d6a7ea 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/moment.min.js +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/moment.min.js @@ -1,5 +1,5 @@ //! moment.js -//! version : 2.29.2 +//! version : 2.29.4 //! license : MIT //! momentjs.com -!function(e,t){"object"==typeof exports&&"undefined"!=typeof module?module.exports=t():"function"==typeof define&&define.amd?define(t):e.moment=t()}(this,function(){"use strict";var e;function c(){return e.apply(null,arguments)}function a(e){return e instanceof Array||"[object Array]"===Object.prototype.toString.call(e)}function o(e){return null!=e&&"[object Object]"===Object.prototype.toString.call(e)}function f(e,t){return Object.prototype.hasOwnProperty.call(e,t)}function u(e){if(Object.getOwnPropertyNames)return 0===Object.getOwnPropertyNames(e).length;for(var t in e)if(f(e,t))return;return 1}function l(e){return void 0===e}function h(e){return"number"==typeof e||"[object Number]"===Object.prototype.toString.call(e)}function s(e){return e instanceof Date||"[object Date]"===Object.prototype.toString.call(e)}function i(e,t){for(var n=[],s=e.length,i=0;i>>0,s=0;sFe(e)?(r=e+1,i-Fe(e)):(r=e,i);return{year:r,dayOfYear:i}}function Ie(e,t,n){var s,i,r=Ee(e.year(),t,n),r=Math.floor((e.dayOfYear()-r-1)/7)+1;return r<1?s=r+je(i=e.year()-1,t,n):r>je(e.year(),t,n)?(s=r-je(e.year(),t,n),i=e.year()+1):(i=e.year(),s=r),{week:s,year:i}}function je(e,t,n){var s=Ee(e,t,n),n=Ee(e+1,t,n);return(Fe(e)-s+n)/7}C("w",["ww",2],"wo","week"),C("W",["WW",2],"Wo","isoWeek"),L("week","w"),L("isoWeek","W"),A("week",5),A("isoWeek",5),de("w",ee),de("ww",ee,J),de("W",ee),de("WW",ee,J),ge(["w","ww","W","WW"],function(e,t,n,s){t[s.substr(0,1)]=Z(e)});function Ze(e,t){return e.slice(t,7).concat(e.slice(0,t))}C("d",0,"do","day"),C("dd",0,0,function(e){return this.localeData().weekdaysMin(this,e)}),C("ddd",0,0,function(e){return this.localeData().weekdaysShort(this,e)}),C("dddd",0,0,function(e){return this.localeData().weekdays(this,e)}),C("e",0,0,"weekday"),C("E",0,0,"isoWeekday"),L("day","d"),L("weekday","e"),L("isoWeekday","E"),A("day",11),A("weekday",11),A("isoWeekday",11),de("d",ee),de("e",ee),de("E",ee),de("dd",function(e,t){return t.weekdaysMinRegex(e)}),de("ddd",function(e,t){return t.weekdaysShortRegex(e)}),de("dddd",function(e,t){return t.weekdaysRegex(e)}),ge(["dd","ddd","dddd"],function(e,t,n,s){s=n._locale.weekdaysParse(e,s,n._strict);null!=s?t.d=s:_(n).invalidWeekday=e}),ge(["d","e","E"],function(e,t,n,s){t[s]=Z(e)});var ze="Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),$e="Sun_Mon_Tue_Wed_Thu_Fri_Sat".split("_"),qe="Su_Mo_Tu_We_Th_Fr_Sa".split("_"),Be=he,Je=he,Qe=he;function Xe(){function e(e,t){return t.length-e.length}for(var t,n,s,i=[],r=[],a=[],o=[],u=0;u<7;u++)s=m([2e3,1]).day(u),t=fe(this.weekdaysMin(s,"")),n=fe(this.weekdaysShort(s,"")),s=fe(this.weekdays(s,"")),i.push(t),r.push(n),a.push(s),o.push(t),o.push(n),o.push(s);i.sort(e),r.sort(e),a.sort(e),o.sort(e),this._weekdaysRegex=new RegExp("^("+o.join("|")+")","i"),this._weekdaysShortRegex=this._weekdaysRegex,this._weekdaysMinRegex=this._weekdaysRegex,this._weekdaysStrictRegex=new RegExp("^("+a.join("|")+")","i"),this._weekdaysShortStrictRegex=new RegExp("^("+r.join("|")+")","i"),this._weekdaysMinStrictRegex=new RegExp("^("+i.join("|")+")","i")}function Ke(){return this.hours()%12||12}function et(e,t){C(e,0,0,function(){return this.localeData().meridiem(this.hours(),this.minutes(),t)})}function tt(e,t){return t._meridiemParse}C("H",["HH",2],0,"hour"),C("h",["hh",2],0,Ke),C("k",["kk",2],0,function(){return this.hours()||24}),C("hmm",0,0,function(){return""+Ke.apply(this)+T(this.minutes(),2)}),C("hmmss",0,0,function(){return""+Ke.apply(this)+T(this.minutes(),2)+T(this.seconds(),2)}),C("Hmm",0,0,function(){return""+this.hours()+T(this.minutes(),2)}),C("Hmmss",0,0,function(){return""+this.hours()+T(this.minutes(),2)+T(this.seconds(),2)}),et("a",!0),et("A",!1),L("hour","h"),A("hour",13),de("a",tt),de("A",tt),de("H",ee),de("h",ee),de("k",ee),de("HH",ee,J),de("hh",ee,J),de("kk",ee,J),de("hmm",te),de("hmmss",ne),de("Hmm",te),de("Hmmss",ne),ye(["H","HH"],Me),ye(["k","kk"],function(e,t,n){e=Z(e);t[Me]=24===e?0:e}),ye(["a","A"],function(e,t,n){n._isPm=n._locale.isPM(e),n._meridiem=e}),ye(["h","hh"],function(e,t,n){t[Me]=Z(e),_(n).bigHour=!0}),ye("hmm",function(e,t,n){var s=e.length-2;t[Me]=Z(e.substr(0,s)),t[De]=Z(e.substr(s)),_(n).bigHour=!0}),ye("hmmss",function(e,t,n){var s=e.length-4,i=e.length-2;t[Me]=Z(e.substr(0,s)),t[De]=Z(e.substr(s,2)),t[Se]=Z(e.substr(i)),_(n).bigHour=!0}),ye("Hmm",function(e,t,n){var s=e.length-2;t[Me]=Z(e.substr(0,s)),t[De]=Z(e.substr(s))}),ye("Hmmss",function(e,t,n){var s=e.length-4,i=e.length-2;t[Me]=Z(e.substr(0,s)),t[De]=Z(e.substr(s,2)),t[Se]=Z(e.substr(i))});var nt=z("Hours",!0);var st,it={calendar:{sameDay:"[Today at] LT",nextDay:"[Tomorrow at] LT",nextWeek:"dddd [at] LT",lastDay:"[Yesterday at] LT",lastWeek:"[Last] dddd [at] LT",sameElse:"L"},longDateFormat:{LTS:"h:mm:ss A",LT:"h:mm A",L:"MM/DD/YYYY",LL:"MMMM D, YYYY",LLL:"MMMM D, YYYY h:mm A",LLLL:"dddd, MMMM D, YYYY h:mm A"},invalidDate:"Invalid date",ordinal:"%d",dayOfMonthOrdinalParse:/\d{1,2}/,relativeTime:{future:"in %s",past:"%s ago",s:"a few seconds",ss:"%d seconds",m:"a minute",mm:"%d minutes",h:"an hour",hh:"%d hours",d:"a day",dd:"%d days",w:"a week",ww:"%d weeks",M:"a month",MM:"%d months",y:"a year",yy:"%d years"},months:Te,monthsShort:Ne,week:{dow:0,doy:6},weekdays:ze,weekdaysMin:qe,weekdaysShort:$e,meridiemParse:/[ap]\.?m?\.?/i},rt={},at={};function ot(e){return e&&e.toLowerCase().replace("_","-")}function ut(e){for(var t,n,s,i,r=0;r=t&&function(e,t){for(var n=Math.min(e.length,t.length),s=0;s=t-1)break;t--}r++}return st}function lt(t){var e;if(void 0===rt[t]&&"undefined"!=typeof module&&module&&module.exports&&null!=t.match("^[^/\\\\]*$"))try{e=st._abbr,require("./locale/"+t),ht(e)}catch(e){rt[t]=null}return rt[t]}function ht(e,t){return e&&((t=l(t)?ct(e):dt(e,t))?st=t:"undefined"!=typeof console&&console.warn&&console.warn("Locale "+e+" not found. Did you forget to load it?")),st._abbr}function dt(e,t){if(null===t)return delete rt[e],null;var n,s=it;if(t.abbr=e,null!=rt[e])S("defineLocaleOverride","use moment.updateLocale(localeName, config) to change an existing locale. moment.defineLocale(localeName, config) should only be used for creating a new locale See http://momentjs.com/guides/#/warnings/define-locale/ for more info."),s=rt[e]._config;else if(null!=t.parentLocale)if(null!=rt[t.parentLocale])s=rt[t.parentLocale]._config;else{if(null==(n=lt(t.parentLocale)))return at[t.parentLocale]||(at[t.parentLocale]=[]),at[t.parentLocale].push({name:e,config:t}),null;s=n._config}return rt[e]=new b(O(s,t)),at[e]&&at[e].forEach(function(e){dt(e.name,e.config)}),ht(e),rt[e]}function ct(e){var t;if(!(e=e&&e._locale&&e._locale._abbr?e._locale._abbr:e))return st;if(!a(e)){if(t=lt(e))return t;e=[e]}return ut(e)}function ft(e){var t=e._a;return t&&-2===_(e).overflow&&(t=t[ve]<0||11xe(t[pe],t[ve])?ke:t[Me]<0||24je(n,r,a)?_(e)._overflowWeeks=!0:null!=o?_(e)._overflowWeekday=!0:(a=Ae(n,s,i,r,a),e._a[pe]=a.year,e._dayOfYear=a.dayOfYear)}(e),null!=e._dayOfYear&&(s=Yt(e._a[pe],n[pe]),(e._dayOfYear>Fe(s)||0===e._dayOfYear)&&(_(e)._overflowDayOfYear=!0),s=Ge(s,0,e._dayOfYear),e._a[ve]=s.getUTCMonth(),e._a[ke]=s.getUTCDate()),t=0;t<3&&null==e._a[t];++t)e._a[t]=a[t]=n[t];for(;t<7;t++)e._a[t]=a[t]=null==e._a[t]?2===t?1:0:e._a[t];24===e._a[Me]&&0===e._a[De]&&0===e._a[Se]&&0===e._a[Ye]&&(e._nextDay=!0,e._a[Me]=0),e._d=(e._useUTC?Ge:Ve).apply(null,a),s=e._useUTC?e._d.getUTCDay():e._d.getDay(),null!=e._tzm&&e._d.setUTCMinutes(e._d.getUTCMinutes()-e._tzm),e._nextDay&&(e._a[Me]=24),e._w&&void 0!==e._w.d&&e._w.d!==s&&(_(e).weekdayMismatch=!0)}}function bt(e){if(e._f!==c.ISO_8601)if(e._f!==c.RFC_2822){e._a=[],_(e).empty=!0;for(var t,n,s,i,r,a=""+e._i,o=a.length,u=0,l=H(e._f,e._locale).match(N)||[],h=l.length,d=0;de.valueOf():e.valueOf()"}),ie.toJSON=function(){return this.isValid()?this.toISOString():null},ie.toString=function(){return this.clone().locale("en").format("ddd MMM DD YYYY HH:mm:ss [GMT]ZZ")},ie.unix=function(){return Math.floor(this.valueOf()/1e3)},ie.valueOf=function(){return this._d.valueOf()-6e4*(this._offset||0)},ie.creationData=function(){return{input:this._i,format:this._f,locale:this._locale,isUTC:this._isUTC,strict:this._strict}},ie.eraName=function(){for(var e,t=this.localeData().eras(),n=0,s=t.length;nthis.clone().month(0).utcOffset()||this.utcOffset()>this.clone().month(5).utcOffset()},ie.isLocal=function(){return!!this.isValid()&&!this._isUTC},ie.isUtcOffset=function(){return!!this.isValid()&&this._isUTC},ie.isUtc=It,ie.isUTC=It,ie.zoneAbbr=function(){return this._isUTC?"UTC":""},ie.zoneName=function(){return this._isUTC?"Coordinated Universal Time":""},ie.dates=n("dates accessor is deprecated. Use date instead.",ne),ie.months=n("months accessor is deprecated. Use month instead",Ue),ie.years=n("years accessor is deprecated. Use year instead",Le),ie.zone=n("moment().zone is deprecated, use moment().utcOffset instead. http://momentjs.com/guides/#/warnings/zone/",function(e,t){return null!=e?(this.utcOffset(e="string"!=typeof e?-e:e,t),this):-this.utcOffset()}),ie.isDSTShifted=n("isDSTShifted is deprecated. See http://momentjs.com/guides/#/warnings/dst-shifted/ for more information",function(){if(!l(this._isDSTShifted))return this._isDSTShifted;var e,t={};return p(t,this),(t=xt(t))._a?(e=(t._isUTC?m:Nt)(t._a),this._isDSTShifted=this.isValid()&&0>>0,s=0;sAe(e)?(r=e+1,t-Ae(e)):(r=e,t);return{year:r,dayOfYear:n}}function qe(e,t,n){var s,i,r=ze(e.year(),t,n),r=Math.floor((e.dayOfYear()-r-1)/7)+1;return r<1?s=r+P(i=e.year()-1,t,n):r>P(e.year(),t,n)?(s=r-P(e.year(),t,n),i=e.year()+1):(i=e.year(),s=r),{week:s,year:i}}function P(e,t,n){var s=ze(e,t,n),t=ze(e+1,t,n);return(Ae(e)-s+t)/7}s("w",["ww",2],"wo","week"),s("W",["WW",2],"Wo","isoWeek"),t("week","w"),t("isoWeek","W"),n("week",5),n("isoWeek",5),k("w",p),k("ww",p,w),k("W",p),k("WW",p,w),Te(["w","ww","W","WW"],function(e,t,n,s){t[s.substr(0,1)]=g(e)});function Be(e,t){return e.slice(t,7).concat(e.slice(0,t))}s("d",0,"do","day"),s("dd",0,0,function(e){return this.localeData().weekdaysMin(this,e)}),s("ddd",0,0,function(e){return this.localeData().weekdaysShort(this,e)}),s("dddd",0,0,function(e){return this.localeData().weekdays(this,e)}),s("e",0,0,"weekday"),s("E",0,0,"isoWeekday"),t("day","d"),t("weekday","e"),t("isoWeekday","E"),n("day",11),n("weekday",11),n("isoWeekday",11),k("d",p),k("e",p),k("E",p),k("dd",function(e,t){return t.weekdaysMinRegex(e)}),k("ddd",function(e,t){return t.weekdaysShortRegex(e)}),k("dddd",function(e,t){return t.weekdaysRegex(e)}),Te(["dd","ddd","dddd"],function(e,t,n,s){s=n._locale.weekdaysParse(e,s,n._strict);null!=s?t.d=s:m(n).invalidWeekday=e}),Te(["d","e","E"],function(e,t,n,s){t[s]=g(e)});var Je="Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),Qe="Sun_Mon_Tue_Wed_Thu_Fri_Sat".split("_"),Xe="Su_Mo_Tu_We_Th_Fr_Sa".split("_"),Ke=v,et=v,tt=v;function nt(){function e(e,t){return t.length-e.length}for(var t,n,s,i=[],r=[],a=[],o=[],u=0;u<7;u++)s=l([2e3,1]).day(u),t=M(this.weekdaysMin(s,"")),n=M(this.weekdaysShort(s,"")),s=M(this.weekdays(s,"")),i.push(t),r.push(n),a.push(s),o.push(t),o.push(n),o.push(s);i.sort(e),r.sort(e),a.sort(e),o.sort(e),this._weekdaysRegex=new RegExp("^("+o.join("|")+")","i"),this._weekdaysShortRegex=this._weekdaysRegex,this._weekdaysMinRegex=this._weekdaysRegex,this._weekdaysStrictRegex=new RegExp("^("+a.join("|")+")","i"),this._weekdaysShortStrictRegex=new RegExp("^("+r.join("|")+")","i"),this._weekdaysMinStrictRegex=new RegExp("^("+i.join("|")+")","i")}function st(){return this.hours()%12||12}function it(e,t){s(e,0,0,function(){return this.localeData().meridiem(this.hours(),this.minutes(),t)})}function rt(e,t){return t._meridiemParse}s("H",["HH",2],0,"hour"),s("h",["hh",2],0,st),s("k",["kk",2],0,function(){return this.hours()||24}),s("hmm",0,0,function(){return""+st.apply(this)+r(this.minutes(),2)}),s("hmmss",0,0,function(){return""+st.apply(this)+r(this.minutes(),2)+r(this.seconds(),2)}),s("Hmm",0,0,function(){return""+this.hours()+r(this.minutes(),2)}),s("Hmmss",0,0,function(){return""+this.hours()+r(this.minutes(),2)+r(this.seconds(),2)}),it("a",!0),it("A",!1),t("hour","h"),n("hour",13),k("a",rt),k("A",rt),k("H",p),k("h",p),k("k",p),k("HH",p,w),k("hh",p,w),k("kk",p,w),k("hmm",ge),k("hmmss",we),k("Hmm",ge),k("Hmmss",we),D(["H","HH"],x),D(["k","kk"],function(e,t,n){e=g(e);t[x]=24===e?0:e}),D(["a","A"],function(e,t,n){n._isPm=n._locale.isPM(e),n._meridiem=e}),D(["h","hh"],function(e,t,n){t[x]=g(e),m(n).bigHour=!0}),D("hmm",function(e,t,n){var s=e.length-2;t[x]=g(e.substr(0,s)),t[T]=g(e.substr(s)),m(n).bigHour=!0}),D("hmmss",function(e,t,n){var s=e.length-4,i=e.length-2;t[x]=g(e.substr(0,s)),t[T]=g(e.substr(s,2)),t[N]=g(e.substr(i)),m(n).bigHour=!0}),D("Hmm",function(e,t,n){var s=e.length-2;t[x]=g(e.substr(0,s)),t[T]=g(e.substr(s))}),D("Hmmss",function(e,t,n){var s=e.length-4,i=e.length-2;t[x]=g(e.substr(0,s)),t[T]=g(e.substr(s,2)),t[N]=g(e.substr(i))});v=de("Hours",!0);var at,ot={calendar:{sameDay:"[Today at] LT",nextDay:"[Tomorrow at] LT",nextWeek:"dddd [at] LT",lastDay:"[Yesterday at] LT",lastWeek:"[Last] dddd [at] LT",sameElse:"L"},longDateFormat:{LTS:"h:mm:ss A",LT:"h:mm A",L:"MM/DD/YYYY",LL:"MMMM D, YYYY",LLL:"MMMM D, YYYY h:mm A",LLLL:"dddd, MMMM D, YYYY h:mm A"},invalidDate:"Invalid date",ordinal:"%d",dayOfMonthOrdinalParse:/\d{1,2}/,relativeTime:{future:"in %s",past:"%s ago",s:"a few seconds",ss:"%d seconds",m:"a minute",mm:"%d minutes",h:"an hour",hh:"%d hours",d:"a day",dd:"%d days",w:"a week",ww:"%d weeks",M:"a month",MM:"%d months",y:"a year",yy:"%d years"},months:Ce,monthsShort:Ue,week:{dow:0,doy:6},weekdays:Je,weekdaysMin:Xe,weekdaysShort:Qe,meridiemParse:/[ap]\.?m?\.?/i},R={},ut={};function lt(e){return e&&e.toLowerCase().replace("_","-")}function ht(e){for(var t,n,s,i,r=0;r=t&&function(e,t){for(var n=Math.min(e.length,t.length),s=0;s=t-1)break;t--}r++}return at}function dt(t){var e;if(void 0===R[t]&&"undefined"!=typeof module&&module&&module.exports&&null!=t.match("^[^/\\\\]*$"))try{e=at._abbr,require("./locale/"+t),ct(e)}catch(e){R[t]=null}return R[t]}function ct(e,t){return e&&((t=o(t)?mt(e):ft(e,t))?at=t:"undefined"!=typeof console&&console.warn&&console.warn("Locale "+e+" not found. Did you forget to load it?")),at._abbr}function ft(e,t){if(null===t)return delete R[e],null;var n,s=ot;if(t.abbr=e,null!=R[e])Q("defineLocaleOverride","use moment.updateLocale(localeName, config) to change an existing locale. moment.defineLocale(localeName, config) should only be used for creating a new locale See http://momentjs.com/guides/#/warnings/define-locale/ for more info."),s=R[e]._config;else if(null!=t.parentLocale)if(null!=R[t.parentLocale])s=R[t.parentLocale]._config;else{if(null==(n=dt(t.parentLocale)))return ut[t.parentLocale]||(ut[t.parentLocale]=[]),ut[t.parentLocale].push({name:e,config:t}),null;s=n._config}return R[e]=new K(X(s,t)),ut[e]&&ut[e].forEach(function(e){ft(e.name,e.config)}),ct(e),R[e]}function mt(e){var t;if(!(e=e&&e._locale&&e._locale._abbr?e._locale._abbr:e))return at;if(!a(e)){if(t=dt(e))return t;e=[e]}return ht(e)}function _t(e){var t=e._a;return t&&-2===m(e).overflow&&(t=t[O]<0||11We(t[Y],t[O])?b:t[x]<0||24P(r,u,l)?m(s)._overflowWeeks=!0:null!=h?m(s)._overflowWeekday=!0:(d=$e(r,a,o,u,l),s._a[Y]=d.year,s._dayOfYear=d.dayOfYear)),null!=e._dayOfYear&&(i=bt(e._a[Y],n[Y]),(e._dayOfYear>Ae(i)||0===e._dayOfYear)&&(m(e)._overflowDayOfYear=!0),h=Ze(i,0,e._dayOfYear),e._a[O]=h.getUTCMonth(),e._a[b]=h.getUTCDate()),t=0;t<3&&null==e._a[t];++t)e._a[t]=c[t]=n[t];for(;t<7;t++)e._a[t]=c[t]=null==e._a[t]?2===t?1:0:e._a[t];24===e._a[x]&&0===e._a[T]&&0===e._a[N]&&0===e._a[Ne]&&(e._nextDay=!0,e._a[x]=0),e._d=(e._useUTC?Ze:je).apply(null,c),r=e._useUTC?e._d.getUTCDay():e._d.getDay(),null!=e._tzm&&e._d.setUTCMinutes(e._d.getUTCMinutes()-e._tzm),e._nextDay&&(e._a[x]=24),e._w&&void 0!==e._w.d&&e._w.d!==r&&(m(e).weekdayMismatch=!0)}}function Tt(e){if(e._f===f.ISO_8601)St(e);else if(e._f===f.RFC_2822)Ot(e);else{e._a=[],m(e).empty=!0;for(var t,n,s,i,r,a=""+e._i,o=a.length,u=0,l=ae(e._f,e._locale).match(te)||[],h=l.length,d=0;de.valueOf():e.valueOf()"}),i.toJSON=function(){return this.isValid()?this.toISOString():null},i.toString=function(){return this.clone().locale("en").format("ddd MMM DD YYYY HH:mm:ss [GMT]ZZ")},i.unix=function(){return Math.floor(this.valueOf()/1e3)},i.valueOf=function(){return this._d.valueOf()-6e4*(this._offset||0)},i.creationData=function(){return{input:this._i,format:this._f,locale:this._locale,isUTC:this._isUTC,strict:this._strict}},i.eraName=function(){for(var e,t=this.localeData().eras(),n=0,s=t.length;nthis.clone().month(0).utcOffset()||this.utcOffset()>this.clone().month(5).utcOffset()},i.isLocal=function(){return!!this.isValid()&&!this._isUTC},i.isUtcOffset=function(){return!!this.isValid()&&this._isUTC},i.isUtc=At,i.isUTC=At,i.zoneAbbr=function(){return this._isUTC?"UTC":""},i.zoneName=function(){return this._isUTC?"Coordinated Universal Time":""},i.dates=e("dates accessor is deprecated. Use date instead.",ve),i.months=e("months accessor is deprecated. Use month instead",Ge),i.years=e("years accessor is deprecated. Use year instead",Ie),i.zone=e("moment().zone is deprecated, use moment().utcOffset instead. http://momentjs.com/guides/#/warnings/zone/",function(e,t){return null!=e?(this.utcOffset(e="string"!=typeof e?-e:e,t),this):-this.utcOffset()}),i.isDSTShifted=e("isDSTShifted is deprecated. See http://momentjs.com/guides/#/warnings/dst-shifted/ for more information",function(){if(!o(this._isDSTShifted))return this._isDSTShifted;var e,t={};return $(t,this),(t=Nt(t))._a?(e=(t._isUTC?l:W)(t._a),this._isDSTShifted=this.isValid()&&0 Date: Fri, 9 Dec 2022 13:47:11 +0000 Subject: [PATCH 25/45] HADOOP-18546. Followup: ITestReadBufferManager fix (#5198) This is a followup to the original HADOOP-18546 patch; cherry-picks of that should include this or follow up with it. Removes risk of race conditions in assertions of ITestReadBufferManager on the state of the in-progress and completed queues by removing assertions brittle to race conditions in scheduling/network IO * Waits for all the executor pool shutdown to complete before making any assertions * Assertions that there are no in progress reads MUST be cut as there may be some and they won't be cancelled. * Assertions that the completed list is without buffers of a closed stream are brittle because if there was an in progress stream which completed after stream.close() then it will end up in the list. Contributed by Steve Loughran --- .../services/ITestReadBufferManager.java | 24 +++++++------------ 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java index 705cc2530d335..eca670fba9059 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java @@ -25,6 +25,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -74,17 +75,14 @@ public void testPurgeBufferManagerForParallelStreams() throws Exception { } } finally { executorService.shutdown(); + // wait for all tasks to finish + executorService.awaitTermination(1, TimeUnit.MINUTES); } ReadBufferManager bufferManager = ReadBufferManager.getBufferManager(); - assertListEmpty("CompletedList", bufferManager.getCompletedReadListCopy()); + // verify there is no work in progress or the readahead queue. assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList()); assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy()); - Assertions.assertThat(bufferManager.getFreeListCopy()) - .describedAs("After closing all streams free list contents should match with " + freeList) - .hasSize(numBuffers) - .containsExactlyInAnyOrderElementsOf(freeList); - } private void assertListEmpty(String listName, List list) { @@ -116,22 +114,18 @@ public void testPurgeBufferManagerForSequentialStream() throws Exception { try { iStream2 = (AbfsInputStream) fs.open(testFilePath).getWrappedStream(); iStream2.read(); - // After closing stream1, none of the buffers associated with stream1 should be present. - assertListDoesnotContainBuffersForIstream(bufferManager.getInProgressCopiedList(), iStream1); - assertListDoesnotContainBuffersForIstream(bufferManager.getCompletedReadListCopy(), iStream1); + // After closing stream1, no queued buffers of stream1 should be present + // assertions can't be made about the state of the other lists as it is + // too prone to race conditions. assertListDoesnotContainBuffersForIstream(bufferManager.getReadAheadQueueCopy(), iStream1); } finally { // closing the stream later. IOUtils.closeStream(iStream2); } - // After closing stream2, none of the buffers associated with stream2 should be present. - assertListDoesnotContainBuffersForIstream(bufferManager.getInProgressCopiedList(), iStream2); - assertListDoesnotContainBuffersForIstream(bufferManager.getCompletedReadListCopy(), iStream2); + // After closing stream2, no queued buffers of stream2 should be present. assertListDoesnotContainBuffersForIstream(bufferManager.getReadAheadQueueCopy(), iStream2); - // After closing both the streams, all lists should be empty. - assertListEmpty("CompletedList", bufferManager.getCompletedReadListCopy()); - assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList()); + // After closing both the streams, read queue should be empty. assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy()); } From c82a6f8425b761b9054e8ebe4258e9389617018b Mon Sep 17 00:00:00 2001 From: Jack Richard Buggins Date: Sat, 10 Dec 2022 14:27:05 +0000 Subject: [PATCH 26/45] HADOOP-18329. Support for IBM Semeru JVM > 11.0.15.0 Vendor Name Changes (#4537) The static boolean PlatformName.IBM_JAVA now identifies Java 11+ IBM Semeru runtimes as IBM JVM releases. Contributed by Jack Buggins. --- .../org/apache/hadoop/util/PlatformName.java | 66 +++++++++++++++++-- .../hadoop/security/ssl/SSLFactory.java | 6 +- .../apache/hadoop/minikdc/TestMiniKdc.java | 37 +++++++++-- 3 files changed, 95 insertions(+), 14 deletions(-) diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/util/PlatformName.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/util/PlatformName.java index eb52839b65ace..c52d5d2135106 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/util/PlatformName.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/util/PlatformName.java @@ -18,6 +18,10 @@ package org.apache.hadoop.util; +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.util.Arrays; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -33,21 +37,71 @@ public class PlatformName { * per the java-vm. */ public static final String PLATFORM_NAME = - (System.getProperty("os.name").startsWith("Windows") - ? System.getenv("os") : System.getProperty("os.name")) - + "-" + System.getProperty("os.arch") - + "-" + System.getProperty("sun.arch.data.model"); + (System.getProperty("os.name").startsWith("Windows") ? + System.getenv("os") : System.getProperty("os.name")) + + "-" + System.getProperty("os.arch") + "-" + + System.getProperty("sun.arch.data.model"); /** * The java vendor name used in this platform. */ public static final String JAVA_VENDOR_NAME = System.getProperty("java.vendor"); + /** + * Define a system class accessor that is open to changes in underlying implementations + * of the system class loader modules. + */ + private static final class SystemClassAccessor extends ClassLoader { + public Class getSystemClass(String className) throws ClassNotFoundException { + return findSystemClass(className); + } + } + /** * A public static variable to indicate the current java vendor is - * IBM java or not. + * IBM and the type is Java Technology Edition which provides its + * own implementations of many security packages and Cipher suites. + * Note that these are not provided in Semeru runtimes: + * See https://developer.ibm.com/languages/java/semeru-runtimes for details. */ - public static final boolean IBM_JAVA = JAVA_VENDOR_NAME.contains("IBM"); + public static final boolean IBM_JAVA = JAVA_VENDOR_NAME.contains("IBM") && + hasIbmTechnologyEditionModules(); + + private static boolean hasIbmTechnologyEditionModules() { + return Arrays.asList( + "com.ibm.security.auth.module.JAASLoginModule", + "com.ibm.security.auth.module.Win64LoginModule", + "com.ibm.security.auth.module.NTLoginModule", + "com.ibm.security.auth.module.AIX64LoginModule", + "com.ibm.security.auth.module.LinuxLoginModule", + "com.ibm.security.auth.module.Krb5LoginModule" + ).stream().anyMatch((module) -> isSystemClassAvailable(module)); + } + + /** + * In rare cases where different behaviour is performed based on the JVM vendor + * this method should be used to test for a unique JVM class provided by the + * vendor rather than using the vendor method. For example if on JVM provides a + * different Kerberos login module testing for that login module being loadable + * before configuring to use it is preferable to using the vendor data. + * + * @param className the name of a class in the JVM to test for + * @return true if the class is available, false otherwise. + */ + private static boolean isSystemClassAvailable(String className) { + return AccessController.doPrivileged((PrivilegedAction) () -> { + try { + // Using ClassLoader.findSystemClass() instead of + // Class.forName(className, false, null) because Class.forName with a null + // ClassLoader only looks at the boot ClassLoader with Java 9 and above + // which doesn't look at all the modules available to the findSystemClass. + new SystemClassAccessor().getSystemClass(className); + return true; + } catch (Exception ignored) { + return false; + } + }); + } public static void main(String[] args) { System.out.println(PLATFORM_NAME); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/SSLFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/SSLFactory.java index fe3233d848d4f..5ab38aa7420e9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/SSLFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/SSLFactory.java @@ -25,7 +25,7 @@ import org.apache.hadoop.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.hadoop.util.PlatformName.JAVA_VENDOR_NAME; +import static org.apache.hadoop.util.PlatformName.IBM_JAVA; import javax.net.ssl.HostnameVerifier; import javax.net.ssl.HttpsURLConnection; @@ -102,11 +102,11 @@ public enum Mode { CLIENT, SERVER } "ssl.server.exclude.cipher.list"; public static final String KEY_MANAGER_SSLCERTIFICATE = - JAVA_VENDOR_NAME.contains("IBM") ? "ibmX509" : + IBM_JAVA ? "ibmX509" : KeyManagerFactory.getDefaultAlgorithm(); public static final String TRUST_MANAGER_SSLCERTIFICATE = - JAVA_VENDOR_NAME.contains("IBM") ? "ibmX509" : + IBM_JAVA ? "ibmX509" : TrustManagerFactory.getDefaultAlgorithm(); public static final String KEYSTORES_FACTORY_CLASS_KEY = diff --git a/hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java b/hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java index 74130cff19b91..45684053a03ab 100644 --- a/hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java +++ b/hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java @@ -38,8 +38,35 @@ import java.util.Arrays; public class TestMiniKdc extends KerberosSecurityTestcase { - private static final boolean IBM_JAVA = System.getProperty("java.vendor") - .contains("IBM"); + private static final boolean IBM_JAVA = shouldUseIbmPackages(); + // duplicated to avoid cycles in the build + private static boolean shouldUseIbmPackages() { + final List ibmTechnologyEditionSecurityModules = Arrays.asList( + "com.ibm.security.auth.module.JAASLoginModule", + "com.ibm.security.auth.module.Win64LoginModule", + "com.ibm.security.auth.module.NTLoginModule", + "com.ibm.security.auth.module.AIX64LoginModule", + "com.ibm.security.auth.module.LinuxLoginModule", + "com.ibm.security.auth.module.Krb5LoginModule" + ); + + if (System.getProperty("java.vendor").contains("IBM")) { + return ibmTechnologyEditionSecurityModules + .stream().anyMatch((module) -> isSystemClassAvailable(module)); + } + + return false; + } + + private static boolean isSystemClassAvailable(String className) { + try { + Class.forName(className); + return true; + } catch (Exception ignored) { + return false; + } + } + @Test public void testMiniKdcStart() { MiniKdc kdc = getKdc(); @@ -117,9 +144,9 @@ public AppConfigurationEntry[] getAppConfigurationEntry(String name) { options.put("debug", "true"); return new AppConfigurationEntry[]{ - new AppConfigurationEntry(getKrb5LoginModuleName(), - AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, - options)}; + new AppConfigurationEntry(getKrb5LoginModuleName(), + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + options)}; } } From 8e249116438d819fcd92d1ea55d1a6143d219d05 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sun, 11 Dec 2022 07:03:49 +0800 Subject: [PATCH 27/45] YARN-11385. Fix hadoop-yarn-server-common module Java Doc Errors. (#5182) --- .../org/apache/hadoop/yarn/lib/ZKClient.java | 36 ++++++++++--------- .../apache/hadoop/yarn/lib/package-info.java | 8 +++-- .../server/AMHeartbeatRequestHandler.java | 1 + .../yarn/server/api/SCMUploaderProtocol.java | 8 ++--- .../hadoop/yarn/server/api/ServerRMProxy.java | 2 +- .../protocolrecords/NMContainerStatus.java | 9 +++-- .../api/protocolrecords/RemoteNode.java | 2 +- .../LocalityMulticastAMRMProxyPolicy.java | 4 +++ .../utils/FederationRegistryClient.java | 1 + .../server/scheduler/ResourceRequestSet.java | 6 ++-- .../security/BaseNMTokenSecretManager.java | 5 +++ .../yarn/server/service/package-info.java | 9 +++-- .../timeline/security/package-info.java | 4 +-- .../server/util/timeline/package-info.java | 4 +-- .../yarn/server/utils/LeveldbIterator.java | 22 ++++++++++-- .../server/utils/YarnServerSecurityUtils.java | 6 ++-- .../yarn/server/volume/csi/package-info.java | 9 +++-- .../hadoop/yarn/server/webapp/LogServlet.java | 3 ++ .../yarn/server/webapp/LogWebService.java | 6 ++++ .../server/webapp/WrappedLogMetaRequest.java | 1 + 20 files changed, 97 insertions(+), 49 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java index ba130c61ba0bf..21cbe20ab486f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java @@ -40,18 +40,19 @@ public class ZKClient { * the zookeeper client library to * talk to zookeeper * @param string the host - * @throws IOException + * @throws IOException if there are I/O errors. */ public ZKClient(String string) throws IOException { zkClient = new ZooKeeper(string, 30000, new ZKWatcher()); } /** - * register the service to a specific path + * register the service to a specific path. + * * @param path the path in zookeeper namespace to register to * @param data the data that is part of this registration - * @throws IOException - * @throws InterruptedException + * @throws IOException if there are I/O errors. + * @throws InterruptedException if any thread has interrupted. */ public void registerService(String path, String data) throws IOException, InterruptedException { @@ -64,13 +65,14 @@ public void registerService(String path, String data) throws } /** - * unregister the service. + * unregister the service. + * * @param path the path at which the service was registered - * @throws IOException - * @throws InterruptedException + * @throws IOException if there are I/O errors. + * @throws InterruptedException if any thread has interrupted. */ public void unregisterService(String path) throws IOException, - InterruptedException { + InterruptedException { try { zkClient.delete(path, -1); } catch(KeeperException ke) { @@ -79,15 +81,16 @@ public void unregisterService(String path) throws IOException, } /** - * list the services registered under a path + * list the services registered under a path. + * * @param path the path under which services are * registered * @return the list of names of services registered - * @throws IOException - * @throws InterruptedException + * @throws IOException if there are I/O errors. + * @throws InterruptedException if any thread has interrupted. */ public List listServices(String path) throws IOException, - InterruptedException { + InterruptedException { List children = null; try { children = zkClient.getChildren(path, false); @@ -98,14 +101,15 @@ public List listServices(String path) throws IOException, } /** - * get data published by the service at the registration address + * get data published by the service at the registration address. + * * @param path the path where the service is registered * @return the data of the registered service - * @throws IOException - * @throws InterruptedException + * @throws IOException if there are I/O errors. + * @throws InterruptedException if any thread has interrupted. */ public String getServiceData(String path) throws IOException, - InterruptedException { + InterruptedException { String data; try { Stat stat = new Stat(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/package-info.java index d4fa452c3aed6..ba287fdfe437f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/package-info.java @@ -15,7 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -@InterfaceAudience.Private + +/** + * This package contains zkClient related classes. + */ +@Private package org.apache.hadoop.yarn.lib; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMHeartbeatRequestHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMHeartbeatRequestHandler.java index 9a73fb308ce0d..cb59d41505deb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMHeartbeatRequestHandler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMHeartbeatRequestHandler.java @@ -153,6 +153,7 @@ public void resetLastResponseId() { /** * Set the UGI for RM connection. + * @param ugi UserGroupInformation. */ public void setUGI(UserGroupInformation ugi) { this.userUgi = ugi; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/SCMUploaderProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/SCMUploaderProtocol.java index 937f648510c00..b73a02af6c502 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/SCMUploaderProtocol.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/SCMUploaderProtocol.java @@ -53,8 +53,8 @@ public interface SCMUploaderProtocol { * to the shared cache * @return response indicating if the newly uploaded resource should be * deleted - * @throws YarnException - * @throws IOException + * @throws YarnException exceptions from yarn servers. + * @throws IOException if there are I/O errors. */ public SCMUploaderNotifyResponse notify(SCMUploaderNotifyRequest request) @@ -73,8 +73,8 @@ public interface SCMUploaderProtocol { * * @param request whether the resource can be uploaded to the shared cache * @return response indicating if resource can be uploaded to the shared cache - * @throws YarnException - * @throws IOException + * @throws YarnException exceptions from yarn servers. + * @throws IOException if there are I/O errors. */ public SCMUploaderCanUploadResponse canUpload(SCMUploaderCanUploadRequest request) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java index 50eed3a75d511..fa7e390a947c3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java @@ -49,7 +49,7 @@ private ServerRMProxy() { * @param protocol Server protocol for which proxy is being requested. * @param Type of proxy. * @return Proxy to the ResourceManager for the specified server protocol. - * @throws IOException + * @throws IOException if there are I/O errors. */ public static T createRMProxy(final Configuration configuration, final Class protocol) throws IOException { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java index 065918d5eb97d..6027ce7452e7d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java @@ -120,14 +120,18 @@ public static NMContainerStatus newInstance(ContainerId containerId, public abstract void setPriority(Priority priority); /** - * Get the time when the container is created + * Get the time when the container is created. + * + * @return CreationTime. */ public abstract long getCreationTime(); public abstract void setCreationTime(long creationTime); /** - * Get the node-label-expression in the original ResourceRequest + * Get the node-label-expression in the original ResourceRequest. + * + * @return NodeLabelExpression. */ public abstract String getNodeLabelExpression(); @@ -167,6 +171,7 @@ public void setExecutionType(ExecutionType executionType) { } /** * Get and set the Allocation tags associated with the container. + * @return Allocation tags. */ public Set getAllocationTags() { return Collections.emptySet(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RemoteNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RemoteNode.java index 67ad5bac294a3..72dcb6e99144c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RemoteNode.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RemoteNode.java @@ -148,7 +148,7 @@ public static RemoteNode newInstance(NodeId nodeId, String httpAddress, /** * Set Node Partition. - * @param nodePartition + * @param nodePartition node Partition. */ @Private @Unstable 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/LocalityMulticastAMRMProxyPolicy.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/LocalityMulticastAMRMProxyPolicy.java index c8f945896e4dc..17cc8390a11a5 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/LocalityMulticastAMRMProxyPolicy.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/LocalityMulticastAMRMProxyPolicy.java @@ -290,6 +290,10 @@ public Map> splitResourceRequests( /** * For unit test to override. + * + * @param bookKeeper bookKeeper + * @param allocationId allocationId. + * @return SubClusterId. */ protected SubClusterId getSubClusterForUnResolvedRequest( AllocationBookkeeper bookKeeper, long allocationId) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationRegistryClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationRegistryClient.java index 1eb120c455469..fa64188a608b9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationRegistryClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationRegistryClient.java @@ -111,6 +111,7 @@ public synchronized void cleanAllApplications() { /** * Write/update the UAM token for an application and a sub-cluster. * + * @param appId ApplicationId. * @param subClusterId sub-cluster id of the token * @param token the UAM of the application * @return whether the amrmToken is added or updated to a new value diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/ResourceRequestSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/ResourceRequestSet.java index cf24bbf361f21..ed615e85c9ca4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/ResourceRequestSet.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/ResourceRequestSet.java @@ -71,7 +71,7 @@ public ResourceRequestSet(ResourceRequestSet other) { * with the same resource name, override it and update accordingly. * * @param ask the new {@link ResourceRequest} - * @throws YarnException + * @throws YarnException indicates exceptions from yarn servers. */ public void addAndOverrideRR(ResourceRequest ask) throws YarnException { if (!this.key.equals(new ResourceRequestSetKey(ask))) { @@ -102,7 +102,7 @@ public void addAndOverrideRR(ResourceRequest ask) throws YarnException { * Merge a requestSet into this one. * * @param requestSet the requestSet to merge - * @throws YarnException + * @throws YarnException indicates exceptions from yarn servers. */ public void addAndOverrideRRSet(ResourceRequestSet requestSet) throws YarnException { @@ -149,7 +149,7 @@ public int getNumContainers() { * Force set the # of containers to ask for this requestSet to a given value. * * @param newValue the new # of containers value - * @throws YarnException + * @throws YarnException indicates exceptions from yarn servers. */ public void setNumContainers(int newValue) throws YarnException { if (this.numContainers == 0) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/BaseNMTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/BaseNMTokenSecretManager.java index 3cbd1dc36dcce..b8b5073119c7f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/BaseNMTokenSecretManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/BaseNMTokenSecretManager.java @@ -111,6 +111,11 @@ public NMTokenIdentifier createIdentifier() { /** * Helper function for creating NMTokens. + * + * @param applicationAttemptId application AttemptId. + * @param nodeId node Id. + * @param applicationSubmitter application Submitter. + * @return NMToken. */ public Token createNMToken(ApplicationAttemptId applicationAttemptId, NodeId nodeId, String applicationSubmitter) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/service/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/service/package-info.java index c448bab134d0e..c27269820ed3c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/service/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/service/package-info.java @@ -19,9 +19,8 @@ * Package org.apache.hadoop.yarn.server.service contains service related * classes. */ -@InterfaceAudience.Private @InterfaceStability.Unstable - +@Private +@Unstable package org.apache.hadoop.yarn.server.service; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/package-info.java index 14a52e342b30e..76ea2064b2b2b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/package-info.java @@ -21,6 +21,6 @@ * to timeline authentication filters and abstract delegation token service for * ATSv1 and ATSv2. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.server.timeline.security; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/package-info.java index 75c69738c5064..0c61b6246cc38 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/package-info.java @@ -20,6 +20,6 @@ * Package org.apache.hadoop.server.util.timeline contains utility classes used * by ATSv1 and ATSv2 on the server side. */ -@InterfaceAudience.Private +@Private package org.apache.hadoop.yarn.server.util.timeline; -import org.apache.hadoop.classification.InterfaceAudience; \ No newline at end of file +import org.apache.hadoop.classification.InterfaceAudience.Private; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/LeveldbIterator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/LeveldbIterator.java index f33cb5f1d8965..463bee7ebab89 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/LeveldbIterator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/LeveldbIterator.java @@ -41,21 +41,28 @@ public class LeveldbIterator implements Iterator>, private DBIterator iter; /** - * Create an iterator for the specified database + * Create an iterator for the specified database. + * + * @param db database. */ public LeveldbIterator(DB db) { iter = db.iterator(); } /** - * Create an iterator for the specified database + * Create an iterator for the specified database. + * + * @param db db. + * @param options ReadOptions. */ public LeveldbIterator(DB db, ReadOptions options) { iter = db.iterator(options); } /** - * Create an iterator using the specified underlying DBIterator + * Create an iterator using the specified underlying DBIterator. + * + * @param iter DB Iterator. */ public LeveldbIterator(DBIterator iter) { this.iter = iter; @@ -64,6 +71,9 @@ public LeveldbIterator(DBIterator iter) { /** * Repositions the iterator so the key of the next BlockElement * returned greater than or equal to the specified targetKey. + * + * @param key key of the next BlockElement. + * @throws DBException db Exception. */ public void seek(byte[] key) throws DBException { try { @@ -116,6 +126,9 @@ public boolean hasNext() throws DBException { /** * Returns the next element in the iteration. + * + * @return the next element in the iteration. + * @throws DBException DB Exception. */ @Override public Map.Entry next() throws DBException { @@ -131,6 +144,9 @@ public Map.Entry next() throws DBException { /** * Returns the next element in the iteration, without advancing the * iteration. + * + * @return the next element in the iteration. + * @throws DBException db Exception. */ public Map.Entry peekNext() throws DBException { try { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerSecurityUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerSecurityUtils.java index c5ae56f3d1090..4ad6a94ab11d0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerSecurityUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerSecurityUtils.java @@ -56,7 +56,7 @@ private YarnServerSecurityUtils() { * current application. * * @return the AMRMTokenIdentifier instance for the current user - * @throws YarnException + * @throws YarnException exceptions from yarn servers. */ public static AMRMTokenIdentifier authorizeRequest() throws YarnException { @@ -137,9 +137,9 @@ public static void updateAMRMToken( * Parses the container launch context and returns a Credential instance that * contains all the tokens from the launch context. * - * @param launchContext + * @param launchContext ContainerLaunchContext. * @return the credential instance - * @throws IOException + * @throws IOException if there are I/O errors. */ public static Credentials parseCredentials( ContainerLaunchContext launchContext) throws IOException { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/volume/csi/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/volume/csi/package-info.java index ef4ffef56464f..64b42c7c43559 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/volume/csi/package-info.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/volume/csi/package-info.java @@ -19,9 +19,8 @@ /** * This package contains common volume related classes. */ -@InterfaceAudience.Private -@InterfaceStability.Unstable +@Private +@Unstable package org.apache.hadoop.yarn.server.volume.csi; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogServlet.java index 86b8d55adc65f..16fac7ac43946 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogServlet.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogServlet.java @@ -187,7 +187,10 @@ private void validateUserInput(ApplicationId applicationId, * Returns the user qualified path name of the remote log directory for * each pre-configured log aggregation file controller. * + * @param user remoteUser. + * @param applicationId applicationId. * @return {@link Response} object containing remote log dir path names + * @throws IOException if there are I/O errors. */ public Response getRemoteLogDirPath(String user, String applicationId) throws IOException { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogWebService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogWebService.java index 1edfd5287ac41..565d4fd8c9e16 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogWebService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogWebService.java @@ -138,6 +138,9 @@ private void initForReadableEndpoints(HttpServletResponse response) { * @param containerIdStr The container ID * @param nmId The Node Manager NodeId * @param redirectedFromNode Whether this is a redirected request from NM + * @param clusterId clusterId the id of the cluster + * @param manualRedirection whether to return a response with a Location + * instead of an automatic redirection * @return The log file's name and current file size */ @GET @@ -242,6 +245,9 @@ public BasicAppInfo getApp(HttpServletRequest req, String appId, * @param size the size of the log file * @param nmId The Node Manager NodeId * @param redirectedFromNode Whether this is the redirect request from NM + * @param clusterId the id of the cluster + * @param manualRedirection whether to return a response with a Location + * instead of an automatic redirection * @return The contents of the container's log file */ @GET diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WrappedLogMetaRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WrappedLogMetaRequest.java index d39eef8cee493..59a88c2618693 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WrappedLogMetaRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WrappedLogMetaRequest.java @@ -155,6 +155,7 @@ public ContainerId getContainerId() { * * @return list of {@link ContainerLogMeta} objects that belong * to the application, attempt or container + * @throws IOException if there are I/O errors. */ public List getContainerLogMetas() throws IOException { ApplicationId applicationId = ApplicationId.fromString(getAppId()); From 07ccbf65fe76076efcef83102b30c27522831f88 Mon Sep 17 00:00:00 2001 From: curie71 <39853223+curie71@users.noreply.github.com> Date: Tue, 13 Dec 2022 12:24:51 +0800 Subject: [PATCH 28/45] HDFS-16868. Fix audit log duplicate issue when an ACE occurs in FSNamesystem. (#5206). Contributed by Beibei Zhao. Signed-off-by: Chris Nauroth Signed-off-by: He Xiaoqiao --- .../hdfs/server/namenode/FSNamesystem.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 5b8bc0ac61cfb..ccffcd0c70ba8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -3621,10 +3621,10 @@ void setQuota(String src, long nsQuota, long ssQuota, StorageType type) final String operationName = getQuotaCommand(nsQuota, ssQuota); final FSPermissionChecker pc = getPermissionChecker(); FSPermissionChecker.setOperationType(operationName); + if(!allowOwnerSetQuota) { + checkSuperuserPrivilege(operationName, src); + } try { - if(!allowOwnerSetQuota) { - checkSuperuserPrivilege(operationName, src); - } writeLock(); try { checkOperation(OperationCategory.WRITE); @@ -7761,8 +7761,8 @@ void addCachePool(CachePoolInfo req, boolean logRetryCache) checkOperation(OperationCategory.WRITE); String poolInfoStr = null; String poolName = req == null ? null : req.getPoolName(); + checkSuperuserPrivilege(operationName, poolName); try { - checkSuperuserPrivilege(operationName, poolName); writeLock(); try { checkOperation(OperationCategory.WRITE); @@ -7788,8 +7788,8 @@ void modifyCachePool(CachePoolInfo req, boolean logRetryCache) checkOperation(OperationCategory.WRITE); String poolNameStr = "{poolName: " + (req == null ? null : req.getPoolName()) + "}"; + checkSuperuserPrivilege(operationName, poolNameStr); try { - checkSuperuserPrivilege(operationName, poolNameStr); writeLock(); try { checkOperation(OperationCategory.WRITE); @@ -7815,8 +7815,8 @@ void removeCachePool(String cachePoolName, boolean logRetryCache) final String operationName = "removeCachePool"; checkOperation(OperationCategory.WRITE); String poolNameStr = "{poolName: " + cachePoolName + "}"; + checkSuperuserPrivilege(operationName, poolNameStr); try { - checkSuperuserPrivilege(operationName, poolNameStr); writeLock(); try { checkOperation(OperationCategory.WRITE); @@ -8017,11 +8017,11 @@ void createEncryptionZone(final String src, final String keyName, SafeModeException, AccessControlException { final String operationName = "createEncryptionZone"; FileStatus resultingStat = null; + checkSuperuserPrivilege(operationName, src); try { Metadata metadata = FSDirEncryptionZoneOp.ensureKeyIsInitialized(dir, keyName, src); final FSPermissionChecker pc = getPermissionChecker(); - checkSuperuserPrivilege(operationName, src); checkOperation(OperationCategory.WRITE); writeLock(); try { @@ -8100,11 +8100,11 @@ void reencryptEncryptionZone(final String zone, final ReencryptAction action, final boolean logRetryCache) throws IOException { final String operationName = "reencryptEncryptionZone"; boolean success = false; + checkSuperuserPrivilege(operationName, zone); try { Preconditions.checkNotNull(zone, "zone is null."); checkOperation(OperationCategory.WRITE); final FSPermissionChecker pc = dir.getPermissionChecker(); - checkSuperuserPrivilege(operationName, zone); checkNameNodeSafeMode("NameNode in safemode, cannot " + action + " re-encryption on zone " + zone); reencryptEncryptionZoneInt(pc, zone, action, logRetryCache); From d24a5c58ec4e6f028f62270eba0086a4e1367a08 Mon Sep 17 00:00:00 2001 From: Ashutosh Gupta Date: Wed, 14 Dec 2022 12:54:08 +0000 Subject: [PATCH 29/45] MAPREDUCE-7428. Fix failures related to Junit 4 to Junit 5 upgrade in org.apache.hadoop.mapreduce.v2.app.webapp (#5209) Contributed by: Ashutosh Gupta --- .../hadoop-mapreduce-client-app/pom.xml | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml index e3b3511c0ce17..ff268cbd049dd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml @@ -124,6 +124,26 @@ assertj-core test + + org.junit.platform + junit-platform-launcher + test + + + org.junit.jupiter + junit-jupiter-api + test + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test + From 19cd2cf098e22f4336ec7647cf45f0b66df71a2e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 14 Dec 2022 14:01:28 +0000 Subject: [PATCH 30/45] HADOOP-18183. s3a audit logs to publish range start/end of GET requests. (#5110) The start and end of the range is set in a new audit param "rg", e.g "?rg=100-200" Contributed by Ankit Saurabh --- .../hadoop/fs/audit/AuditConstants.java | 5 +++ .../fs/s3a/audit/impl/LoggingAuditor.java | 32 +++++++++++++++ .../markdown/tools/hadoop-aws/auditing.md | 1 + .../fs/s3a/audit/AbstractAuditingTest.java | 24 +++++++++++ .../audit/TestHttpReferrerAuditHeader.java | 41 +++++++++++++++++++ 5 files changed, 103 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java index 0929c2be03acf..ffca6097c4733 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java @@ -90,6 +90,11 @@ private AuditConstants() { */ public static final String PARAM_PROCESS = "ps"; + /** + * Header: Range for GET request data: {@value}. + */ + public static final String PARAM_RANGE = "rg"; + /** * Task Attempt ID query header: {@value}. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java index da1f5b59bdc36..feb926a0bfca2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java @@ -25,6 +25,7 @@ import java.util.Map; import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,6 +36,7 @@ import org.apache.hadoop.fs.s3a.audit.AWSRequestAnalyzer; import org.apache.hadoop.fs.s3a.audit.AuditFailureException; import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader; import org.apache.hadoop.security.UserGroupInformation; @@ -110,6 +112,14 @@ public class LoggingAuditor */ private Collection filters; + /** + * Log for warning of problems getting the range of GetObjectRequest + * will only log of a problem once per process instance. + * This is to avoid logs being flooded with errors. + */ + private static final LogExactlyOnce WARN_INCORRECT_RANGE = + new LogExactlyOnce(LOG); + /** * Create the auditor. * The UGI current user is used to provide the principal; @@ -230,6 +240,26 @@ private class LoggingAuditSpan extends AbstractAuditSpanImpl { private final HttpReferrerAuditHeader referrer; + /** + * Attach Range of data for GetObject Request. + * @param request given get object request + */ + private void attachRangeFromRequest(AmazonWebServiceRequest request) { + if (request instanceof GetObjectRequest) { + long[] rangeValue = ((GetObjectRequest) request).getRange(); + if (rangeValue == null || rangeValue.length == 0) { + return; + } + if (rangeValue.length != 2) { + WARN_INCORRECT_RANGE.warn("Expected range to contain 0 or 2 elements." + + " Got {} elements. Ignoring.", rangeValue.length); + return; + } + String combinedRangeValue = String.format("%d-%d", rangeValue[0], rangeValue[1]); + referrer.set(AuditConstants.PARAM_RANGE, combinedRangeValue); + } + } + private final String description; private LoggingAuditSpan( @@ -314,6 +344,8 @@ public void set(final String key, final String value) { @Override public T beforeExecution( final T request) { + // attach range for GetObject requests + attachRangeFromRequest(request); // build the referrer header final String header = referrer.buildHttpReferrer(); // update the outer class's field. diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md index 8ccc36cf83bb1..d7b95b14877a4 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md @@ -232,6 +232,7 @@ If any of the field values were `null`, the field is omitted. | `p2` | Path 2 of operation | `s3a://alice-london/path2` | | `pr` | Principal | `alice` | | `ps` | Unique process UUID | `235865a0-d399-4696-9978-64568db1b51c` | +| `rg` | GET request range | `100-200` | | `ta` | Task Attempt ID (S3A committer) | | | `t0` | Thread 0: thread span was created in | `100` | | `t1` | Thread 1: thread this operation was executed in | `200` | diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java index c76e3fa968f92..f5e5cd5e954af 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java @@ -20,8 +20,10 @@ import java.io.IOException; import java.util.Map; +import java.util.function.Consumer; import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; import org.junit.After; import org.junit.Before; import org.slf4j.Logger; @@ -138,6 +140,17 @@ protected GetObjectMetadataRequest head() { requestFactory.newGetObjectMetadataRequest("/")); } + /** + * Create a GetObject request and modify it before passing it through auditor. + * @param modifyRequest Consumer Interface for changing the request before passing to the auditor + * @return the request + */ + protected GetObjectRequest get(Consumer modifyRequest) { + GetObjectRequest req = requestFactory.newGetObjectRequest("/"); + modifyRequest.accept(req); + return manager.beforeExecution(req); + } + /** * Assert a head request fails as there is no * active span. @@ -210,4 +223,15 @@ protected void assertMapContains(final Map params, .isEqualTo(expected); } + /** + * Assert the map does not contain the key, i.e, it is null. + * @param params map of params + * @param key key + */ + protected void assertMapNotContains(final Map params, final String key) { + assertThat(params.get(key)) + .describedAs(key) + .isNull(); + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java index b653d24d4168a..af94e1455fc30 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java @@ -23,6 +23,7 @@ import java.util.regex.Matcher; import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; @@ -46,6 +47,7 @@ import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PATH; import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PATH2; import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PRINCIPAL; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_RANGE; import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD0; import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD1; import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_TIMESTAMP; @@ -115,6 +117,7 @@ public void testHttpReferrerPatchesTheRequest() throws Throwable { assertThat(span.getTimestamp()) .describedAs("Timestamp of " + span) .isEqualTo(ts); + assertMapNotContains(params, PARAM_RANGE); assertMapContains(params, PARAM_TIMESTAMP, Long.toString(ts)); @@ -309,6 +312,44 @@ public void testStripWrappedQuotes() throws Throwable { expectStrippedField("\"\"\"b\"", "b"); } + /** + * Verify that correct range is getting published in header. + */ + @Test + public void testGetObjectRange() throws Throwable { + AuditSpan span = span(); + GetObjectRequest request = get(getObjectRequest -> getObjectRequest.setRange(100, 200)); + Map headers + = request.getCustomRequestHeaders(); + assertThat(headers) + .describedAs("Custom headers") + .containsKey(HEADER_REFERRER); + String header = headers.get(HEADER_REFERRER); + LOG.info("Header is {}", header); + Map params + = HttpReferrerAuditHeader.extractQueryParameters(header); + assertMapContains(params, PARAM_RANGE, "100-200"); + } + + /** + * Verify that no range is getting added to the header in request without range. + */ + @Test + public void testGetObjectWithoutRange() throws Throwable { + AuditSpan span = span(); + GetObjectRequest request = get(getObjectRequest -> {}); + Map headers + = request.getCustomRequestHeaders(); + assertThat(headers) + .describedAs("Custom headers") + .containsKey(HEADER_REFERRER); + String header = headers.get(HEADER_REFERRER); + LOG.info("Header is {}", header); + Map params + = HttpReferrerAuditHeader.extractQueryParameters(header); + assertMapNotContains(params, PARAM_RANGE); + } + /** * Expect a field with quote stripping to match the expected value. * @param str string to strip From 60b7a92df371800dee3593301f58402b2b909d7b Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Wed, 14 Dec 2022 15:55:44 +0100 Subject: [PATCH 31/45] HADOOP-18569. NFS Gateway may release buffer too early (#5212) (cherry picked from commit df4812df65d01889ba93bce1415e01461500208d) --- .../java/org/apache/hadoop/oncrpc/RpcProgram.java | 13 +++++++++++-- .../main/java/org/apache/hadoop/oncrpc/RpcUtil.java | 8 +++++--- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcProgram.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcProgram.java index 252eae64b5359..53647126463ab 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcProgram.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcProgram.java @@ -26,6 +26,7 @@ import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.util.ReferenceCountUtil; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.oncrpc.RpcAcceptedReply.AcceptState; import org.apache.hadoop.oncrpc.security.VerifierNone; @@ -163,8 +164,16 @@ public void stopDaemons() {} public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { RpcInfo info = (RpcInfo) msg; + try { + channelRead(ctx, info); + } finally { + ReferenceCountUtil.release(info.data()); + } + } + + private void channelRead(ChannelHandlerContext ctx, RpcInfo info) + throws Exception { RpcCall call = (RpcCall) info.header(); - SocketAddress remoteAddress = info.remoteAddress(); if (LOG.isTraceEnabled()) { LOG.trace(program + " procedure #" + call.getProcedure()); @@ -256,4 +265,4 @@ public int getPort() { public int getPortmapUdpTimeoutMillis() { return portmapUdpTimeoutMillis; } -} \ No newline at end of file +} diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java index caba13105cca3..edf42444b8b76 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java @@ -129,15 +129,17 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) RpcInfo info = null; try { RpcCall callHeader = RpcCall.read(in); - ByteBuf dataBuffer = Unpooled.wrappedBuffer(in.buffer() - .slice()); + ByteBuf dataBuffer = buf.slice(b.position(), b.remaining()); info = new RpcInfo(callHeader, dataBuffer, ctx, ctx.channel(), remoteAddress); } catch (Exception exc) { LOG.info("Malformed RPC request from " + remoteAddress); } finally { - buf.release(); + // only release buffer if it is not passed to downstream handler + if (info == null) { + buf.release(); + } } if (info != null) { From d688a5cef69ddbe04bc7ffcd66e134abb03d3785 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Thu, 15 Dec 2022 01:09:38 +0800 Subject: [PATCH 32/45] YARN-11350. [Federation] Router Support DelegationToken With ZK. (#5131) --- .../FederationDelegationTokenStateStore.java | 35 + .../impl/MemoryFederationStateStore.java | 30 + .../store/impl/SQLFederationStateStore.java | 25 + .../ZKFederationStateStoreOpDurations.java | 49 ++ .../impl/ZookeeperFederationStateStore.java | 690 +++++++++++++++++- .../store/records/RouterStoreToken.java | 9 + .../impl/pb/RouterStoreTokenPBImpl.java | 9 + ...FederationRouterRMTokenInputValidator.java | 105 +++ .../utils/FederationStateStoreFacade.java | 47 +- .../impl/FederationStateStoreBaseTest.java | 15 + .../impl/TestMemoryFederationStateStore.java | 58 +- .../impl/TestSQLFederationStateStore.java | 18 + .../TestZookeeperFederationStateStore.java | 123 +++- .../FederationStateStoreService.java | 25 + .../RouterDelegationTokenSecretManager.java | 25 + 15 files changed, 1223 insertions(+), 40 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationDelegationTokenStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationDelegationTokenStateStore.java index 384295fd09dcc..452bcf9d4ad87 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationDelegationTokenStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationDelegationTokenStateStore.java @@ -112,4 +112,39 @@ RouterRMTokenResponse removeStoredToken(RouterRMTokenRequest request) */ RouterRMTokenResponse getTokenByRouterStoreToken(RouterRMTokenRequest request) throws YarnException, IOException; + + /** + * The Router Supports incrementDelegationTokenSeqNum. + * + * @return DelegationTokenSeqNum. + */ + int incrementDelegationTokenSeqNum(); + + /** + * The Router Supports getDelegationTokenSeqNum. + * + * @return DelegationTokenSeqNum. + */ + int getDelegationTokenSeqNum(); + + /** + * The Router Supports setDelegationTokenSeqNum. + * + * @param seqNum DelegationTokenSeqNum. + */ + void setDelegationTokenSeqNum(int seqNum); + + /** + * The Router Supports getCurrentKeyId. + * + * @return CurrentKeyId. + */ + int getCurrentKeyId(); + + /** + * The Router Supports incrementCurrentKeyId. + * + * @return CurrentKeyId. + */ + int incrementCurrentKeyId(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java index 2d210ab37496c..41ade680be2e3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java @@ -27,6 +27,7 @@ import java.util.Set; import java.util.TimeZone; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.Comparator; @@ -110,6 +111,8 @@ public class MemoryFederationStateStore implements FederationStateStore { private Map policies; private RouterRMDTSecretManagerState routerRMSecretManagerState; private int maxAppsInStateStore; + private AtomicInteger sequenceNum; + private AtomicInteger masterKeyId; private final MonotonicClock clock = new MonotonicClock(); @@ -126,6 +129,8 @@ public void init(Configuration conf) { maxAppsInStateStore = conf.getInt( YarnConfiguration.FEDERATION_STATESTORE_MAX_APPLICATIONS, YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_MAX_APPLICATIONS); + sequenceNum = new AtomicInteger(); + masterKeyId = new AtomicInteger(); } @Override @@ -534,6 +539,31 @@ public RouterRMTokenResponse getTokenByRouterStoreToken(RouterRMTokenRequest req return RouterRMTokenResponse.newInstance(resultToken); } + @Override + public int incrementDelegationTokenSeqNum() { + return sequenceNum.incrementAndGet(); + } + + @Override + public int getDelegationTokenSeqNum() { + return sequenceNum.get(); + } + + @Override + public void setDelegationTokenSeqNum(int seqNum) { + sequenceNum.set(seqNum); + } + + @Override + public int getCurrentKeyId() { + return masterKeyId.get(); + } + + @Override + public int incrementCurrentKeyId() { + return masterKeyId.incrementAndGet(); + } + private void storeOrUpdateRouterRMDT(RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate, boolean isUpdate) throws IOException { Map rmDTState = routerRMSecretManagerState.getTokenState(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java index 2bf2658944e5f..1e3f3a12f3dcb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java @@ -1394,4 +1394,29 @@ public RouterRMTokenResponse getTokenByRouterStoreToken(RouterRMTokenRequest req throws YarnException, IOException { throw new NotImplementedException("Code is not implemented"); } + + @Override + public int incrementDelegationTokenSeqNum() { + return 0; + } + + @Override + public int getDelegationTokenSeqNum() { + return 0; + } + + @Override + public void setDelegationTokenSeqNum(int seqNum) { + return; + } + + @Override + public int getCurrentKeyId() { + return 0; + } + + @Override + public int incrementCurrentKeyId() { + return 0; + } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZKFederationStateStoreOpDurations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZKFederationStateStoreOpDurations.java index 113e4850a5709..54b8b5f4ddae1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZKFederationStateStoreOpDurations.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZKFederationStateStoreOpDurations.java @@ -89,6 +89,27 @@ public final class ZKFederationStateStoreOpDurations implements MetricsSource { @Metric("Duration for a update reservation homeSubCluster call") private MutableRate updateReservationHomeSubCluster; + @Metric("Duration for a store new master key call") + private MutableRate storeNewMasterKey; + + @Metric("Duration for a remove new master key call") + private MutableRate removeStoredMasterKey; + + @Metric("Duration for a get master key by delegation key call") + private MutableRate getMasterKeyByDelegationKey; + + @Metric("Duration for a store new token call") + private MutableRate storeNewToken; + + @Metric("Duration for a update stored token call") + private MutableRate updateStoredToken; + + @Metric("Duration for a remove stored token call") + private MutableRate removeStoredToken; + + @Metric("Duration for a get token by router store token call") + private MutableRate getTokenByRouterStoreToken; + protected static final MetricsInfo RECORD_INFO = info("ZKFederationStateStoreOpDurations", "Durations of ZKFederationStateStore calls"); @@ -187,4 +208,32 @@ public void addDeleteReservationHomeSubClusterDuration(long startTime, long endT public void addUpdateReservationHomeSubClusterDuration(long startTime, long endTime) { updateReservationHomeSubCluster.add(endTime - startTime); } + + public void addStoreNewMasterKeyDuration(long startTime, long endTime) { + storeNewMasterKey.add(endTime - startTime); + } + + public void removeStoredMasterKeyDuration(long startTime, long endTime) { + removeStoredMasterKey.add(endTime - startTime); + } + + public void getMasterKeyByDelegationKeyDuration(long startTime, long endTime) { + getMasterKeyByDelegationKey.add(endTime - startTime); + } + + public void getStoreNewTokenDuration(long startTime, long endTime) { + storeNewToken.add(endTime - startTime); + } + + public void updateStoredTokenDuration(long startTime, long endTime) { + updateStoredToken.add(endTime - startTime); + } + + public void removeStoredTokenDuration(long startTime, long endTime) { + removeStoredToken.add(endTime - startTime); + } + + public void getTokenByRouterStoreTokenDuration(long startTime, long endTime) { + getTokenByRouterStoreToken.add(endTime - startTime); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java index f4d45f5a723c1..95903b81d18ad 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java @@ -17,9 +17,12 @@ package org.apache.hadoop.yarn.server.federation.store.impl; -import static org.apache.hadoop.util.curator.ZKCuratorManager.getNodePath; - import java.io.IOException; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Calendar; import java.util.List; @@ -27,9 +30,11 @@ import java.util.Comparator; import java.util.stream.Collectors; -import org.apache.commons.lang3.NotImplementedException; +import org.apache.curator.framework.recipes.shared.SharedCount; +import org.apache.curator.framework.recipes.shared.VersionedValue; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.util.curator.ZKCuratorManager; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -37,6 +42,7 @@ import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto; import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterInfoProto; import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterPolicyConfigurationProto; +import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.store.FederationStateStore; import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest; import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse; @@ -87,14 +93,18 @@ import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterIdPBImpl; import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterInfoPBImpl; import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterPolicyConfigurationPBImpl; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; import org.apache.hadoop.yarn.server.federation.store.utils.FederationApplicationHomeSubClusterStoreInputValidator; import org.apache.hadoop.yarn.server.federation.store.utils.FederationMembershipStateStoreInputValidator; import org.apache.hadoop.yarn.server.federation.store.utils.FederationPolicyStoreInputValidator; import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils; import org.apache.hadoop.yarn.server.federation.store.utils.FederationReservationHomeSubClusterStoreInputValidator; +import org.apache.hadoop.yarn.server.federation.store.utils.FederationRouterRMTokenInputValidator; import org.apache.hadoop.yarn.server.records.Version; import org.apache.hadoop.yarn.api.records.ReservationId; import org.apache.hadoop.yarn.util.Clock; +import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.SystemClock; import org.apache.zookeeper.data.ACL; import org.slf4j.Logger; @@ -103,11 +113,14 @@ import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException; import static org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils.filterHomeSubCluster; +import static org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE; +import static org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE_DEFAULT; +import static org.apache.hadoop.util.curator.ZKCuratorManager.getNodePath; /** * ZooKeeper implementation of {@link FederationStateStore}. - * * The znode structure is as follows: + * * ROOT_DIR_PATH * |--- MEMBERSHIP * | |----- SC1 @@ -121,6 +134,14 @@ * |--- RESERVATION * | |----- RESERVATION1 * | |----- RESERVATION2 + * |--- ROUTER_RM_DT_SECRET_MANAGER_ROOT + * | |----- ROUTER_RM_DELEGATION_TOKENS_ROOT + * | | |----- RM_DELEGATION_TOKEN_1 + * | | |----- RM_DELEGATION_TOKEN_2 + * | | |----- RM_DELEGATION_TOKEN_3 + * | |----- ROUTER_RM_DT_MASTER_KEYS_ROOT + * | | |----- DELEGATION_KEY_1 + * | |----- ROUTER_RM_DT_SEQUENTIAL_NUMBER */ public class ZookeeperFederationStateStore implements FederationStateStore { @@ -132,9 +153,29 @@ public class ZookeeperFederationStateStore implements FederationStateStore { private final static String ROOT_ZNODE_NAME_POLICY = "policies"; private final static String ROOT_ZNODE_NAME_RESERVATION = "reservation"; + /** Store Delegation Token Node. */ + private final static String ROUTER_RM_DT_SECRET_MANAGER_ROOT = "router_rm_dt_secret_manager_root"; + private static final String ROUTER_RM_DT_MASTER_KEYS_ROOT_ZNODE_NAME = + "router_rm_dt_master_keys_root"; + private static final String ROUTER_RM_DELEGATION_TOKENS_ROOT_ZNODE_NAME = + "router_rm_delegation_tokens_root"; + private static final String ROUTER_RM_DT_SEQUENTIAL_NUMBER_ZNODE_NAME = + "router_rm_dt_sequential_number"; + private static final String ROUTER_RM_DT_MASTER_KEY_ID_ZNODE_NAME = + "router_rm_dt_master_key_id"; + private static final String ROUTER_RM_DELEGATION_KEY_PREFIX = "delegation_key_"; + private static final String ROUTER_RM_DELEGATION_TOKEN_PREFIX = "rm_delegation_token_"; + /** Interface to Zookeeper. */ private ZKCuratorManager zkManager; + /** Store sequenceNum. **/ + private int seqNumBatchSize; + private int currentSeqNum; + private int currentMaxSeqNum; + private SharedCount delTokSeqCounter; + private SharedCount keyIdSeqCounter; + /** Directory to store the state store data. */ private String baseZNode; @@ -144,6 +185,13 @@ public class ZookeeperFederationStateStore implements FederationStateStore { private String reservationsZNode; private int maxAppsInStateStore; + /** Directory to store the delegation token data. **/ + private String routerRMDTSecretManagerRoot; + private String routerRMDTMasterKeysRootPath; + private String routerRMDelegationTokensRootPath; + private String routerRMSequenceNumberPath; + private String routerRMMasterKeyIdPath; + private volatile Clock clock = SystemClock.getInstance(); @VisibleForTesting @@ -152,6 +200,7 @@ public class ZookeeperFederationStateStore implements FederationStateStore { @Override public void init(Configuration conf) throws YarnException { + LOG.info("Initializing ZooKeeper connection"); maxAppsInStateStore = conf.getInt( @@ -174,6 +223,17 @@ public void init(Configuration conf) throws YarnException { policiesZNode = getNodePath(baseZNode, ROOT_ZNODE_NAME_POLICY); reservationsZNode = getNodePath(baseZNode, ROOT_ZNODE_NAME_RESERVATION); + // delegation token znodes + routerRMDTSecretManagerRoot = getNodePath(baseZNode, ROUTER_RM_DT_SECRET_MANAGER_ROOT); + routerRMDTMasterKeysRootPath = getNodePath(routerRMDTSecretManagerRoot, + ROUTER_RM_DT_MASTER_KEYS_ROOT_ZNODE_NAME); + routerRMDelegationTokensRootPath = getNodePath(routerRMDTSecretManagerRoot, + ROUTER_RM_DELEGATION_TOKENS_ROOT_ZNODE_NAME); + routerRMSequenceNumberPath = getNodePath(routerRMDTSecretManagerRoot, + ROUTER_RM_DT_SEQUENTIAL_NUMBER_ZNODE_NAME); + routerRMMasterKeyIdPath = getNodePath(routerRMDTSecretManagerRoot, + ROUTER_RM_DT_MASTER_KEY_ID_ZNODE_NAME); + // Create base znode for each entity try { List zkAcl = ZKCuratorManager.getZKAcls(conf); @@ -181,14 +241,68 @@ public void init(Configuration conf) throws YarnException { zkManager.createRootDirRecursively(appsZNode, zkAcl); zkManager.createRootDirRecursively(policiesZNode, zkAcl); zkManager.createRootDirRecursively(reservationsZNode, zkAcl); + zkManager.createRootDirRecursively(routerRMDTSecretManagerRoot, zkAcl); + zkManager.createRootDirRecursively(routerRMDTMasterKeysRootPath, zkAcl); + zkManager.createRootDirRecursively(routerRMDelegationTokensRootPath, zkAcl); } catch (Exception e) { String errMsg = "Cannot create base directories: " + e.getMessage(); FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg); } + + // Distributed sequenceNum. + try { + seqNumBatchSize = conf.getInt(ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE, + ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE_DEFAULT); + + delTokSeqCounter = new SharedCount(zkManager.getCurator(), routerRMSequenceNumberPath, 0); + + if (delTokSeqCounter != null) { + delTokSeqCounter.start(); + } + + // the first batch range should be allocated during this starting window + // by calling the incrSharedCount + currentSeqNum = incrSharedCount(delTokSeqCounter, seqNumBatchSize); + currentMaxSeqNum = currentSeqNum + seqNumBatchSize; + + LOG.info("Fetched initial range of seq num, from {} to {} ", + currentSeqNum + 1, currentMaxSeqNum); + } catch (Exception e) { + throw new YarnException("Could not start Sequence Counter.", e); + } + + // Distributed masterKeyId. + try { + keyIdSeqCounter = new SharedCount(zkManager.getCurator(), routerRMMasterKeyIdPath, 0); + if (keyIdSeqCounter != null) { + keyIdSeqCounter.start(); + } + } catch (Exception e) { + throw new YarnException("Could not start Master KeyId Counter", e); + } } @Override public void close() throws Exception { + + try { + if (delTokSeqCounter != null) { + delTokSeqCounter.close(); + delTokSeqCounter = null; + } + } catch (Exception e) { + LOG.error("Could not Stop Delegation Token Counter.", e); + } + + try { + if (keyIdSeqCounter != null) { + keyIdSeqCounter.close(); + keyIdSeqCounter = null; + } + } catch (Exception e) { + LOG.error("Could not stop Master KeyId Counter.", e); + } + if (zkManager != null) { zkManager.close(); } @@ -886,45 +1000,599 @@ public UpdateReservationHomeSubClusterResponse updateReservationHomeSubCluster( return UpdateReservationHomeSubClusterResponse.newInstance(); } + /** + * ZookeeperFederationStateStore Supports Store NewMasterKey. + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterMasterKeyResponse storeNewMasterKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // For the verification of the request, after passing the verification, + // the request and the internal objects will not be empty and can be used directly. + FederationRouterRMTokenInputValidator.validate(request); + + // Parse the delegationKey from the request and get the ZK storage path. + DelegationKey delegationKey = convertMasterKeyToDelegationKey(request); + String nodeCreatePath = getMasterKeyZNodePathByDelegationKey(delegationKey); + LOG.debug("Storing RMDelegationKey_{}, ZkNodePath = {}.", delegationKey.getKeyId(), + nodeCreatePath); + + // Write master key data to zk. + try (ByteArrayOutputStream os = new ByteArrayOutputStream(); + DataOutputStream fsOut = new DataOutputStream(os)) { + delegationKey.write(fsOut); + put(nodeCreatePath, os.toByteArray(), false); + } + + // Get the stored masterKey from zk. + RouterMasterKey masterKeyFromZK = getRouterMasterKeyFromZK(nodeCreatePath); + long end = clock.getTime(); + opDurations.addStoreNewMasterKeyDuration(start, end); + return RouterMasterKeyResponse.newInstance(masterKeyFromZK); } + /** + * ZookeeperFederationStateStore Supports Remove MasterKey. + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterMasterKeyResponse removeStoredMasterKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // For the verification of the request, after passing the verification, + // the request and the internal objects will not be empty and can be used directly. + FederationRouterRMTokenInputValidator.validate(request); + + try { + // Parse the delegationKey from the request and get the ZK storage path. + RouterMasterKey masterKey = request.getRouterMasterKey(); + DelegationKey delegationKey = convertMasterKeyToDelegationKey(request); + String nodeRemovePath = getMasterKeyZNodePathByDelegationKey(delegationKey); + LOG.debug("Removing RMDelegationKey_{}, ZkNodePath = {}.", delegationKey.getKeyId(), + nodeRemovePath); + + // Check if the path exists, Throws an exception if the path does not exist. + if (!exists(nodeRemovePath)) { + throw new YarnException("ZkNodePath = " + nodeRemovePath + " not exists!"); + } + + // try to remove masterKey. + zkManager.delete(nodeRemovePath); + long end = clock.getTime(); + opDurations.removeStoredMasterKeyDuration(start, end); + return RouterMasterKeyResponse.newInstance(masterKey); + } catch (Exception e) { + throw new YarnException(e); + } } + /** + * ZookeeperFederationStateStore Supports Remove MasterKey. + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterMasterKeyResponse getMasterKeyByDelegationKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // For the verification of the request, after passing the verification, + // the request and the internal objects will not be empty and can be used directly. + FederationRouterRMTokenInputValidator.validate(request); + + try { + + // Parse the delegationKey from the request and get the ZK storage path. + DelegationKey delegationKey = convertMasterKeyToDelegationKey(request); + String nodePath = getMasterKeyZNodePathByDelegationKey(delegationKey); + + // Check if the path exists, Throws an exception if the path does not exist. + if (!exists(nodePath)) { + throw new YarnException("ZkNodePath = " + nodePath + " not exists!"); + } + + // Get the stored masterKey from zk. + RouterMasterKey routerMasterKey = getRouterMasterKeyFromZK(nodePath); + long end = clock.getTime(); + opDurations.getMasterKeyByDelegationKeyDuration(start, end); + return RouterMasterKeyResponse.newInstance(routerMasterKey); + } catch (Exception e) { + throw new YarnException(e); + } + } + + /** + * Get MasterKeyZNodePath based on DelegationKey. + * + * @param delegationKey delegationKey. + * @return masterKey ZNodePath. + */ + private String getMasterKeyZNodePathByDelegationKey(DelegationKey delegationKey) { + return getMasterKeyZNodePathByKeyId(delegationKey.getKeyId()); } + /** + * Get MasterKeyZNodePath based on KeyId. + * + * @param keyId master key id. + * @return masterKey ZNodePath. + */ + private String getMasterKeyZNodePathByKeyId(int keyId) { + String nodeName = ROUTER_RM_DELEGATION_KEY_PREFIX + keyId; + return getNodePath(routerRMDTMasterKeysRootPath, nodeName); + } + + /** + * Get RouterMasterKey from ZK. + * + * @param nodePath The path where masterKey is stored in zk. + * + * @return RouterMasterKey. + * @throws IOException An IO Error occurred. + */ + private RouterMasterKey getRouterMasterKeyFromZK(String nodePath) + throws IOException { + try { + byte[] data = get(nodePath); + if ((data == null) || (data.length == 0)) { + return null; + } + + ByteArrayInputStream bin = new ByteArrayInputStream(data); + DataInputStream din = new DataInputStream(bin); + DelegationKey key = new DelegationKey(); + key.readFields(din); + + return RouterMasterKey.newInstance(key.getKeyId(), + ByteBuffer.wrap(key.getEncodedKey()), key.getExpiryDate()); + } catch (Exception ex) { + LOG.error("No node in path {}.", nodePath); + throw new IOException(ex); + } + } + + /** + * ZookeeperFederationStateStore Supports Store RMDelegationTokenIdentifier. + * + * The stored token method is a synchronized method + * used to ensure that storeNewToken is a thread-safe method. + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return routerRMTokenResponse, the response contains the RouterStoreToken. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterRMTokenResponse storeNewToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // We verify the RouterRMTokenRequest to ensure that the request is not empty, + // and that the internal RouterStoreToken is not empty. + FederationRouterRMTokenInputValidator.validate(request); + + try { + + // add delegationToken + storeOrUpdateRouterRMDT(request, false); + + // Get the stored delegationToken from ZK and return. + RouterStoreToken resultStoreToken = getStoreTokenFromZK(request); + long end = clock.getTime(); + opDurations.getStoreNewTokenDuration(start, end); + return RouterRMTokenResponse.newInstance(resultStoreToken); + } catch (YarnException | IOException e) { + throw e; + } catch (Exception e) { + throw new YarnException(e); + } } + /** + * ZookeeperFederationStateStore Supports Update RMDelegationTokenIdentifier. + * + * The update stored token method is a synchronized method + * used to ensure that storeNewToken is a thread-safe method. + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return routerRMTokenResponse, the response contains the RouterStoreToken. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterRMTokenResponse updateStoredToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // We verify the RouterRMTokenRequest to ensure that the request is not empty, + // and that the internal RouterStoreToken is not empty. + FederationRouterRMTokenInputValidator.validate(request); + + try { + + // get the Token storage path + String nodePath = getStoreTokenZNodePathByTokenRequest(request); + + // updateStoredToken needs to determine whether the zkNode exists. + // If it exists, update the token data. + // If it does not exist, write the new token data directly. + boolean pathExists = true; + if (!exists(nodePath)) { + pathExists = false; + } + + if (pathExists) { + // update delegationToken + storeOrUpdateRouterRMDT(request, true); + } else { + // add new delegationToken + storeNewToken(request); + } + + // Get the stored delegationToken from ZK and return. + RouterStoreToken resultStoreToken = getStoreTokenFromZK(request); + long end = clock.getTime(); + opDurations.updateStoredTokenDuration(start, end); + return RouterRMTokenResponse.newInstance(resultStoreToken); + } catch (YarnException | IOException e) { + throw e; + } catch (Exception e) { + throw new YarnException(e); + } } + /** + * ZookeeperFederationStateStore Supports Remove RMDelegationTokenIdentifier. + * + * The remove stored token method is a synchronized method + * used to ensure that storeNewToken is a thread-safe method. + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return routerRMTokenResponse, the response contains the RouterStoreToken. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterRMTokenResponse removeStoredToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // We verify the RouterRMTokenRequest to ensure that the request is not empty, + // and that the internal RouterStoreToken is not empty. + FederationRouterRMTokenInputValidator.validate(request); + + try { + + // get the Token storage path + String nodePath = getStoreTokenZNodePathByTokenRequest(request); + + // If the path to be deleted does not exist, throw an exception directly. + if (!exists(nodePath)) { + throw new YarnException("ZkNodePath = " + nodePath + " not exists!"); + } + + // Check again, first get the data from ZK, + // if the data is not empty, then delete it + RouterStoreToken storeToken = getStoreTokenFromZK(request); + if (storeToken != null) { + zkManager.delete(nodePath); + } + + // return deleted token data. + long end = clock.getTime(); + opDurations.removeStoredTokenDuration(start, end); + return RouterRMTokenResponse.newInstance(storeToken); + } catch (YarnException | IOException e) { + throw e; + } catch (Exception e) { + throw new YarnException(e); + } } + /** + * The Router Supports GetTokenByRouterStoreToken. + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return RouterRMTokenResponse. + * @throws YarnException if the call to the state store is unsuccessful + * @throws IOException An IO Error occurred + */ @Override public RouterRMTokenResponse getTokenByRouterStoreToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // We verify the RouterRMTokenRequest to ensure that the request is not empty, + // and that the internal RouterStoreToken is not empty. + FederationRouterRMTokenInputValidator.validate(request); + + try { + + // Before get the token, + // we need to determine whether the path where the token is stored exists. + // If it doesn't exist, we will throw an exception. + String nodePath = getStoreTokenZNodePathByTokenRequest(request); + if (!exists(nodePath)) { + throw new YarnException("ZkNodePath = " + nodePath + " not exists!"); + } + + // Get the stored delegationToken from ZK and return. + RouterStoreToken resultStoreToken = getStoreTokenFromZK(request); + // return deleted token data. + long end = clock.getTime(); + opDurations.getTokenByRouterStoreTokenDuration(start, end); + return RouterRMTokenResponse.newInstance(resultStoreToken); + } catch (YarnException | IOException e) { + throw e; + } catch (Exception e) { + throw new YarnException(e); + } + } + + /** + * Convert MasterKey to DelegationKey. + * + * Before using this function, + * please use FederationRouterRMTokenInputValidator to verify the request. + * By default, the request is not empty, and the internal object is not empty. + * + * @param request RouterMasterKeyRequest + * @return DelegationKey. + */ + private DelegationKey convertMasterKeyToDelegationKey(RouterMasterKeyRequest request) { + RouterMasterKey masterKey = request.getRouterMasterKey(); + return convertMasterKeyToDelegationKey(masterKey); + } + + /** + * Convert MasterKey to DelegationKey. + * + * @param masterKey masterKey. + * @return DelegationKey. + */ + private DelegationKey convertMasterKeyToDelegationKey(RouterMasterKey masterKey) { + ByteBuffer keyByteBuf = masterKey.getKeyBytes(); + byte[] keyBytes = new byte[keyByteBuf.remaining()]; + keyByteBuf.get(keyBytes); + return new DelegationKey(masterKey.getKeyId(), masterKey.getExpiryDate(), keyBytes); + } + + /** + * Check if a path exists in zk. + * + * @param path Path to be checked. + * @return Returns true if the path exists, false if the path does not exist. + * @throws Exception When an exception to access zk occurs. + */ + @VisibleForTesting + boolean exists(final String path) throws Exception { + return zkManager.exists(path); + } + + /** + * Add or update delegationToken. + * + * Before using this function, + * please use FederationRouterRMTokenInputValidator to verify the request. + * By default, the request is not empty, and the internal object is not empty. + * + * @param request storeToken + * @param isUpdate true, update the token; false, create a new token. + * @throws Exception exception occurs. + */ + private void storeOrUpdateRouterRMDT(RouterRMTokenRequest request, boolean isUpdate) + throws Exception { + + RouterStoreToken routerStoreToken = request.getRouterStoreToken(); + String nodeCreatePath = getStoreTokenZNodePathByTokenRequest(request); + LOG.debug("nodeCreatePath = {}, isUpdate = {}", nodeCreatePath, isUpdate); + put(nodeCreatePath, routerStoreToken.toByteArray(), isUpdate); + } + + /** + * Get ZNode Path of StoreToken. + * + * Before using this method, we should use FederationRouterRMTokenInputValidator + * to verify the request,ensure that the request is not empty, + * and ensure that the object in the request is not empty. + * + * @param request RouterMasterKeyRequest. + * @return RouterRMToken ZNode Path. + * @throws IOException io exception occurs. + */ + private String getStoreTokenZNodePathByTokenRequest(RouterRMTokenRequest request) + throws IOException { + RouterStoreToken routerStoreToken = request.getRouterStoreToken(); + YARNDelegationTokenIdentifier identifier = routerStoreToken.getTokenIdentifier(); + return getStoreTokenZNodePathByIdentifier(identifier); + } + + /** + * Get ZNode Path of StoreToken. + * + * @param identifier YARNDelegationTokenIdentifier + * @return RouterRMToken ZNode Path. + */ + private String getStoreTokenZNodePathByIdentifier(YARNDelegationTokenIdentifier identifier) { + String nodePath = getNodePath(routerRMDelegationTokensRootPath, + ROUTER_RM_DELEGATION_TOKEN_PREFIX + identifier.getSequenceNumber()); + return nodePath; + } + + /** + * Get RouterStoreToken from ZK. + * + * @param request RouterMasterKeyRequest. + * @return RouterStoreToken. + * @throws IOException io exception occurs. + */ + private RouterStoreToken getStoreTokenFromZK(RouterRMTokenRequest request) throws IOException { + RouterStoreToken routerStoreToken = request.getRouterStoreToken(); + YARNDelegationTokenIdentifier identifier = routerStoreToken.getTokenIdentifier(); + return getStoreTokenFromZK(identifier); + } + + /** + * Get RouterStoreToken from ZK. + * + * @param identifier YARN DelegationToken Identifier. + * @return RouterStoreToken. + * @throws IOException io exception occurs. + */ + private RouterStoreToken getStoreTokenFromZK(YARNDelegationTokenIdentifier identifier) + throws IOException { + // get the Token storage path + String nodePath = getStoreTokenZNodePathByIdentifier(identifier); + return getStoreTokenFromZK(nodePath); + } + + /** + * Get RouterStoreToken from ZK. + * + * @param nodePath Znode location where data is stored. + * @return RouterStoreToken. + * @throws IOException io exception occurs. + */ + private RouterStoreToken getStoreTokenFromZK(String nodePath) + throws IOException { + try { + byte[] data = get(nodePath); + if ((data == null) || (data.length == 0)) { + return null; + } + ByteArrayInputStream bin = new ByteArrayInputStream(data); + DataInputStream din = new DataInputStream(bin); + RouterStoreToken storeToken = Records.newRecord(RouterStoreToken.class); + storeToken.readFields(din); + return storeToken; + } catch (Exception ex) { + LOG.error("No node in path [{}]", nodePath, ex); + throw new IOException(ex); + } + } + + /** + * Increase SequenceNum. For zk, this is a distributed value. + * To ensure data consistency, we will use the synchronized keyword. + * + * For ZookeeperFederationStateStore, in order to reduce the interaction with ZK, + * we will apply for SequenceNum from ZK in batches(Apply + * when currentSeqNum >= currentMaxSeqNum), + * and assign this value to the variable currentMaxSeqNum. + * + * When calling the method incrementDelegationTokenSeqNum, + * if currentSeqNum < currentMaxSeqNum, we return ++currentMaxSeqNum, + * When currentSeqNum >= currentMaxSeqNum, we re-apply SequenceNum from zk. + * + * @return SequenceNum. + */ + @Override + public int incrementDelegationTokenSeqNum() { + // The secret manager will keep a local range of seq num which won't be + // seen by peers, so only when the range is exhausted it will ask zk for + // another range again + if (currentSeqNum >= currentMaxSeqNum) { + try { + // after a successful batch request, we can get the range starting point + currentSeqNum = incrSharedCount(delTokSeqCounter, seqNumBatchSize); + currentMaxSeqNum = currentSeqNum + seqNumBatchSize; + LOG.info("Fetched new range of seq num, from {} to {} ", + currentSeqNum + 1, currentMaxSeqNum); + } catch (InterruptedException e) { + // The ExpirationThread is just finishing.. so don't do anything.. + LOG.debug("Thread interrupted while performing token counter increment", e); + Thread.currentThread().interrupt(); + } catch (Exception e) { + throw new RuntimeException("Could not increment shared counter !!", e); + } + } + return ++currentSeqNum; + } + + /** + * Increment the value of the shared variable. + * + * @param sharedCount zk SharedCount. + * @param batchSize batch size. + * @return new SequenceNum. + * @throws Exception exception occurs. + */ + private int incrSharedCount(SharedCount sharedCount, int batchSize) + throws Exception { + while (true) { + // Loop until we successfully increment the counter + VersionedValue versionedValue = sharedCount.getVersionedValue(); + if (sharedCount.trySetCount(versionedValue, versionedValue.getValue() + batchSize)) { + return versionedValue.getValue(); + } + } + } + + /** + * Get DelegationToken SeqNum. + * + * @return delegationTokenSeqNum. + */ + @Override + public int getDelegationTokenSeqNum() { + return delTokSeqCounter.getCount(); + } + + /** + * Set DelegationToken SeqNum. + * + * @param seqNum sequenceNum. + */ + @Override + public void setDelegationTokenSeqNum(int seqNum) { + try { + delTokSeqCounter.setCount(seqNum); + } catch (Exception e) { + throw new RuntimeException("Could not set shared counter !!", e); + } + } + + /** + * Get Current KeyId. + * + * @return currentKeyId. + */ + @Override + public int getCurrentKeyId() { + return keyIdSeqCounter.getCount(); + } + + /** + * The Router Supports incrementCurrentKeyId. + * + * @return CurrentKeyId. + */ + @Override + public int incrementCurrentKeyId() { + try { + // It should be noted that the BatchSize of MasterKeyId defaults to 1. + incrSharedCount(keyIdSeqCounter, 1); + } catch (InterruptedException e) { + // The ExpirationThread is just finishing.. so don't do anything.. + LOG.debug("Thread interrupted while performing Master keyId increment", e); + Thread.currentThread().interrupt(); + } catch (Exception e) { + throw new RuntimeException("Could not increment shared Master keyId counter !!", e); + } + return keyIdSeqCounter.getCount(); } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java index d6eade878020f..29f86903f9144 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java @@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.util.Records; +import java.io.DataInput; import java.io.IOException; @Private @@ -53,4 +54,12 @@ public static RouterStoreToken newInstance(YARNDelegationTokenIdentifier identif @Private @Unstable public abstract void setRenewDate(Long renewDate); + + @Private + @Unstable + public abstract byte[] toByteArray() throws IOException; + + @Private + @Unstable + public abstract void readFields(DataInput in) throws IOException; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java index 32e148cb5b7ba..df6030a3f0d68 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java @@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.RouterStoreTokenProtoOrBuilder; import java.io.ByteArrayInputStream; +import java.io.DataInput; import java.io.DataInputStream; import java.io.IOException; @@ -168,4 +169,12 @@ private YARNDelegationTokenIdentifierProto convertToProtoFormat( YARNDelegationTokenIdentifier delegationTokenIdentifier) { return delegationTokenIdentifier.getProto(); } + + public byte[] toByteArray() throws IOException { + return builder.build().toByteArray(); + } + + public void readFields(DataInput in) throws IOException { + builder.mergeFrom((DataInputStream) in); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java new file mode 100644 index 0000000000000..40fe1f36cfb96 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java @@ -0,0 +1,105 @@ +/** + * 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.federation.store.utils; + +import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; +import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException; +import org.apache.hadoop.yarn.server.federation.store.records.RouterRMTokenRequest; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyRequest; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class FederationRouterRMTokenInputValidator { + + private static final Logger LOG = + LoggerFactory.getLogger(FederationRouterRMTokenInputValidator.class); + + private FederationRouterRMTokenInputValidator() { + } + + /** + * We will check with the RouterRMTokenRequest{@link RouterRMTokenRequest} + * to ensure that the request object is not empty and that the RouterStoreToken is not empty. + * + * @param request RouterRMTokenRequest Request. + * @throws FederationStateStoreInvalidInputException if the request is invalid. + */ + public static void validate(RouterRMTokenRequest request) + throws FederationStateStoreInvalidInputException { + + if (request == null) { + String message = "Missing RouterRMToken Request." + + " Please try again by specifying a router rm token information."; + LOG.warn(message); + throw new FederationStateStoreInvalidInputException(message); + } + + RouterStoreToken storeToken = request.getRouterStoreToken(); + if (storeToken == null) { + String message = "Missing RouterStoreToken." + + " Please try again by specifying a router rm token information."; + LOG.warn(message); + throw new FederationStateStoreInvalidInputException(message); + } + + try { + YARNDelegationTokenIdentifier identifier = storeToken.getTokenIdentifier(); + if (identifier == null) { + String message = "Missing YARNDelegationTokenIdentifier." + + " Please try again by specifying a router rm token information."; + LOG.warn(message); + throw new FederationStateStoreInvalidInputException(message); + } + } catch (Exception e) { + throw new FederationStateStoreInvalidInputException(e); + } + } + + /** + * We will check with the RouterMasterKeyRequest{@link RouterMasterKeyRequest} + * to ensure that the request object is not empty and that the RouterMasterKey is not empty. + * + * @param request RouterMasterKey Request. + * @throws FederationStateStoreInvalidInputException if the request is invalid. + */ + public static void validate(RouterMasterKeyRequest request) + throws FederationStateStoreInvalidInputException { + + // Verify the request to ensure that the request is not empty, + // if the request is found to be empty, an exception will be thrown. + if (request == null) { + String message = "Missing RouterMasterKey Request." + + " Please try again by specifying a router master key request information."; + LOG.warn(message); + throw new FederationStateStoreInvalidInputException(message); + } + + // Check whether the masterKey is empty, + // if the masterKey is empty, throw an exception message. + RouterMasterKey masterKey = request.getRouterMasterKey(); + if (masterKey == null) { + String message = "Missing RouterMasterKey." + + " Please try again by specifying a router master key information."; + LOG.warn(message); + throw new FederationStateStoreInvalidInputException(message); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java index fc1e442ab9cc1..e1ebce828922f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java @@ -504,7 +504,7 @@ public void deleteReservationHomeSubCluster(ReservationId reservationId) throws * @param defaultValue the default implementation for fallback * @param type the class for which a retry proxy is required * @param retryPolicy the policy for retrying method call failures - * @param The type of the instance + * @param The type of the instance. * @return a retry proxy for the specified interface */ public static Object createRetryInstance(Configuration conf, @@ -859,6 +859,51 @@ public RouterRMTokenResponse getTokenByRouterStoreToken(RMDelegationTokenIdentif return stateStore.getTokenByRouterStoreToken(request); } + /** + * stateStore provides DelegationTokenSeqNum increase. + * + * @return delegationTokenSequenceNumber. + */ + public int incrementDelegationTokenSeqNum() { + return stateStore.incrementDelegationTokenSeqNum(); + } + + /** + * Get SeqNum from stateStore. + * + * @return delegationTokenSequenceNumber. + */ + public int getDelegationTokenSeqNum() { + return stateStore.getDelegationTokenSeqNum(); + } + + /** + * Set SeqNum from stateStore. + * + * @param seqNum delegationTokenSequenceNumber. + */ + public void setDelegationTokenSeqNum(int seqNum) { + stateStore.setDelegationTokenSeqNum(seqNum); + } + + /** + * Get CurrentKeyId from stateStore. + * + * @return currentKeyId. + */ + public int getCurrentKeyId() { + return stateStore.getCurrentKeyId(); + } + + /** + * stateStore provides CurrentKeyId increase. + * + * @return currentKeyId. + */ + public int incrementCurrentKeyId() { + return stateStore.incrementCurrentKeyId(); + } + /** * Get the number of active cluster nodes. * diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java index 3786f7cccc0c8..7fb1e327e855b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java @@ -97,6 +97,12 @@ public abstract class FederationStateStoreBaseTest { protected abstract FederationStateStore createStateStore(); + protected abstract void checkRouterMasterKey(DelegationKey delegationKey, + RouterMasterKey routerMasterKey) throws YarnException, IOException; + + protected abstract void checkRouterStoreToken(RMDelegationTokenIdentifier identifier, + RouterStoreToken token) throws YarnException, IOException; + private Configuration conf; @Before @@ -876,6 +882,8 @@ public void testStoreNewMasterKey() throws Exception { Assert.assertEquals(routerMasterKey.getKeyId(), routerMasterKeyResp.getKeyId()); Assert.assertEquals(routerMasterKey.getKeyBytes(), routerMasterKeyResp.getKeyBytes()); Assert.assertEquals(routerMasterKey.getExpiryDate(), routerMasterKeyResp.getExpiryDate()); + + checkRouterMasterKey(key, routerMasterKey); } @Test @@ -949,6 +957,9 @@ public void testStoreNewToken() throws IOException, YarnException { Assert.assertNotNull(storeTokenResp); Assert.assertEquals(storeToken.getRenewDate(), storeTokenResp.getRenewDate()); Assert.assertEquals(storeToken.getTokenIdentifier(), storeTokenResp.getTokenIdentifier()); + + checkRouterStoreToken(identifier, storeToken); + checkRouterStoreToken(identifier, storeTokenResp); } @Test @@ -981,6 +992,8 @@ public void testUpdateStoredToken() throws IOException, YarnException { Assert.assertNotNull(updateTokenResp); Assert.assertEquals(updateToken.getRenewDate(), updateTokenResp.getRenewDate()); Assert.assertEquals(updateToken.getTokenIdentifier(), updateTokenResp.getTokenIdentifier()); + + checkRouterStoreToken(identifier, updateTokenResp); } @Test @@ -1029,5 +1042,7 @@ public void testGetTokenByRouterStoreToken() throws IOException, YarnException { Assert.assertNotNull(getStoreTokenResp); Assert.assertEquals(getStoreTokenResp.getRenewDate(), storeToken.getRenewDate()); Assert.assertEquals(getStoreTokenResp.getTokenIdentifier(), storeToken.getTokenIdentifier()); + + checkRouterStoreToken(identifier, getStoreTokenResp); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java index 70dda2227d0d2..0ea714ff06e2f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java @@ -18,14 +18,29 @@ package org.apache.hadoop.yarn.server.federation.store.impl; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; +import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.store.FederationStateStore; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.apache.hadoop.yarn.server.federation.store.records.RouterRMDTSecretManagerState; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Set; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; /** * Unit tests for MemoryFederationStateStore. */ -public class TestMemoryFederationStateStore - extends FederationStateStoreBaseTest { +public class TestMemoryFederationStateStore extends FederationStateStoreBaseTest { @Override protected FederationStateStore createStateStore() { @@ -34,4 +49,43 @@ protected FederationStateStore createStateStore() { super.setConf(conf); return new MemoryFederationStateStore(); } + + @Override + protected void checkRouterMasterKey(DelegationKey delegationKey, + RouterMasterKey routerMasterKey) throws YarnException, IOException { + MemoryFederationStateStore memoryStateStore = + MemoryFederationStateStore.class.cast(this.getStateStore()); + RouterRMDTSecretManagerState secretManagerState = + memoryStateStore.getRouterRMSecretManagerState(); + assertNotNull(secretManagerState); + + Set delegationKeys = secretManagerState.getMasterKeyState(); + assertNotNull(delegationKeys); + + assertTrue(delegationKeys.contains(delegationKey)); + + RouterMasterKey resultRouterMasterKey = RouterMasterKey.newInstance(delegationKey.getKeyId(), + ByteBuffer.wrap(delegationKey.getEncodedKey()), delegationKey.getExpiryDate()); + assertEquals(resultRouterMasterKey, routerMasterKey); + } + + @Override + protected void checkRouterStoreToken(RMDelegationTokenIdentifier identifier, + RouterStoreToken token) throws YarnException, IOException { + MemoryFederationStateStore memoryStateStore = + MemoryFederationStateStore.class.cast(this.getStateStore()); + RouterRMDTSecretManagerState secretManagerState = + memoryStateStore.getRouterRMSecretManagerState(); + assertNotNull(secretManagerState); + + Map tokenStateMap = + secretManagerState.getTokenState(); + assertNotNull(tokenStateMap); + + assertTrue(tokenStateMap.containsKey(identifier)); + + YARNDelegationTokenIdentifier tokenIdentifier = token.getTokenIdentifier(); + assertTrue(tokenIdentifier instanceof RMDelegationTokenIdentifier); + assertEquals(identifier, tokenIdentifier); + } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java index cddcf29ffb079..befdf4897637c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java @@ -18,12 +18,14 @@ package org.apache.hadoop.yarn.server.federation.store.impl; import org.apache.commons.lang3.NotImplementedException; +import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.Time; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ReservationId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.store.FederationStateStore; import org.apache.hadoop.yarn.server.federation.store.metrics.FederationStateStoreClientMetrics; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; @@ -33,6 +35,8 @@ import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterRequest; import org.apache.hadoop.yarn.server.federation.store.records.UpdateReservationHomeSubClusterRequest; import org.apache.hadoop.yarn.server.federation.store.records.DeleteReservationHomeSubClusterRequest; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils; import org.junit.Assert; import org.junit.Test; @@ -592,4 +596,18 @@ public void testRemoveStoredToken() throws IOException, YarnException { public void testGetTokenByRouterStoreToken() throws IOException, YarnException { super.testGetTokenByRouterStoreToken(); } + + @Override + protected void checkRouterMasterKey(DelegationKey delegationKey, + RouterMasterKey routerMasterKey) throws YarnException, IOException { + // TODO: This part of the code will be completed in YARN-11349 and + // will be used to verify whether the RouterMasterKey stored in the DB is as expected. + } + + @Override + protected void checkRouterStoreToken(RMDelegationTokenIdentifier identifier, + RouterStoreToken token) throws YarnException, IOException { + // TODO: This part of the code will be completed in YARN-11349 and + // will be used to verify whether the RouterStoreToken stored in the DB is as expected. + } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestZookeeperFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestZookeeperFederationStateStore.java index 18396cb821566..ba22a1e1894d5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestZookeeperFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestZookeeperFederationStateStore.java @@ -17,9 +17,11 @@ package org.apache.hadoop.yarn.server.federation.store.impl; +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; import java.io.IOException; +import java.nio.ByteBuffer; -import org.apache.commons.lang3.NotImplementedException; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.RetryNTimes; @@ -29,27 +31,52 @@ import org.apache.hadoop.metrics2.MetricsRecord; import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl; import org.apache.hadoop.metrics2.impl.MetricsRecords; +import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.util.Time; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.store.FederationStateStore; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyRequest; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyResponse; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; +import org.apache.hadoop.yarn.util.Records; import org.junit.After; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.util.curator.ZKCuratorManager.getNodePath; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNotNull; /** * Unit tests for ZookeeperFederationStateStore. */ -public class TestZookeeperFederationStateStore - extends FederationStateStoreBaseTest { +public class TestZookeeperFederationStateStore extends FederationStateStoreBaseTest { private static final Logger LOG = LoggerFactory.getLogger(TestZookeeperFederationStateStore.class); + private static final String ZNODE_FEDERATIONSTORE = + "/federationstore"; + private static final String ZNODE_ROUTER_RM_DT_SECRET_MANAGER_ROOT = + "/router_rm_dt_secret_manager_root"; + private static final String ZNODE_ROUTER_RM_DELEGATION_TOKENS_ROOT_ZNODE_NAME = + "/router_rm_delegation_tokens_root"; + private static final String ZNODE_ROUTER_RM_DT_MASTER_KEYS_ROOT_ZNODE_NAME = + "/router_rm_dt_master_keys_root/"; + private static final String ROUTER_RM_DELEGATION_TOKEN_PREFIX = "rm_delegation_token_"; + private static final String ROUTER_RM_DELEGATION_KEY_PREFIX = "delegation_key_"; + + private static final String ZNODE_DT_PREFIX = ZNODE_FEDERATIONSTORE + + ZNODE_ROUTER_RM_DT_SECRET_MANAGER_ROOT + ZNODE_ROUTER_RM_DELEGATION_TOKENS_ROOT_ZNODE_NAME; + private static final String ZNODE_MASTER_KEY_PREFIX = ZNODE_FEDERATIONSTORE + + ZNODE_ROUTER_RM_DT_SECRET_MANAGER_ROOT + ZNODE_ROUTER_RM_DT_MASTER_KEYS_ROOT_ZNODE_NAME; + /** Zookeeper test server. */ private static TestingServer curatorTestingServer; private static CuratorFramework curatorFramework; @@ -171,38 +198,82 @@ public void testMetricsInited() throws Exception { MetricsRecords.assertMetric(record, "UpdateReservationHomeSubClusterNumOps", expectOps); } - @Test(expected = NotImplementedException.class) - public void testStoreNewMasterKey() throws Exception { - super.testStoreNewMasterKey(); + private RouterStoreToken getStoreTokenFromZK(String nodePath) + throws YarnException { + try { + byte[] data = curatorFramework.getData().forPath(nodePath); + if ((data == null) || (data.length == 0)) { + return null; + } + ByteArrayInputStream bin = new ByteArrayInputStream(data); + DataInputStream din = new DataInputStream(bin); + RouterStoreToken storeToken = Records.newRecord(RouterStoreToken.class); + storeToken.readFields(din); + return storeToken; + } catch (Exception e) { + throw new YarnException(e); + } } - @Test(expected = NotImplementedException.class) - public void testGetMasterKeyByDelegationKey() throws YarnException, IOException { - super.testGetMasterKeyByDelegationKey(); + private RouterMasterKey getRouterMasterKeyFromZK(String nodePath) throws YarnException { + try { + byte[] data = curatorFramework.getData().forPath(nodePath); + ByteArrayInputStream bin = new ByteArrayInputStream(data); + DataInputStream din = new DataInputStream(bin); + DelegationKey zkDT = new DelegationKey(); + zkDT.readFields(din); + RouterMasterKey zkRouterMasterKey = RouterMasterKey.newInstance( + zkDT.getKeyId(), ByteBuffer.wrap(zkDT.getEncodedKey()), zkDT.getExpiryDate()); + return zkRouterMasterKey; + } catch (Exception e) { + throw new YarnException(e); + } } - @Test(expected = NotImplementedException.class) - public void testRemoveStoredMasterKey() throws YarnException, IOException { - super.testRemoveStoredMasterKey(); + private boolean isExists(String path) throws YarnException { + try { + return (curatorFramework.checkExists().forPath(path) != null); + } catch (Exception e) { + throw new YarnException(e); + } } - @Test(expected = NotImplementedException.class) - public void testStoreNewToken() throws IOException, YarnException { - super.testStoreNewToken(); - } + protected void checkRouterMasterKey(DelegationKey delegationKey, + RouterMasterKey routerMasterKey) throws YarnException, IOException { + // Check for MasterKey stored in ZK + RouterMasterKeyRequest routerMasterKeyRequest = + RouterMasterKeyRequest.newInstance(routerMasterKey); - @Test(expected = NotImplementedException.class) - public void testUpdateStoredToken() throws IOException, YarnException { - super.testUpdateStoredToken(); - } + // Get Data From zk. + String nodeName = ROUTER_RM_DELEGATION_KEY_PREFIX + delegationKey.getKeyId(); + String nodePath = ZNODE_MASTER_KEY_PREFIX + nodeName; + RouterMasterKey zkRouterMasterKey = getRouterMasterKeyFromZK(nodePath); - @Test(expected = NotImplementedException.class) - public void testRemoveStoredToken() throws IOException, YarnException { - super.testRemoveStoredToken(); + // Call the getMasterKeyByDelegationKey interface to get the returned result. + // The zk data should be consistent with the returned data. + RouterMasterKeyResponse response = getStateStore(). + getMasterKeyByDelegationKey(routerMasterKeyRequest); + assertNotNull(response); + RouterMasterKey respRouterMasterKey = response.getRouterMasterKey(); + assertEquals(routerMasterKey, respRouterMasterKey); + assertEquals(routerMasterKey, zkRouterMasterKey); + assertEquals(zkRouterMasterKey, respRouterMasterKey); } - @Test(expected = NotImplementedException.class) - public void testGetTokenByRouterStoreToken() throws IOException, YarnException { - super.testGetTokenByRouterStoreToken(); + protected void checkRouterStoreToken(RMDelegationTokenIdentifier identifier, + RouterStoreToken token) throws YarnException, IOException { + // Get delegationToken Path + String nodeName = ROUTER_RM_DELEGATION_TOKEN_PREFIX + identifier.getSequenceNumber(); + String nodePath = getNodePath(ZNODE_DT_PREFIX, nodeName); + + // Check if the path exists, we expect the result to exist. + assertTrue(isExists(nodePath)); + + // Check whether the token (paramStoreToken) + // We generated is consistent with the data stored in zk. + // We expect data to be consistent. + RouterStoreToken zkRouterStoreToken = getStoreTokenFromZK(nodePath); + assertNotNull(zkRouterStoreToken); + assertEquals(token, zkRouterStoreToken); } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java index 3cc4dab238618..923768729191b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java @@ -423,6 +423,31 @@ public RouterRMTokenResponse getTokenByRouterStoreToken(RouterRMTokenRequest req return stateStoreClient.getTokenByRouterStoreToken(request); } + @Override + public int incrementDelegationTokenSeqNum() { + return stateStoreClient.incrementDelegationTokenSeqNum(); + } + + @Override + public int getDelegationTokenSeqNum() { + return stateStoreClient.getDelegationTokenSeqNum(); + } + + @Override + public void setDelegationTokenSeqNum(int seqNum) { + stateStoreClient.setDelegationTokenSeqNum(seqNum); + } + + @Override + public int getCurrentKeyId() { + return stateStoreClient.getCurrentKeyId(); + } + + @Override + public int incrementCurrentKeyId() { + return stateStoreClient.incrementCurrentKeyId(); + } + /** * Create a thread that cleans up the app. * @param stage rm-start/rm-stop. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java index de74c41fe7928..79790cde3010d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java @@ -251,4 +251,29 @@ public synchronized Map getAllTokens() { } return allTokens; } + + @Override + protected synchronized int incrementDelegationTokenSeqNum() { + return federationFacade.incrementDelegationTokenSeqNum(); + } + + @Override + protected synchronized int getDelegationTokenSeqNum() { + return federationFacade.getDelegationTokenSeqNum(); + } + + @Override + protected synchronized void setDelegationTokenSeqNum(int seqNum) { + federationFacade.setDelegationTokenSeqNum(seqNum); + } + + @Override + protected synchronized int getCurrentKeyId() { + return federationFacade.getCurrentKeyId(); + } + + @Override + protected synchronized int incrementCurrentKeyId() { + return federationFacade.incrementCurrentKeyId(); + } } From 734318404a79c2172a2d0c319f4dbb9c90e8dabd Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 14 Dec 2022 18:21:03 +0000 Subject: [PATCH 33/45] HADOOP-18526. Leak of S3AInstrumentation instances via hadoop Metrics references (#5144) This has triggered an OOM in a process which was churning through s3a fs instances; the increased memory footprint of IOStatistics amplified what must have been a long-standing issue with FS instances being created and not closed() * Makes sure instrumentation is closed when the FS is closed. * Uses a weak reference from metrics to instrumentation, so even if the FS wasn't closed (see HADOOP-18478), this back reference would not cause the S3AInstrumentation reference to be retained. * If S3AFileSystem is configured to log at TRACE it will log the calling stack of initialize(), so help identify where the instance is being created. This should help track down the cause of instance leakage. Contributed by Steve Loughran. --- .../hadoop/fs/impl/WeakRefMetricsSource.java | 97 ++++++++++++++++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 28 ++--- .../hadoop/fs/s3a/S3AInstrumentation.java | 76 ++++++++++--- .../hadoop/fs/s3a/ITestS3AClosedFS.java | 12 ++ .../fs/s3a/TestInstrumentationLifecycle.java | 104 ++++++++++++++++++ .../src/test/resources/log4j.properties | 2 + 6 files changed, 289 insertions(+), 30 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WeakRefMetricsSource.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInstrumentationLifecycle.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WeakRefMetricsSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WeakRefMetricsSource.java new file mode 100644 index 0000000000000..146773857934c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WeakRefMetricsSource.java @@ -0,0 +1,97 @@ +/* + * 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.fs.impl; + +import java.lang.ref.WeakReference; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.metrics2.MetricsCollector; +import org.apache.hadoop.metrics2.MetricsSource; + +import static java.util.Objects.requireNonNull; + +/** + * A weak referenced metrics source which avoids hanging on to large objects + * if somehow they don't get fully closed/cleaned up. + * The JVM may clean up all objects which are only weakly referenced whenever + * it does a GC, even if there is no memory pressure. + * To avoid these refs being removed, always keep a strong reference around + * somewhere. + */ +@InterfaceAudience.Private +public class WeakRefMetricsSource implements MetricsSource { + + /** + * Name to know when unregistering. + */ + private final String name; + + /** + * Underlying metrics source. + */ + private final WeakReference sourceWeakReference; + + /** + * Constructor. + * @param name Name to know when unregistering. + * @param source metrics source + */ + public WeakRefMetricsSource(final String name, final MetricsSource source) { + this.name = name; + this.sourceWeakReference = new WeakReference<>(requireNonNull(source)); + } + + /** + * If the weak reference is non null, update the metrics. + * @param collector to contain the resulting metrics snapshot + * @param all if true, return all metrics even if unchanged. + */ + @Override + public void getMetrics(final MetricsCollector collector, final boolean all) { + MetricsSource metricsSource = sourceWeakReference.get(); + if (metricsSource != null) { + metricsSource.getMetrics(collector, all); + } + } + + /** + * Name to know when unregistering. + * @return the name passed in during construction. + */ + public String getName() { + return name; + } + + /** + * Get the source, will be null if the reference has been GC'd + * @return the source reference + */ + public MetricsSource getSource() { + return sourceWeakReference.get(); + } + + @Override + public String toString() { + return "WeakRefMetricsSource{" + + "name='" + name + '\'' + + ", sourceWeakReference is " + + (sourceWeakReference.get() == null ? "unset" : "set") + + '}'; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 3e6f2322d3b00..420a92788c82e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -138,7 +138,6 @@ import org.apache.hadoop.fs.statistics.DurationTracker; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.fs.statistics.IOStatistics; -import org.apache.hadoop.fs.statistics.IOStatisticsLogging; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.fs.statistics.IOStatisticsContext; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; @@ -459,6 +458,13 @@ public void initialize(URI name, Configuration originalConf) AuditSpan span = null; try { LOG.debug("Initializing S3AFileSystem for {}", bucket); + if (LOG.isTraceEnabled()) { + // log a full trace for deep diagnostics of where an object is created, + // for tracking down memory leak issues. + LOG.trace("Filesystem for {} created; fs.s3a.impl.disable.cache = {}", + name, originalConf.getBoolean("fs.s3a.impl.disable.cache", false), + new RuntimeException(super.toString())); + } // clone the configuration into one with propagated bucket options Configuration conf = propagateBucketOptions(originalConf, bucket); // HADOOP-17894. remove references to s3a stores in JCEKS credentials. @@ -3999,22 +4005,18 @@ public void close() throws IOException { } isClosed = true; LOG.debug("Filesystem {} is closed", uri); - if (getConf() != null) { - String iostatisticsLoggingLevel = - getConf().getTrimmed(IOSTATISTICS_LOGGING_LEVEL, - IOSTATISTICS_LOGGING_LEVEL_DEFAULT); - logIOStatisticsAtLevel(LOG, iostatisticsLoggingLevel, getIOStatistics()); - } try { super.close(); } finally { stopAllServices(); - } - // Log IOStatistics at debug. - if (LOG.isDebugEnabled()) { - // robust extract and convert to string - LOG.debug("Statistics for {}: {}", uri, - IOStatisticsLogging.ioStatisticsToPrettyString(getIOStatistics())); + // log IO statistics, including of any file deletion during + // superclass close + if (getConf() != null) { + String iostatisticsLoggingLevel = + getConf().getTrimmed(IOSTATISTICS_LOGGING_LEVEL, + IOSTATISTICS_LOGGING_LEVEL_DEFAULT); + logIOStatisticsAtLevel(LOG, iostatisticsLoggingLevel, getIOStatistics()); + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index 46568ec2a8d3d..9d33efa9d01b0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.impl.WeakRefMetricsSource; import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; @@ -160,7 +161,10 @@ public class S3AInstrumentation implements Closeable, MetricsSource, private final DurationTrackerFactory durationTrackerFactory; - private String metricsSourceName; + /** + * Weak reference so there's no back reference to the instrumentation. + */ + private WeakRefMetricsSource metricsSourceReference; private final MetricsRegistry registry = new MetricsRegistry("s3aFileSystem").setContext(CONTEXT); @@ -233,19 +237,33 @@ public S3AInstrumentation(URI name) { new MetricDurationTrackerFactory()); } + /** + * Get the current metrics system; demand creating. + * @return a metric system, creating if need be. + */ @VisibleForTesting - public MetricsSystem getMetricsSystem() { + static MetricsSystem getMetricsSystem() { synchronized (METRICS_SYSTEM_LOCK) { if (metricsSystem == null) { metricsSystem = new MetricsSystemImpl(); metricsSystem.init(METRICS_SYSTEM_NAME); + LOG.debug("Metrics system inited {}", metricsSystem); } } return metricsSystem; } /** - * Register this instance as a metrics source. + * Does the instrumentation have a metrics system? + * @return true if the metrics system is present. + */ + @VisibleForTesting + static boolean hasMetricSystem() { + return metricsSystem != null; + } + + /** + * Register this instance as a metrics source via a weak reference. * @param name s3a:// URI for the associated FileSystem instance */ private void registerAsMetricsSource(URI name) { @@ -257,8 +275,9 @@ private void registerAsMetricsSource(URI name) { number = ++metricsSourceNameCounter; } String msName = METRICS_SOURCE_BASENAME + number; - metricsSourceName = msName + "-" + name.getHost(); - metricsSystem.register(metricsSourceName, "", this); + String metricsSourceName = msName + "-" + name.getHost(); + metricsSourceReference = new WeakRefMetricsSource(metricsSourceName, this); + metricsSystem.register(metricsSourceName, "", metricsSourceReference); } /** @@ -680,19 +699,42 @@ public void getMetrics(MetricsCollector collector, boolean all) { registry.snapshot(collector.addRecord(registry.info().name()), true); } + /** + * if registered with the metrics, return the + * name of the source. + * @return the name of the metrics, or null if this instance is not bonded. + */ + public String getMetricSourceName() { + return metricsSourceReference != null + ? metricsSourceReference.getName() + : null; + } + public void close() { - synchronized (METRICS_SYSTEM_LOCK) { - // it is critical to close each quantile, as they start a scheduled - // task in a shared thread pool. - throttleRateQuantile.stop(); - metricsSystem.unregisterSource(metricsSourceName); - metricsSourceActiveCounter--; - int activeSources = metricsSourceActiveCounter; - if (activeSources == 0) { - LOG.debug("Shutting down metrics publisher"); - metricsSystem.publishMetricsNow(); - metricsSystem.shutdown(); - metricsSystem = null; + if (metricsSourceReference != null) { + // get the name + String name = metricsSourceReference.getName(); + LOG.debug("Unregistering metrics for {}", name); + // then set to null so a second close() is a noop here. + metricsSourceReference = null; + synchronized (METRICS_SYSTEM_LOCK) { + // it is critical to close each quantile, as they start a scheduled + // task in a shared thread pool. + if (metricsSystem == null) { + LOG.debug("there is no metric system to unregister {} from", name); + return; + } + throttleRateQuantile.stop(); + + metricsSystem.unregisterSource(name); + metricsSourceActiveCounter--; + int activeSources = metricsSourceActiveCounter; + if (activeSources == 0) { + LOG.debug("Shutting down metrics publisher"); + metricsSystem.publishMetricsNow(); + metricsSystem.shutdown(); + metricsSystem = null; + } } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java index 79772ec9dadfe..327b0fab288f7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java @@ -103,4 +103,16 @@ public void testClosedOpen() throws Exception { () -> getFileSystem().open(path("to-open"))); } + @Test + public void testClosedInstrumentation() throws Exception { + // no metrics + Assertions.assertThat(S3AInstrumentation.hasMetricSystem()) + .describedAs("S3AInstrumentation.hasMetricSystem()") + .isFalse(); + + Assertions.assertThat(getFileSystem().getIOStatistics()) + .describedAs("iostatistics of %s", getFileSystem()) + .isNotNull(); + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInstrumentationLifecycle.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInstrumentationLifecycle.java new file mode 100644 index 0000000000000..d8b9247008cc8 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInstrumentationLifecycle.java @@ -0,0 +1,104 @@ +/* + * 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.fs.s3a; + +import java.net.URI; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.fs.impl.WeakRefMetricsSource; +import org.apache.hadoop.metrics2.MetricsSource; +import org.apache.hadoop.metrics2.MetricsSystem; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.s3a.S3AInstrumentation.getMetricsSystem; +import static org.apache.hadoop.fs.s3a.Statistic.DIRECTORIES_CREATED; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Test the {@link S3AInstrumentation} lifecycle, in particular how + * it binds to hadoop metrics through a {@link WeakRefMetricsSource} + * and that it will deregister itself in {@link S3AInstrumentation#close()}. + */ +public class TestInstrumentationLifecycle extends AbstractHadoopTestBase { + + @Test + public void testDoubleClose() throws Throwable { + S3AInstrumentation instrumentation = new S3AInstrumentation(new URI("s3a://example/")); + + // the metric system is created in the constructor + assertThat(S3AInstrumentation.hasMetricSystem()) + .describedAs("S3AInstrumentation.hasMetricSystem()") + .isTrue(); + // ask for a metric + String metricName = DIRECTORIES_CREATED.getSymbol(); + assertThat(instrumentation.lookupMetric(metricName)) + .describedAs("lookupMetric(%s) while open", metricName) + .isNotNull(); + + MetricsSystem activeMetrics = getMetricsSystem(); + final String metricSourceName = instrumentation.getMetricSourceName(); + final MetricsSource source = activeMetrics.getSource(metricSourceName); + // verify the source is registered through a weak ref, and that the + // reference maps to the instance. + Assertions.assertThat(source) + .describedAs("metric source %s", metricSourceName) + .isNotNull() + .isInstanceOf(WeakRefMetricsSource.class) + .extracting(m -> ((WeakRefMetricsSource) m).getSource()) + .isSameAs(instrumentation); + + // this will close the metrics system + instrumentation.close(); + + // iostats is still valid + assertThat(instrumentation.getIOStatistics()) + .describedAs("iostats of %s", instrumentation) + .isNotNull(); + + // no metrics + assertThat(S3AInstrumentation.hasMetricSystem()) + .describedAs("S3AInstrumentation.hasMetricSystem()") + .isFalse(); + + // metric lookup still works, so any invocation of an s3a + // method which still updates a metric also works + assertThat(instrumentation.lookupMetric(metricName)) + .describedAs("lookupMetric(%s) when closed", metricName) + .isNotNull(); + + // which we can implicitly verify by asking for it and + // verifying that we get given a different one back + // from the demand-created instance + MetricsSystem metrics2 = getMetricsSystem(); + assertThat(metrics2) + .describedAs("metric system 2") + .isNotSameAs(activeMetrics); + + // this is going to be a no-op + instrumentation.close(); + + // which we can verify because the metrics system doesn't + // get closed this time + assertThat(getMetricsSystem()) + .describedAs("metric system 3") + .isSameAs(metrics2); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index 0ec8d52042807..306a79a20a204 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -53,6 +53,8 @@ log4j.logger.org.apache.hadoop.ipc.Server=WARN # for debugging low level S3a operations, uncomment these lines # Log all S3A classes log4j.logger.org.apache.hadoop.fs.s3a=DEBUG +# when logging at trace, the stack of the initialize() call is logged +#log4j.logger.org.apache.hadoop.fs.s3a.S3AFileSystem=TRACE #log4j.logger.org.apache.hadoop.fs.s3a.S3AUtils=INFO #log4j.logger.org.apache.hadoop.fs.s3a.Listing=INFO log4j.logger.org.apache.hadoop.fs.s3a.SDKV2Upgrade=WARN From e8a6089f190109187d021533c90aad2e21a8f756 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Thu, 15 Dec 2022 06:37:56 +0800 Subject: [PATCH 34/45] YARN-11358. [Federation] Add FederationInterceptor#allow-partial-result config. (#5056) --- .../hadoop/yarn/conf/YarnConfiguration.java | 5 ++ .../src/main/resources/yarn-default.xml | 14 ++++ .../webapp/FederationInterceptorREST.java | 27 ++++++-- .../webapp/TestFederationInterceptorREST.java | 5 ++ .../TestFederationInterceptorRESTRetry.java | 64 ++++++++++++++++++- 5 files changed, 108 insertions(+), 7 deletions(-) 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 25513fec90ce9..316a6421889bd 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 @@ -4294,6 +4294,11 @@ public static boolean isAclEnabled(Configuration conf) { ROUTER_PREFIX + "webapp.cross-origin.enabled"; public static final boolean DEFAULT_ROUTER_WEBAPP_ENABLE_CORS_FILTER = false; + /** Router Interceptor Allow Partial Result Enable. **/ + public static final String ROUTER_INTERCEPTOR_ALLOW_PARTIAL_RESULT_ENABLED = + ROUTER_PREFIX + "interceptor.allow-partial-result.enable"; + public static final boolean DEFAULT_ROUTER_INTERCEPTOR_ALLOW_PARTIAL_RESULT_ENABLED = false; + //////////////////////////////// // CSI Volume configs //////////////////////////////// diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml index e1cc6adbe52ec..ec2ac71adc263 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml @@ -5085,4 +5085,18 @@ + + yarn.router.interceptor.allow-partial-result.enable + false + + This configuration represents whether to allow the interceptor to + return partial SubCluster results. + If true, we will ignore the exception to some subClusters during the calling process, + and return result. + If false, if an exception occurs in a subCluster during the calling process, + an exception will be thrown directly. + Default is false. + + + 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/FederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java index a21be7b4e1b76..61edfb363d0f8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java @@ -140,6 +140,7 @@ public class FederationInterceptorREST extends AbstractRESTRequestInterceptor { private boolean returnPartialReport; private boolean appInfosCacheEnabled; private int appInfosCacheCount; + private boolean allowPartialResult; private long submitIntervalTime; private Map interceptors; @@ -194,6 +195,10 @@ public void init(String user) { appInfosCaches = new LRUCacheHashMap<>(appInfosCacheCount, true); } + allowPartialResult = conf.getBoolean( + YarnConfiguration.ROUTER_INTERCEPTOR_ALLOW_PARTIAL_RESULT_ENABLED, + YarnConfiguration.DEFAULT_ROUTER_INTERCEPTOR_ALLOW_PARTIAL_RESULT_ENABLED); + submitIntervalTime = conf.getTimeDuration( YarnConfiguration.ROUTER_CLIENTRM_SUBMIT_INTERVAL_TIME, YarnConfiguration.DEFAULT_CLIENTRM_SUBMIT_INTERVAL_TIME, TimeUnit.MILLISECONDS); @@ -975,10 +980,13 @@ public NodesInfo getNodes(String states) { }); } catch (NotFoundException e) { LOG.error("get all active sub cluster(s) error.", e); + throw e; } catch (YarnException e) { LOG.error("getNodes by states = {} error.", states, e); + throw new YarnRuntimeException(e); } catch (IOException e) { LOG.error("getNodes by states = {} error with io error.", states, e); + throw new YarnRuntimeException(e); } // Delete duplicate from all the node reports got from all the available @@ -2070,9 +2078,10 @@ private Map invokeConcurrent(Collection c Map results = new HashMap<>(); - // Send the requests in parallel - CompletionService> compSvc = - new ExecutorCompletionService<>(this.threadpool); + // If there is a sub-cluster access error, + // we should choose whether to throw exception information according to user configuration. + // Send the requests in parallel. + CompletionService> compSvc = new ExecutorCompletionService<>(threadpool); // This part of the code should be able to expose the accessed Exception information. // We use Pair to store related information. The left value of the Pair is the response, @@ -2105,9 +2114,10 @@ private Map invokeConcurrent(Collection c if (response != null) { results.put(clusterId, response); } - - Exception exception = pair.getRight(); - if (exception != null) { + Exception exception = pair.getValue(); + // If allowPartialResult=false, it means that if an exception occurs in a subCluster, + // an exception will be thrown directly. + if (!allowPartialResult && exception != null) { throw exception; } } catch (Throwable e) { @@ -2178,4 +2188,9 @@ private SubClusterInfo getHomeSubClusterInfoByReservationId(String resId) public LRUCacheHashMap getAppInfosCaches() { return appInfosCaches; } + + @VisibleForTesting + public void setAllowPartialResult(boolean allowPartialResult) { + this.allowPartialResult = allowPartialResult; + } } \ No newline at end of file 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/TestFederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java index 4c50e5198dc03..7c82e71ea9b7a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java @@ -749,6 +749,7 @@ public void testGetLabelsOnNode() throws Exception { Assert.assertTrue(nodeLabelsName.contains("y")); // null request + interceptor.setAllowPartialResult(false); NodeLabelsInfo nodeLabelsInfo2 = interceptor.getLabelsOnNode(null, "node2"); Assert.assertNotNull(nodeLabelsInfo2); Assert.assertEquals(0, nodeLabelsInfo2.getNodeLabelsName().size()); @@ -1183,6 +1184,8 @@ public void testWebAddressWithScheme() { @Test public void testCheckUserAccessToQueue() throws Exception { + interceptor.setAllowPartialResult(false); + // Case 1: Only queue admin user can access other user's information HttpServletRequest mockHsr = mockHttpServletRequestByUserName("non-admin"); String errorMsg1 = "User=non-admin doesn't haven access to queue=queue " + @@ -1212,6 +1215,8 @@ public void testCheckUserAccessToQueue() throws Exception { // Case 5: get OK only for SUBMIT_APP acl for "yarn" user checkUserAccessToQueueFailed("queue", "yarn", QueueACL.ADMINISTER_QUEUE, "admin"); checkUserAccessToQueueSuccess("queue", "yarn", QueueACL.SUBMIT_APPLICATIONS, "admin"); + + interceptor.setAllowPartialResult(true); } private void checkUserAccessToQueueSuccess(String queue, String userName, 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/TestFederationInterceptorRESTRetry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorRESTRetry.java index e2b2103c7deb8..790cf410bed75 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorRESTRetry.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorRESTRetry.java @@ -25,6 +25,7 @@ import javax.ws.rs.core.Response; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -42,6 +43,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo; import org.apache.hadoop.yarn.server.router.clientrm.PassThroughClientRequestInterceptor; import org.apache.hadoop.yarn.server.router.clientrm.TestableFederationClientInterceptor; import org.apache.hadoop.yarn.webapp.NotFoundException; @@ -81,10 +83,16 @@ public class TestFederationInterceptorRESTRetry @Override public void setUp() { super.setUpConfig(); + + Configuration conf = this.getConf(); + + // Compatible with historical test cases, we set router.allow-partial-result.enable=false. + conf.setBoolean(YarnConfiguration.ROUTER_INTERCEPTOR_ALLOW_PARTIAL_RESULT_ENABLED, false); + interceptor = new TestableFederationInterceptorREST(); stateStore = new MemoryFederationStateStore(); - stateStore.init(this.getConf()); + stateStore.init(conf); FederationStateStoreFacade.getInstance().reinitialize(stateStore, getConf()); stateStoreUtil = new FederationStateStoreTestUtil(stateStore); @@ -516,4 +524,58 @@ private void checkEmptyMetrics(ClusterMetricsInfo response) { Assert.assertEquals(0, response.getActiveNodes()); Assert.assertEquals(0, response.getShutdownNodes()); } + + @Test + public void testGetNodesOneBadSCAllowPartial() throws Exception { + // We set allowPartialResult to true. + // In this test case, we set up a subCluster, + // and the subCluster status is bad, we can't get the response, + // an exception should be thrown at this time. + interceptor.setAllowPartialResult(true); + setupCluster(Arrays.asList(bad2)); + + NodesInfo nodesInfo = interceptor.getNodes(null); + Assert.assertNotNull(nodesInfo); + + // We need to set allowPartialResult=false + interceptor.setAllowPartialResult(false); + } + + @Test + public void testGetNodesTwoBadSCsAllowPartial() throws Exception { + // We set allowPartialResult to true. + // In this test case, we set up 2 subClusters, + // and the status of these 2 subClusters is bad. When we call the interface, + // an exception should be returned. + interceptor.setAllowPartialResult(true); + setupCluster(Arrays.asList(bad1, bad2)); + + NodesInfo nodesInfo = interceptor.getNodes(null); + Assert.assertNotNull(nodesInfo); + + // We need to set allowPartialResult=false + interceptor.setAllowPartialResult(false); + } + + @Test + public void testGetNodesOneBadOneGoodAllowPartial() throws Exception { + + // allowPartialResult = true, + // We tolerate exceptions and return normal results + interceptor.setAllowPartialResult(true); + setupCluster(Arrays.asList(good, bad2)); + + NodesInfo response = interceptor.getNodes(null); + Assert.assertNotNull(response); + Assert.assertEquals(1, response.getNodes().size()); + // Check if the only node came from Good SubCluster + Assert.assertEquals(good.getId(), + Long.toString(response.getNodes().get(0).getLastHealthUpdate())); + + // allowPartialResult = false, + // We do not tolerate exceptions and will throw exceptions directly + interceptor.setAllowPartialResult(false); + + setupCluster(Arrays.asList(good, bad2)); + } } \ No newline at end of file From 060b080d490791d4c50c4a5d3c5efa78a726e00e Mon Sep 17 00:00:00 2001 From: Mehakmeet Singh Date: Thu, 15 Dec 2022 10:19:18 +0530 Subject: [PATCH 35/45] HADOOP-18574. Changing log level of IOStatistics increment to make the DEBUG logs less noisy (#5223) Contributed by: Mehakmeet Singh --- .../apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java index 0471703b3b040..6db3820891969 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java @@ -190,7 +190,7 @@ public long incrementCounter(final String key, final long value) { return counter.get(); } else { long l = incAtomicLong(counter, value); - LOG.debug("Incrementing counter {} by {} with final value {}", + LOG.trace("Incrementing counter {} by {} with final value {}", key, value, l); return l; } From 3806b08aaf9ba124411314691178683e2b900910 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 15 Dec 2022 11:42:36 +0000 Subject: [PATCH 36/45] HADOOP-18573. Improve error reporting on non-standard kerberos names (#5221) The kerberos RPC does not declare any restriction on characters used in kerberos names, though implementations MAY be more restrictive. If the kerberos controller supports use non-conventional principal names *and the kerberos admin chooses to use them* this can confuse some of the parsing. The obvious solution is for the enterprise admins to "not do that" as a lot of things break, bits of hadoop included. Harden the hadoop code slightly so at least we fail more gracefully, so people can then get in touch with their sysadmin and tell them to stop it. --- .../apache/hadoop/security/ShellBasedIdMapping.java | 11 +++++++---- .../src/main/java/org/apache/hadoop/util/Shell.java | 3 ++- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java index c28471a3bdad9..49fd9194e5a30 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java @@ -38,6 +38,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.util.Shell.bashQuote; + /** * A simple shell-based implementation of {@link IdMappingServiceProvider} * Map id to user name or group name. It does update every 15 minutes. Only a @@ -472,26 +474,27 @@ synchronized private void updateMapIncr(final String name, boolean updated = false; updateStaticMapping(); + String name2 = bashQuote(name); if (OS.startsWith("Linux") || OS.equals("SunOS") || OS.contains("BSD")) { if (isGrp) { updated = updateMapInternal(gidNameMap, "group", - getName2IdCmdNIX(name, true), ":", + getName2IdCmdNIX(name2, true), ":", staticMapping.gidMapping); } else { updated = updateMapInternal(uidNameMap, "user", - getName2IdCmdNIX(name, false), ":", + getName2IdCmdNIX(name2, false), ":", staticMapping.uidMapping); } } else { // Mac if (isGrp) { updated = updateMapInternal(gidNameMap, "group", - getName2IdCmdMac(name, true), "\\s+", + getName2IdCmdMac(name2, true), "\\s+", staticMapping.gidMapping); } else { updated = updateMapInternal(uidNameMap, "user", - getName2IdCmdMac(name, false), "\\s+", + getName2IdCmdMac(name2, false), "\\s+", staticMapping.uidMapping); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java index 65978f3c5f59c..91868365b1346 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java @@ -146,7 +146,8 @@ public static void checkWindowsCommandLineLength(String...commands) * @param arg the argument to quote * @return the quoted string */ - static String bashQuote(String arg) { + @InterfaceAudience.Private + public static String bashQuote(String arg) { StringBuilder buffer = new StringBuilder(arg.length() + 2); buffer.append('\'') .append(arg.replace("'", "'\\''")) From 3d253893fb0e3af0f683db69ae92431efffa5535 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 15 Dec 2022 16:45:05 +0000 Subject: [PATCH 37/45] HADOOP-18561. Update commons-net to 3.9.0 (#5214) Addresses CVE-2021-37533, which *only* relates to FTP. Applications not using the ftp:// filesystem, which, as anyone who has used it will know is very minimal and so rarely used, is not a critical part of the project. Furthermore, the FTP-related issue is at worst information leakage if someone connects to a malicious server. This is a due diligence PR rather than an emergency fix. Contributed by Steve Loughran --- LICENSE-binary | 2 +- hadoop-project/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index c4aa63df8805f..142f751a3098c 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -251,7 +251,7 @@ commons-collections:commons-collections:3.2.2 commons-daemon:commons-daemon:1.0.13 commons-io:commons-io:2.8.0 commons-logging:commons-logging:1.1.3 -commons-net:commons-net:3.8.0 +commons-net:commons-net:3.9.0 de.ruedigermoeller:fst:2.50 io.grpc:grpc-api:1.26.0 io.grpc:grpc-context:1.26.0 diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 17df3f1449779..daebcfb5d99ab 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -123,7 +123,7 @@ 1.1.3 1.1 3.6.1 - 3.8.0 + 3.9.0 1.10.0 2.0.2 From 36f50c59a01a08b5444c8077cfee4ddf18b97d1d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 15 Dec 2022 17:08:25 +0000 Subject: [PATCH 38/45] HADOOP-18577. ABFS: Add probes of readahead fix (#5205) Followup patch to HADOOP-18456 as part of HADOOP-18521, ABFS ReadBufferManager buffer sharing across concurrent HTTP requests Add probes of readahead fix aid in checking safety of hadoop ABFS client across different releases. * ReadBufferManager constructor logs the fact it is safe at TRACE * AbfsInputStream declares it is fixed in toString() by including fs.azure.capability.readahead.safe" in the result. The ABFS FileSystem hasPathCapability("fs.azure.capability.readahead.safe") probe returns true to indicate the client's readahead manager has been fixed to be safe when prefetching. All Hadoop releases for which probe this returns false and for which the probe "fs.capability.etags.available" returns true at risk of returning invalid data when reading ADLS Gen2/Azure storage data. Contributed by Steve Loughran. --- .../fs/azurebfs/AzureBlobFileSystem.java | 7 +++ .../azurebfs/constants/InternalConstants.java | 46 +++++++++++++++++++ .../fs/azurebfs/services/AbfsInputStream.java | 8 ++-- .../azurebfs/services/ReadBufferManager.java | 1 + .../ITestFileSystemInitialization.java | 32 +++++++++++++ .../services/ITestReadBufferManager.java | 26 ++++++++++- 6 files changed, 115 insertions(+), 5 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 21501d28f4238..5534b5fb44a51 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -117,6 +117,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BLOCK_UPLOAD_BUFFER_DIR; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_UPLOAD_ACTIVE_BLOCKS_DEFAULT; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DATA_BLOCKS_BUFFER_DEFAULT; +import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtLevel; import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator; @@ -235,6 +236,7 @@ public String toString() { sb.append("uri=").append(uri); sb.append(", user='").append(abfsStore.getUser()).append('\''); sb.append(", primaryUserGroup='").append(abfsStore.getPrimaryGroup()).append('\''); + sb.append("[" + CAPABILITY_SAFE_READAHEAD + "]"); sb.append('}'); return sb.toString(); } @@ -1636,6 +1638,11 @@ public boolean hasPathCapability(final Path path, final String capability) new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.HAS_PATH_CAPABILITY, tracingHeaderFormat, listener)); + + // probe for presence of the HADOOP-18546 readahead fix. + case CAPABILITY_SAFE_READAHEAD: + return true; + default: return super.hasPathCapability(p, capability); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java new file mode 100644 index 0000000000000..12d4f14d92a00 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java @@ -0,0 +1,46 @@ +/* + * 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.fs.azurebfs.constants; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * Constants which are used internally and which don't fit into the other + * classes. + * For use within the {@code hadoop-azure} module only. + */ +@InterfaceAudience.Private +public final class InternalConstants { + + private InternalConstants() { + } + + /** + * Does this version of the store have safe readahead? + * Possible combinations of this and the probe + * {@code "fs.capability.etags.available"}. + *
    + *
  1. {@value}: store is safe
  2. + *
  3. !etags: store is safe
  4. + *
  5. etags && !{@value}: store is UNSAFE
  6. + *
+ */ + public static final String CAPABILITY_SAFE_READAHEAD = + "fs.azure.capability.readahead.safe"; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 8f12484a55c9d..fdeaf70177571 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -50,6 +50,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_KB; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN; +import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD; import static org.apache.hadoop.util.StringUtils.toLowerCase; /** @@ -828,11 +829,12 @@ public IOStatistics getIOStatistics() { @Override public String toString() { final StringBuilder sb = new StringBuilder(super.toString()); + sb.append("AbfsInputStream@(").append(this.hashCode()).append("){"); + sb.append("[" + CAPABILITY_SAFE_READAHEAD + "]"); if (streamStatistics != null) { - sb.append("AbfsInputStream@(").append(this.hashCode()).append("){"); - sb.append(streamStatistics.toString()); - sb.append("}"); + sb.append(", ").append(streamStatistics); } + sb.append("}"); return sb.toString(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index ac84f0b27cf12..031545f57a193 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -101,6 +101,7 @@ private void init() { // hide instance constructor private ReadBufferManager() { + LOGGER.trace("Creating readbuffer manager with HADOOP-18546 patch"); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java index 8b60dd801cb30..f7d4a5b7a83e7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java @@ -20,14 +20,22 @@ import java.net.URI; +import org.assertj.core.api.Assertions; import org.junit.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.services.AuthType; +import static org.apache.hadoop.fs.CommonPathCapabilities.ETAGS_AVAILABLE; +import static org.apache.hadoop.fs.CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME; +import static org.apache.hadoop.fs.CommonPathCapabilities.FS_ACLS; +import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD; +import static org.junit.Assume.assumeTrue; + /** * Test AzureBlobFileSystem initialization. */ @@ -74,4 +82,28 @@ public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception { assertNotNull("working directory", fs.getWorkingDirectory()); } } + + @Test + public void testFileSystemCapabilities() throws Throwable { + final AzureBlobFileSystem fs = getFileSystem(); + + final Path p = new Path("}"); + // etags always present + Assertions.assertThat(fs.hasPathCapability(p, ETAGS_AVAILABLE)) + .describedAs("path capability %s in %s", ETAGS_AVAILABLE, fs) + .isTrue(); + // readahead always correct + Assertions.assertThat(fs.hasPathCapability(p, CAPABILITY_SAFE_READAHEAD)) + .describedAs("path capability %s in %s", CAPABILITY_SAFE_READAHEAD, fs) + .isTrue(); + + // etags-over-rename and ACLs are either both true or both false. + final boolean etagsAcrossRename = fs.hasPathCapability(p, ETAGS_PRESERVED_IN_RENAME); + final boolean acls = fs.hasPathCapability(p, FS_ACLS); + Assertions.assertThat(etagsAcrossRename) + .describedAs("capabilities %s=%s and %s=%s in %s", + ETAGS_PRESERVED_IN_RENAME, etagsAcrossRename, + FS_ACLS, acls, fs) + .isEqualTo(acls); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java index eca670fba9059..a57430fa808cc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java @@ -44,9 +44,24 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; +import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD; +import static org.apache.hadoop.test.LambdaTestUtils.eventually; public class ITestReadBufferManager extends AbstractAbfsIntegrationTest { + /** + * Time before the JUnit test times out for eventually() clauses + * to fail. This copes with slow network connections and debugging + * sessions, yet still allows for tests to fail with meaningful + * messages. + */ + public static final int TIMEOUT_OFFSET = 5 * 60_000; + + /** + * Interval between eventually preobes. + */ + public static final int PROBE_INTERVAL_MILLIS = 1_000; + public ITestReadBufferManager() throws Exception { } @@ -61,6 +76,11 @@ public void testPurgeBufferManagerForParallelStreams() throws Exception { } ExecutorService executorService = Executors.newFixedThreadPool(4); AzureBlobFileSystem fs = getABFSWithReadAheadConfig(); + // verify that the fs has the capability to validate the fix + Assertions.assertThat(fs.hasPathCapability(new Path("/"), CAPABILITY_SAFE_READAHEAD)) + .describedAs("path capability %s in %s", CAPABILITY_SAFE_READAHEAD, fs) + .isTrue(); + try { for (int i = 0; i < 4; i++) { final String fileName = methodName.getMethodName() + i; @@ -80,9 +100,11 @@ public void testPurgeBufferManagerForParallelStreams() throws Exception { } ReadBufferManager bufferManager = ReadBufferManager.getBufferManager(); - // verify there is no work in progress or the readahead queue. - assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList()); + // readahead queue is empty assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy()); + // verify the in progress list eventually empties out. + eventually(getTestTimeoutMillis() - TIMEOUT_OFFSET, PROBE_INTERVAL_MILLIS, () -> + assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList())); } private void assertListEmpty(String listName, List list) { From 69a98d8e75b637924c90e1ed555a8996214bf5cf Mon Sep 17 00:00:00 2001 From: Happy-shi <35164746+Happy-shi@users.noreply.github.com> Date: Fri, 16 Dec 2022 11:07:41 +0800 Subject: [PATCH 39/45] HDFS-16866. Fix a typo in Dispatcher (#5202) Signed-off-by: Tao Li --- .../org/apache/hadoop/hdfs/server/balancer/Balancer.java | 4 ++-- .../apache/hadoop/hdfs/server/balancer/Dispatcher.java | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java index 02004f337c10a..077365147129e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java @@ -708,12 +708,12 @@ public String toString() { Result newResult(ExitStatus exitStatus, long bytesLeftToMove, long bytesBeingMoved) { return new Result(exitStatus, bytesLeftToMove, bytesBeingMoved, - dispatcher.getBytesMoved(), dispatcher.getBblocksMoved()); + dispatcher.getBytesMoved(), dispatcher.getBlocksMoved()); } Result newResult(ExitStatus exitStatus) { return new Result(exitStatus, -1, -1, dispatcher.getBytesMoved(), - dispatcher.getBblocksMoved()); + dispatcher.getBlocksMoved()); } /** Run an iteration for all datanodes. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java index 5c66d66912072..98a6d8449b629 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java @@ -164,7 +164,7 @@ synchronized int allocate(int n) { } } - /** Aloocate a single lot of items */ + /** Allocate a single lot of items. */ int allocate() { return allocate(lotSize); } @@ -1127,7 +1127,7 @@ long getBytesMoved() { return nnc.getBytesMoved().get(); } - long getBblocksMoved() { + long getBlocksMoved() { return nnc.getBlocksMoved().get(); } @@ -1234,7 +1234,7 @@ public boolean dispatchAndCheckContinue() throws InterruptedException { */ private long dispatchBlockMoves() throws InterruptedException { final long bytesLastMoved = getBytesMoved(); - final long blocksLastMoved = getBblocksMoved(); + final long blocksLastMoved = getBlocksMoved(); final Future[] futures = new Future[sources.size()]; int concurrentThreads = Math.min(sources.size(), @@ -1284,7 +1284,7 @@ public void run() { waitForMoveCompletion(targets); LOG.info("Total bytes (blocks) moved in this iteration {} ({})", StringUtils.byteDesc(getBytesMoved() - bytesLastMoved), - (getBblocksMoved() - blocksLastMoved)); + (getBlocksMoved() - blocksLastMoved)); return getBytesMoved() - bytesLastMoved; } From 9092879df708cd297aa87d25bc6454394ee61a7b Mon Sep 17 00:00:00 2001 From: Xing Lin Date: Fri, 16 Dec 2022 08:46:14 -0800 Subject: [PATCH 40/45] HDFS-16852. Skip KeyProviderCache shutdown hook registration if already shutting down (#5160) Signed-off-by: Erik Krogen --- .../main/java/org/apache/hadoop/hdfs/KeyProviderCache.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java index d8dd485101bce..ee97b96ea78c8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java @@ -68,8 +68,11 @@ public void onRemoval( }) .build(); - ShutdownHookManager.get().addShutdownHook(new KeyProviderCacheFinalizer(), - SHUTDOWN_HOOK_PRIORITY); + // Register the shutdown hook when not in shutdown + if (!ShutdownHookManager.get().isShutdownInProgress()) { + ShutdownHookManager.get().addShutdownHook( + new KeyProviderCacheFinalizer(), SHUTDOWN_HOOK_PRIORITY); + } } public KeyProvider get(final Configuration conf, From 67d64a23cbdf9ca20a1fe403c7444165b1a5e4d5 Mon Sep 17 00:00:00 2001 From: Chengbing Liu Date: Sat, 17 Dec 2022 01:15:11 +0800 Subject: [PATCH 41/45] HADOOP-18567. LogThrottlingHelper: properly trigger dependent recorders in cases of infrequent logging (#5215) Signed-off-by: Erik Krogen Co-authored-by: Chengbing Liu --- .../apache/hadoop/log/LogThrottlingHelper.java | 16 +++++++++++++--- .../hadoop/log/TestLogThrottlingHelper.java | 12 ++++++++++++ 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogThrottlingHelper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogThrottlingHelper.java index af5f852143389..ad5ad7b04d080 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogThrottlingHelper.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogThrottlingHelper.java @@ -262,9 +262,15 @@ public LogAction record(String recorderName, long currentTimeMs, if (primaryRecorderName.equals(recorderName) && currentTimeMs - minLogPeriodMs >= lastLogTimestampMs) { lastLogTimestampMs = currentTimeMs; - for (LoggingAction log : currentLogs.values()) { - log.setShouldLog(); - } + currentLogs.replaceAll((key, log) -> { + LoggingAction newLog = log; + if (log.hasLogged()) { + // create a fresh log since the old one has already been logged + newLog = new LoggingAction(log.getValueCount()); + } + newLog.setShouldLog(); + return newLog; + }); } if (currentLog.shouldLog()) { currentLog.setHasLogged(); @@ -357,6 +363,10 @@ private void setHasLogged() { hasLogged = true; } + private int getValueCount() { + return stats.length; + } + private void recordValues(double... values) { if (values.length != stats.length) { throw new IllegalArgumentException("received " + values.length + diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogThrottlingHelper.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogThrottlingHelper.java index d0eeea3e51393..6c627116f8cb9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogThrottlingHelper.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogThrottlingHelper.java @@ -142,6 +142,18 @@ public void testPrimaryAndDependentLoggers() { assertTrue(helper.record("bar", 0).shouldLog()); } + @Test + public void testInfrequentPrimaryAndDependentLoggers() { + helper = new LogThrottlingHelper(LOG_PERIOD, "foo", timer); + + assertTrue(helper.record("foo", 0).shouldLog()); + assertTrue(helper.record("bar", 0).shouldLog()); + + // Both should log once the period has elapsed + assertTrue(helper.record("foo", LOG_PERIOD).shouldLog()); + assertTrue(helper.record("bar", LOG_PERIOD).shouldLog()); + } + @Test public void testMultipleLoggersWithValues() { helper = new LogThrottlingHelper(LOG_PERIOD, "foo", timer); From 5d08fade656b90aeb89a85e23c6b8ee6b25dd1fb Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Sun, 18 Dec 2022 12:19:33 +0000 Subject: [PATCH 42/45] HADOOP-18577. Followup: javadoc fix (#5232) Fixes a javadoc error which came with HADOOP-18577. ABFS: Add probes of readahead fix (#5205) Part of the HADOOP-18521 ABFS readahead fix; MUST be included. Contributed by Steve Loughran --- .../hadoop/fs/azurebfs/constants/InternalConstants.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java index 12d4f14d92a00..85603b0bfd8ab 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java @@ -37,8 +37,8 @@ private InternalConstants() { * {@code "fs.capability.etags.available"}. *
    *
  1. {@value}: store is safe
  2. - *
  3. !etags: store is safe
  4. - *
  5. etags && !{@value}: store is UNSAFE
  6. + *
  7. no etags: store is safe
  8. + *
  9. etags and not {@value}: store is UNSAFE
  10. *
*/ public static final String CAPABILITY_SAFE_READAHEAD = From a023375d6ecdf3d40cd13be27a04e38f183555d2 Mon Sep 17 00:00:00 2001 From: PJ Fanning Date: Sun, 18 Dec 2022 13:25:10 +0100 Subject: [PATCH 43/45] HADOOP-18575. Make XML transformer factory more lenient (#5224) Due diligence followup to HADOOP-18469. Add secure XML parser factories to XMLUtils (#4940) Contributed by P J Fanning --- .../java/org/apache/hadoop/util/XMLUtils.java | 34 ++++++++++++++++--- .../org/apache/hadoop/util/TestXMLUtils.java | 12 +++++++ 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java index a8040ad90a893..7aa4b2bcc5828 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java @@ -29,6 +29,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.xml.sax.SAXException; import java.io.*; @@ -41,6 +43,9 @@ @InterfaceStability.Unstable public class XMLUtils { + private static final Logger LOG = + LoggerFactory.getLogger(XMLUtils.class); + public static final String DISALLOW_DOCTYPE_DECL = "http://apache.org/xml/features/disallow-doctype-decl"; public static final String LOAD_EXTERNAL_DECL = @@ -138,8 +143,8 @@ public static TransformerFactory newSecureTransformerFactory() throws TransformerConfigurationException { TransformerFactory trfactory = TransformerFactory.newInstance(); trfactory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true); - trfactory.setAttribute(XMLConstants.ACCESS_EXTERNAL_DTD, ""); - trfactory.setAttribute(XMLConstants.ACCESS_EXTERNAL_STYLESHEET, ""); + bestEffortSetAttribute(trfactory, XMLConstants.ACCESS_EXTERNAL_DTD, ""); + bestEffortSetAttribute(trfactory, XMLConstants.ACCESS_EXTERNAL_STYLESHEET, ""); return trfactory; } @@ -156,8 +161,29 @@ public static SAXTransformerFactory newSecureSAXTransformerFactory() throws TransformerConfigurationException { SAXTransformerFactory trfactory = (SAXTransformerFactory) SAXTransformerFactory.newInstance(); trfactory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true); - trfactory.setAttribute(XMLConstants.ACCESS_EXTERNAL_DTD, ""); - trfactory.setAttribute(XMLConstants.ACCESS_EXTERNAL_STYLESHEET, ""); + bestEffortSetAttribute(trfactory, XMLConstants.ACCESS_EXTERNAL_DTD, ""); + bestEffortSetAttribute(trfactory, XMLConstants.ACCESS_EXTERNAL_STYLESHEET, ""); return trfactory; } + + /** + * Set an attribute value on a {@link TransformerFactory}. If the TransformerFactory + * does not support the attribute, the method just returns false and + * logs the issue at debug level. + * + * @param transformerFactory to update + * @param name of the attribute to set + * @param value to set on the attribute + * @return whether the attribute was successfully set + */ + static boolean bestEffortSetAttribute(TransformerFactory transformerFactory, + String name, Object value) { + try { + transformerFactory.setAttribute(name, value); + return true; + } catch (Throwable t) { + LOG.debug("Issue setting TransformerFactory attribute {}: {}", name, t.toString()); + } + return false; + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestXMLUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestXMLUtils.java index ec1b74338a113..0ebec96213b19 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestXMLUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestXMLUtils.java @@ -20,10 +20,12 @@ import java.io.InputStream; import java.io.StringReader; import java.io.StringWriter; +import javax.xml.XMLConstants; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.SAXParser; import javax.xml.transform.Transformer; import javax.xml.transform.TransformerException; +import javax.xml.transform.TransformerFactory; import javax.xml.transform.dom.DOMSource; import javax.xml.transform.stream.StreamResult; import javax.xml.transform.stream.StreamSource; @@ -31,6 +33,7 @@ import org.apache.hadoop.test.AbstractHadoopTestBase; import org.assertj.core.api.Assertions; +import org.junit.Assert; import org.junit.Test; import org.w3c.dom.Document; import org.xml.sax.InputSource; @@ -128,6 +131,15 @@ public void testExternalDtdWithSecureSAXTransformerFactory() throws Exception { } } + @Test + public void testBestEffortSetAttribute() throws Exception { + TransformerFactory factory = TransformerFactory.newInstance(); + Assert.assertFalse("unexpected attribute results in return of false", + XMLUtils.bestEffortSetAttribute(factory, "unsupportedAttribute false", "abc")); + Assert.assertTrue("expected attribute results in return of false", + XMLUtils.bestEffortSetAttribute(factory, XMLConstants.ACCESS_EXTERNAL_DTD, "")); + } + private static InputStream getResourceStream(final String filename) { return TestXMLUtils.class.getResourceAsStream(filename); } From 998be8051db6b04f26b902244e64008a3667a02c Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 19 Dec 2022 12:54:37 +0000 Subject: [PATCH 44/45] HADOOP-18470. Update index md with section on ABFS prefetching --- hadoop-project/src/site/markdown/index.md.vm | 77 +++++++++++++------- 1 file changed, 50 insertions(+), 27 deletions(-) diff --git a/hadoop-project/src/site/markdown/index.md.vm b/hadoop-project/src/site/markdown/index.md.vm index 05478ea50ac0e..5e0a46449fae9 100644 --- a/hadoop-project/src/site/markdown/index.md.vm +++ b/hadoop-project/src/site/markdown/index.md.vm @@ -23,11 +23,29 @@ Overview of Changes Users are encouraged to read the full set of release notes. This page provides an overview of the major changes. +Azure ABFS: Critical Stream Prefetch Fix +--------------------------------------------- + +The abfs has a critical bug fix +[HADOOP-18546](https://issues.apache.org/jira/browse/HADOOP-18546). +*ABFS. Disable purging list of in-progress reads in abfs stream close().* + +All users of the abfs connector in hadoop releases 3.3.2+ MUST either upgrade +or disable prefetching by setting `fs.azure.readaheadqueue.depth` to `0` + +Consult the parent JIRA [HADOOP-18521](https://issues.apache.org/jira/browse/HADOOP-18521) +*ABFS ReadBufferManager buffer sharing across concurrent HTTP requests* +for root cause analysis, details on what is affected, and mitigations. + + Vectored IO API --------------- +[HADOOP-18103](https://issues.apache.org/jira/browse/HADOOP-18103). +*High performance vectored read API in Hadoop* + The `PositionedReadable` interface has now added an operation for -Vectored (also known as Scatter/Gather IO): +Vectored IO (also known as Scatter/Gather IO): ```java void readVectored(List ranges, IntFunction allocate) @@ -38,25 +56,25 @@ possibly in parallel, with results potentially coming in out-of-order. 1. The default implementation uses a series of `readFully()` calls, so delivers equivalent performance. -2. The local filesystem uses java native IO calls for higher performance reads than `readFully()` +2. The local filesystem uses java native IO calls for higher performance reads than `readFully()`. 3. The S3A filesystem issues parallel HTTP GET requests in different threads. -Benchmarking of (modified) ORC and Parquet clients through `file://` and `s3a://` -show tangible improvements in query times. +Benchmarking of enhanced Apache ORC and Apache Parquet clients through `file://` and `s3a://` +show significant improvements in query performance. Further Reading: [FsDataInputStream](./hadoop-project-dist/hadoop-common/filesystem/fsdatainputstream.html). -Manifest Committer for Azure ABFS and google GCS performance ------------------------------------------------------------- +Mapreduce: Manifest Committer for Azure ABFS and google GCS +---------------------------------------------------------- -A new "intermediate manifest committer" uses a manifest file +The new _Intermediate Manifest Committer_ uses a manifest file to commit the work of successful task attempts, rather than renaming directories. Job commit is matter of reading all the manifests, creating the destination directories (parallelized) and renaming the files, again in parallel. -This is fast and correct on Azure Storage and Google GCS, +This is both fast and correct on Azure Storage and Google GCS, and should be used there instead of the classic v1/v2 file output committers. @@ -69,24 +87,6 @@ More details are available in the [manifest committer](./hadoop-mapreduce-client/hadoop-mapreduce-client-core/manifest_committer.html). documentation. -Transitive CVE fixes --------------------- - -A lot of dependencies have been upgraded to address recent CVEs. -Many of the CVEs were not actually exploitable through the Hadoop -so much of this work is just due diligence. -However applications which have all the library is on a class path may -be vulnerable, and the ugprades should also reduce the number of false -positives security scanners report. - -We have not been able to upgrade every single dependency to the latest -version there is. Some of those changes are just going to be incompatible. -If you have concerns about the state of a specific library, consult the apache JIRA -issue tracker to see what discussions have taken place about the library in question. - -As an open source project, contributions in this area are always welcome, -especially in testing the active branches, testing applications downstream of -those branches and of whether updated dependencies trigger regressions. HDFS: Router Based Federation ----------------------------- @@ -96,7 +96,6 @@ A lot of effort has been invested into stabilizing/improving the HDFS Router Bas 1. HDFS-13522, HDFS-16767 & Related Jiras: Allow Observer Reads in HDFS Router Based Federation. 2. HDFS-13248: RBF supports Client Locality - HDFS: Dynamic Datanode Reconfiguration -------------------------------------- @@ -109,6 +108,29 @@ cluster-wide Datanode Restarts. See [DataNode.java](https://github.com/apache/hadoop/blob/branch-3.3.5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L346-L361) for the list of dynamically reconfigurable attributes. + +Transitive CVE fixes +-------------------- + +A lot of dependencies have been upgraded to address recent CVEs. +Many of the CVEs were not actually exploitable through the Hadoop +so much of this work is just due diligence. +However applications which have all the library is on a class path may +be vulnerable, and the ugprades should also reduce the number of false +positives security scanners report. + +We have not been able to upgrade every single dependency to the latest +version there is. Some of those changes are just going to be incompatible. +If you have concerns about the state of a specific library, consult the pache JIRA +issue tracker to see whether a JIRA has been filed, discussions have taken place about +the library in question, and whether or not there is already a fix in the pipeline. +*Please don't file new JIRAs about dependency-X.Y.Z having a CVE without +searching for any existing issue first* + +As an open source project, contributions in this area are always welcome, +especially in testing the active branches, testing applications downstream of +those branches and of whether updated dependencies trigger regressions. + Getting Started =============== @@ -119,3 +141,4 @@ which shows you how to set up a single-node Hadoop installation. Then move on to the [Cluster Setup](./hadoop-project-dist/hadoop-common/ClusterSetup.html) to learn how to set up a multi-node Hadoop installation. + From 8730458d0588b0af1eae4c7b4ccf65d877406535 Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Tue, 20 Dec 2022 10:38:01 +0800 Subject: [PATCH 45/45] YARN-11349. Fix CheckStyle. --- .../AbstractDelegationTokenSecretManager.java | 4 +- .../MySQL/FederationStateStoreStoredProcs.sql | 71 +++ .../MySQL/FederationStateStoreTables.sql | 42 ++ .../MySQL/dropStoreProcedures.sql | 28 + .../FederationStateStore/MySQL/dropTables.sql | 8 + .../FederationStateStoreStoredProcs.sql | 538 +++++++++++++++++ .../SQLServer/FederationStateStoreTables.sql | 124 ++++ .../SQLServer/dropStoreProcedures.sql | 42 ++ .../SQLServer/dropTables.sql | 12 + .../impl/MemoryFederationStateStore.java | 25 +- .../store/impl/SQLFederationStateStore.java | 557 ++++++++++++++++-- .../records/RouterRMDTSecretManagerState.java | 4 +- .../store/records/RouterStoreToken.java | 25 +- .../impl/pb/RouterStoreTokenPBImpl.java | 28 + .../federation/store/sql/DatabaseProduct.java | 125 ++++ .../store/sql/FederationQueryRunner.java | 303 ++++++++++ .../store/sql/FederationSQLOutParameter.java | 91 +++ .../store/sql/ResultSetHandler.java | 30 + .../store/sql/RouterMasterKeyHandler.java | 68 +++ .../store/sql/RouterStoreTokenHandler.java | 83 +++ .../federation/store/sql/RowCountHandler.java | 56 ++ .../federation/store/sql/package-info.java | 17 + ...FederationRouterRMTokenInputValidator.java | 5 +- .../utils/FederationStateStoreUtils.java | 69 ++- .../utils/FederationStateStoreFacade.java | 36 ++ .../proto/yarn_server_federation_protos.proto | 1 + .../impl/FederationStateStoreBaseTest.java | 33 +- .../impl/HSQLDBFederationStateStore.java | 125 ++++ .../impl/TestMemoryFederationStateStore.java | 2 +- .../impl/TestSQLFederationStateStore.java | 185 ++++-- .../RouterDelegationTokenSupport.java | 65 ++ .../token/delegation/package-info.java | 20 + .../RouterDelegationTokenSecretManager.java | 84 +++ .../TestFederationClientInterceptor.java | 18 +- 34 files changed, 2783 insertions(+), 141 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/DatabaseProduct.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationQueryRunner.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationSQLOutParameter.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/ResultSetHandler.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterMasterKeyHandler.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterStoreTokenHandler.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RowCountHandler.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/package-info.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/RouterDelegationTokenSupport.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java index d0c0fac6e88df..8aaf9bbd8de17 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java @@ -866,9 +866,9 @@ private String getTokenRealOwner(TokenIdent id) { /** * Add token stats to the owner to token count mapping. * - * @param id + * @param id token id. */ - private void addTokenForOwnerStats(TokenIdent id) { + protected void addTokenForOwnerStats(TokenIdent id) { String realOwner = getTokenRealOwner(id); tokenOwnerStats.put(realOwner, tokenOwnerStats.getOrDefault(realOwner, 0L)+1); diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql index 6461cf2bd75e4..fadb144f421d1 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql @@ -219,4 +219,75 @@ BEGIN SELECT ROW_COUNT() INTO rowCount_OUT; END // +CREATE PROCEDURE sp_addMasterKey( + IN keyId_IN bigint, IN masterKey_IN varchar(1024), + OUT rowCount_OUT int) +BEGIN + INSERT INTO masterKeys(keyId, masterKey) + (SELECT keyId_IN, masterKey_IN + FROM masterKeys + WHERE keyId = keyId_IN + HAVING COUNT(*) = 0); + SELECT ROW_COUNT() INTO rowCount_OUT; +END // + +CREATE PROCEDURE sp_getMasterKey( + IN keyId_IN bigint, + OUT masterKey_OUT varchar(1024)) +BEGIN + SELECT masterKey INTO masterKey_OUT + FROM masterKeys + WHERE keyId = keyId_IN; +END // + +CREATE PROCEDURE sp_deleteMasterKey( + IN keyId_IN bigint, OUT rowCount_OUT int) +BEGIN + DELETE FROM masterKeys + WHERE keyId = keyId_IN; + SELECT ROW_COUNT() INTO rowCount_OUT; +END // + +CREATE PROCEDURE sp_addDelegationToken( + IN sequenceNum_IN bigint, IN tokenIdent_IN varchar(1024), + IN token_IN varchar(1024), IN renewDate_IN bigint, + OUT rowCount_OUT int) +BEGIN + INSERT INTO delegationTokens(sequenceNum, tokenIdent, token, renewDate) + (SELECT sequenceNum_IN, tokenIdent_IN, token_IN, renewDate_IN + FROM delegationTokens + WHERE sequenceNum = sequenceNum_IN + HAVING COUNT(*) = 0); + SELECT ROW_COUNT() INTO rowCount_OUT; +END // + +CREATE PROCEDURE sp_getDelegationToken( + IN sequenceNum_IN bigint, OUT tokenIdent_OUT varchar(1024), + OUT token_OUT varchar(1024), OUT renewDate_OUT bigint) +BEGIN + SELECT tokenIdent INTO tokenIdent_OUT, token INTO token_OUT, renewDate INTO renewDate_OUT + FROM delegationTokens + WHERE sequenceNum = sequenceNum_IN; +END // + +CREATE PROCEDURE sp_updateDelegationToken( + IN sequenceNum_IN bigint, IN tokenIdent_IN varchar(1024), + IN token_IN varchar(1024), IN renewDate_IN bigint, OUT rowCount_OUT int) +BEGIN + UPDATE delegationTokens + SET tokenIdent = tokenIdent_IN, + token = token_IN, + renewDate = renewDate_IN + WHERE sequenceNum = sequenceNum_IN; + SELECT ROW_COUNT() INTO rowCount_OUT; +END // + +CREATE PROCEDURE sp_deleteDelegationToken( + IN sequenceNum_IN bigint, OUT rowCount_OUT int) +BEGIN + DELETE FROM delegationTokens + WHERE sequenceNum = sequenceNum_IN; + SELECT ROW_COUNT() INTO rowCount_OUT; +END // + DELIMITER ; diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql index d61a10f998b3e..eaefb5d20016a 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql @@ -52,4 +52,46 @@ CREATE TABLE reservationsHomeSubCluster ( reservationId varchar(128) NOT NULL, homeSubCluster varchar(256) NOT NULL, CONSTRAINT pk_reservationId PRIMARY KEY (reservationId) +); + +CREATE TABLE masterKeys ( + keyId bigint NOT NULL, + masterKey varchar(1024) NOT NULL, + CONSTRAINT pk_keyId PRIMARY KEY (keyId) +); + +CREATE TABLE delegationTokens +( + sequenceNum bigint NOT NULL, + tokenIdent varchar(1024) NOT NULL, + token varchar(1024) NOT NULL, + renewDate bigint NOT NULL, + CONSTRAINT pk_sequenceNum PRIMARY KEY (sequenceNum) +); + +CREATE TABLE sequenceTable ( + sequenceName varchar(255) NOT NULL, + nextVal bigint(20) NOT NULL, + CONSTRAINT pk_sequenceName PRIMARY KEY (sequenceName) +); + +CREATE TABLE masterKeys ( + keyId bigint NOT NULL, + masterKey varchar(1024) NOT NULL, + CONSTRAINT pk_keyId PRIMARY KEY (keyId) +); + +CREATE TABLE delegationTokens +( + sequenceNum bigint NOT NULL, + tokenIdent varchar(1024) NOT NULL, + token varchar(1024) NOT NULL, + renewDate bigint NOT NULL, + CONSTRAINT pk_sequenceNum PRIMARY KEY (sequenceNum) +); + +CREATE TABLE sequenceTable ( + sequenceName varchar(255) NOT NULL, + nextVal bigint(20) NOT NULL, + CONSTRAINT pk_sequenceName PRIMARY KEY (sequenceName) ); \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql index a2f0b882b3ff6..e041c079a0871 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql @@ -55,3 +55,31 @@ DROP PROCEDURE sp_getReservationsHomeSubCluster; DROP PROCEDURE sp_deleteReservationHomeSubCluster; DROP PROCEDURE sp_updateReservationHomeSubCluster; + +DROP PROCEDURE sp_addMasterKey; + +DROP PROCEDURE sp_getMasterKey; + +DROP PROCEDURE sp_deleteMasterKey; + +DROP PROCEDURE sp_addDelegationToken; + +DROP PROCEDURE sp_getDelegationToken; + +DROP PROCEDURE sp_updateDelegationToken; + +DROP PROCEDURE sp_deleteDelegationToken; + +DROP PROCEDURE sp_addMasterKey; + +DROP PROCEDURE sp_getMasterKey; + +DROP PROCEDURE sp_deleteMasterKey; + +DROP PROCEDURE sp_addDelegationToken; + +DROP PROCEDURE sp_getDelegationToken; + +DROP PROCEDURE sp_updateDelegationToken; + +DROP PROCEDURE sp_deleteDelegationToken; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql index d29f8652c153e..2dec594088c6d 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql @@ -27,3 +27,11 @@ DROP TABLE membership; DROP TABLE policies; DROP TABLE reservationsHomeSubCluster; + +DROP TABLE masterKeys; + +DROP TABLE delegationTokens; + +DROP TABLE masterKeys; + +DROP TABLE delegationTokens; diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoredProcs.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoredProcs.sql index cc8a79d627367..651379200585c 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoredProcs.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoredProcs.sql @@ -699,4 +699,542 @@ AS BEGIN ) WITH log END CATCH END; +GO + +IF OBJECT_ID ( '[sp_addMasterKey]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_addMasterKey]; +GO + +CREATE PROCEDURE [dbo].[sp_addMasterKey] + @keyId_IN BIGINT + @masterKey_IN VARCHAR(1024), + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + -- If application to sub-cluster map doesn't exist, insert it. + -- Otherwise don't change the current mapping. + IF NOT EXISTS (SELECT TOP 1 * + FROM [dbo].[masterKeys] + WHERE [keyId] = @keyId_IN) + + INSERT INTO [dbo].[masterKeys] ( + [keyId], + [masterKey]) + VALUES ( + @keyId_IN, + @masterKey_IN); + -- End of the IF block + + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_getMasterKey]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_getMasterKey]; +GO + +CREATE PROCEDURE [dbo].[sp_getMasterKey] + @keyId_IN bigint, + @masterKey_OUT VARCHAR(1024) OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + + SELECT @masterKey_OUT = [masterKey] + FROM [dbo].[masterKeys] + WHERE [keyId] = @keyId_IN; + + END TRY + + BEGIN CATCH + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_deleteMasterKey]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_deleteMasterKey]; +GO + +CREATE PROCEDURE [dbo].[sp_deleteMasterKey] + @keyId_IN bigint, + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + + DELETE FROM [dbo].[masterKeys] + WHERE [keyId] = @keyId_IN; + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_addDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_addDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_addDelegationToken] + @sequenceNum_IN BIGINT, + @tokenIdent_IN VARCHAR(1024), + @token_IN VARCHAR(1024), + @renewDate_IN BIGINT, + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + -- If application to sub-cluster map doesn't exist, insert it. + -- Otherwise don't change the current mapping. + IF NOT EXISTS (SELECT TOP 1 * + FROM [dbo].[delegationTokens] + WHERE [sequenceNum] = @sequenceNum_IN) + + INSERT INTO [dbo].[delegationTokens] ( + [sequenceNum], + [tokenIdent], + [token], + [renewDate]) + VALUES ( + @sequenceNum_IN, + @tokenIdent_IN, + @token_IN, + @renewDate_IN); + -- End of the IF block + + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_getDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_getDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_getDelegationToken] + @sequenceNum_IN BIGINT, + @tokenIdent_OUT VARCHAR(1024) OUTPUT, + @token_OUT VARCHAR(1024) OUTPUT, + @renewDate_OUT BIGINT OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + + SELECT @tokenIdent_OUT = [tokenIdent], + @token_OUT = [token], + @renewDate_OUT = [renewDate] + FROM [dbo].[delegationTokens] + WHERE [sequenceNum] = @sequenceNum_IN; + + END TRY + + BEGIN CATCH + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_updateDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_updateDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_updateDelegationToken] + @sequenceNum_IN BIGINT, + @tokenIdent_IN VARCHAR(1024), + @token_IN VARCHAR(1024), + @renewDate_IN BIGINT, + @rowCount_OUT BIGINT OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + + UPDATE [dbo].[delegationTokens] + SET [tokenIdent] = @tokenIdent_IN, + [token] = @token_IN, + [renewDate] = @renewDate_IN + WHERE [sequenceNum] = @sequenceNum_IN; + SELECT @rowCount_OUT = @@ROWCOUNT; + + END TRY + + BEGIN CATCH + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_deleteDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_deleteDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_deleteDelegationToken] + @sequenceNum_IN bigint, + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + + DELETE FROM [dbo].[delegationTokens] + WHERE [sequenceNum] = @sequenceNum_IN; + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_addMasterKey]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_addMasterKey]; +GO + +CREATE PROCEDURE [dbo].[sp_addMasterKey] + @keyId_IN BIGINT + @masterKey_IN VARCHAR(1024), + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + -- If application to sub-cluster map doesn't exist, insert it. + -- Otherwise don't change the current mapping. + IF NOT EXISTS (SELECT TOP 1 * + FROM [dbo].[masterKeys] + WHERE [keyId] = @keyId_IN) + + INSERT INTO [dbo].[masterKeys] ( + [keyId], + [masterKey]) + VALUES ( + @keyId_IN, + @masterKey_IN); + -- End of the IF block + + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_getMasterKey]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_getMasterKey]; +GO + +CREATE PROCEDURE [dbo].[sp_getMasterKey] + @keyId_IN bigint, + @masterKey_OUT VARCHAR(1024) OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + + SELECT @masterKey_OUT = [masterKey] + FROM [dbo].[masterKeys] + WHERE [keyId] = @keyId_IN; + + END TRY + + BEGIN CATCH + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_deleteMasterKey]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_deleteMasterKey]; +GO + +CREATE PROCEDURE [dbo].[sp_deleteMasterKey] + @keyId_IN bigint, + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + + DELETE FROM [dbo].[masterKeys] + WHERE [keyId] = @keyId_IN; + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_addDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_addDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_addDelegationToken] + @sequenceNum_IN BIGINT, + @tokenIdent_IN VARCHAR(1024), + @token_IN VARCHAR(1024), + @renewDate_IN BIGINT, + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + -- If application to sub-cluster map doesn't exist, insert it. + -- Otherwise don't change the current mapping. + IF NOT EXISTS (SELECT TOP 1 * + FROM [dbo].[delegationTokens] + WHERE [sequenceNum] = @sequenceNum_IN) + + INSERT INTO [dbo].[delegationTokens] ( + [sequenceNum], + [tokenIdent], + [token], + [renewDate]) + VALUES ( + @sequenceNum_IN, + @tokenIdent_IN, + @token_IN, + @renewDate_IN); + -- End of the IF block + + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_getDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_getDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_getDelegationToken] + @sequenceNum_IN BIGINT, + @tokenIdent_OUT VARCHAR(1024) OUTPUT, + @token_OUT VARCHAR(1024) OUTPUT, + @renewDate_OUT BIGINT OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + + SELECT @tokenIdent_OUT = [tokenIdent], + @token_OUT = [token], + @renewDate_OUT = [renewDate] + FROM [dbo].[delegationTokens] + WHERE [sequenceNum] = @sequenceNum_IN; + + END TRY + + BEGIN CATCH + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_updateDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_updateDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_updateDelegationToken] + @sequenceNum_IN BIGINT, + @tokenIdent_IN VARCHAR(1024), + @token_IN VARCHAR(1024), + @renewDate_IN BIGINT, + @rowCount_OUT BIGINT OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + + UPDATE [dbo].[delegationTokens] + SET [tokenIdent] = @tokenIdent_IN, + [token] = @token_IN, + [renewDate] = @renewDate_IN + WHERE [sequenceNum] = @sequenceNum_IN; + SELECT @rowCount_OUT = @@ROWCOUNT; + + END TRY + + BEGIN CATCH + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; +GO + +IF OBJECT_ID ( '[sp_deleteDelegationToken]', 'P' ) IS NOT NULL + DROP PROCEDURE [sp_deleteDelegationToken]; +GO + +CREATE PROCEDURE [dbo].[sp_deleteDelegationToken] + @sequenceNum_IN bigint, + @rowCount_OUT int OUTPUT +AS BEGIN + DECLARE @errorMessage nvarchar(4000) + + BEGIN TRY + BEGIN TRAN + + DELETE FROM [dbo].[delegationTokens] + WHERE [sequenceNum] = @sequenceNum_IN; + SELECT @rowCount_OUT = @@ROWCOUNT; + + COMMIT TRAN + END TRY + + BEGIN CATCH + ROLLBACK TRAN + + SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE()) + + /* raise error and terminate the execution */ + RAISERROR(@errorMessage, --- Error Message + 1, -- Severity + -1 -- State + ) WITH log + END CATCH +END; GO \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql index fb8a1bff554b3..9af902333a75b 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql @@ -155,4 +155,128 @@ IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables ELSE PRINT 'Table reservationsHomeSubCluster exists, no operation required...' GO +GO + +IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables + WHERE name = 'masterKeys' + AND schema_id = SCHEMA_ID('dbo')) + BEGIN + PRINT 'Table masterKeys does not exist, create it...' + + SET ANSI_NULLS ON + + SET QUOTED_IDENTIFIER ON + + SET ANSI_PADDING ON + + CREATE TABLE [dbo].[masterKeys]( + keyId BIGINT NOT NULL, + masterKey VARCHAR(1024) NOT NULL, + CONSTRAINT [pk_keyId] PRIMARY KEY + ( + [keyId] + ) + ) + + SET ANSI_PADDING OFF + + PRINT 'Table masterKeys created.' + END +ELSE + PRINT 'Table masterKeys exists, no operation required...' + GO +GO + +IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables + WHERE name = 'delegationTokens' + AND schema_id = SCHEMA_ID('dbo')) + BEGIN + PRINT 'Table delegationTokens does not exist, create it...' + + SET ANSI_NULLS ON + + SET QUOTED_IDENTIFIER ON + + SET ANSI_PADDING ON + + CREATE TABLE [dbo].[delegationTokens]( + sequenceNum BIGINT NOT NULL, + tokenIdent VARCHAR(1024) NOT NULL, + token VARCHAR(1024) NOT NULL, + renewDate BIGINT NOT NULL, + CONSTRAINT [pk_sequenceNum] PRIMARY KEY + ( + [sequenceNum] + ) + ) + + SET ANSI_PADDING OFF + + PRINT 'Table delegationTokens created.' + END +ELSE + PRINT 'Table delegationTokens exists, no operation required...' + GO +GO + +IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables + WHERE name = 'masterKeys' + AND schema_id = SCHEMA_ID('dbo')) + BEGIN + PRINT 'Table masterKeys does not exist, create it...' + + SET ANSI_NULLS ON + + SET QUOTED_IDENTIFIER ON + + SET ANSI_PADDING ON + + CREATE TABLE [dbo].[masterKeys]( + keyId BIGINT NOT NULL, + masterKey VARCHAR(1024) NOT NULL, + CONSTRAINT [pk_keyId] PRIMARY KEY + ( + [keyId] + ) + ) + + SET ANSI_PADDING OFF + + PRINT 'Table masterKeys created.' + END +ELSE + PRINT 'Table masterKeys exists, no operation required...' + GO +GO + +IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables + WHERE name = 'delegationTokens' + AND schema_id = SCHEMA_ID('dbo')) + BEGIN + PRINT 'Table delegationTokens does not exist, create it...' + + SET ANSI_NULLS ON + + SET QUOTED_IDENTIFIER ON + + SET ANSI_PADDING ON + + CREATE TABLE [dbo].[delegationTokens]( + sequenceNum BIGINT NOT NULL, + tokenIdent VARCHAR(1024) NOT NULL, + token VARCHAR(1024) NOT NULL, + renewDate BIGINT NOT NULL, + CONSTRAINT [pk_sequenceNum] PRIMARY KEY + ( + [sequenceNum] + ) + ) + + SET ANSI_PADDING OFF + + PRINT 'Table delegationTokens created.' + END +ELSE + PRINT 'Table delegationTokens exists, no operation required...' + GO GO \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropStoreProcedures.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropStoreProcedures.sql index 6204df2f418f2..1dd226b6af6b4 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropStoreProcedures.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropStoreProcedures.sql @@ -74,3 +74,45 @@ GO DROP PROCEDURE IF EXISTS [sp_deleteReservationHomeSubCluster]; GO + +DROP PROCEDURE IF EXISTS [sp_addMasterKey]; +GO + +DROP PROCEDURE IF EXISTS [sp_getMasterKey]; +GO + +DROP PROCEDURE IF EXISTS [sp_deleteMasterKey]; +GO + +DROP PROCEDURE IF EXISTS [sp_addDelegationToken]; +GO + +DROP PROCEDURE IF EXISTS [sp_getDelegationToken]; +GO + +DROP PROCEDURE IF EXISTS [sp_updateDelegationToken]; +GO + +DROP PROCEDURE IF EXISTS [sp_deleteDelegationToken]; +GO + +DROP PROCEDURE IF EXISTS [sp_addMasterKey]; +GO + +DROP PROCEDURE IF EXISTS [sp_getMasterKey]; +GO + +DROP PROCEDURE IF EXISTS [sp_deleteMasterKey]; +GO + +DROP PROCEDURE IF EXISTS [sp_addDelegationToken]; +GO + +DROP PROCEDURE IF EXISTS [sp_getDelegationToken]; +GO + +DROP PROCEDURE IF EXISTS [sp_updateDelegationToken]; +GO + +DROP PROCEDURE IF EXISTS [sp_deleteDelegationToken]; +GO diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropTables.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropTables.sql index 9bcacb7f8851a..ac0afe4126d2a 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropTables.sql +++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/dropTables.sql @@ -32,3 +32,15 @@ GO DROP TABLE [reservationsHomeSubCluster]; GO + +DROP TABLE [masterKeys]; +GO + +DROP TABLE [delegationTokens]; +GO + +DROP TABLE [masterKeys]; +GO + +DROP TABLE [delegationTokens]; +GO diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java index 41ade680be2e3..d44c30eef28b1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java @@ -494,8 +494,7 @@ public RouterRMTokenResponse storeNewToken(RouterRMTokenRequest request) RouterStoreToken storeToken = request.getRouterStoreToken(); RMDelegationTokenIdentifier tokenIdentifier = (RMDelegationTokenIdentifier) storeToken.getTokenIdentifier(); - Long renewDate = storeToken.getRenewDate(); - storeOrUpdateRouterRMDT(tokenIdentifier, renewDate, false); + storeOrUpdateRouterRMDT(tokenIdentifier, storeToken, false); return RouterRMTokenResponse.newInstance(storeToken); } @@ -505,10 +504,10 @@ public RouterRMTokenResponse updateStoredToken(RouterRMTokenRequest request) RouterStoreToken storeToken = request.getRouterStoreToken(); RMDelegationTokenIdentifier tokenIdentifier = (RMDelegationTokenIdentifier) storeToken.getTokenIdentifier(); - Long renewDate = storeToken.getRenewDate(); - Map rmDTState = routerRMSecretManagerState.getTokenState(); + Map rmDTState = + routerRMSecretManagerState.getTokenState(); rmDTState.remove(tokenIdentifier); - storeOrUpdateRouterRMDT(tokenIdentifier, renewDate, true); + storeOrUpdateRouterRMDT(tokenIdentifier, storeToken, true); return RouterRMTokenResponse.newInstance(storeToken); } @@ -518,7 +517,8 @@ public RouterRMTokenResponse removeStoredToken(RouterRMTokenRequest request) RouterStoreToken storeToken = request.getRouterStoreToken(); RMDelegationTokenIdentifier tokenIdentifier = (RMDelegationTokenIdentifier) storeToken.getTokenIdentifier(); - Map rmDTState = routerRMSecretManagerState.getTokenState(); + Map rmDTState = + routerRMSecretManagerState.getTokenState(); rmDTState.remove(tokenIdentifier); return RouterRMTokenResponse.newInstance(storeToken); } @@ -529,13 +529,13 @@ public RouterRMTokenResponse getTokenByRouterStoreToken(RouterRMTokenRequest req RouterStoreToken storeToken = request.getRouterStoreToken(); RMDelegationTokenIdentifier tokenIdentifier = (RMDelegationTokenIdentifier) storeToken.getTokenIdentifier(); - Map rmDTState = routerRMSecretManagerState.getTokenState(); + Map rmDTState = + routerRMSecretManagerState.getTokenState(); if (!rmDTState.containsKey(tokenIdentifier)) { LOG.info("Router RMDelegationToken: {} does not exist.", tokenIdentifier); throw new IOException("Router RMDelegationToken: " + tokenIdentifier + " does not exist."); } - RouterStoreToken resultToken = - RouterStoreToken.newInstance(tokenIdentifier, rmDTState.get(tokenIdentifier)); + RouterStoreToken resultToken = rmDTState.get(tokenIdentifier); return RouterRMTokenResponse.newInstance(resultToken); } @@ -565,13 +565,14 @@ public int incrementCurrentKeyId() { } private void storeOrUpdateRouterRMDT(RMDelegationTokenIdentifier rmDTIdentifier, - Long renewDate, boolean isUpdate) throws IOException { - Map rmDTState = routerRMSecretManagerState.getTokenState(); + RouterStoreToken routerStoreToken, boolean isUpdate) throws IOException { + Map rmDTState = + routerRMSecretManagerState.getTokenState(); if (rmDTState.containsKey(rmDTIdentifier)) { LOG.info("Error storing info for RMDelegationToken: {}.", rmDTIdentifier); throw new IOException("Router RMDelegationToken: " + rmDTIdentifier + "is already stored."); } - rmDTState.put(rmDTIdentifier, renewDate); + rmDTState.put(rmDTIdentifier, routerStoreToken); if (!isUpdate) { routerRMSecretManagerState.setDtSequenceNumber(rmDTIdentifier.getSequenceNumber()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java index 1e3f3a12f3dcb..f16fe673ce3b9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java @@ -33,10 +33,12 @@ import org.apache.commons.lang3.NotImplementedException; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ReservationId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.store.FederationStateStore; import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException; import org.apache.hadoop.yarn.server.federation.store.metrics.FederationStateStoreClientMetrics; @@ -86,11 +88,19 @@ import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyResponse; import org.apache.hadoop.yarn.server.federation.store.records.RouterRMTokenRequest; import org.apache.hadoop.yarn.server.federation.store.records.RouterRMTokenResponse; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; import org.apache.hadoop.yarn.server.federation.store.utils.FederationApplicationHomeSubClusterStoreInputValidator; import org.apache.hadoop.yarn.server.federation.store.utils.FederationMembershipStateStoreInputValidator; import org.apache.hadoop.yarn.server.federation.store.utils.FederationPolicyStoreInputValidator; -import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils; import org.apache.hadoop.yarn.server.federation.store.utils.FederationReservationHomeSubClusterStoreInputValidator; +import org.apache.hadoop.yarn.server.federation.store.utils.FederationRouterRMTokenInputValidator; +import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils; +import org.apache.hadoop.yarn.server.federation.store.sql.FederationSQLOutParameter; +import org.apache.hadoop.yarn.server.federation.store.sql.FederationQueryRunner; +import org.apache.hadoop.yarn.server.federation.store.sql.RouterMasterKeyHandler; +import org.apache.hadoop.yarn.server.federation.store.sql.RouterStoreTokenHandler; +import org.apache.hadoop.yarn.server.federation.store.sql.RowCountHandler; import org.apache.hadoop.yarn.server.records.Version; import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.MonotonicClock; @@ -100,6 +110,13 @@ import org.apache.hadoop.classification.VisibleForTesting; import com.zaxxer.hikari.HikariDataSource; +import static java.sql.Types.INTEGER; +import static java.sql.Types.VARCHAR; +import static java.sql.Types.BIGINT; +import static org.apache.hadoop.yarn.server.federation.store.sql.FederationQueryRunner.YARN_ROUTER_CURRENT_KEY_ID; +import static org.apache.hadoop.yarn.server.federation.store.sql.FederationQueryRunner.YARN_ROUTER_SEQUENCE_NUM; +import static org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils.convertMasterKeyToDelegationKey; + /** * SQL implementation of {@link FederationStateStore}. */ @@ -164,6 +181,27 @@ public class SQLFederationStateStore implements FederationStateStore { protected static final String CALL_SP_UPDATE_RESERVATION_HOME_SUBCLUSTER = "{call sp_updateReservationHomeSubCluster(?, ?, ?)}"; + protected static final String CALL_SP_ADD_MASTERKEY = + "{call sp_addMasterKey(?, ?, ?)}"; + + protected static final String CALL_SP_GET_MASTERKEY = + "{call sp_getMasterKey(?, ?)}"; + + protected static final String CALL_SP_DELETE_MASTERKEY = + "{call sp_deleteMasterKey(?, ?)}"; + + protected static final String CALL_SP_ADD_DELEGATIONTOKEN = + "{call sp_addDelegationToken(?, ?, ?, ?, ?)}"; + + protected static final String CALL_SP_GET_DELEGATIONTOKEN = + "{call sp_getDelegationToken(?, ?, ?, ?)}"; + + protected static final String CALL_SP_UPDATE_DELEGATIONTOKEN = + "{call sp_updateDelegationToken(?, ?, ?, ?, ?)}"; + + protected static final String CALL_SP_DELETE_DELEGATIONTOKEN = + "{call sp_deleteDelegationToken(?, ?)}"; + private Calendar utcCalendar = Calendar.getInstance(TimeZone.getTimeZone("UTC")); @@ -247,7 +285,7 @@ public SubClusterRegisterResponse registerSubCluster( cstmt.setString("state_IN", subClusterInfo.getState().toString()); cstmt.setLong("lastStartTime_IN", subClusterInfo.getLastStartTime()); cstmt.setString("capability_IN", subClusterInfo.getCapability()); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -302,7 +340,7 @@ public SubClusterDeregisterResponse deregisterSubCluster( // Set the parameters for the stored procedure cstmt.setString("subClusterId_IN", subClusterId.getId()); cstmt.setString("state_IN", state.toString()); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -356,7 +394,7 @@ public SubClusterHeartbeatResponse subClusterHeartbeat( cstmt.setString("subClusterId_IN", subClusterId.getId()); cstmt.setString("state_IN", state.toString()); cstmt.setString("capability_IN", subClusterHeartbeatRequest.getCapability()); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -408,14 +446,14 @@ public GetSubClusterInfoResponse getSubCluster( cstmt.setString("subClusterId_IN", subClusterId.getId()); // Set the parameters for the stored procedure - cstmt.registerOutParameter("amRMServiceAddress_OUT", java.sql.Types.VARCHAR); - cstmt.registerOutParameter("clientRMServiceAddress_OUT", java.sql.Types.VARCHAR); - cstmt.registerOutParameter("rmAdminServiceAddress_OUT", java.sql.Types.VARCHAR); - cstmt.registerOutParameter("rmWebServiceAddress_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("amRMServiceAddress_OUT", VARCHAR); + cstmt.registerOutParameter("clientRMServiceAddress_OUT", VARCHAR); + cstmt.registerOutParameter("rmAdminServiceAddress_OUT", VARCHAR); + cstmt.registerOutParameter("rmWebServiceAddress_OUT", VARCHAR); cstmt.registerOutParameter("lastHeartBeat_OUT", java.sql.Types.TIMESTAMP); - cstmt.registerOutParameter("state_OUT", java.sql.Types.VARCHAR); - cstmt.registerOutParameter("lastStartTime_OUT", java.sql.Types.BIGINT); - cstmt.registerOutParameter("capability_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("state_OUT", VARCHAR); + cstmt.registerOutParameter("lastStartTime_OUT", BIGINT); + cstmt.registerOutParameter("capability_OUT", VARCHAR); // Execute the query long startTime = clock.getTime(); @@ -548,8 +586,8 @@ public AddApplicationHomeSubClusterResponse addApplicationHomeSubCluster( // Set the parameters for the stored procedure cstmt.setString("applicationId_IN", appId.toString()); cstmt.setString("homeSubCluster_IN", subClusterId.getId()); - cstmt.registerOutParameter("storedHomeSubCluster_OUT", java.sql.Types.VARCHAR); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("storedHomeSubCluster_OUT", VARCHAR); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -625,7 +663,7 @@ public UpdateApplicationHomeSubClusterResponse updateApplicationHomeSubCluster( // Set the parameters for the stored procedure cstmt.setString("applicationId_IN", appId.toString()); cstmt.setString("homeSubCluster_IN", subClusterId.getId()); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -677,7 +715,7 @@ public GetApplicationHomeSubClusterResponse getApplicationHomeSubCluster( // Set the parameters for the stored procedure cstmt.setString("applicationId_IN", applicationId.toString()); - cstmt.registerOutParameter("homeSubCluster_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("homeSubCluster_OUT", VARCHAR); // Execute the query long startTime = clock.getTime(); @@ -775,7 +813,7 @@ public DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubCluster( // Set the parameters for the stored procedure cstmt.setString("applicationId_IN", applicationId.toString()); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -825,7 +863,7 @@ public GetSubClusterPolicyConfigurationResponse getPolicyConfiguration( // Set the parameters for the stored procedure cstmt.setString("queue_IN", request.getQueue()); - cstmt.registerOutParameter("policyType_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("policyType_OUT", VARCHAR); cstmt.registerOutParameter("params_OUT", java.sql.Types.VARBINARY); // Execute the query @@ -877,7 +915,7 @@ public SetSubClusterPolicyConfigurationResponse setPolicyConfiguration( cstmt.setString("queue_IN", policyConf.getQueue()); cstmt.setString("policyType_IN", policyConf.getType()); cstmt.setBytes("params_IN", getByteArray(policyConf.getParams())); - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -984,6 +1022,22 @@ protected Connection getConnection() throws SQLException { return dataSource.getConnection(); } + /** + * Get a connection from the DataSource pool. + * + * @param isCommitted Whether to enable automatic transaction commit. + * If set to true, turn on transaction autocommit, + * if set to false, turn off transaction autocommit. + * + * @return a connection from the DataSource pool. + * @throws SQLException on failure. + */ + protected Connection getConnection(boolean isCommitted) throws SQLException { + Connection dbConn = getConnection(); + dbConn.setAutoCommit(isCommitted); + return dbConn; + } + @VisibleForTesting protected CallableStatement getCallableStatement(String procedure) throws SQLException { @@ -1029,9 +1083,9 @@ public AddReservationHomeSubClusterResponse addReservationHomeSubCluster( // 2)IN homeSubCluster_IN varchar(256) cstmt.setString("homeSubCluster_IN", subClusterId.getId()); // 3) OUT storedHomeSubCluster_OUT varchar(256) - cstmt.registerOutParameter("storedHomeSubCluster_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("storedHomeSubCluster_OUT", VARCHAR); // 4) OUT rowCount_OUT int - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -1119,7 +1173,7 @@ public GetReservationHomeSubClusterResponse getReservationHomeSubCluster( // 1)IN reservationId_IN varchar(128) cstmt.setString("reservationId_IN", reservationId.toString()); // 2)OUT homeSubCluster_OUT varchar(256) - cstmt.registerOutParameter("homeSubCluster_OUT", java.sql.Types.VARCHAR); + cstmt.registerOutParameter("homeSubCluster_OUT", VARCHAR); // Execute the query long startTime = clock.getTime(); @@ -1237,7 +1291,7 @@ public DeleteReservationHomeSubClusterResponse deleteReservationHomeSubCluster( // 1)IN reservationId_IN varchar(128) cstmt.setString("reservationId_IN", reservationId.toString()); // 2)OUT rowCount_OUT int - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -1306,7 +1360,7 @@ public UpdateReservationHomeSubClusterResponse updateReservationHomeSubCluster( // 2)IN homeSubCluster_IN varchar(256) cstmt.setString("homeSubCluster_IN", subClusterId.getId()); // 3)OUT rowCount_OUT int - cstmt.registerOutParameter("rowCount_OUT", java.sql.Types.INTEGER); + cstmt.registerOutParameter("rowCount_OUT", INTEGER); // Execute the query long startTime = clock.getTime(); @@ -1353,70 +1407,503 @@ public Connection getConn() { return conn; } + /** + * SQLFederationStateStore Supports Store New MasterKey. + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey. + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterMasterKeyResponse storeNewMasterKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2: Parse the parameters and serialize the DelegationKey as a string. + DelegationKey delegationKey = convertMasterKeyToDelegationKey(request); + int keyId = delegationKey.getKeyId(); + String delegationKeyStr = FederationStateStoreUtils.encodeWritable(delegationKey); + + // Step3. store data in database. + try { + + FederationSQLOutParameter rowCountOUT = + new FederationSQLOutParameter<>("rowCount_OUT", INTEGER, Integer.class); + + // Execute the query + long startTime = clock.getTime(); + Integer rowCount = getRowCountByProcedureSQL(CALL_SP_ADD_MASTERKEY, keyId, + delegationKeyStr, rowCountOUT); + long stopTime = clock.getTime(); + + // We hope that 1 record can be written to the database. + // If the number of records is not 1, it means that the data was written incorrectly. + if (rowCount != 1) { + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "Wrong behavior during the insertion of masterKey, keyId = %s. " + + "please check the records of the database.", String.valueOf(keyId)); + } + FederationStateStoreClientMetrics.succeededStateStoreCall(stopTime - startTime); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + FederationStateStoreUtils.logAndThrowRetriableException(e, LOG, + "Unable to insert the newly masterKey, keyId = %s.", String.valueOf(keyId)); + } + + // Step4. Query Data from the database and return the result. + return getMasterKeyByDelegationKey(request); } + /** + * SQLFederationStateStore Supports Remove MasterKey. + * + * Defined the sp_deleteMasterKey procedure. + * This procedure requires 1 input parameters, 1 output parameters. + * Input parameters + * 1. IN keyId_IN int + * Output parameters + * 2. OUT rowCount_OUT int + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterMasterKeyResponse removeStoredMasterKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2: Parse parameters and get KeyId. + RouterMasterKey paramMasterKey = request.getRouterMasterKey(); + int paramKeyId = paramMasterKey.getKeyId(); + + // Step3. Clear data from database. + try { + + // Execute the query + long startTime = clock.getTime(); + FederationSQLOutParameter rowCountOUT = + new FederationSQLOutParameter<>("rowCount_OUT", INTEGER, Integer.class); + Integer rowCount = getRowCountByProcedureSQL(CALL_SP_DELETE_MASTERKEY, + paramKeyId, rowCountOUT); + long stopTime = clock.getTime(); + + // if it is equal to 0 it means the call + // did not delete the reservation from FederationStateStore + if (rowCount == 0) { + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "masterKeyId = %s does not exist.", String.valueOf(paramKeyId)); + } else if (rowCount != 1) { + // if it is different from 1 it means the call + // had a wrong behavior. Maybe the database is not set correctly. + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "Wrong behavior during deleting the keyId %s. " + + "The database is expected to delete 1 record, " + + "but the number of deleted records returned by the database is greater than 1, " + + "indicating that a duplicate masterKey occurred during the deletion process.", + paramKeyId); + } + + LOG.info("Delete from the StateStore the keyId: {}.", paramKeyId); + FederationStateStoreClientMetrics.succeededStateStoreCall(stopTime - startTime); + return RouterMasterKeyResponse.newInstance(paramMasterKey); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + FederationStateStoreUtils.logAndThrowRetriableException(e, LOG, + "Unable to delete the keyId %s.", paramKeyId); + } + + throw new YarnException("Unable to delete the masterKey, keyId = " + paramKeyId); } + /** + * SQLFederationStateStore Supports Remove MasterKey. + * + * Defined the sp_getMasterKey procedure. + * this procedure requires 2 parameters. + * Input parameters: + * 1. IN keyId_IN int + * Output parameters: + * 2. OUT masterKey_OUT varchar(1024) + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterMasterKeyResponse getMasterKeyByDelegationKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2: Parse parameters and get KeyId. + RouterMasterKey paramMasterKey = request.getRouterMasterKey(); + int paramKeyId = paramMasterKey.getKeyId(); + + // Step3: Call the stored procedure to get the result. + try { + + FederationQueryRunner runner = new FederationQueryRunner(); + FederationSQLOutParameter masterKeyOUT = + new FederationSQLOutParameter<>("masterKey_OUT", VARCHAR, String.class); + + // Execute the query + long startTime = clock.getTime(); + RouterMasterKey routerMasterKey = runner.execute( + conn, CALL_SP_GET_MASTERKEY, new RouterMasterKeyHandler(), paramKeyId, masterKeyOUT); + long stopTime = clock.getTime(); + + LOG.info("Got the information about the specified masterKey = {} according to keyId = {}.", + routerMasterKey, paramKeyId); + + FederationStateStoreClientMetrics.succeededStateStoreCall(stopTime - startTime); + + // Return query result. + return RouterMasterKeyResponse.newInstance(routerMasterKey); + + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + FederationStateStoreUtils.logAndThrowRetriableException(e, LOG, + "Unable to obtain the masterKey information according to %s.", + String.valueOf(paramKeyId)); + } + + // Throw exception information + throw new YarnException( + "Unable to obtain the masterKey information according to " + paramKeyId); } + /** + * SQLFederationStateStore Supports Store RMDelegationTokenIdentifier. + * + * Defined the sp_addDelegationToken procedure. + * This procedure requires 4 input parameters, 1 output parameters. + * Input parameters: + * 1. IN sequenceNum_IN int + * 2. IN tokenIdent_IN varchar(1024) + * 3. IN token_IN varchar(1024) + * 4. IN renewDate_IN bigint + * Output parameters: + * 5. OUT rowCount_OUT int + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return routerRMTokenResponse, the response contains the RouterStoreToken. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterRMTokenResponse storeNewToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2. store data in database. + try { + long duration = addOrUpdateToken(request, true); + FederationStateStoreClientMetrics.succeededStateStoreCall(duration); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + throw new YarnException(e); + } + + // Step3. Query Data from the database and return the result. + return getTokenByRouterStoreToken(request); } + /** + * SQLFederationStateStore Supports Update RMDelegationTokenIdentifier. + * + * Defined the sp_updateDelegationToken procedure. + * This procedure requires 4 input parameters, 1 output parameters. + * Input parameters: + * 1. IN sequenceNum_IN int + * 2. IN tokenIdent_IN varchar(1024) + * 3. IN token_IN varchar(1024) + * 4. IN renewDate_IN bigint + * Output parameters: + * 5. OUT rowCount_OUT int + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return routerRMTokenResponse, the response contains the RouterStoreToken. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterRMTokenResponse updateStoredToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2. update data in database. + try { + long duration = addOrUpdateToken(request, false); + FederationStateStoreClientMetrics.succeededStateStoreCall(duration); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + throw new YarnException(e); + } + + // Step3. Query Data from the database and return the result. + return getTokenByRouterStoreToken(request); } + /** + * Add Or Update RMDelegationTokenIdentifier. + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @param isAdd true, addData; false, updateData. + * @return method operation time. + * @throws IOException An IO Error occurred. + * @throws SQLException An SQL Error occurred. + * @throws YarnException if the call to the state store is unsuccessful. + */ + private long addOrUpdateToken(RouterRMTokenRequest request, boolean isAdd) + throws IOException, SQLException, YarnException { + + // Parse parameters and get KeyId. + RouterStoreToken routerStoreToken = request.getRouterStoreToken(); + YARNDelegationTokenIdentifier identifier = routerStoreToken.getTokenIdentifier(); + String tokenIdentifier = FederationStateStoreUtils.encodeWritable(identifier); + String tokenInfo = routerStoreToken.getTokenInfo(); + long renewDate = routerStoreToken.getRenewDate(); + int sequenceNum = identifier.getSequenceNumber(); + + FederationQueryRunner runner = new FederationQueryRunner(); + FederationSQLOutParameter rowCountOUT = + new FederationSQLOutParameter<>("rowCount_OUT", INTEGER, Integer.class); + + // Execute the query + long startTime = clock.getTime(); + String procedure = isAdd ? CALL_SP_ADD_DELEGATIONTOKEN : CALL_SP_UPDATE_DELEGATIONTOKEN; + Integer rowCount = runner.execute(conn, procedure, new RowCountHandler("rowCount_OUT"), + sequenceNum, tokenIdentifier, tokenInfo, renewDate, rowCountOUT); + long stopTime = clock.getTime(); + + // Get rowCount + // In the process of updating the code, rowCount may be 0 or 1; + // if rowCount=1, it is as expected, indicating that we have updated the Token correctly; + // if rowCount=0, it is not as expected, + // indicating that we have not updated the Token correctly. + if (rowCount != 1) { + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "Wrong behavior during the insertion of delegationToken, tokenId = %s. " + + "Please check the records of the database.", String.valueOf(sequenceNum)); + } + + // return execution time + return (stopTime - startTime); + } + + /** + * SQLFederationStateStore Supports Remove RMDelegationTokenIdentifier. + * + * Defined the sp_deleteDelegationToken procedure. + * This procedure requires 1 input parameters, 1 output parameters. + * Input parameters: + * 1. IN sequenceNum_IN bigint + * Output parameters: + * 2. OUT rowCount_OUT int + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return routerRMTokenResponse, the response contains the RouterStoreToken. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterRMTokenResponse removeStoredToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2: Parse parameters and get KeyId. + RouterStoreToken routerStoreToken = request.getRouterStoreToken(); + YARNDelegationTokenIdentifier identifier = routerStoreToken.getTokenIdentifier(); + int sequenceNum = identifier.getSequenceNumber(); + + try { + + FederationSQLOutParameter rowCountOUT = + new FederationSQLOutParameter<>("rowCount_OUT", INTEGER, Integer.class); + + // Execute the query + long startTime = clock.getTime(); + Integer rowCount = getRowCountByProcedureSQL(CALL_SP_DELETE_DELEGATIONTOKEN, + sequenceNum, rowCountOUT); + long stopTime = clock.getTime(); + + // if it is equal to 0 it means the call + // did not delete the reservation from FederationStateStore + if (rowCount == 0) { + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "TokenId %s does not exist", String.valueOf(sequenceNum)); + } else if (rowCount != 1) { + // if it is different from 1 it means the call + // had a wrong behavior. Maybe the database is not set correctly. + FederationStateStoreUtils.logAndThrowStoreException(LOG, + "Wrong behavior during deleting the delegationToken %s. " + + "The database is expected to delete 1 record, " + + "but the number of deleted records returned by the database is greater than 1, " + + "indicating that a duplicate tokenId occurred during the deletion process.", + String.valueOf(sequenceNum)); + } + + LOG.info("Delete from the StateStore the delegationToken, tokenId = {}.", sequenceNum); + FederationStateStoreClientMetrics.succeededStateStoreCall(stopTime - startTime); + return RouterRMTokenResponse.newInstance(routerStoreToken); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + FederationStateStoreUtils.logAndThrowRetriableException(e, LOG, + "Unable to delete the delegationToken, tokenId = %s.", sequenceNum); + } + throw new YarnException("Unable to delete the delegationToken, tokenId = " + sequenceNum); } + /** + * The Router Supports GetTokenByRouterStoreToken. + * + * @param request The request contains RouterRMToken (RMDelegationTokenIdentifier and renewDate) + * @return RouterRMTokenResponse. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterRMTokenResponse getTokenByRouterStoreToken(RouterRMTokenRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + // Step1: Verify parameters to ensure that key fields are not empty. + FederationRouterRMTokenInputValidator.validate(request); + + // Step2: Parse parameters and get KeyId. + RouterStoreToken routerStoreToken = request.getRouterStoreToken(); + YARNDelegationTokenIdentifier identifier = routerStoreToken.getTokenIdentifier(); + int sequenceNum = identifier.getSequenceNumber(); + + try { + FederationQueryRunner runner = new FederationQueryRunner(); + FederationSQLOutParameter tokenIdentOUT = + new FederationSQLOutParameter<>("tokenIdent_OUT", VARCHAR, String.class); + FederationSQLOutParameter tokenOUT = + new FederationSQLOutParameter<>("token_OUT", VARCHAR, String.class); + FederationSQLOutParameter renewDateOUT = + new FederationSQLOutParameter<>("renewDate_OUT", BIGINT, Long.class); + + // Execute the query + long startTime = clock.getTime(); + RouterStoreToken resultToken = runner.execute(conn, CALL_SP_GET_DELEGATIONTOKEN, + new RouterStoreTokenHandler(), sequenceNum, tokenIdentOUT, tokenOUT, renewDateOUT); + long stopTime = clock.getTime(); + + FederationStateStoreClientMetrics.succeededStateStoreCall(stopTime - startTime); + return RouterRMTokenResponse.newInstance(resultToken); + } catch (SQLException e) { + FederationStateStoreClientMetrics.failedStateStoreCall(); + FederationStateStoreUtils.logAndThrowRetriableException(e, LOG, + "Unable to get the delegationToken, tokenId = %s.", String.valueOf(sequenceNum)); + } + + // Throw exception information + throw new YarnException("Unable to get the delegationToken, tokenId = " + sequenceNum); } + /** + * Call Procedure to get RowCount. + * + * @param procedure procedureSQL. + * @param params procedure params. + * @return RowCount. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + private int getRowCountByProcedureSQL(String procedure, Object... params) throws SQLException { + FederationQueryRunner runner = new FederationQueryRunner(); + // Execute the query + Integer rowCount = runner.execute(conn, procedure, + new RowCountHandler("rowCount_OUT"), params); + return rowCount; + } + + /** + * Increment DelegationToken SeqNum. + * + * @return delegationTokenSeqNum. + */ @Override public int incrementDelegationTokenSeqNum() { - return 0; + return querySequenceTable(YARN_ROUTER_SEQUENCE_NUM, true); } + /** + * Get DelegationToken SeqNum. + * + * @return delegationTokenSeqNum. + */ @Override public int getDelegationTokenSeqNum() { - return 0; + return querySequenceTable(YARN_ROUTER_SEQUENCE_NUM, false); } @Override public void setDelegationTokenSeqNum(int seqNum) { - return; + Connection connection = null; + try { + connection = getConnection(false); + FederationQueryRunner runner = new FederationQueryRunner(); + runner.updateSequenceTable(connection, YARN_ROUTER_SEQUENCE_NUM, seqNum); + } catch (Exception e) { + throw new RuntimeException("Could not update sequence table!!", e); + } finally { + // Return to the pool the CallableStatement + try { + FederationStateStoreUtils.returnToPool(LOG, null, connection); + } catch (YarnException e) { + LOG.error("close connection error.", e); + } + } } + /** + * Get Current KeyId. + * + * @return currentKeyId. + */ @Override public int getCurrentKeyId() { - return 0; + return querySequenceTable(YARN_ROUTER_CURRENT_KEY_ID, false); } + /** + * The Router Supports incrementCurrentKeyId. + * + * @return CurrentKeyId. + */ @Override public int incrementCurrentKeyId() { - return 0; + return querySequenceTable(YARN_ROUTER_CURRENT_KEY_ID, true); + } + + private int querySequenceTable(String sequenceName, boolean isUpdate){ + Connection connection = null; + try { + connection = getConnection(false); + FederationQueryRunner runner = new FederationQueryRunner(); + return runner.selectOrUpdateSequenceTable(connection, sequenceName, isUpdate); + } catch (Exception e) { + throw new RuntimeException("Could not query sequence table!!", e); + } finally { + // Return to the pool the CallableStatement + try { + FederationStateStoreUtils.returnToPool(LOG, null, connection); + } catch (YarnException e) { + LOG.error("close connection error.", e); + } + } } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterRMDTSecretManagerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterRMDTSecretManagerState.java index 85a8002c91c9c..62a89f419dd6a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterRMDTSecretManagerState.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterRMDTSecretManagerState.java @@ -28,13 +28,13 @@ public class RouterRMDTSecretManagerState { // DTIdentifier -> renewDate - private Map delegationTokenState = new HashMap<>(); + private Map delegationTokenState = new HashMap<>(); private Set masterKeyState = new HashSet<>(); private int dtSequenceNumber = 0; - public Map getTokenState() { + public Map getTokenState() { return delegationTokenState; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java index 29f86903f9144..a20297399b7f5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java @@ -18,7 +18,9 @@ package org.apache.hadoop.yarn.server.federation.store.records; import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.util.Records; @@ -41,14 +43,25 @@ public static RouterStoreToken newInstance(YARNDelegationTokenIdentifier identif @Private @Unstable - public abstract YARNDelegationTokenIdentifier getTokenIdentifier() throws IOException; + public static RouterStoreToken newInstance(YARNDelegationTokenIdentifier identifier, + Long renewdate, String tokenInfo) { + RouterStoreToken storeToken = Records.newRecord(RouterStoreToken.class); + storeToken.setIdentifier(identifier); + storeToken.setRenewDate(renewdate); + storeToken.setTokenInfo(tokenInfo); + return storeToken; + } @Private @Unstable - public abstract void setIdentifier(YARNDelegationTokenIdentifier identifier); + public abstract YARNDelegationTokenIdentifier getTokenIdentifier() throws IOException; @Private @Unstable + public abstract void setIdentifier(YARNDelegationTokenIdentifier identifier); + + @Public + @Stable public abstract Long getRenewDate(); @Private @@ -62,4 +75,12 @@ public static RouterStoreToken newInstance(YARNDelegationTokenIdentifier identif @Private @Unstable public abstract void readFields(DataInput in) throws IOException; + + @Public + @Stable + public abstract String getTokenInfo(); + + @Private + @Unstable + public abstract void setTokenInfo(String tokenInfo); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java index df6030a3f0d68..a89cf06f0b110 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/RouterStoreTokenPBImpl.java @@ -47,6 +47,7 @@ public class RouterStoreTokenPBImpl extends RouterStoreToken { private YARNDelegationTokenIdentifier rMDelegationTokenIdentifier = null; private Long renewDate; + private String tokenInfo; public RouterStoreTokenPBImpl() { builder = RouterStoreTokenProto.newBuilder(); @@ -84,6 +85,10 @@ private void mergeLocalToBuilder() { if (this.renewDate != null) { builder.setRenewDate(this.renewDate); } + + if (this.tokenInfo != null) { + builder.setTokenInfo(this.tokenInfo); + } } private void maybeInitBuilder() { @@ -164,6 +169,29 @@ public void setRenewDate(Long renewDate) { this.renewDate = renewDate; this.builder.setRenewDate(renewDate); } + @Override + public String getTokenInfo() { + RouterStoreTokenProtoOrBuilder p = viaProto ? proto : builder; + if (this.tokenInfo != null) { + return this.tokenInfo; + } + if (!p.hasTokenInfo()) { + return null; + } + this.tokenInfo = p.getTokenInfo(); + return this.tokenInfo; + } + + @Override + public void setTokenInfo(String tokenInfo) { + maybeInitBuilder(); + if (tokenInfo == null) { + builder.clearTokenInfo(); + return; + } + this.tokenInfo = tokenInfo; + this.builder.setTokenInfo(tokenInfo); + } private YARNDelegationTokenIdentifierProto convertToProtoFormat( YARNDelegationTokenIdentifier delegationTokenIdentifier) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/DatabaseProduct.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/DatabaseProduct.java new file mode 100644 index 0000000000000..830ecb65e01e1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/DatabaseProduct.java @@ -0,0 +1,125 @@ +/** + * 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.federation.store.sql; + +import org.apache.hadoop.classification.InterfaceAudience.Private; + +import java.sql.Connection; +import java.sql.SQLException; + +@Private +public final class DatabaseProduct { + + public enum DbType {MYSQL, SQLSERVER, POSTGRES, UNDEFINED, HSQLDB} + + private static final String SQL_SERVER_NAME = "sqlserver"; + private static final String MYSQL_NAME = "mysql"; + private static final String MARIADB_NAME = "mariadb"; + private static final String HSQLDB_NAME = "hsqldatabase"; + + private DatabaseProduct() { + } + + public static DbType getDbType(Connection conn) throws SQLException { + if (conn == null) { + return DbType.UNDEFINED; + } + String productName = getProductName(conn); + return getDbType(productName); + } + + /** + * We get DBType based on ProductName. + * + * @param productName productName. + * @return DbType. + */ + private static DbType getDbType(String productName) { + DbType dbt; + productName = productName.replaceAll("\\s+", "").toLowerCase(); + if (productName.contains(SQL_SERVER_NAME)) { + dbt = DbType.SQLSERVER; + } else if (productName.contains(MYSQL_NAME) || productName.contains(MARIADB_NAME)) { + dbt = DbType.MYSQL; + } else if (productName.contains(HSQLDB_NAME)) { + dbt = DbType.HSQLDB; + } else { + dbt = DbType.UNDEFINED; + } + return dbt; + } + + /** + * We get ProductName based on metadata in SQL Connection. + * + * @param conn SQL Connection + * @return DB ProductName (Like MySQL SQLSERVER etc.) + */ + private static String getProductName(Connection conn) throws SQLException { + return conn.getMetaData().getDatabaseProductName(); + } + + /** + * We add for update to SQL according to different database types. + * This statement can ensure that a row of records in the database is only updated by one thread. + * + * @param dbType type of database. + * @param selectStatement querySQL. + * @return SQL after adding for update. + * @throws SQLException SQL exception. + */ + public static String addForUpdateClause(DbType dbType, String selectStatement) + throws SQLException { + switch (dbType) { + case MYSQL: + case HSQLDB: + return selectStatement + " for update"; + case SQLSERVER: + String modifier = " with (updlock)"; + int wherePos = selectStatement.toUpperCase().indexOf(" WHERE "); + if (wherePos < 0) { + return selectStatement + modifier; + } + return selectStatement.substring(0, wherePos) + modifier + + selectStatement.substring(wherePos, selectStatement.length()); + default: + String msg = "Unrecognized database product name <" + dbType + ">"; + throw new SQLException(msg); + } + } + + public static boolean isDuplicateKeyError(DbType dbType, SQLException ex) { + switch (dbType) { + case MYSQL: + if((ex.getErrorCode() == 1022 || ex.getErrorCode() == 1062 || ex.getErrorCode() == 1586) && + "23000".equals(ex.getSQLState())) { + return true; + } + break; + case SQLSERVER: + if ((ex.getErrorCode() == 2627 || ex.getErrorCode() == 2601) && "23000".equals(ex.getSQLState())) { + return true; + } + break; + default: + return false; + } + return false; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationQueryRunner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationQueryRunner.java new file mode 100644 index 0000000000000..a0de0ee39f4bb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationQueryRunner.java @@ -0,0 +1,303 @@ +/** + * 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.federation.store.sql; + +import org.apache.hadoop.classification.VisibleForTesting; + +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.CallableStatement; +import java.sql.ResultSet; +import java.util.Arrays; + +import org.apache.hadoop.yarn.server.federation.store.sql.DatabaseProduct.DbType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.yarn.server.federation.store.sql.DatabaseProduct.isDuplicateKeyError; + +/** + * QueryRunner is used to execute stored procedure SQL and parse the returned results. + */ +public class FederationQueryRunner { + + public final static String YARN_ROUTER_SEQUENCE_NUM = "YARN_ROUTER_SEQUENCE_NUM"; + + public final static String YARN_ROUTER_CURRENT_KEY_ID = "YARN_ROUTER_CURRENT_KEY_ID"; + + public final static String QUERY_SEQUENCE_TABLE_SQL = + "SELECT nextVal FROM sequenceTable WHERE sequenceName = %s"; + + public final static String INSERT_SEQUENCE_TABLE_SQL = "" + + "INSERT INTO sequenceTable(sequenceName, nextVal) VALUES(%s, %d)"; + + public final static String UPDATE_SEQUENCE_TABLE_SQL = "" + + "UPDATE sequenceTable SET nextVal = %d WHERE sequenceName = %s"; + + public static final Logger LOG = LoggerFactory.getLogger(FederationQueryRunner.class); + + /** + * Execute Stored Procedure SQL. + * + * @param conn Database Connection. + * @param procedure Stored Procedure SQL. + * @param rsh Result Set handler. + * @param params List of stored procedure parameters. + * @param Generic T. + * @return Stored Procedure Result Set. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + public T execute(Connection conn, String procedure, ResultSetHandler rsh, Object... params) + throws SQLException { + if (conn == null) { + throw new SQLException("Null connection"); + } + + if (procedure == null) { + throw new SQLException("Null Procedure SQL statement"); + } + + if (rsh == null) { + throw new SQLException("Null ResultSetHandler"); + } + + CallableStatement stmt = null; + T results = null; + + try { + stmt = this.getCallableStatement(conn, procedure); + this.fillStatement(stmt, params); + stmt.executeUpdate(); + this.retrieveOutParameters(stmt, params); + results = rsh.handle(params); + } catch (SQLException e) { + this.rethrow(e, procedure, params); + } finally { + close(stmt); + } + return results; + } + + /** + * Get CallableStatement from Conn. + * + * @param conn Database Connection. + * @param procedure Stored Procedure SQL. + * @return CallableStatement. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + @VisibleForTesting + protected CallableStatement getCallableStatement(Connection conn, String procedure) + throws SQLException { + return conn.prepareCall(procedure); + } + + /** + * Set Statement parameters. + * + * @param stmt CallableStatement. + * @param params Stored procedure parameters. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + public void fillStatement(CallableStatement stmt, Object... params) + throws SQLException { + for (int i = 0; i < params.length; i++) { + if (params[i] != null) { + if (stmt != null) { + if (params[i] instanceof FederationSQLOutParameter) { + FederationSQLOutParameter sqlOutParameter = (FederationSQLOutParameter) params[i]; + sqlOutParameter.register(stmt, i + 1); + } else { + stmt.setObject(i + 1, params[i]); + } + } + } + } + } + + /** + * Close Statement. + * + * @param stmt CallableStatement. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + public void close(Statement stmt) throws SQLException { + if (stmt != null) { + stmt.close(); + stmt = null; + } + } + + /** + * Retrieve execution result from CallableStatement. + * + * @param stmt CallableStatement. + * @param params Stored procedure parameters. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + private void retrieveOutParameters(CallableStatement stmt, Object[] params) throws SQLException { + if (params != null && stmt != null) { + for (int i = 0; i < params.length; i++) { + if (params[i] instanceof FederationSQLOutParameter) { + FederationSQLOutParameter sqlOutParameter = (FederationSQLOutParameter) params[i]; + sqlOutParameter.setValue(stmt, i + 1); + } + } + } + } + + /** + * Re-throw SQL exception. + * + * @param cause SQLException. + * @param sql Stored Procedure SQL. + * @param params Stored procedure parameters. + * @throws SQLException An exception occurred when calling a stored procedure. + */ + protected void rethrow(SQLException cause, String sql, Object... params) + throws SQLException { + + String causeMessage = cause.getMessage(); + if (causeMessage == null) { + causeMessage = ""; + } + + StringBuffer msg = new StringBuffer(causeMessage); + msg.append(" Query: "); + msg.append(sql); + msg.append(" Parameters: "); + + if (params == null) { + msg.append("[]"); + } else { + msg.append(Arrays.deepToString(params)); + } + + SQLException e = new SQLException(msg.toString(), cause.getSQLState(), cause.getErrorCode()); + e.setNextException(cause); + throw e; + } + + /** + * + * @param connection + * @param sequenceName + */ + public int selectOrUpdateSequenceTable(Connection connection, String sequenceName, + boolean isUpdate) throws SQLException { + + int maxSequenceValue = 0; + boolean insertDone = false; + boolean committed = false; + Statement statement = null; + + try { + + // Step1. Query SequenceValue. + while (maxSequenceValue == 0) { + // Query SQL. + String sql = String.format(QUERY_SEQUENCE_TABLE_SQL, quoteString(sequenceName)); + DbType dbType = DatabaseProduct.getDbType(connection); + String forUpdateSQL = DatabaseProduct.addForUpdateClause(dbType, sql); + statement = connection.createStatement(); + ResultSet rs = statement.executeQuery(forUpdateSQL); + if (rs.next()) { + maxSequenceValue = rs.getInt("nextVal"); + } else if (insertDone) { + throw new SQLException("Invalid state of SEQUENCE_TABLE for " + sequenceName); + } else { + insertDone = true; + close(statement); + statement = connection.createStatement(); + String insertSQL = String.format(INSERT_SEQUENCE_TABLE_SQL, quoteString(sequenceName), 1); + try { + statement.executeUpdate(insertSQL); + } catch (SQLException e) { + // If the record is already inserted by some other thread continue to select. + if (isDuplicateKeyError(dbType, e)) { + continue; + } + LOG.error("Unable to insert into SEQUENCE_TABLE for {}.", e); + throw e; + } finally { + close(statement); + } + } + } + + // Step2. Increase SequenceValue. + if (isUpdate) { + int nextSequenceValue = maxSequenceValue + 1; + close(statement); + statement = connection.createStatement(); + String updateSQL = + String.format(UPDATE_SEQUENCE_TABLE_SQL, nextSequenceValue, quoteString(sequenceName)); + statement.executeUpdate(updateSQL); + maxSequenceValue = nextSequenceValue; + } + + connection.commit(); + committed = true; + return maxSequenceValue; + } catch (Exception e) { + throw new SQLException("Unable to selectOrUpdateSequenceTable due to: " + e.getMessage()); + } finally { + if (!committed) { + rollbackDBConn(connection); + } + close(statement); + } + } + + public void updateSequenceTable(Connection connection, String sequenceName, int sequenceValue) + throws SQLException { + String updateSQL = + String.format(UPDATE_SEQUENCE_TABLE_SQL, sequenceValue, quoteString(sequenceName)); + boolean committed = false; + Statement statement = null; + try { + statement = connection.createStatement(); + statement.executeUpdate(updateSQL); + connection.commit(); + committed = true; + } catch (SQLException e) { + throw new SQLException("Unable to updateSequenceTable due to: " + e.getMessage()); + } finally { + if (!committed) { + rollbackDBConn(connection); + } + close(statement); + } + } + + static void rollbackDBConn(Connection dbConn) { + try { + if (dbConn != null && !dbConn.isClosed()) { + dbConn.rollback(); + } + } catch (SQLException e) { + LOG.warn("Failed to rollback db connection ", e); + } + } + + static String quoteString(String input) { + return "'" + input + "'"; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationSQLOutParameter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationSQLOutParameter.java new file mode 100644 index 0000000000000..890e3e1e851d8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/FederationSQLOutParameter.java @@ -0,0 +1,91 @@ +/** + * 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.federation.store.sql; + +import java.sql.CallableStatement; +import java.sql.SQLException; + +/** + * SQLOutParameter is used to set the output parameters of the stored procedure. + * @param Generic T. + */ +public class FederationSQLOutParameter { + private final int sqlType; + private final Class javaType; + private T value = null; + private String paramName; + + public FederationSQLOutParameter(String paramName, int sqlType, Class javaType) { + this.paramName = paramName; + this.sqlType = sqlType; + this.javaType = javaType; + } + + public FederationSQLOutParameter(int sqlType, Class javaType, T value) { + this.sqlType = sqlType; + this.javaType = javaType; + this.value = value; + } + + public int getSqlType() { + return sqlType; + } + + public Class getJavaType() { + return javaType; + } + + public T getValue() { + return value; + } + + public void setValue(T value) { + this.value = value; + } + + public String getParamName() { + return paramName; + } + + public void setParamName(String paramName) { + this.paramName = paramName; + } + + void setValue(CallableStatement stmt, int index) throws SQLException { + Object object = stmt.getObject(index); + value = javaType.cast(object); + } + + void register(CallableStatement stmt, int index) throws SQLException { + stmt.registerOutParameter(index, sqlType); + if (value != null) { + stmt.setObject(index, value); + } + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("OutParameter: [") + .append("SqlType: ").append(sqlType).append(", ") + .append("JavaType: ").append(javaType).append(", ") + .append("Value: ").append(value) + .append("]"); + return sb.toString(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/ResultSetHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/ResultSetHandler.java new file mode 100644 index 0000000000000..1e793d0ecbdba --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/ResultSetHandler.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.yarn.server.federation.store.sql; + +import java.sql.SQLException; + +/** + * Result Set Handler. + * + * @param Generic T. + */ +public interface ResultSetHandler { + T handle(Object... params) throws SQLException; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterMasterKeyHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterMasterKeyHandler.java new file mode 100644 index 0000000000000..d02855be65fba --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterMasterKeyHandler.java @@ -0,0 +1,68 @@ +/** + * 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.federation.store.sql; + +import org.apache.hadoop.security.token.delegation.DelegationKey; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils; +import org.apache.hadoop.yarn.util.Records; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.sql.SQLException; + +/** + * RouterMasterKey Handler. + * Used to parse the result information of the output parameter into the RouterMasterKey type. + */ +public class RouterMasterKeyHandler implements ResultSetHandler { + + private final static String MASTERKEY_OUT = "masterKey_OUT"; + + @Override + public RouterMasterKey handle(Object... params) throws SQLException { + RouterMasterKey routerMasterKey = Records.newRecord(RouterMasterKey.class); + for (Object param : params) { + if (param != null && param instanceof FederationSQLOutParameter) { + FederationSQLOutParameter parameter = (FederationSQLOutParameter) param; + String paramName = parameter.getParamName(); + Object parmaValue = parameter.getValue(); + if (StringUtils.equalsIgnoreCase(paramName, MASTERKEY_OUT)) { + DelegationKey key = getDelegationKey(parmaValue); + routerMasterKey.setKeyId(key.getKeyId()); + routerMasterKey.setKeyBytes(ByteBuffer.wrap(key.getEncodedKey())); + routerMasterKey.setExpiryDate(key.getExpiryDate()); + } + } + } + return routerMasterKey; + } + + private DelegationKey getDelegationKey(Object paramMasterKey) throws SQLException { + try { + DelegationKey key = new DelegationKey(); + String masterKey = String.valueOf(paramMasterKey); + FederationStateStoreUtils.decodeWritable(key, masterKey); + return key; + } catch (IOException e) { + throw new SQLException(e); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterStoreTokenHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterStoreTokenHandler.java new file mode 100644 index 0000000000000..37a0c5c244c21 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RouterStoreTokenHandler.java @@ -0,0 +1,83 @@ +/** + * 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.federation.store.sql; + +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; +import org.apache.hadoop.yarn.util.Records; + +import java.io.IOException; +import java.sql.SQLException; + +import static org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils.decodeWritable; + +/** + * RouterStoreToken Handler. + * Used to parse the result information of the output parameter into the RouterStoreToken type. + */ +public class RouterStoreTokenHandler implements ResultSetHandler { + + private final static String TOKENIDENT_OUT = "tokenIdent_OUT"; + private final static String TOKEN_OUT = "token_OUT"; + private final static String RENEWDATE_OUT = "renewDate_OUT"; + + @Override + public RouterStoreToken handle(Object... params) throws SQLException { + RouterStoreToken storeToken = Records.newRecord(RouterStoreToken.class); + for (Object param : params) { + if (param != null && param instanceof FederationSQLOutParameter) { + FederationSQLOutParameter parameter = (FederationSQLOutParameter) param; + String paramName = parameter.getParamName(); + Object parmaValue = parameter.getValue(); + if (StringUtils.equalsIgnoreCase(paramName, TOKENIDENT_OUT)) { + YARNDelegationTokenIdentifier identifier = getYARNDelegationTokenIdentifier(parmaValue); + storeToken.setIdentifier(identifier); + } else if (StringUtils.equalsIgnoreCase(paramName, TOKEN_OUT)) { + String tokenInfo = getTokenInfo(parmaValue); + storeToken.setTokenInfo(tokenInfo); + } else if(StringUtils.equalsIgnoreCase(paramName, RENEWDATE_OUT)){ + Long renewDate = getRenewDate(parmaValue); + storeToken.setRenewDate(renewDate); + } + } + } + return storeToken; + } + + private YARNDelegationTokenIdentifier getYARNDelegationTokenIdentifier(Object tokenIdent) + throws SQLException { + try { + YARNDelegationTokenIdentifier resultIdentifier = + Records.newRecord(YARNDelegationTokenIdentifier.class); + decodeWritable(resultIdentifier, String.valueOf(tokenIdent)); + return resultIdentifier; + } catch (IOException e) { + throw new SQLException(e); + } + } + + private String getTokenInfo(Object tokenInfo) { + return String.valueOf(tokenInfo); + } + + private Long getRenewDate(Object renewDate) { + return Long.parseLong(String.valueOf(renewDate)); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RowCountHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RowCountHandler.java new file mode 100644 index 0000000000000..4af0b52e0e9ea --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/RowCountHandler.java @@ -0,0 +1,56 @@ +/** + * 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.federation.store.sql; + +import org.apache.hadoop.util.StringUtils; + +import java.sql.SQLException; + +/** + * RowCount Handler. + * Used to parse out the rowCount information of the output parameter. + */ +public class RowCountHandler implements ResultSetHandler { + + private String rowCountParamName; + + public RowCountHandler(String paramName) { + this.rowCountParamName = paramName; + } + + @Override + public Integer handle(Object... params) throws SQLException { + Integer result = 0; + for (Object param : params) { + if (param != null && param instanceof FederationSQLOutParameter) { + FederationSQLOutParameter parameter = (FederationSQLOutParameter) param; + String paramName = parameter.getParamName(); + Object parmaValue = parameter.getValue(); + if (StringUtils.equalsIgnoreCase(paramName, rowCountParamName)) { + result = getRowCount(parmaValue); + } + } + } + return result; + } + + private Integer getRowCount(Object rowCount) { + return Integer.parseInt(String.valueOf(rowCount)); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/package-info.java new file mode 100644 index 0000000000000..d6bca3fd23611 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/sql/package-info.java @@ -0,0 +1,17 @@ +/** + * 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.federation.store.sql; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java index 40fe1f36cfb96..b4c6bd890e9f5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationRouterRMTokenInputValidator.java @@ -15,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.yarn.server.federation.store.utils; import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyRequest; import org.apache.hadoop.yarn.server.federation.store.records.RouterRMTokenRequest; import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; -import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyRequest; -import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java index f14867a0e6563..aba8ddac2f81f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java @@ -18,16 +18,27 @@ package org.apache.hadoop.yarn.server.federation.store.utils; +import java.io.IOException; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.nio.ByteBuffer; import java.sql.CallableStatement; import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; +import java.util.Base64; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreException; import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException; import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreRetriableException; import org.apache.hadoop.yarn.server.federation.store.metrics.FederationStateStoreClientMetrics; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyRequest; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,7 +47,6 @@ /** * Common utility methods used by the store implementations. - * */ public final class FederationStateStoreUtils { @@ -329,4 +339,61 @@ public static boolean filterHomeSubCluster(SubClusterId filterSubCluster, return false; } + + /** + * Encode for Writable objects. + * This method will convert the writable object to a base64 string. + * + * @param key Writable Key. + * @return base64 string. + * @throws IOException raised on errors performing I/O. + */ + public static String encodeWritable(Writable key) throws IOException { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(bos); + key.write(dos); + dos.flush(); + return Base64.getUrlEncoder().encodeToString(bos.toByteArray()); + } + + /** + * Decode Base64 string to Writable object. + * + * @param w Writable Key. + * @param idStr base64 string. + * @throws IOException raised on errors performing I/O. + */ + public static void decodeWritable(Writable w, String idStr) throws IOException { + DataInputStream in = new DataInputStream( + new ByteArrayInputStream(Base64.getUrlDecoder().decode(idStr))); + w.readFields(in); + } + + /** + * Convert MasterKey to DelegationKey. + * + * Before using this function, + * please use FederationRouterRMTokenInputValidator to verify the request. + * By default, the request is not empty, and the internal object is not empty. + * + * @param request RouterMasterKeyRequest + * @return DelegationKey. + */ + public static DelegationKey convertMasterKeyToDelegationKey(RouterMasterKeyRequest request) { + RouterMasterKey masterKey = request.getRouterMasterKey(); + return convertMasterKeyToDelegationKey(masterKey); + } + + /** + * Convert MasterKey to DelegationKey. + * + * @param masterKey masterKey. + * @return DelegationKey. + */ + private static DelegationKey convertMasterKeyToDelegationKey(RouterMasterKey masterKey) { + ByteBuffer keyByteBuf = masterKey.getKeyBytes(); + byte[] keyBytes = new byte[keyByteBuf.remaining()]; + keyByteBuf.get(keyBytes); + return new DelegationKey(masterKey.getKeyId(), masterKey.getExpiryDate(), keyBytes); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java index e1ebce828922f..ca57da8e5345d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java @@ -809,6 +809,24 @@ public void storeNewToken(RMDelegationTokenIdentifier identifier, stateStore.storeNewToken(request); } + /** + * The Router Supports Store RMDelegationTokenIdentifier{@link RMDelegationTokenIdentifier}. + * + * @param identifier delegation tokens from the RM. + * @param renewDate renewDate. + * @param tokenInfo tokenInfo. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ + public void storeNewToken(RMDelegationTokenIdentifier identifier, + long renewDate, String tokenInfo) throws YarnException, IOException { + LOG.info("storing RMDelegation token with sequence number: {}.", + identifier.getSequenceNumber()); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); + RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); + stateStore.storeNewToken(request); + } + /** * The Router Supports Update RMDelegationTokenIdentifier{@link RMDelegationTokenIdentifier}. * @@ -826,6 +844,24 @@ public void updateStoredToken(RMDelegationTokenIdentifier identifier, stateStore.updateStoredToken(request); } + /** + * The Router Supports Update RMDelegationTokenIdentifier{@link RMDelegationTokenIdentifier}. + * + * @param identifier delegation tokens from the RM + * @param renewDate renewDate + * @param tokenInfo tokenInfo. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ + public void updateStoredToken(RMDelegationTokenIdentifier identifier, + long renewDate, String tokenInfo) throws YarnException, IOException { + LOG.info("updating RMDelegation token with sequence number: {}.", + identifier.getSequenceNumber()); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); + RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); + stateStore.updateStoredToken(request); + } + /** * The Router Supports Remove RMDelegationTokenIdentifier{@link RMDelegationTokenIdentifier}. * diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto index 0544a26e4c5a2..26fc77e01f2ad 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto @@ -234,6 +234,7 @@ message RouterMasterKeyResponseProto { message RouterStoreTokenProto { optional YARNDelegationTokenIdentifierProto token_identifier = 1; optional int64 renew_date = 2; + optional string token_info = 3; } message RouterRMTokenRequestProto { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java index 7fb1e327e855b..c93115ccfd381 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.sql.SQLException; import java.util.Calendar; import java.util.List; import java.util.Set; @@ -98,10 +99,10 @@ public abstract class FederationStateStoreBaseTest { protected abstract FederationStateStore createStateStore(); protected abstract void checkRouterMasterKey(DelegationKey delegationKey, - RouterMasterKey routerMasterKey) throws YarnException, IOException; + RouterMasterKey routerMasterKey) throws YarnException, IOException, SQLException; protected abstract void checkRouterStoreToken(RMDelegationTokenIdentifier identifier, - RouterStoreToken token) throws YarnException, IOException; + RouterStoreToken token) throws YarnException, IOException, SQLException; private Configuration conf; @@ -937,16 +938,17 @@ public void testRemoveStoredMasterKey() throws YarnException, IOException { } @Test - public void testStoreNewToken() throws IOException, YarnException { + public void testStoreNewToken() throws IOException, YarnException, SQLException { // prepare parameters RMDelegationTokenIdentifier identifier = new RMDelegationTokenIdentifier( new Text("owner1"), new Text("renewer1"), new Text("realuser1")); int sequenceNumber = 1; identifier.setSequenceNumber(sequenceNumber); Long renewDate = Time.now(); + String tokenInfo = "tokenInfo"; // store new rm-token - RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); RouterRMTokenResponse routerRMTokenResponse = stateStore.storeNewToken(request); @@ -957,33 +959,33 @@ public void testStoreNewToken() throws IOException, YarnException { Assert.assertNotNull(storeTokenResp); Assert.assertEquals(storeToken.getRenewDate(), storeTokenResp.getRenewDate()); Assert.assertEquals(storeToken.getTokenIdentifier(), storeTokenResp.getTokenIdentifier()); + Assert.assertEquals(storeToken.getTokenInfo(), storeTokenResp.getTokenInfo()); - checkRouterStoreToken(identifier, storeToken); checkRouterStoreToken(identifier, storeTokenResp); } @Test - public void testUpdateStoredToken() throws IOException, YarnException { + public void testUpdateStoredToken() throws IOException, YarnException, SQLException { // prepare saveToken parameters RMDelegationTokenIdentifier identifier = new RMDelegationTokenIdentifier( new Text("owner2"), new Text("renewer2"), new Text("realuser2")); int sequenceNumber = 2; + String tokenInfo = "tokenInfo"; identifier.setSequenceNumber(sequenceNumber); Long renewDate = Time.now(); // store new rm-token - RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); RouterRMTokenResponse routerRMTokenResponse = stateStore.storeNewToken(request); Assert.assertNotNull(routerRMTokenResponse); // prepare updateToken parameters Long renewDate2 = Time.now(); - int sequenceNumber2 = 3; - identifier.setSequenceNumber(sequenceNumber2); + String tokenInfo2 = "tokenInfo2"; // update rm-token - RouterStoreToken updateToken = RouterStoreToken.newInstance(identifier, renewDate2); + RouterStoreToken updateToken = RouterStoreToken.newInstance(identifier, renewDate2, tokenInfo2); RouterRMTokenRequest updateTokenRequest = RouterRMTokenRequest.newInstance(updateToken); RouterRMTokenResponse updateTokenResponse = stateStore.updateStoredToken(updateTokenRequest); @@ -992,6 +994,7 @@ public void testUpdateStoredToken() throws IOException, YarnException { Assert.assertNotNull(updateTokenResp); Assert.assertEquals(updateToken.getRenewDate(), updateTokenResp.getRenewDate()); Assert.assertEquals(updateToken.getTokenIdentifier(), updateTokenResp.getTokenIdentifier()); + Assert.assertEquals(updateToken.getTokenInfo(), updateTokenResp.getTokenInfo()); checkRouterStoreToken(identifier, updateTokenResp); } @@ -1004,9 +1007,10 @@ public void testRemoveStoredToken() throws IOException, YarnException { int sequenceNumber = 3; identifier.setSequenceNumber(sequenceNumber); Long renewDate = Time.now(); + String tokenInfo = "tokenInfo"; // store new rm-token - RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); RouterRMTokenResponse routerRMTokenResponse = stateStore.storeNewToken(request); Assert.assertNotNull(routerRMTokenResponse); @@ -1021,16 +1025,17 @@ public void testRemoveStoredToken() throws IOException, YarnException { } @Test - public void testGetTokenByRouterStoreToken() throws IOException, YarnException { + public void testGetTokenByRouterStoreToken() throws IOException, YarnException, SQLException { // prepare saveToken parameters RMDelegationTokenIdentifier identifier = new RMDelegationTokenIdentifier( new Text("owner4"), new Text("renewer4"), new Text("realuser4")); int sequenceNumber = 4; identifier.setSequenceNumber(sequenceNumber); Long renewDate = Time.now(); + String tokenInfo = "tokenInfo"; // store new rm-token - RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate); + RouterStoreToken storeToken = RouterStoreToken.newInstance(identifier, renewDate, tokenInfo); RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(storeToken); RouterRMTokenResponse routerRMTokenResponse = stateStore.storeNewToken(request); Assert.assertNotNull(routerRMTokenResponse); @@ -1041,7 +1046,7 @@ public void testGetTokenByRouterStoreToken() throws IOException, YarnException { RouterStoreToken getStoreTokenResp = getRouterRMTokenResp.getRouterStoreToken(); Assert.assertNotNull(getStoreTokenResp); Assert.assertEquals(getStoreTokenResp.getRenewDate(), storeToken.getRenewDate()); - Assert.assertEquals(getStoreTokenResp.getTokenIdentifier(), storeToken.getTokenIdentifier()); + Assert.assertEquals(storeToken.getTokenInfo(), getStoreTokenResp.getTokenInfo()); checkRouterStoreToken(identifier, getStoreTokenResp); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/HSQLDBFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/HSQLDBFederationStateStore.java index b3bb0764dfa4f..73b65feb48e22 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/HSQLDBFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/HSQLDBFederationStateStore.java @@ -75,6 +75,26 @@ public class HSQLDBFederationStateStore extends SQLFederationStateStore { + " homeSubCluster varchar(256) NOT NULL," + " CONSTRAINT pk_reservationId PRIMARY KEY (reservationId))"; + private static final String TABLE_MASTERKEYS = + " CREATE TABLE masterKeys (" + + " keyId bigint NOT NULL," + + " masterKey varchar(1024) NOT NULL," + + " CONSTRAINT pk_keyId PRIMARY KEY (keyId))"; + + private static final String TABLE_DELEGATIONTOKENS = + " CREATE TABLE delegationTokens (" + + " sequenceNum bigint NOT NULL," + + " tokenIdent varchar(1024) NOT NULL," + + " token varchar(1024) NOT NULL," + + " renewDate bigint NOT NULL," + + " CONSTRAINT pk_sequenceNum PRIMARY KEY (sequenceNum))"; + + private static final String TABLE_SEQUENCETABLE = + " CREATE TABLE sequenceTable (" + + " sequenceName varchar(255) NOT NULL," + + " nextVal bigint NOT NULL," + + " CONSTRAINT pk_sequenceName PRIMARY KEY (sequenceName))"; + private static final String SP_REGISTERSUBCLUSTER = "CREATE PROCEDURE sp_registerSubCluster(" + " IN subClusterId_IN varchar(256)," @@ -318,6 +338,99 @@ public class HSQLDBFederationStateStore extends SQLFederationStateStore { + " WHERE reservationId = reservationId_IN;" + " SET rowCount_OUT = 2; END"; + protected static final String SP_DROP_ADDMASTERKEY = "DROP PROCEDURE sp_addMasterKey"; + + protected static final String SP_ADDMASTERKEY = + "CREATE PROCEDURE sp_addMasterKey(" + + " IN keyId_IN int, IN masterKey_IN varchar(1024)," + + " OUT rowCount_OUT int)" + + " MODIFIES SQL DATA BEGIN ATOMIC " + + " INSERT INTO masterKeys(keyId, masterKey)" + + " (SELECT keyId_IN, masterKey_IN" + + " FROM masterKeys " + + " WHERE keyId = keyId_IN " + + " HAVING COUNT(*) = 0);" + + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT;" + + " END"; + + protected static final String SP_DROP_GETMASTERKEY = "DROP PROCEDURE sp_getMasterKey"; + + protected static final String SP_GETMASTERKEY = + "CREATE PROCEDURE sp_getMasterKey(" + + " IN keyId_IN int," + + " OUT masterKey_OUT varchar(1024))" + + " MODIFIES SQL DATA BEGIN ATOMIC " + + " SELECT masterKey INTO masterKey_OUT " + + " FROM masterKeys " + + " WHERE keyId = keyId_IN; " + + " END "; + + protected static final String SP_DROP_DELETEMASTERKEY = "DROP PROCEDURE sp_deleteMasterKey"; + + protected static final String SP_DELETEMASTERKEY = + "CREATE PROCEDURE sp_deleteMasterKey(" + + " IN keyId_IN int, OUT rowCount_OUT int)" + + " MODIFIES SQL DATA BEGIN ATOMIC" + + " DELETE FROM masterKeys WHERE keyId = keyId_IN;" + + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; END"; + + protected static final String SP_DROP_ADD_DELEGATIONTOKEN = + "DROP PROCEDURE sp_addDelegationToken"; + + protected static final String SP_ADD_DELEGATIONTOKEN = + "CREATE PROCEDURE sp_addDelegationToken(" + + " IN sequenceNum_IN bigint, IN tokenIdent_IN varchar(1024)," + + " IN token_IN varchar(1024), IN renewDate_IN bigint, OUT rowCount_OUT int)" + + " MODIFIES SQL DATA BEGIN ATOMIC " + + " INSERT INTO delegationTokens(sequenceNum, tokenIdent, token, renewDate)" + + " (SELECT sequenceNum_IN, tokenIdent_IN, token_IN, renewDate_IN" + + " FROM delegationTokens" + + " WHERE sequenceNum = sequenceNum_IN" + + " HAVING COUNT(*) = 0);" + + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT;" + + " END"; + + protected static final String SP_DROP_GET_DELEGATIONTOKEN = + "DROP PROCEDURE sp_getDelegationToken"; + + protected static final String SP_GET_DELEGATIONTOKEN = + "CREATE PROCEDURE sp_getDelegationToken(" + + " IN sequenceNum_IN bigint, OUT tokenIdent_OUT varchar(1024), " + + " OUT token_OUT varchar(1024), OUT renewDate_OUT bigint)" + + " MODIFIES SQL DATA BEGIN ATOMIC " + + " SELECT tokenIdent, token, renewDate INTO " + + " tokenIdent_OUT, token_OUT, renewDate_OUT" + + " FROM delegationTokens" + + " WHERE sequenceNum = sequenceNum_IN; " + + " END "; + + protected static final String SP_DROP_UPDATE_DELEGATIONTOKEN = + "DROP PROCEDURE sp_updateDelegationToken"; + + protected static final String SP_UPDATE_DELEGATIONTOKEN = + "CREATE PROCEDURE sp_updateDelegationToken(" + + " IN sequenceNum_IN bigint, IN tokenIdent_IN varchar(1024)," + + " IN token_IN varchar(1024), IN renewDate_IN bigint, OUT rowCount_OUT int)" + + " MODIFIES SQL DATA BEGIN ATOMIC" + + " UPDATE delegationTokens" + + " SET tokenIdent = tokenIdent_IN," + + " token = token_IN, renewDate = renewDate_IN" + + " WHERE sequenceNum = sequenceNum_IN;" + + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; " + + " END "; + + protected static final String SP_DROP_DELETE_DELEGATIONTOKEN = + "DROP PROCEDURE sp_deleteDelegationToken"; + + protected static final String SP_DELETE_DELEGATIONTOKEN = + "CREATE PROCEDURE sp_deleteDelegationToken(" + + " IN sequenceNum_IN bigint, OUT rowCount_OUT int)" + + " MODIFIES SQL DATA BEGIN ATOMIC" + + " DELETE FROM delegationTokens" + + " WHERE sequenceNum = sequenceNum_IN;" + + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; " + + " END "; + private List tables = new ArrayList<>(); @Override @@ -333,6 +446,9 @@ public void init(Configuration conf) { conn.prepareStatement(TABLE_MEMBERSHIP).execute(); conn.prepareStatement(TABLE_POLICIES).execute(); conn.prepareStatement(TABLE_RESERVATIONSHOMESUBCLUSTER).execute(); + conn.prepareStatement(TABLE_MASTERKEYS).execute(); + conn.prepareStatement(TABLE_DELEGATIONTOKENS).execute(); + conn.prepareStatement(TABLE_SEQUENCETABLE).execute(); conn.prepareStatement(SP_REGISTERSUBCLUSTER).execute(); conn.prepareStatement(SP_DEREGISTERSUBCLUSTER).execute(); @@ -356,6 +472,15 @@ public void init(Configuration conf) { conn.prepareStatement(SP_DELETERESERVATIONHOMESUBCLUSTER).execute(); conn.prepareStatement(SP_UPDATERESERVATIONHOMESUBCLUSTER).execute(); + conn.prepareStatement(SP_ADDMASTERKEY).execute(); + conn.prepareStatement(SP_GETMASTERKEY).execute(); + conn.prepareStatement(SP_DELETEMASTERKEY).execute(); + + conn.prepareStatement(SP_ADD_DELEGATIONTOKEN).execute(); + conn.prepareStatement(SP_GET_DELEGATIONTOKEN).execute(); + conn.prepareStatement(SP_UPDATE_DELEGATIONTOKEN).execute(); + conn.prepareStatement(SP_DELETE_DELEGATIONTOKEN).execute(); + LOG.info("Database Init: Complete"); } catch (Exception e) { LOG.error("ERROR: failed to initialize HSQLDB {}.", e.getMessage()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java index 0ea714ff06e2f..5548dab1b8c06 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestMemoryFederationStateStore.java @@ -78,7 +78,7 @@ protected void checkRouterStoreToken(RMDelegationTokenIdentifier identifier, memoryStateStore.getRouterRMSecretManagerState(); assertNotNull(secretManagerState); - Map tokenStateMap = + Map tokenStateMap = secretManagerState.getTokenState(); assertNotNull(tokenStateMap); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java index befdf4897637c..05ee85efa1a93 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java @@ -17,7 +17,6 @@ package org.apache.hadoop.yarn.server.federation.store.impl; -import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.Time; @@ -37,6 +36,13 @@ import org.apache.hadoop.yarn.server.federation.store.records.DeleteReservationHomeSubClusterRequest; import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyRequest; +import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyResponse; +import org.apache.hadoop.yarn.server.federation.store.sql.DatabaseProduct; +import org.apache.hadoop.yarn.server.federation.store.sql.FederationSQLOutParameter; +import org.apache.hadoop.yarn.server.federation.store.sql.FederationQueryRunner; +import org.apache.hadoop.yarn.server.federation.store.sql.RouterMasterKeyHandler; +import org.apache.hadoop.yarn.server.federation.store.sql.RouterStoreTokenHandler; import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils; import org.junit.Assert; import org.junit.Test; @@ -48,20 +54,30 @@ import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.DatabaseMetaData; import java.util.ArrayList; import java.util.List; +import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_GET_MASTERKEY; import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_ADD_RESERVATION_HOME_SUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_GET_RESERVATION_HOME_SUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_GET_RESERVATIONS_HOME_SUBCLUSTER; -import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_UPDATE_RESERVATION_HOME_SUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_DELETE_RESERVATION_HOME_SUBCLUSTER; +import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_GET_DELEGATIONTOKEN; +import static org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore.CALL_SP_UPDATE_RESERVATION_HOME_SUBCLUSTER; +import static org.apache.hadoop.yarn.server.federation.store.sql.DatabaseProduct.DbType; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_DROP_ADDRESERVATIONHOMESUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_ADDRESERVATIONHOMESUBCLUSTER2; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_DROP_UPDATERESERVATIONHOMESUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_UPDATERESERVATIONHOMESUBCLUSTER2; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_DROP_DELETERESERVATIONHOMESUBCLUSTER; import static org.apache.hadoop.yarn.server.federation.store.impl.HSQLDBFederationStateStore.SP_DELETERESERVATIONHOMESUBCLUSTER2; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static java.sql.Types.VARCHAR; +import static java.sql.Types.BIGINT; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Unit tests for SQLFederationStateStore. @@ -74,6 +90,7 @@ public class TestSQLFederationStateStore extends FederationStateStoreBaseTest { private static final String DATABASE_URL = "jdbc:hsqldb:mem:state"; private static final String DATABASE_USERNAME = "SA"; private static final String DATABASE_PASSWORD = ""; + private SQLFederationStateStore sqlFederationStateStore = null; @Override protected FederationStateStore createStateStore() { @@ -90,7 +107,8 @@ protected FederationStateStore createStateStore() { DATABASE_URL + System.currentTimeMillis()); conf.setInt(YarnConfiguration.FEDERATION_STATESTORE_MAX_APPLICATIONS, 10); super.setConf(conf); - return new HSQLDBFederationStateStore(); + sqlFederationStateStore = new HSQLDBFederationStateStore(); + return sqlFederationStateStore; } @Test @@ -103,13 +121,13 @@ public void testSqlConnectionsCreatedCount() throws YarnException { stateStore.registerSubCluster( SubClusterRegisterRequest.newInstance(subClusterInfo)); - Assert.assertEquals(subClusterInfo, querySubClusterInfo(subClusterId)); + assertEquals(subClusterInfo, querySubClusterInfo(subClusterId)); addApplicationHomeSC(appId, subClusterId); - Assert.assertEquals(subClusterId, queryApplicationHomeSC(appId)); + assertEquals(subClusterId, queryApplicationHomeSC(appId)); // Verify if connection is created only once at statestore init - Assert.assertEquals(1, + assertEquals(1, FederationStateStoreClientMetrics.getNumConnections()); } @@ -266,9 +284,9 @@ public void testCheckAddReservationHomeSubCluster() throws Exception { CALL_SP_ADD_RESERVATION_HOME_SUBCLUSTER, reservationId.toString(), subHomeClusterId); // validation results - Assert.assertNotNull(resultHC); - Assert.assertEquals(subHomeClusterId, resultHC.subHomeClusterId); - Assert.assertEquals(1, resultHC.dbUpdateCount); + assertNotNull(resultHC); + assertEquals(subHomeClusterId, resultHC.subHomeClusterId); + assertEquals(1, resultHC.dbUpdateCount); } /** @@ -297,9 +315,9 @@ public void testCheckGetReservationHomeSubCluster() throws Exception { ReservationHomeSC resultHC = getReservationHomeSubCluster(sqlFederationStateStore, CALL_SP_GET_RESERVATION_HOME_SUBCLUSTER, reservationId.toString()); - Assert.assertNotNull(resultHC); - Assert.assertEquals(subHomeClusterId, resultHC.subHomeClusterId); - Assert.assertEquals(reservationId.toString(), resultHC.reservationId); + assertNotNull(resultHC); + assertEquals(subHomeClusterId, resultHC.subHomeClusterId); + assertEquals(reservationId.toString(), resultHC.reservationId); } /** @@ -335,18 +353,18 @@ public void testCheckGetReservationsHomeSubCluster() throws Exception { List reservationHomeSubClusters = getReservationsHomeSubCluster( sqlFederationStateStore, CALL_SP_GET_RESERVATIONS_HOME_SUBCLUSTER); - Assert.assertNotNull(reservationHomeSubClusters); - Assert.assertEquals(2, reservationHomeSubClusters.size()); + assertNotNull(reservationHomeSubClusters); + assertEquals(2, reservationHomeSubClusters.size()); ReservationHomeSC resultHC1 = reservationHomeSubClusters.get(0); - Assert.assertNotNull(resultHC1); - Assert.assertEquals(reservationId1.toString(), resultHC1.reservationId); - Assert.assertEquals(subHomeClusterId1, resultHC1.subHomeClusterId); + assertNotNull(resultHC1); + assertEquals(reservationId1.toString(), resultHC1.reservationId); + assertEquals(subHomeClusterId1, resultHC1.subHomeClusterId); ReservationHomeSC resultHC2 = reservationHomeSubClusters.get(1); - Assert.assertNotNull(resultHC2); - Assert.assertEquals(reservationId2.toString(), resultHC2.reservationId); - Assert.assertEquals(subHomeClusterId2, resultHC2.subHomeClusterId); + assertNotNull(resultHC2); + assertEquals(reservationId2.toString(), resultHC2.reservationId); + assertEquals(subHomeClusterId2, resultHC2.subHomeClusterId); } /** @@ -378,8 +396,8 @@ public void testCheckUpdateReservationHomeSubCluster() throws Exception { // verify that the subHomeClusterId corresponding to reservationId is SC-1 ReservationHomeSC resultHC = getReservationHomeSubCluster(sqlFederationStateStore, CALL_SP_GET_RESERVATION_HOME_SUBCLUSTER, reservationId.toString()); - Assert.assertNotNull(resultHC); - Assert.assertEquals(subHomeClusterId, resultHC.subHomeClusterId); + assertNotNull(resultHC); + assertEquals(subHomeClusterId, resultHC.subHomeClusterId); // prepare to update parameters String newSubHomeClusterId = "SC-2"; @@ -387,14 +405,14 @@ public void testCheckUpdateReservationHomeSubCluster() throws Exception { updateReservationHomeSubCluster(sqlFederationStateStore, CALL_SP_UPDATE_RESERVATION_HOME_SUBCLUSTER, reservationId.toString(), newSubHomeClusterId); - Assert.assertNotNull(reservationHomeSubCluster); - Assert.assertEquals(1, reservationHomeSubCluster.dbUpdateCount); + assertNotNull(reservationHomeSubCluster); + assertEquals(1, reservationHomeSubCluster.dbUpdateCount); // verify that the subHomeClusterId corresponding to reservationId is SC-2 ReservationHomeSC resultHC2 = getReservationHomeSubCluster(sqlFederationStateStore, CALL_SP_GET_RESERVATION_HOME_SUBCLUSTER, reservationId.toString()); - Assert.assertNotNull(resultHC2); - Assert.assertEquals(newSubHomeClusterId, resultHC2.subHomeClusterId); + assertNotNull(resultHC2); + assertEquals(newSubHomeClusterId, resultHC2.subHomeClusterId); } /** @@ -426,14 +444,14 @@ public void testCheckDeleteReservationHomeSubCluster() throws Exception { ReservationHomeSC resultHC = deleteReservationHomeSubCluster(sqlFederationStateStore, CALL_SP_DELETE_RESERVATION_HOME_SUBCLUSTER, reservationId.toString()); - Assert.assertNotNull(resultHC); - Assert.assertEquals(1, resultHC.dbUpdateCount); + assertNotNull(resultHC); + assertEquals(1, resultHC.dbUpdateCount); // call getReservationHomeSubCluster to get the result ReservationHomeSC resultHC1 = getReservationHomeSubCluster(sqlFederationStateStore, CALL_SP_GET_RESERVATION_HOME_SUBCLUSTER, reservationId.toString()); - Assert.assertNotNull(resultHC1); - Assert.assertEquals(null, resultHC1.subHomeClusterId); + assertNotNull(resultHC1); + assertEquals(null, resultHC1.subHomeClusterId); } /** @@ -562,52 +580,91 @@ public void testDeleteReservationHomeSubClusterAbnormalSituation() throws Except () -> stateStore.deleteReservationHomeSubCluster(delRequest)); } - @Test(expected = NotImplementedException.class) - public void testStoreNewMasterKey() throws Exception { - super.testStoreNewMasterKey(); - } - - @Test(expected = NotImplementedException.class) - public void testGetMasterKeyByDelegationKey() throws YarnException, IOException { - super.testGetMasterKeyByDelegationKey(); - } + @Override + protected void checkRouterMasterKey(DelegationKey delegationKey, + RouterMasterKey routerMasterKey) throws YarnException, IOException, SQLException { + // Check for MasterKey stored in DB. + RouterMasterKeyRequest routerMasterKeyRequest = + RouterMasterKeyRequest.newInstance(routerMasterKey); - @Test(expected = NotImplementedException.class) - public void testRemoveStoredMasterKey() throws YarnException, IOException { - super.testRemoveStoredMasterKey(); + // Query Data from DB. + Connection conn = sqlFederationStateStore.getConn(); + int paramKeyId = delegationKey.getKeyId(); + FederationQueryRunner runner = new FederationQueryRunner(); + FederationSQLOutParameter masterKeyOUT = + new FederationSQLOutParameter<>("masterKey_OUT", VARCHAR, String.class); + RouterMasterKey sqlRouterMasterKey = runner.execute( + conn, CALL_SP_GET_MASTERKEY, new RouterMasterKeyHandler(), paramKeyId, masterKeyOUT); + + // Check Data. + RouterMasterKeyResponse response = getStateStore(). + getMasterKeyByDelegationKey(routerMasterKeyRequest); + assertNotNull(response); + RouterMasterKey respRouterMasterKey = response.getRouterMasterKey(); + assertEquals(routerMasterKey, respRouterMasterKey); + assertEquals(routerMasterKey, sqlRouterMasterKey); + assertEquals(sqlRouterMasterKey, respRouterMasterKey); } - @Test(expected = NotImplementedException.class) - public void testStoreNewToken() throws IOException, YarnException { - super.testStoreNewToken(); + @Override + protected void checkRouterStoreToken(RMDelegationTokenIdentifier identifier, + RouterStoreToken token) throws YarnException, IOException, SQLException { + // Get SequenceNum. + int sequenceNum = identifier.getSequenceNumber(); + + // Query Data from DB. + Connection conn = sqlFederationStateStore.getConn(); + FederationQueryRunner runner = new FederationQueryRunner(); + FederationSQLOutParameter tokenIdentOUT = + new FederationSQLOutParameter<>("tokenIdent_OUT", VARCHAR, String.class); + FederationSQLOutParameter tokenOUT = + new FederationSQLOutParameter<>("token_OUT", VARCHAR, String.class); + FederationSQLOutParameter renewDateOUT = + new FederationSQLOutParameter<>("renewDate_OUT", BIGINT, Long.class); + RouterStoreToken sqlRouterStoreToken = runner.execute(conn, CALL_SP_GET_DELEGATIONTOKEN, + new RouterStoreTokenHandler(), sequenceNum, tokenIdentOUT, tokenOUT, renewDateOUT); + + assertEquals(token, sqlRouterStoreToken); } - @Test(expected = NotImplementedException.class) - public void testUpdateStoredToken() throws IOException, YarnException { - super.testUpdateStoredToken(); + @Test + public void testCheckHSQLDB() throws SQLException { + FederationStateStore stateStore = getStateStore(); + SQLFederationStateStore sqlFederationStateStore = (SQLFederationStateStore) stateStore; + Connection conn = sqlFederationStateStore.getConn(); + DbType dbType = DatabaseProduct.getDbType(conn); + assertEquals(DbType.HSQLDB, dbType); } - @Test(expected = NotImplementedException.class) - public void testRemoveStoredToken() throws IOException, YarnException { - super.testRemoveStoredToken(); + @Test + public void testGetDbTypeNullConn() throws SQLException { + DbType dbType = DatabaseProduct.getDbType(null); + assertEquals(DbType.UNDEFINED, dbType); } - @Test(expected = NotImplementedException.class) - public void testGetTokenByRouterStoreToken() throws IOException, YarnException { - super.testGetTokenByRouterStoreToken(); + @Test + public void testGetDBTypeEmptyConn() throws SQLException { + Connection connection = mock(Connection.class); + DatabaseMetaData metaData = mock(DatabaseMetaData.class); + when(metaData.getDatabaseProductName()).thenReturn(""); + when(connection.getMetaData()).thenReturn(metaData); + DbType dbType = DatabaseProduct.getDbType(connection); + assertEquals(DbType.UNDEFINED, dbType); } - @Override - protected void checkRouterMasterKey(DelegationKey delegationKey, - RouterMasterKey routerMasterKey) throws YarnException, IOException { - // TODO: This part of the code will be completed in YARN-11349 and - // will be used to verify whether the RouterMasterKey stored in the DB is as expected. + @Test + public void testCheckForHSQLDBUpdateSQL() throws SQLException { + String sql = "select sequenceName, nextVal from sequenceTable"; + String hsqlDBSQL = DatabaseProduct.addForUpdateClause(DbType.HSQLDB, sql); + String expectUpdateSQL = "select sequenceName, nextVal from sequenceTable for update"; + assertEquals(expectUpdateSQL, hsqlDBSQL); } - @Override - protected void checkRouterStoreToken(RMDelegationTokenIdentifier identifier, - RouterStoreToken token) throws YarnException, IOException { - // TODO: This part of the code will be completed in YARN-11349 and - // will be used to verify whether the RouterStoreToken stored in the DB is as expected. + @Test + public void testCheckForSqlServerDBUpdateSQL() throws SQLException { + String sql = "select sequenceName, nextVal from sequenceTable"; + String sqlServerDBSQL = DatabaseProduct.addForUpdateClause(DbType.SQLSERVER, sql); + String expectUpdateSQL = "select sequenceName, nextVal from sequenceTable with (updlock)"; + assertEquals(expectUpdateSQL, sqlServerDBSQL); } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/RouterDelegationTokenSupport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/RouterDelegationTokenSupport.java new file mode 100644 index 0000000000000..d530f751cb20a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/RouterDelegationTokenSupport.java @@ -0,0 +1,65 @@ +/* + * 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.security.token.delegation; + +import org.apache.hadoop.io.WritableUtils; +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.DelegationTokenInformation; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Base64; + +/** + * Workaround for serialization of {@link DelegationTokenInformation} through package access. + * Future version of Hadoop should add this to DelegationTokenInformation itself. + */ +public final class RouterDelegationTokenSupport { + + private RouterDelegationTokenSupport() { + } + + public static String encodeDelegationTokenInformation(DelegationTokenInformation token) { + try { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(bos); + WritableUtils.writeVInt(out, token.password.length); + out.write(token.password); + out.writeLong(token.renewDate); + out.flush(); + byte[] tokenInfoBytes = bos.toByteArray(); + return Base64.getUrlEncoder().encodeToString(tokenInfoBytes); + } catch (IOException ex) { + throw new RuntimeException("Failed to encode token.", ex); + } + } + + public static DelegationTokenInformation decodeDelegationTokenInformation(byte[] tokenBytes) + throws IOException { + DataInputStream in = new DataInputStream(new ByteArrayInputStream(tokenBytes)); + DelegationTokenInformation token = new DelegationTokenInformation(0, null); + int len = WritableUtils.readVInt(in); + token.password = new byte[len]; + in.readFully(token.password); + token.renewDate = in.readLong(); + return token; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java new file mode 100644 index 0000000000000..3a1cb3e69ae5f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/security/token/delegation/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Router security token delegation. **/ +package org.apache.hadoop.security.token.delegation; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java index 79790cde3010d..606329ea194a2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/security/RouterDelegationTokenSecretManager.java @@ -21,13 +21,16 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager; import org.apache.hadoop.security.token.delegation.DelegationKey; +import org.apache.hadoop.security.token.delegation.RouterDelegationTokenSupport; import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKey; import org.apache.hadoop.yarn.server.federation.store.records.RouterMasterKeyResponse; import org.apache.hadoop.yarn.server.federation.store.records.RouterRMTokenResponse; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +41,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.Base64; /** * A Router specific delegation token secret manager. @@ -137,6 +141,29 @@ public void storeNewToken(RMDelegationTokenIdentifier identifier, } } + /** + * The Router Supports Store new Token. + * + * @param identifier RMDelegationToken. + * @param tokenInfo DelegationTokenInformation. + */ + public void storeNewToken(RMDelegationTokenIdentifier identifier, + DelegationTokenInformation tokenInfo) { + try { + String token = + RouterDelegationTokenSupport.encodeDelegationTokenInformation(tokenInfo); + long renewDate = tokenInfo.getRenewDate(); + + federationFacade.storeNewToken(identifier, renewDate, token); + } catch (Exception e) { + if (!shouldIgnoreException(e)) { + LOG.error("Error in storing RMDelegationToken with sequence number: {}.", + identifier.getSequenceNumber()); + ExitUtil.terminate(1, e); + } + } + } + /** * The Router Supports Update Token. * @@ -157,6 +184,27 @@ public void updateStoredToken(RMDelegationTokenIdentifier id, long renewDate) th } } + /** + * The Router Supports Update Token. + * + * @param identifier RMDelegationToken. + * @param tokenInfo DelegationTokenInformation. + */ + public void updateStoredToken(RMDelegationTokenIdentifier identifier, + DelegationTokenInformation tokenInfo) { + try { + long renewDate = tokenInfo.getRenewDate(); + String token = RouterDelegationTokenSupport.encodeDelegationTokenInformation(tokenInfo); + federationFacade.updateStoredToken(identifier, renewDate, token); + } catch (Exception e) { + if (!shouldIgnoreException(e)) { + LOG.error("Error in updating persisted RMDelegationToken with sequence number: {}.", + identifier.getSequenceNumber()); + ExitUtil.terminate(1, e); + } + } + } + /** * The Router Supports Remove Token. * @@ -257,6 +305,42 @@ protected synchronized int incrementDelegationTokenSeqNum() { return federationFacade.incrementDelegationTokenSeqNum(); } + @Override + protected void storeToken(RMDelegationTokenIdentifier rmDelegationTokenIdentifier, + DelegationTokenInformation tokenInfo) throws IOException { + this.currentTokens.put(rmDelegationTokenIdentifier, tokenInfo); + this.addTokenForOwnerStats(rmDelegationTokenIdentifier); + storeNewToken(rmDelegationTokenIdentifier, tokenInfo); + } + + @Override + protected void updateToken(RMDelegationTokenIdentifier rmDelegationTokenIdentifier, + DelegationTokenInformation tokenInfo) throws IOException { + this.currentTokens.put(rmDelegationTokenIdentifier, tokenInfo); + updateStoredToken(rmDelegationTokenIdentifier, tokenInfo); + } + + @Override + protected DelegationTokenInformation getTokenInfo( + RMDelegationTokenIdentifier ident) { + // First check if I have this.. + DelegationTokenInformation tokenInfo = currentTokens.get(ident); + if (tokenInfo == null) { + try { + RouterRMTokenResponse response = federationFacade.getTokenByRouterStoreToken(ident); + RouterStoreToken routerStoreToken = response.getRouterStoreToken(); + String tokenStr = routerStoreToken.getTokenInfo(); + byte[] tokenBytes = Base64.getUrlDecoder().decode(tokenStr); + tokenInfo = RouterDelegationTokenSupport.decodeDelegationTokenInformation(tokenBytes); + } catch (Exception e) { + LOG.error("Error retrieving tokenInfo [" + ident.getSequenceNumber() + + "] from StateStore.", e); + throw new YarnRuntimeException(e); + } + } + return tokenInfo; + } + @Override protected synchronized int getDelegationTokenSeqNum() { return federationFacade.getDelegationTokenSeqNum(); 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/TestFederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java index 2488fc73b07b3..6f7248a0866b4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java @@ -138,6 +138,7 @@ import org.apache.hadoop.yarn.server.federation.policies.manager.UniformBroadcastPolicyManager; import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore; import org.apache.hadoop.yarn.server.federation.store.records.RouterRMDTSecretManagerState; +import org.apache.hadoop.yarn.server.federation.store.records.RouterStoreToken; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade; @@ -1617,14 +1618,17 @@ public void testGetDelegationToken() throws IOException, YarnException { RouterRMDTSecretManagerState managerState = stateStore.getRouterRMSecretManagerState(); Assert.assertNotNull(managerState); - Map delegationTokenState = managerState.getTokenState(); + Map delegationTokenState = + managerState.getTokenState(); Assert.assertNotNull(delegationTokenState); Assert.assertTrue(delegationTokenState.containsKey(rMDelegationTokenIdentifier)); long tokenRenewInterval = this.getConf().getLong( YarnConfiguration.RM_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, YarnConfiguration.RM_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT); - long renewDate = delegationTokenState.get(rMDelegationTokenIdentifier); + RouterStoreToken resultRouterStoreToken = delegationTokenState.get(rMDelegationTokenIdentifier); + Assert.assertNotNull(resultRouterStoreToken); + long renewDate = resultRouterStoreToken.getRenewDate(); Assert.assertEquals(issueDate + tokenRenewInterval, renewDate); } @@ -1667,10 +1671,13 @@ public void testRenewDelegationToken() throws IOException, YarnException { // Step3. Compare whether the expirationTime returned to // the client is consistent with the renewDate in the stateStore RouterRMDTSecretManagerState managerState = stateStore.getRouterRMSecretManagerState(); - Map delegationTokenState = managerState.getTokenState(); + Map delegationTokenState = + managerState.getTokenState(); Assert.assertNotNull(delegationTokenState); Assert.assertTrue(delegationTokenState.containsKey(rMDelegationTokenIdentifier)); - long renewDate = delegationTokenState.get(rMDelegationTokenIdentifier); + RouterStoreToken resultRouterStoreToken = delegationTokenState.get(rMDelegationTokenIdentifier); + Assert.assertNotNull(resultRouterStoreToken); + long renewDate = resultRouterStoreToken.getRenewDate(); Assert.assertEquals(expDate, renewDate); } @@ -1700,7 +1707,8 @@ public void testCancelDelegationToken() throws IOException, YarnException { // Step3. Query the data in the StateStore and confirm that the Delegation has been deleted. // At this point, the size of delegationTokenState should be 0. RouterRMDTSecretManagerState managerState = stateStore.getRouterRMSecretManagerState(); - Map delegationTokenState = managerState.getTokenState(); + Map delegationTokenState = + managerState.getTokenState(); Assert.assertNotNull(delegationTokenState); Assert.assertEquals(0, delegationTokenState.size()); }