From f58945d7d172fe70984a259eeb8e26da05624c91 Mon Sep 17 00:00:00 2001 From: Tom Date: Wed, 8 Nov 2023 06:25:21 -0800 Subject: [PATCH 01/10] HDFS-16791. Add getEnclosingRoot() API to filesystem interface and implementations (#6198) The enclosing root path is a common ancestor that should be used for temp and staging dirs as well as within encryption zones and other restricted directories. Contributed by Tom McCormick --- .../apache/hadoop/fs/AbstractFileSystem.java | 18 +++ .../java/org/apache/hadoop/fs/FileSystem.java | 18 +++ .../apache/hadoop/fs/FilterFileSystem.java | 5 + .../java/org/apache/hadoop/fs/FilterFs.java | 5 + .../hadoop/fs/viewfs/ViewFileSystem.java | 37 +++++ .../org/apache/hadoop/fs/viewfs/ViewFs.java | 17 ++ .../site/markdown/filesystem/filesystem.md | 35 +++- .../hadoop/fs/TestGetEnclosingRoot.java | 94 +++++++++++ .../apache/hadoop/fs/TestHarFileSystem.java | 2 + .../AbstractContractGetEnclosingRoot.java | 103 ++++++++++++ .../TestLocalFSContractGetEnclosingRoot.java | 32 ++++ .../TestRawlocalContractGetEnclosingRoot.java | 32 ++++ .../org/apache/hadoop/hdfs/DFSClient.java | 10 ++ .../hadoop/hdfs/DFSOpsCountStatistics.java | 1 + .../hadoop/hdfs/DistributedFileSystem.java | 27 ++++ .../hadoop/hdfs/protocol/ClientProtocol.java | 8 + .../ClientNamenodeProtocolTranslatorPB.java | 18 +++ .../main/proto/ClientNamenodeProtocol.proto | 10 ++ .../hadoop/hdfs/protocol/TestReadOnly.java | 1 + .../router/RouterClientProtocol.java | 33 ++++ .../federation/router/RouterRpcServer.java | 5 + .../router/TestRouterMountTable.java | 25 +++ ...amenodeProtocolServerSideTranslatorPB.java | 22 ++- .../hdfs/server/namenode/FSNamesystem.java | 15 ++ .../server/namenode/NameNodeRpcServer.java | 7 + .../fs/viewfs/TestViewFileSystemHdfs.java | 149 ++++++++++++++---- .../org/apache/hadoop/hdfs/DFSTestUtil.java | 27 ++++ .../hdfs/TestDistributedFileSystem.java | 5 + .../apache/hadoop/hdfs/TestEnclosingRoot.java | 149 ++++++++++++++++++ 29 files changed, 878 insertions(+), 32 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetEnclosingRoot.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetEnclosingRoot.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractGetEnclosingRoot.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractGetEnclosingRoot.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEnclosingRoot.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java index a4737c548c8fa..63b5bc7d94ac3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java @@ -1638,6 +1638,24 @@ public MultipartUploaderBuilder createMultipartUploader(Path basePath) return null; } + /** + * Return path of the enclosing root for a given path + * The enclosing root path is a common ancestor that should be used for temp and staging dirs + * as well as within encryption zones and other restricted directories. + * + * Call makeQualified on the param path to ensure its part of the correct filesystem + * + * @param path file path to find the enclosing root path for + * @return a path to the enclosing root + * @throws IOException early checks like failure to resolve path cause IO failures + */ + @InterfaceAudience.Public + @InterfaceStability.Unstable + public Path getEnclosingRoot(Path path) throws IOException { + makeQualified(path); + return makeQualified(new Path("/")); + } + /** * Helper method that throws an {@link UnsupportedOperationException} for the * current {@link FileSystem} method being called. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index 52425211dc688..0213772ab6a5c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -4944,6 +4944,24 @@ public CompletableFuture build() throws IOException { } + /** + * Return path of the enclosing root for a given path. + * The enclosing root path is a common ancestor that should be used for temp and staging dirs + * as well as within encryption zones and other restricted directories. + * + * Call makeQualified on the param path to ensure its part of the correct filesystem. + * + * @param path file path to find the enclosing root path for + * @return a path to the enclosing root + * @throws IOException early checks like failure to resolve path cause IO failures + */ + @InterfaceAudience.Public + @InterfaceStability.Unstable + public Path getEnclosingRoot(Path path) throws IOException { + this.makeQualified(path); + return this.makeQualified(new Path("/")); + } + /** * Create a multipart uploader. * @param basePath file path under which all files are uploaded diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java index cdbe51e330701..a85cf2ff5a17e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java @@ -732,6 +732,11 @@ protected CompletableFuture openFileWithOptions( return fs.openFileWithOptions(pathHandle, parameters); } + @Override + public Path getEnclosingRoot(Path path) throws IOException { + return fs.getEnclosingRoot(path); + } + @Override public boolean hasPathCapability(final Path path, final String capability) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java index 7d979b37b4a50..df010e3dae7f3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java @@ -459,4 +459,9 @@ public MultipartUploaderBuilder createMultipartUploader(final Path basePath) throws IOException { return myFs.createMultipartUploader(basePath); } + + @Override + public Path getEnclosingRoot(Path path) throws IOException { + return myFs.getEnclosingRoot(path); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java index b4cf96ea5996a..c111886e369aa 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java @@ -1370,6 +1370,24 @@ public boolean hasPathCapability(Path path, String capability) } } + @Override + public Path getEnclosingRoot(Path path) throws IOException { + InodeTree.ResolveResult res; + try { + res = fsState.resolve(getUriPath(path), true); + } catch (FileNotFoundException ex) { + NotInMountpointException mountPointEx = + new NotInMountpointException(path, + String.format("getEnclosingRoot - %s", ex.getMessage())); + mountPointEx.initCause(ex); + throw mountPointEx; + } + Path mountPath = new Path(res.resolvedPath); + Path enclosingPath = res.targetFileSystem.getEnclosingRoot(new Path(getUriPath(path))); + return fixRelativePart(this.makeQualified(enclosingPath.depth() > mountPath.depth() + ? enclosingPath : mountPath)); + } + /** * An instance of this class represents an internal dir of the viewFs * that is internal dir of the mount table. @@ -1919,6 +1937,25 @@ public Collection getAllStoragePolicies() } return allPolicies; } + + @Override + public Path getEnclosingRoot(Path path) throws IOException { + InodeTree.ResolveResult res; + try { + res = fsState.resolve((path.toString()), true); + } catch (FileNotFoundException ex) { + NotInMountpointException mountPointEx = + new NotInMountpointException(path, + String.format("getEnclosingRoot - %s", ex.getMessage())); + mountPointEx.initCause(ex); + throw mountPointEx; + } + Path fullPath = new Path(res.resolvedPath); + Path enclosingPath = res.targetFileSystem.getEnclosingRoot(path); + return enclosingPath.depth() > fullPath.depth() + ? enclosingPath + : fullPath; + } } enum RenameStrategy { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java index 5f54c9cdd06aa..3d85015bea47e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java @@ -1477,5 +1477,22 @@ public void setStoragePolicy(Path path, String policyName) throws IOException { throw readOnlyMountTable("setStoragePolicy", path); } + + @Override + public Path getEnclosingRoot(Path path) throws IOException { + InodeTree.ResolveResult res; + try { + res = fsState.resolve((path.toString()), true); + } catch (FileNotFoundException ex) { + NotInMountpointException mountPointEx = + new NotInMountpointException(path, + String.format("getEnclosingRoot - %s", ex.getMessage())); + mountPointEx.initCause(ex); + throw mountPointEx; + } + Path fullPath = new Path(res.resolvedPath); + Path enclosingPath = res.targetFileSystem.getEnclosingRoot(path); + return enclosingPath.depth() > fullPath.depth() ? enclosingPath : fullPath; + } } } diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md index fafe2819cf66a..5fba8a2515bb4 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md @@ -601,7 +601,40 @@ on the filesystem. 1. The outcome of this operation MUST be identical to the value of `getFileStatus(P).getBlockSize()`. -1. By inference, it MUST be > 0 for any file of length > 0. +2. By inference, it MUST be > 0 for any file of length > 0. + +### `Path getEnclosingRoot(Path p)` + +This method is used to find a root directory for a path given. This is useful for creating +staging and temp directories in the same enclosing root directory. There are constraints around how +renames are allowed to atomically occur (ex. across hdfs volumes or across encryption zones). + +For any two paths p1 and p2 that do not have the same enclosing root, `rename(p1, p2)` is expected to fail or will not +be atomic. + +For object stores, even with the same enclosing root, there is no guarantee file or directory rename is atomic + +The following statement is always true: +`getEnclosingRoot(p) == getEnclosingRoot(getEnclosingRoot(p))` + + +```python +path in ancestors(FS, p) or path == p: +isDir(FS, p) +``` + +#### Preconditions + +The path does not have to exist, but the path does need to be valid and reconcilable by the filesystem +* if a linkfallback is used all paths are reconcilable +* if a linkfallback is not used there must be a mount point covering the path + + +#### Postconditions + +* The path returned will not be null, if there is no deeper enclosing root, the root path ("/") will be returned. +* The path returned is a directory + ## State Changing Operations diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetEnclosingRoot.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetEnclosingRoot.java new file mode 100644 index 0000000000000..8bbab36d53096 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetEnclosingRoot.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.test.HadoopTestBase; +import org.junit.Test; + +public class TestGetEnclosingRoot extends HadoopTestBase { + @Test + public void testEnclosingRootEquivalence() throws IOException { + FileSystem fs = getFileSystem(); + Path root = path("/"); + Path foobar = path("/foo/bar"); + + assertEquals(root, fs.getEnclosingRoot(root)); + assertEquals(root, fs.getEnclosingRoot(foobar)); + assertEquals(root, fs.getEnclosingRoot(fs.getEnclosingRoot(foobar))); + assertEquals(fs.getEnclosingRoot(root), fs.getEnclosingRoot(foobar)); + + assertEquals(root, fs.getEnclosingRoot(path(foobar.toString()))); + assertEquals(root, fs.getEnclosingRoot(fs.getEnclosingRoot(path(foobar.toString())))); + assertEquals(fs.getEnclosingRoot(root), fs.getEnclosingRoot(path(foobar.toString()))); + } + + @Test + public void testEnclosingRootPathExists() throws Exception { + FileSystem fs = getFileSystem(); + Path root = path("/"); + Path foobar = path("/foo/bar"); + fs.mkdirs(foobar); + + assertEquals(root, fs.getEnclosingRoot(foobar)); + assertEquals(root, fs.getEnclosingRoot(path(foobar.toString()))); + } + + @Test + public void testEnclosingRootPathDNE() throws Exception { + FileSystem fs = getFileSystem(); + Path foobar = path("/foo/bar"); + Path root = path("/"); + + assertEquals(root, fs.getEnclosingRoot(foobar)); + assertEquals(root, fs.getEnclosingRoot(path(foobar.toString()))); + } + + @Test + public void testEnclosingRootWrapped() throws Exception { + FileSystem fs = getFileSystem(); + Path root = path("/"); + + assertEquals(root, fs.getEnclosingRoot(new Path("/foo/bar"))); + + UserGroupInformation ugi = UserGroupInformation.createRemoteUser("foo"); + Path p = ugi.doAs((PrivilegedExceptionAction) () -> { + FileSystem wFs = getFileSystem(); + return wFs.getEnclosingRoot(new Path("/foo/bar")); + }); + assertEquals(root, p); + } + + private FileSystem getFileSystem() throws IOException { + return FileSystem.get(new Configuration()); + } + + /** + * Create a path under the test path provided by + * the FS contract. + * @param filepath path string in + * @return a path qualified by the test filesystem + * @throws IOException IO problems + */ + private Path path(String filepath) throws IOException { + return getFileSystem().makeQualified( + new Path(filepath)); + }} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java index b227e16908828..0287b7ec1fb84 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java @@ -255,6 +255,8 @@ MultipartUploaderBuilder createMultipartUploader(Path basePath) FSDataOutputStream append(Path f, int bufferSize, Progressable progress, boolean appendToNewBlock) throws IOException; + + Path getEnclosingRoot(Path path) throws IOException; } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetEnclosingRoot.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetEnclosingRoot.java new file mode 100644 index 0000000000000..9564c31725d06 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetEnclosingRoot.java @@ -0,0 +1,103 @@ +/** + * 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.contract; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public abstract class AbstractContractGetEnclosingRoot extends AbstractFSContractTestBase { + private static final Logger LOG = LoggerFactory.getLogger(AbstractContractGetEnclosingRoot.class); + + @Test + public void testEnclosingRootEquivalence() throws IOException { + FileSystem fs = getFileSystem(); + Path root = path("/"); + Path foobar = path("/foo/bar"); + + assertEquals("Ensure getEnclosingRoot on the root directory returns the root directory", + root, fs.getEnclosingRoot(foobar)); + assertEquals("Ensure getEnclosingRoot called on itself returns the root directory", + root, fs.getEnclosingRoot(fs.getEnclosingRoot(foobar))); + assertEquals( + "Ensure getEnclosingRoot for different paths in the same enclosing root " + + "returns the same path", + fs.getEnclosingRoot(root), fs.getEnclosingRoot(foobar)); + assertEquals("Ensure getEnclosingRoot on a path returns the root directory", + root, fs.getEnclosingRoot(methodPath())); + assertEquals("Ensure getEnclosingRoot called on itself on a path returns the root directory", + root, fs.getEnclosingRoot(fs.getEnclosingRoot(methodPath()))); + assertEquals( + "Ensure getEnclosingRoot for different paths in the same enclosing root " + + "returns the same path", + fs.getEnclosingRoot(root), + fs.getEnclosingRoot(methodPath())); + } + + + @Test + public void testEnclosingRootPathExists() throws Exception { + FileSystem fs = getFileSystem(); + Path root = path("/"); + Path foobar = methodPath(); + fs.mkdirs(foobar); + + assertEquals( + "Ensure getEnclosingRoot returns the root directory when the root directory exists", + root, fs.getEnclosingRoot(foobar)); + assertEquals("Ensure getEnclosingRoot returns the root directory when the directory exists", + root, fs.getEnclosingRoot(foobar)); + } + + @Test + public void testEnclosingRootPathDNE() throws Exception { + FileSystem fs = getFileSystem(); + Path foobar = path("/foo/bar"); + Path root = path("/"); + + // . + assertEquals( + "Ensure getEnclosingRoot returns the root directory even when the path does not exist", + root, fs.getEnclosingRoot(foobar)); + assertEquals( + "Ensure getEnclosingRoot returns the root directory even when the path does not exist", + root, fs.getEnclosingRoot(methodPath())); + } + + @Test + public void testEnclosingRootWrapped() throws Exception { + FileSystem fs = getFileSystem(); + Path root = path("/"); + + assertEquals("Ensure getEnclosingRoot returns the root directory when the directory exists", + root, fs.getEnclosingRoot(new Path("/foo/bar"))); + + UserGroupInformation ugi = UserGroupInformation.createRemoteUser("foo"); + Path p = ugi.doAs((PrivilegedExceptionAction) () -> { + FileSystem wFs = getContract().getTestFileSystem(); + return wFs.getEnclosingRoot(new Path("/foo/bar")); + }); + assertEquals("Ensure getEnclosingRoot works correctly within a wrapped FileSystem", root, p); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractGetEnclosingRoot.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractGetEnclosingRoot.java new file mode 100644 index 0000000000000..9819245ba6867 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractGetEnclosingRoot.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs.contract.localfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractGetEnclosingRoot; +import org.apache.hadoop.fs.contract.AbstractFSContract; + + +public class TestLocalFSContractGetEnclosingRoot + extends AbstractContractGetEnclosingRoot { + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new LocalFSContract(conf); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractGetEnclosingRoot.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractGetEnclosingRoot.java new file mode 100644 index 0000000000000..7e99cb7b88ed6 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractGetEnclosingRoot.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs.contract.rawlocal; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractGetEnclosingRoot; +import org.apache.hadoop.fs.contract.AbstractFSContract; + + +public class TestRawlocalContractGetEnclosingRoot extends AbstractContractGetEnclosingRoot { + + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new RawlocalFSContract(conf); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 116ed41703858..cbe7516b0ede0 100755 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -3529,4 +3529,14 @@ public DatanodeInfo[] slowDatanodeReport() throws IOException { } } + public Path getEnclosingRoot(String src) throws IOException { + checkOpen(); + try (TraceScope ignored = newPathTraceScope("getEnclosingRoot", src)) { + return namenode.getEnclosingRoot(src); + } catch (RemoteException re) { + throw re.unwrapRemoteException(AccessControlException.class, + UnresolvedPathException.class); + } + } + } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java index 05d9c562392cf..3fbd40d41d01a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java @@ -64,6 +64,7 @@ public enum OpType { GET_EC_CODECS("op_get_ec_codecs"), GET_EC_POLICY("op_get_ec_policy"), GET_EC_POLICIES("op_get_ec_policies"), + GET_ENCLOSING_ROOT("op_get_enclosing_root"), GET_ENCRYPTION_ZONE("op_get_encryption_zone"), GET_FILE_BLOCK_LOCATIONS("op_get_file_block_locations"), GET_FILE_CHECKSUM(CommonStatisticNames.OP_GET_FILE_CHECKSUM), diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 16093c8e9920c..17c39f6c55b75 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -4011,4 +4011,31 @@ public LocatedBlocks next(final FileSystem fs, final Path p) } }.resolve(this, absF); } + + /** + * Return path of the enclosing root for a given path + * The enclosing root path is a common ancestor that should be used for temp and staging dirs + * as well as within encryption zones and other restricted directories. + * + * @param path file path to find the enclosing root path for + * @return a path to the enclosing root + * @throws IOException early checks like failure to resolve path cause IO failures + */ + public Path getEnclosingRoot(final Path path) throws IOException { + statistics.incrementReadOps(1); + storageStatistics.incrementOpCounter(OpType.GET_ENCLOSING_ROOT); + Preconditions.checkNotNull(path); + Path absF = fixRelativePart(path); + return new FileSystemLinkResolver() { + @Override + public Path doCall(final Path p) throws IOException { + return dfs.getEnclosingRoot(getPathName(p)); + } + + @Override + public Path next(final FileSystem fs, final Path p) throws IOException { + return fs.getEnclosingRoot(p); + } + }.resolve(this, absF); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java index 4f2da496a1a3d..b56b7916ff798 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java @@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.crypto.CryptoProtocolVersion; import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; import org.apache.hadoop.ha.HAServiceProtocol; import org.apache.hadoop.hdfs.AddBlockFlag; @@ -1888,4 +1889,11 @@ BatchedEntries listOpenFiles(long prevId, @ReadOnly DatanodeInfo[] getSlowDatanodeReport() throws IOException; + /** + * Get the enclosing root for a path. + */ + @Idempotent + @ReadOnly(isCoordinated = true) + Path getEnclosingRoot(String src) throws IOException; + } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java index ae4a84ead6552..543f0a58e6ec6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FsServerDefaults; import org.apache.hadoop.fs.Options.Rename; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.QuotaUsage; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.XAttr; @@ -127,6 +128,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEnclosingRootRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEnclosingRootResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto; @@ -1669,4 +1672,19 @@ public HAServiceProtocol.HAServiceState getHAServiceState() } } + @Override + public Path getEnclosingRoot(String filename) throws IOException { + final GetEnclosingRootRequestProto.Builder builder = + GetEnclosingRootRequestProto.newBuilder(); + builder.setFilename(filename); + final GetEnclosingRootRequestProto req = builder.build(); + try { + final GetEnclosingRootResponseProto response = + rpcProxy.getEnclosingRoot(null, req); + return new Path(response.getEnclosingRootPath()); + } catch (ServiceException e) { + throw getRemoteException(e); + } + } + } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto index 60792b5b6c94c..5ad3fe96f08f1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto @@ -428,6 +428,14 @@ message GetPreferredBlockSizeResponseProto { message GetSlowDatanodeReportRequestProto { } +message GetEnclosingRootRequestProto { + optional string filename = 1; +} + +message GetEnclosingRootResponseProto { + optional string enclosingRootPath = 1; +} + message GetSlowDatanodeReportResponseProto { repeated DatanodeInfoProto datanodeInfoProto = 1; } @@ -1080,4 +1088,6 @@ service ClientNamenodeProtocol { returns(HAServiceStateResponseProto); rpc getSlowDatanodeReport(GetSlowDatanodeReportRequestProto) returns(GetSlowDatanodeReportResponseProto); + rpc getEnclosingRoot(GetEnclosingRootRequestProto) + returns(GetEnclosingRootResponseProto); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestReadOnly.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestReadOnly.java index f7ea7bcd76dde..c225a98c8244a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestReadOnly.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestReadOnly.java @@ -56,6 +56,7 @@ public class TestReadOnly { "listCachePools", "getAclStatus", "getEZForPath", + "getEnclosingRoot", "listEncryptionZones", "listReencryptionStatus", "getXAttrs", diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java index ba0abc11e0276..1f18608deda21 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java @@ -147,6 +147,9 @@ public class RouterClientProtocol implements ClientProtocol { /** Time out when getting the mount statistics. */ private long mountStatusTimeOut; + /** Default nameservice enabled. */ + private final boolean defaultNameServiceEnabled; + /** Identifier for the super user. */ private String superUser; /** Identifier for the super group. */ @@ -196,6 +199,9 @@ public class RouterClientProtocol implements ClientProtocol { this.routerCacheAdmin = new RouterCacheAdmin(rpcServer); this.securityManager = rpcServer.getRouterSecurityManager(); this.rbfRename = new RouterFederationRename(rpcServer, conf); + this.defaultNameServiceEnabled = conf.getBoolean( + RBFConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE_ENABLE, + RBFConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE_ENABLE_DEFAULT); } @Override @@ -1967,6 +1973,33 @@ public DatanodeInfo[] getSlowDatanodeReport() throws IOException { return rpcServer.getSlowDatanodeReport(true, 0); } + @Override + public Path getEnclosingRoot(String src) throws IOException { + Path mountPath = null; + if (defaultNameServiceEnabled) { + mountPath = new Path("/"); + } + + if (subclusterResolver instanceof MountTableResolver) { + MountTableResolver mountTable = (MountTableResolver) subclusterResolver; + if (mountTable.getMountPoint(src) != null) { + mountPath = new Path(mountTable.getMountPoint(src).getSourcePath()); + } + } + + if (mountPath == null) { + throw new IOException(String.format("No mount point for %s", src)); + } + + EncryptionZone zone = getEZForPath(src); + if (zone == null) { + return mountPath; + } else { + Path zonePath = new Path(zone.getPath()); + return zonePath.depth() > mountPath.depth() ? zonePath : mountPath; + } + } + @Override public HAServiceProtocol.HAServiceState getHAServiceState() { if (rpcServer.isSafeMode()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java index 2aa2eae5305d0..9d7c1263f09bf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java @@ -1610,6 +1610,11 @@ public DatanodeInfo[] getSlowDatanodeReport() throws IOException { return clientProto.getSlowDatanodeReport(); } + @Override // ClientProtocol + public Path getEnclosingRoot(String src) throws IOException { + return clientProto.getEnclosingRoot(src); + } + @Override // NamenodeProtocol public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size, long minBlockSize, long hotBlockTimeInterval, StorageType storageType) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTable.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTable.java index a346c1a241a80..8812f83febb80 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTable.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTable.java @@ -763,4 +763,29 @@ public void testListStatusMountPoint() throws Exception { nnFs0.delete(new Path("/testLsMountEntryDest"), true); } } + + @Test + public void testGetEnclosingRoot() throws Exception { + + // Add a read only entry + MountTable readOnlyEntry = MountTable.newInstance( + "/readonly", Collections.singletonMap("ns0", "/testdir")); + readOnlyEntry.setReadOnly(true); + assertTrue(addMountTable(readOnlyEntry)); + assertEquals(routerFs.getEnclosingRoot(new Path("/readonly")), new Path("/readonly")); + + assertEquals(routerFs.getEnclosingRoot(new Path("/regular")), new Path("/")); + assertEquals(routerFs.getEnclosingRoot(new Path("/regular")), + routerFs.getEnclosingRoot(routerFs.getEnclosingRoot(new Path("/regular")))); + + // Add a regular entry + MountTable regularEntry = MountTable.newInstance( + "/regular", Collections.singletonMap("ns0", "/testdir")); + assertTrue(addMountTable(regularEntry)); + assertEquals(routerFs.getEnclosingRoot(new Path("/regular")), new Path("/regular")); + + // path does not need to exist + assertEquals(routerFs.getEnclosingRoot(new Path("/regular/pathDNE")), new Path("/regular")); + + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java index 79c122cf5bae0..25fcdc3080df0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java @@ -25,8 +25,6 @@ import java.util.Map; import java.util.stream.Collectors; -import org.apache.hadoop.thirdparty.protobuf.ByteString; -import org.apache.hadoop.thirdparty.protobuf.ProtocolStringList; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries; @@ -40,6 +38,7 @@ import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FsServerDefaults; import org.apache.hadoop.fs.Options.Rename; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.QuotaUsage; import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse; import org.apache.hadoop.hdfs.protocol.BatchedDirectoryListing; @@ -134,6 +133,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEnclosingRootRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEnclosingRootResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto; import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto; @@ -303,7 +304,8 @@ import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto; import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto; import org.apache.hadoop.security.token.Token; - +import org.apache.hadoop.thirdparty.protobuf.ByteString; +import org.apache.hadoop.thirdparty.protobuf.ProtocolStringList; import org.apache.hadoop.thirdparty.protobuf.RpcController; import org.apache.hadoop.thirdparty.protobuf.ServiceException; @@ -2074,4 +2076,18 @@ public GetSlowDatanodeReportResponseProto getSlowDatanodeReport(RpcController co throw new ServiceException(e); } } + + @Override + public GetEnclosingRootResponseProto getEnclosingRoot( + RpcController controller, GetEnclosingRootRequestProto req) + throws ServiceException { + try { + Path enclosingRootPath = server.getEnclosingRoot(req.getFilename()); + return GetEnclosingRootResponseProto.newBuilder() + .setEnclosingRootPath(enclosingRootPath.toUri().toString()) + .build(); + } catch (IOException e) { + throw new ServiceException(e); + } + } } 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 7918daf6b9db8..3c6efb78dd1fd 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 @@ -9109,4 +9109,19 @@ private void checkBlockLocationsWhenObserver(LocatedBlocks blocks, String src) } } } + + /** + * Get the enclosing root for the specified path. + * + * @param srcArg the path of a file or directory to get the EZ for. + * @return the enclosing root of the path or null if none. + */ + Path getEnclosingRoot(final String srcArg) throws IOException { + EncryptionZone ez = getEZForPath(srcArg); + if (ez != null) { + return new Path(ez.getPath()); + } else { + return new Path("/"); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java index f02688d1629f4..3a8357f4bddf4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java @@ -2677,4 +2677,11 @@ public Long getNextSPSPath() throws IOException { } return namesystem.getBlockManager().getSPSManager().getNextPathId(); } + + @Override // ClientProtocol + public Path getEnclosingRoot(String src) + throws IOException { + checkNNStartup(); + return namesystem.getEnclosingRoot(src); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java index 62dc3076d00a6..afa6ccaf2dad4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java @@ -57,6 +57,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY; import static org.apache.hadoop.fs.FileSystem.TRASH_PREFIX; +import org.apache.hadoop.test.LambdaTestUtils; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -191,34 +192,38 @@ Path getTrashRootInFallBackFS() throws IOException { @Test public void testTrashRootsAfterEncryptionZoneDeletion() throws Exception { - final Path zone = new Path("/EZ"); - fsTarget.mkdirs(zone); - final Path zone1 = new Path("/EZ/zone1"); - fsTarget.mkdirs(zone1); - - DFSTestUtil.createKey("test_key", cluster, CONF); - HdfsAdmin hdfsAdmin = new HdfsAdmin(cluster.getURI(0), CONF); - final EnumSet provisionTrash = - EnumSet.of(CreateEncryptionZoneFlag.PROVISION_TRASH); - hdfsAdmin.createEncryptionZone(zone1, "test_key", provisionTrash); - - final Path encFile = new Path(zone1, "encFile"); - DFSTestUtil.createFile(fsTarget, encFile, 10240, (short) 1, 0xFEED); - - Configuration clientConf = new Configuration(CONF); - clientConf.setLong(FS_TRASH_INTERVAL_KEY, 1); - clientConf.set("fs.default.name", fsTarget.getUri().toString()); - FsShell shell = new FsShell(clientConf); - - //Verify file deletion within EZ - DFSTestUtil.verifyDelete(shell, fsTarget, encFile, true); - assertTrue("ViewFileSystem trash roots should include EZ file trash", - (fsView.getTrashRoots(true).size() == 1)); - - //Verify deletion of EZ - DFSTestUtil.verifyDelete(shell, fsTarget, zone, true); - assertTrue("ViewFileSystem trash roots should include EZ zone trash", - (fsView.getTrashRoots(true).size() == 2)); + try { + final Path zone = new Path("/EZ"); + fsTarget.mkdirs(zone); + final Path zone1 = new Path("/EZ/zone1"); + fsTarget.mkdirs(zone1); + + DFSTestUtil.createKey("test_key", cluster, CONF); + HdfsAdmin hdfsAdmin = new HdfsAdmin(cluster.getURI(0), CONF); + final EnumSet provisionTrash = + EnumSet.of(CreateEncryptionZoneFlag.PROVISION_TRASH); + hdfsAdmin.createEncryptionZone(zone1, "test_key", provisionTrash); + + final Path encFile = new Path(zone1, "encFile"); + DFSTestUtil.createFile(fsTarget, encFile, 10240, (short) 1, 0xFEED); + + Configuration clientConf = new Configuration(CONF); + clientConf.setLong(FS_TRASH_INTERVAL_KEY, 1); + clientConf.set("fs.default.name", fsTarget.getUri().toString()); + FsShell shell = new FsShell(clientConf); + + //Verify file deletion within EZ + DFSTestUtil.verifyDelete(shell, fsTarget, encFile, true); + assertTrue("ViewFileSystem trash roots should include EZ file trash", + (fsView.getTrashRoots(true).size() == 1)); + + //Verify deletion of EZ + DFSTestUtil.verifyDelete(shell, fsTarget, zone, true); + assertTrue("ViewFileSystem trash roots should include EZ zone trash", + (fsView.getTrashRoots(true).size() == 2)); + } finally { + DFSTestUtil.deleteKey("test_key", cluster); + } } @Test @@ -506,4 +511,92 @@ public void testInternalDirectoryPermissions() throws IOException { assertEquals(fs.getFileStatus(subDirOfInternalDir).getPermission(), fs.getFileStatus(subDirOfRealDir).getPermission()); } + + private Path getViewFsPath(Path path, FileSystem fs) { + return fs.makeQualified(path); + } + + private Path getViewFsPath(String path, FileSystem fs) { + return getViewFsPath(new Path(path), fs); + } + + @Test + public void testEnclosingRootsBase() throws Exception { + try { + final Path zone = new Path("/data/EZ"); + fsTarget.mkdirs(zone); + final Path zone1 = new Path("/data/EZ/zone1"); + fsTarget.mkdirs(zone1); + + DFSTestUtil.createKey("test_key", cluster, 0, CONF); + HdfsAdmin hdfsAdmin = new HdfsAdmin(cluster.getURI(0), CONF); + final EnumSet provisionTrash = + EnumSet.of(CreateEncryptionZoneFlag.PROVISION_TRASH); + hdfsAdmin.createEncryptionZone(zone1, "test_key", provisionTrash); + assertEquals(fsView.getEnclosingRoot(zone), getViewFsPath("/data", fsView)); + assertEquals(fsView.getEnclosingRoot(zone1), getViewFsPath(zone1, fsView)); + + Path nn02Ez = new Path("/mountOnNn2/EZ"); + fsTarget2.mkdirs(nn02Ez); + assertEquals(fsView.getEnclosingRoot((nn02Ez)), getViewFsPath("/mountOnNn2", fsView)); + HdfsAdmin hdfsAdmin2 = new HdfsAdmin(cluster.getURI(1), CONF); + DFSTestUtil.createKey("test_key", cluster, 1, CONF); + hdfsAdmin2.createEncryptionZone(nn02Ez, "test_key", provisionTrash); + assertEquals(fsView.getEnclosingRoot((nn02Ez)), getViewFsPath(nn02Ez, fsView)); + assertEquals(fsView.getEnclosingRoot(new Path(nn02Ez, "dir/dir2/file")), + getViewFsPath(nn02Ez, fsView)); + + // With viewfs:// scheme + assertEquals(fsView.getEnclosingRoot(fsView.getWorkingDirectory()), + getViewFsPath("/user", fsView)); + } finally { + DFSTestUtil.deleteKey("test_key", cluster, 0); + } + } + + @Test + public void testEnclosingRootFailure() throws Exception { + LambdaTestUtils.intercept(NotInMountpointException.class, + ()-> fsView.getEnclosingRoot(new Path("/does/not/exist"))); + + final Path zone = new Path("/data/EZ"); + Path fs1 = fsTarget.makeQualified(zone); + + LambdaTestUtils.intercept(IllegalArgumentException.class, + ()-> fsView.getEnclosingRoot(fs1)); + LambdaTestUtils.intercept(IllegalArgumentException.class, + ()-> fsView.getEnclosingRoot(new Path("hdfs://fakeAuthority/"))); + } + + @Test + public void testEnclosingRootWrapped() throws Exception { + try { + final Path zone = new Path("/data/EZ"); + fsTarget.mkdirs(zone); + final Path zone1 = new Path("/data/EZ/testZone1"); + fsTarget.mkdirs(zone1); + + DFSTestUtil.createKey("test_key", cluster, 0, CONF); + HdfsAdmin hdfsAdmin = new HdfsAdmin(cluster.getURI(0), CONF); + final EnumSet provisionTrash = + EnumSet.of(CreateEncryptionZoneFlag.PROVISION_TRASH); + hdfsAdmin.createEncryptionZone(zone1, "test_key", provisionTrash); + + UserGroupInformation ugi = UserGroupInformation.createRemoteUser("foo"); + Path p = ugi.doAs((PrivilegedExceptionAction) () -> { + FileSystem wFs = FileSystem.get(FsConstants.VIEWFS_URI, this.conf); + return wFs.getEnclosingRoot(zone); + }); + assertEquals(p, getViewFsPath("/data", fsView)); + p = ugi.doAs((PrivilegedExceptionAction) () -> { + FileSystem wFs = FileSystem.get(FsConstants.VIEWFS_URI, this.conf); + return wFs.getEnclosingRoot(zone1); + }); + assertEquals(p, getViewFsPath(zone1, fsView)); + + + } finally { + DFSTestUtil.deleteKey("test_key", cluster, 0); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java index 594b46f6cacb0..e816edd3110de 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java @@ -1875,6 +1875,33 @@ public static void createKey(String keyName, MiniDFSCluster cluster, provider.flush(); } + /** + * Helper function to delete a key in the Key Provider. Defaults + * to the first indexed NameNode's Key Provider. + * + * @param keyName The name of the key to create + * @param cluster The cluster to create it in + */ + public static void deleteKey(String keyName, MiniDFSCluster cluster) + throws NoSuchAlgorithmException, IOException { + deleteKey(keyName, cluster, 0); + } + + /** + * Helper function to delete a key in the Key Provider. + * + * @param keyName The name of the key to create + * @param cluster The cluster to create it in + * @param idx The NameNode index + */ + public static void deleteKey(String keyName, MiniDFSCluster cluster, int idx) + throws NoSuchAlgorithmException, IOException { + NameNode nn = cluster.getNameNode(idx); + KeyProvider provider = nn.getNamesystem().getProvider(); + provider.deleteKey(keyName); + provider.flush(); + } + /** * @return the node which is expected to run the recovery of the * given block, which is known to be under construction inside the diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java index 9e8c11d7b0660..8eb048c14235c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java @@ -969,6 +969,11 @@ public void testStatistics2() throws IOException, NoSuchAlgorithmException { checkStatistics(dfs, ++readOps, writeOps, 0); checkOpStatistics(OpType.GET_ENCRYPTION_ZONE, opCount + 1); + opCount = getOpStatistics(OpType.GET_ENCLOSING_ROOT); + dfs.getEnclosingRoot(dir); + checkStatistics(dfs, ++readOps, writeOps, 0); + checkOpStatistics(OpType.GET_ENCLOSING_ROOT, opCount + 1); + opCount = getOpStatistics(OpType.GET_SNAPSHOTTABLE_DIRECTORY_LIST); dfs.getSnapshottableDirListing(); checkStatistics(dfs, ++readOps, writeOps, 0); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEnclosingRoot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEnclosingRoot.java new file mode 100644 index 0000000000000..85204e7a1a5b3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEnclosingRoot.java @@ -0,0 +1,149 @@ +/** + * 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 java.io.File; +import java.util.EnumSet; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.crypto.key.JavaKeyStoreProvider; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileSystemTestHelper; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag; +import org.apache.hadoop.hdfs.client.HdfsAdmin; +import org.apache.hadoop.hdfs.server.namenode.EncryptionFaultInjector; +import org.apache.hadoop.hdfs.server.namenode.EncryptionZoneManager; +import org.apache.hadoop.test.AbstractHadoopTestBase; +import org.apache.hadoop.test.GenericTestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.slf4j.event.Level; + +import static org.assertj.core.api.Assertions.assertThat; + +public class TestEnclosingRoot extends AbstractHadoopTestBase { + private static final Logger LOG = LoggerFactory.getLogger(TestEnclosingRoot.class); + private static final String TEST_KEY = "test_key"; + private static final EnumSet NO_TRASH = + EnumSet.of(CreateEncryptionZoneFlag.NO_TRASH); + + private Configuration conf; + private FileSystemTestHelper fsHelper; + + private MiniDFSCluster cluster; + private HdfsAdmin dfsAdmin; + private DistributedFileSystem fs; + private File testRootDir; + + private String getKeyProviderURI() { + return JavaKeyStoreProvider.SCHEME_NAME + "://file" + + new Path(testRootDir.toString(), "test.jks").toUri(); + } + + @Before + public void setup() throws Exception { + conf = new HdfsConfiguration(); + fsHelper = new FileSystemTestHelper(); + // Set up java key store + String testRoot = fsHelper.getTestRootDir(); + testRootDir = new File(testRoot).getAbsoluteFile(); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH, + getKeyProviderURI()); + conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true); + // Lower the batch size for testing + conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES, + 2); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); + cluster.waitActive(); + GenericTestUtils.setLogLevel( + LoggerFactory.getLogger(EncryptionZoneManager.class), Level.TRACE); + fs = cluster.getFileSystem(); + dfsAdmin = new HdfsAdmin(cluster.getURI(), conf); + setProvider(); + // Create a test key + DFSTestUtil.createKey(TEST_KEY, cluster, conf); + } + + protected void setProvider() { + // Need to set the client's KeyProvider to the NN's for JKS, + // else the updates do not get flushed properly + fs.getClient().setKeyProvider(cluster.getNameNode().getNamesystem() + .getProvider()); + } + + @After + public void teardown() { + try { + if (cluster != null) { + cluster.shutdown(); + cluster = null; + } + } finally { + EncryptionFaultInjector.instance = new EncryptionFaultInjector(); + } + } + + @Test + /** + * Testing basic operations for getEnclosingRoot with dfs/DistributedFileSystem + */ + public void testBasicOperations() throws Exception { + final Path rootDir = new Path("/"); + final Path zone1 = new Path(rootDir, "zone1"); + + // Ensure that the root "/" returns the root without mount points or encryption zones + assertThat(fs.getEnclosingRoot(rootDir)) + .describedAs("enclosing root of %s", rootDir) + .isEqualTo(rootDir); + + // Ensure a dir returns the root without mount points or encryption zones + assertThat(fs.getEnclosingRoot(zone1)) + .describedAs("enclosing root of %s", zone1) + .isEqualTo(rootDir); + + // create an encryption zone + fs.mkdirs(zone1); + dfsAdmin.createEncryptionZone(zone1, TEST_KEY, NO_TRASH); + + // Ensure that the root "/" returns the root with an encryption zone present + assertThat(fs.getEnclosingRoot(rootDir)) + .describedAs("enclosing root of %s", rootDir) + .isEqualTo(rootDir); + + // Ensure that the encryption zone path itself returns correctly as itself + assertThat(fs.getEnclosingRoot(zone1)) + .describedAs("enclosing root of %s", zone1) + .isEqualTo(zone1); + + // Ensure that a path where the file does not exist returns the encryption zone root path + final Path zone1FileDNE = new Path(zone1, "newDNE.txt"); + assertThat(fs.getEnclosingRoot(zone1FileDNE)) + .describedAs("enclosing root of %s", zone1FileDNE) + .isEqualTo(zone1); + + // Ensure that a path where the dir does not exist returns the encryption zone root path + final Path zone1DirDNE = new Path(zone1, "zone2/newDNE.txt"); + assertThat(fs.getEnclosingRoot(zone1DirDNE)) + .describedAs("enclosing root of %s", zone1DirDNE) + .isEqualTo(zone1); + } +} From a32097a921b6a256c82ee8c2a83aa1990c635e0d Mon Sep 17 00:00:00 2001 From: K0K0V0K <109747532+K0K0V0K@users.noreply.github.com> Date: Thu, 9 Nov 2023 10:14:14 +0100 Subject: [PATCH 02/10] HADOOP-18954. Filter NaN values from JMX json interface. (#6229). Reviewed-by: Ferenc Erdelyi Signed-off-by: He Xiaoqiao --- .../fs/CommonConfigurationKeysPublic.java | 8 +++ .../org/apache/hadoop/http/HttpServer2.java | 15 ++++- .../org/apache/hadoop/jmx/JMXJsonServlet.java | 51 ++++++++++----- .../hadoop/jmx/JMXJsonServletNaNFiltered.java | 49 ++++++++++++++ .../src/main/resources/core-default.xml | 12 ++++ .../apache/hadoop/jmx/TestJMXJsonServlet.java | 9 ++- .../jmx/TestJMXJsonServletNaNFiltered.java | 65 +++++++++++++++++++ .../metrics2/lib/TestMutableMetrics.java | 2 + 8 files changed, 190 insertions(+), 21 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServletNaNFiltered.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/jmx/TestJMXJsonServletNaNFiltered.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java index 006144e64ad15..24a3167b3db2d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java @@ -1076,5 +1076,13 @@ public class CommonConfigurationKeysPublic { public static final String IPC_SERVER_METRICS_UPDATE_RUNNER_INTERVAL = "ipc.server.metrics.update.runner.interval"; public static final int IPC_SERVER_METRICS_UPDATE_RUNNER_INTERVAL_DEFAULT = 5000; + + /** + * @see + * + * core-default.xml + */ + public static final String JMX_NAN_FILTER = "hadoop.http.jmx.nan-filter.enabled"; + public static final boolean JMX_NAN_FILTER_DEFAULT = false; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java index 515148e929817..d4bfa41b21c4c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java @@ -56,6 +56,7 @@ import javax.servlet.http.HttpServletResponse; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.jmx.JMXJsonServletNaNFiltered; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; import com.sun.jersey.spi.container.servlet.ServletContainer; @@ -117,6 +118,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.JMX_NAN_FILTER; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.JMX_NAN_FILTER_DEFAULT; + /** * Create a Jetty embedded server to answer http requests. The primary goal is * to serve up status information for the server. There are three contexts: @@ -785,7 +789,7 @@ private void initializeWebServer(String name, String hostName, } } - addDefaultServlets(); + addDefaultServlets(conf); addPrometheusServlet(conf); addAsyncProfilerServlet(contexts, conf); } @@ -976,12 +980,17 @@ private void setContextAttributes(ServletContextHandler context, /** * Add default servlets. + * @param configuration the hadoop configuration */ - protected void addDefaultServlets() { + protected void addDefaultServlets(Configuration configuration) { // set up default servlets addServlet("stacks", "/stacks", StackServlet.class); addServlet("logLevel", "/logLevel", LogLevel.Servlet.class); - addServlet("jmx", "/jmx", JMXJsonServlet.class); + addServlet("jmx", "/jmx", + configuration.getBoolean(JMX_NAN_FILTER, JMX_NAN_FILTER_DEFAULT) + ? JMXJsonServletNaNFiltered.class + : JMXJsonServlet.class + ); addServlet("conf", "/conf", ConfServlet.class); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java index 85f2d2828562d..f089db502783e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java @@ -17,12 +17,12 @@ package org.apache.hadoop.jmx; -import com.fasterxml.jackson.core.JsonFactory; -import com.fasterxml.jackson.core.JsonGenerator; -import org.apache.hadoop.http.HttpServer2; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - +import java.io.IOException; +import java.io.PrintWriter; +import java.lang.management.ManagementFactory; +import java.lang.reflect.Array; +import java.util.Iterator; +import java.util.Set; import javax.management.AttributeNotFoundException; import javax.management.InstanceNotFoundException; import javax.management.IntrospectionException; @@ -42,12 +42,14 @@ import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import java.io.IOException; -import java.io.PrintWriter; -import java.lang.management.ManagementFactory; -import java.lang.reflect.Array; -import java.util.Iterator; -import java.util.Set; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.NotImplementedException; +import org.apache.hadoop.http.HttpServer2; /* * This servlet is based off of the JMXProxyServlet from Tomcat 7.0.14. It has @@ -136,6 +138,7 @@ public class JMXJsonServlet extends HttpServlet { * Json Factory to create Json generators for write objects in json format */ protected transient JsonFactory jsonFactory; + /** * Initialize this servlet. */ @@ -386,10 +389,10 @@ private void writeAttribute(JsonGenerator jg, ObjectName oname, MBeanAttributeIn private void writeAttribute(JsonGenerator jg, String attName, Object value) throws IOException { jg.writeFieldName(attName); - writeObject(jg, value); + writeObject(jg, value, attName); } - private void writeObject(JsonGenerator jg, Object value) throws IOException { + private void writeObject(JsonGenerator jg, Object value, String attName) throws IOException { if(value == null) { jg.writeNull(); } else { @@ -399,9 +402,11 @@ private void writeObject(JsonGenerator jg, Object value) throws IOException { int len = Array.getLength(value); for (int j = 0; j < len; j++) { Object item = Array.get(value, j); - writeObject(jg, item); + writeObject(jg, item, attName); } jg.writeEndArray(); + } else if (extraCheck(value)) { + extraWrite(value, attName, jg); } else if(value instanceof Number) { Number n = (Number)value; jg.writeNumber(n.toString()); @@ -421,7 +426,7 @@ private void writeObject(JsonGenerator jg, Object value) throws IOException { TabularData tds = (TabularData)value; jg.writeStartArray(); for(Object entry : tds.values()) { - writeObject(jg, entry); + writeObject(jg, entry, attName); } jg.writeEndArray(); } else { @@ -429,4 +434,18 @@ private void writeObject(JsonGenerator jg, Object value) throws IOException { } } } + + /** + * In case you need to modify the logic, how java objects transforms to json, + * you can overwrite this method to return true in case special handling + * @param value the object what should be judged + * @return true, if it needs special transformation + */ + protected boolean extraCheck(Object value) { + return false; + } + + protected void extraWrite(Object value, String attName, JsonGenerator jg) throws IOException { + throw new NotImplementedException(); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServletNaNFiltered.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServletNaNFiltered.java new file mode 100644 index 0000000000000..40e3c1a168f9d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServletNaNFiltered.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.jmx; + +import java.io.IOException; +import java.util.Objects; + +import com.fasterxml.jackson.core.JsonGenerator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * For example in case of MutableGauge we are using numbers, + * but not implementing Number interface, + * so we skip class check here because we can not be sure NaN values are wrapped + * with classes which implements the Number interface + */ +public class JMXJsonServletNaNFiltered extends JMXJsonServlet { + + private static final Logger LOG = + LoggerFactory.getLogger(JMXJsonServletNaNFiltered.class); + + @Override + protected boolean extraCheck(Object value) { + return Objects.equals("NaN", Objects.toString(value).trim()); + } + + @Override + protected void extraWrite(Object value, String attName, JsonGenerator jg) throws IOException { + LOG.debug("The {} attribute with value: {} was identified as NaN " + + "and will be replaced with 0.0", attName, value); + jg.writeNumber(0.0); + } +} 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 31d980353bf26..c86fd8b98609b 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 @@ -65,6 +65,18 @@ + + hadoop.http.jmx.nan-filter.enabled + false + + The REST API of the JMX interface can return with NaN values + if the attribute represent a 0.0/0.0 value. + Some JSON parser by default can not parse json attributes like foo:NaN. + If this filter is enabled the NaN values will be converted to 0.0 values, + to make json parse less complicated. + + + diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/jmx/TestJMXJsonServlet.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/jmx/TestJMXJsonServlet.java index 035090ef65b72..ba7de6f437ee5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/jmx/TestJMXJsonServlet.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/jmx/TestJMXJsonServlet.java @@ -62,10 +62,15 @@ public static void assertReFind(String re, String value) { result = readOutput(new URL(baseUrl, "/jmx?qry=java.lang:type=Memory")); assertReFind("\"name\"\\s*:\\s*\"java.lang:type=Memory\"", result); assertReFind("\"modelerType\"", result); - + + System.setProperty("THE_TEST_OF_THE_NAN_VALUES", String.valueOf(Float.NaN)); result = readOutput(new URL(baseUrl, "/jmx")); assertReFind("\"name\"\\s*:\\s*\"java.lang:type=Memory\"", result); - + assertReFind( + "\"key\"\\s*:\\s*\"THE_TEST_OF_THE_NAN_VALUES\"\\s*,\\s*\"value\"\\s*:\\s*\"NaN\"", + result + ); + // test to get an attribute of a mbean result = readOutput(new URL(baseUrl, "/jmx?get=java.lang:type=Memory::HeapMemoryUsage")); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/jmx/TestJMXJsonServletNaNFiltered.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/jmx/TestJMXJsonServletNaNFiltered.java new file mode 100644 index 0000000000000..52a52be80a35c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/jmx/TestJMXJsonServletNaNFiltered.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.jmx; + +import java.net.URL; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.http.HttpServer2; +import org.apache.hadoop.http.HttpServerFunctionalTest; + +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.JMX_NAN_FILTER; + +public class TestJMXJsonServletNaNFiltered extends HttpServerFunctionalTest { + private static HttpServer2 server; + private static URL baseUrl; + + @BeforeClass public static void setup() throws Exception { + Configuration configuration = new Configuration(); + configuration.setBoolean(JMX_NAN_FILTER, true); + server = createTestServer(configuration); + server.start(); + baseUrl = getServerURL(server); + } + + @AfterClass public static void cleanup() throws Exception { + server.stop(); + } + + public static void assertReFind(String re, String value) { + Pattern p = Pattern.compile(re); + Matcher m = p.matcher(value); + assertTrue("'"+p+"' does not match "+value, m.find()); + } + + @Test public void testQuery() throws Exception { + System.setProperty("THE_TEST_OF_THE_NAN_VALUES", String.valueOf(Float.NaN)); + String result = readOutput(new URL(baseUrl, "/jmx")); + assertReFind("\"name\"\\s*:\\s*\"java.lang:type=Memory\"", result); + assertReFind( + "\"key\"\\s*:\\s*\"THE_TEST_OF_THE_NAN_VALUES\"\\s*,\\s*\"value\"\\s*:\\s*0.0", + result + ); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/lib/TestMutableMetrics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/lib/TestMutableMetrics.java index 2cc3d706f0c20..1ebc0cbdbf23d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/lib/TestMutableMetrics.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/lib/TestMutableMetrics.java @@ -634,5 +634,7 @@ public void testMutableGaugeFloat() { assertEquals(3.2f, mgf.value(), 0.0); mgf.incr(); assertEquals(4.2f, mgf.value(), 0.0); + mgf.set(Float.NaN); + assertEquals(Float.NaN, mgf.value(), 0.0); } } From 342e6caba189a5639a26ff3f27ac6af7ef5bd881 Mon Sep 17 00:00:00 2001 From: LiuGuH <444506464@qq.com> Date: Tue, 14 Nov 2023 02:16:31 +0800 Subject: [PATCH 03/10] HDFS-17249. Fix TestDFSUtil.testIsValidName() unit test failure (#6249) Contributed by liuguanghua. --- .../org/apache/hadoop/hdfs/DFSUtilClient.java | 7 +++-- .../org/apache/hadoop/hdfs/TestDFSUtil.java | 27 ++++++++++++++----- 2 files changed, 25 insertions(+), 9 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java index 71cff2e3915b0..b2fc472aad835 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java @@ -661,9 +661,12 @@ public static boolean isValidName(String src) { String[] components = StringUtils.split(src, '/'); for (int i = 0; i < components.length; i++) { String element = components[i]; + // For Windows, we must allow the : in the drive letter. + if (Shell.WINDOWS && i == 1 && element.endsWith(":")) { + continue; + } if (element.equals(".") || - // For Windows, we must allow the : in the drive letter. - (!Shell.WINDOWS && i == 1 && element.contains(":")) || + (element.contains(":")) || (element.contains("/"))) { return false; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java index f8e8e4120c43f..5d7110d3d9a8b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java @@ -83,6 +83,7 @@ import org.apache.hadoop.security.alias.JavaKeyStoreProvider; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; +import org.apache.hadoop.util.Shell; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -865,13 +866,25 @@ public void testLocalhostReverseLookup() { @Test (timeout=15000) public void testIsValidName() { - assertFalse(DFSUtil.isValidName("/foo/../bar")); - assertFalse(DFSUtil.isValidName("/foo/./bar")); - assertFalse(DFSUtil.isValidName("/foo//bar")); - assertTrue(DFSUtil.isValidName("/")); - assertTrue(DFSUtil.isValidName("/bar/")); - assertFalse(DFSUtil.isValidName("/foo/:/bar")); - assertFalse(DFSUtil.isValidName("/foo:bar")); + String validPaths[] = new String[]{"/", "/bar/"}; + for (String path : validPaths) { + assertTrue("Should have been accepted '" + path + "'", DFSUtil.isValidName(path)); + } + + String invalidPaths[] = + new String[]{"/foo/../bar", "/foo/./bar", "/foo//bar", "/foo/:/bar", "/foo:bar"}; + for (String path : invalidPaths) { + assertFalse("Should have been rejected '" + path + "'", DFSUtil.isValidName(path)); + } + + String windowsPath = "/C:/foo/bar"; + if (Shell.WINDOWS) { + assertTrue("Should have been accepted '" + windowsPath + "' in windows os.", + DFSUtil.isValidName(windowsPath)); + } else { + assertFalse("Should have been rejected '" + windowsPath + "' in unix os.", + DFSUtil.isValidName(windowsPath)); + } } @Test(timeout=5000) From 000a39ba2d2131ac158e23b35eae8c1329681bff Mon Sep 17 00:00:00 2001 From: Anuj Modi <128447756+anujmodi2021@users.noreply.github.com> Date: Mon, 13 Nov 2023 11:36:33 -0800 Subject: [PATCH 04/10] HADOOP-18872: [ABFS] [BugFix] Misreporting Retry Count for Sub-sequential and Parallel Operations (#6019) Contributed by Anuj Modi --- .../fs/azurebfs/services/AbfsClient.java | 201 ++++++++++-------- .../azurebfs/services/AbfsRestOperation.java | 33 ++- .../ITestAzureBlobFileSystemDelete.java | 31 ++- .../ITestAzureBlobFileSystemListStatus.java | 141 +++++++++--- .../fs/azurebfs/TestTracingContext.java | 76 +++---- .../azurebfs/services/AbfsClientTestUtil.java | 162 ++++++++++++++ .../fs/azurebfs/services/ITestAbfsClient.java | 3 +- .../services/ITestAbfsRestOperation.java | 1 + .../TestAbfsRestOperationMockFailures.java | 47 +--- 9 files changed, 502 insertions(+), 193 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java 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 45cb538d0b007..9c1f590da9c5a 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 @@ -260,26 +260,25 @@ AbfsUriQueryBuilder createDefaultUriQueryBuilder() { return abfsUriQueryBuilder; } - public AbfsRestOperation createFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException { + public AbfsRestOperation createFilesystem(TracingContext tracingContext) + throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.CreateFileSystem, - this, - HTTP_METHOD_PUT, - url, - requestHeaders); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CreateFileSystem, + HTTP_METHOD_PUT, url, requestHeaders); op.execute(tracingContext); return op; } - public AbfsRestOperation setFilesystemProperties(final String properties, TracingContext tracingContext) throws AzureBlobFileSystemException { + public AbfsRestOperation setFilesystemProperties(final String properties, + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); - // JDK7 does not support PATCH, so to workaround the issue we will use + // JDK7 does not support PATCH, so to work around the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); @@ -291,9 +290,8 @@ public AbfsRestOperation setFilesystemProperties(final String properties, Tracin abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.SetFileSystemProperties, - this, HTTP_METHOD_PUT, url, requestHeaders); @@ -316,9 +314,8 @@ public AbfsRestOperation listPath(final String relativePath, final boolean recur appendSASTokenToQuery(relativePath, SASTokenProvider.LIST_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.ListPaths, - this, HTTP_METHOD_GET, url, requestHeaders); @@ -333,9 +330,8 @@ public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext) abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.GetFileSystemProperties, - this, HTTP_METHOD_HEAD, url, requestHeaders); @@ -350,9 +346,8 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.DeleteFileSystem, - this, HTTP_METHOD_DELETE, url, requestHeaders); @@ -396,9 +391,8 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.CreatePath, - this, HTTP_METHOD_PUT, url, requestHeaders); @@ -431,9 +425,8 @@ public AbfsRestOperation acquireLease(final String path, int duration, TracingCo final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.LeasePath, - this, HTTP_METHOD_POST, url, requestHeaders); @@ -451,9 +444,8 @@ public AbfsRestOperation renewLease(final String path, final String leaseId, final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.LeasePath, - this, HTTP_METHOD_POST, url, requestHeaders); @@ -471,9 +463,8 @@ public AbfsRestOperation releaseLease(final String path, final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.LeasePath, - this, HTTP_METHOD_POST, url, requestHeaders); @@ -491,9 +482,8 @@ public AbfsRestOperation breakLease(final String path, final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.LeasePath, - this, HTTP_METHOD_POST, url, requestHeaders); @@ -646,9 +636,8 @@ private boolean checkIsDir(AbfsHttpOperation result) { @VisibleForTesting AbfsRestOperation createRenameRestOperation(URL url, List requestHeaders) { - AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.RenamePath, - this, HTTP_METHOD_PUT, url, requestHeaders); @@ -766,7 +755,8 @@ public AbfsRestOperation append(final String path, final byte[] buffer, abfsUriQueryBuilder, cachedSasToken); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperationForAppend(AbfsRestOperationType.Append, + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.Append, HTTP_METHOD_PUT, url, requestHeaders, @@ -801,7 +791,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, if (reqParams.isAppendBlob() && appendSuccessCheckOp(op, path, (reqParams.getPosition() + reqParams.getLength()), tracingContext)) { - final AbfsRestOperation successOp = getAbfsRestOperationForAppend( + final AbfsRestOperation successOp = getAbfsRestOperation( AbfsRestOperationType.Append, HTTP_METHOD_PUT, url, @@ -819,38 +809,6 @@ && appendSuccessCheckOp(op, path, return op; } - /** - * Returns the rest operation for append. - * @param operationType The AbfsRestOperationType. - * @param httpMethod specifies the httpMethod. - * @param url specifies the url. - * @param requestHeaders This includes the list of request headers. - * @param buffer The buffer to write into. - * @param bufferOffset The buffer offset. - * @param bufferLength The buffer Length. - * @param sasTokenForReuse The sasToken. - * @return AbfsRestOperation op. - */ - @VisibleForTesting - AbfsRestOperation getAbfsRestOperationForAppend(final AbfsRestOperationType operationType, - final String httpMethod, - final URL url, - final List requestHeaders, - final byte[] buffer, - final int bufferOffset, - final int bufferLength, - final String sasTokenForReuse) { - return new AbfsRestOperation( - operationType, - this, - httpMethod, - url, - requestHeaders, - buffer, - bufferOffset, - bufferLength, sasTokenForReuse); - } - /** * Returns true if the status code lies in the range of user error. * @param responseStatusCode http response status code. @@ -907,9 +865,8 @@ public AbfsRestOperation flush(final String path, final long position, abfsUriQueryBuilder, cachedSasToken); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.Flush, - this, HTTP_METHOD_PUT, url, requestHeaders, sasTokenForReuse); @@ -934,9 +891,8 @@ public AbfsRestOperation setPathProperties(final String path, final String prope appendSASTokenToQuery(path, SASTokenProvider.SET_PROPERTIES_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.SetPathProperties, - this, HTTP_METHOD_PUT, url, requestHeaders); @@ -963,9 +919,8 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.GetPathStatus, - this, HTTP_METHOD_HEAD, url, requestHeaders); @@ -988,9 +943,8 @@ public AbfsRestOperation read(final String path, final long position, final byte abfsUriQueryBuilder, cachedSasToken); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.ReadFile, - this, HTTP_METHOD_GET, url, requestHeaders, @@ -1063,9 +1017,8 @@ public AbfsRestOperation deleteIdempotencyCheckOp(final AbfsRestOperation op) { && DEFAULT_DELETE_CONSIDERED_IDEMPOTENT) { // Server has returned HTTP 404, which means path no longer // exists. Assuming delete result to be idempotent, return success. - final AbfsRestOperation successOp = new AbfsRestOperation( + final AbfsRestOperation successOp = getAbfsRestOperation( AbfsRestOperationType.DeletePath, - this, HTTP_METHOD_DELETE, op.getUrl(), op.getRequestHeaders()); @@ -1098,9 +1051,8 @@ public AbfsRestOperation setOwner(final String path, final String owner, final S appendSASTokenToQuery(path, SASTokenProvider.SET_OWNER_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.SetOwner, - this, AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); @@ -1124,9 +1076,8 @@ public AbfsRestOperation setPermission(final String path, final String permissio appendSASTokenToQuery(path, SASTokenProvider.SET_PERMISSION_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.SetPermissions, - this, AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); @@ -1159,9 +1110,8 @@ public AbfsRestOperation setAcl(final String path, final String aclSpecString, f appendSASTokenToQuery(path, SASTokenProvider.SET_ACL_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.SetAcl, - this, AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); @@ -1184,9 +1134,8 @@ public AbfsRestOperation getAclStatus(final String path, final boolean useUPN, appendSASTokenToQuery(path, SASTokenProvider.GET_ACL_OPERATION, abfsUriQueryBuilder); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( + final AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.GetAcl, - this, AbfsHttpConstants.HTTP_METHOD_HEAD, url, requestHeaders); @@ -1211,9 +1160,11 @@ public AbfsRestOperation checkAccess(String path, String rwx, TracingContext tra abfsUriQueryBuilder.addQuery(QUERY_FS_ACTION, rwx); appendSASTokenToQuery(path, SASTokenProvider.CHECK_ACCESS_OPERATION, abfsUriQueryBuilder); URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.CheckAccess, this, - AbfsHttpConstants.HTTP_METHOD_HEAD, url, createDefaultHeaders()); + AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CheckAccess, + AbfsHttpConstants.HTTP_METHOD_HEAD, + url, + createDefaultHeaders()); op.execute(tracingContext); return op; } @@ -1238,7 +1189,7 @@ public static String getDirectoryQueryParameter(final String path) { } /** - * If configured for SAS AuthType, appends SAS token to queryBuilder + * If configured for SAS AuthType, appends SAS token to queryBuilder. * @param path * @param operation * @param queryBuilder @@ -1250,7 +1201,7 @@ private String appendSASTokenToQuery(String path, String operation, AbfsUriQuery } /** - * If configured for SAS AuthType, appends SAS token to queryBuilder + * If configured for SAS AuthType, appends SAS token to queryBuilder. * @param path * @param operation * @param queryBuilder @@ -1459,4 +1410,82 @@ public void addCallback(ListenableFuture future, FutureCallback callba protected AccessTokenProvider getTokenProvider() { return tokenProvider; } + + /** + * Creates an AbfsRestOperation with additional parameters for buffer and SAS token. + * + * @param operationType The type of the operation. + * @param httpMethod The HTTP method of the operation. + * @param url The URL associated with the operation. + * @param requestHeaders The list of HTTP headers for the request. + * @param buffer The byte buffer containing data for the operation. + * @param bufferOffset The offset within the buffer where the data starts. + * @param bufferLength The length of the data within the buffer. + * @param sasTokenForReuse The SAS token for reusing authentication. + * @return An AbfsRestOperation instance. + */ + AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType, + final String httpMethod, + final URL url, + final List requestHeaders, + final byte[] buffer, + final int bufferOffset, + final int bufferLength, + final String sasTokenForReuse) { + return new AbfsRestOperation( + operationType, + this, + httpMethod, + url, + requestHeaders, + buffer, + bufferOffset, + bufferLength, + sasTokenForReuse); + } + + /** + * Creates an AbfsRestOperation with basic parameters and no buffer or SAS token. + * + * @param operationType The type of the operation. + * @param httpMethod The HTTP method of the operation. + * @param url The URL associated with the operation. + * @param requestHeaders The list of HTTP headers for the request. + * @return An AbfsRestOperation instance. + */ + AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType, + final String httpMethod, + final URL url, + final List requestHeaders) { + return new AbfsRestOperation( + operationType, + this, + httpMethod, + url, + requestHeaders + ); + } + + /** + * Creates an AbfsRestOperation with parameters including request headers and SAS token. + * + * @param operationType The type of the operation. + * @param httpMethod The HTTP method of the operation. + * @param url The URL associated with the operation. + * @param requestHeaders The list of HTTP headers for the request. + * @param sasTokenForReuse The SAS token for reusing authentication. + * @return An AbfsRestOperation instance. + */ + AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType, + final String httpMethod, + final URL url, + final List requestHeaders, + final String sasTokenForReuse) { + return new AbfsRestOperation( + operationType, + this, + httpMethod, + url, + requestHeaders, sasTokenForReuse); + } } 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 6402be72ddc37..f40cd2cea81ed 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 @@ -82,6 +82,11 @@ public class AbfsRestOperation { */ private String failureReason; + /** + * This variable stores the tracing context used for last Rest Operation. + */ + private TracingContext lastUsedTracingContext; + /** * Checks if there is non-null HTTP response. * @return true if there is a non-null HTTP response from the ABFS call. @@ -197,10 +202,13 @@ String getSasToken() { public void execute(TracingContext tracingContext) throws AzureBlobFileSystemException { + // Since this might be a sub-sequential or parallel rest operation + // triggered by a single file system call, using a new tracing context. + lastUsedTracingContext = createNewTracingContext(tracingContext); try { IOStatisticsBinding.trackDurationOfInvocation(abfsCounters, AbfsStatistic.getStatNameFromHttpCall(method), - () -> completeExecute(tracingContext)); + () -> completeExecute(lastUsedTracingContext)); } catch (AzureBlobFileSystemException aze) { throw aze; } catch (IOException e) { @@ -214,7 +222,7 @@ public void execute(TracingContext tracingContext) * HTTP operations. * @param tracingContext TracingContext instance to track correlation IDs */ - private void completeExecute(TracingContext tracingContext) + void completeExecute(TracingContext tracingContext) throws AzureBlobFileSystemException { // see if we have latency reports from the previous requests String latencyHeader = getClientLatency(); @@ -409,4 +417,25 @@ private void incrementCounter(AbfsStatistic statistic, long value) { abfsCounters.incrementCounter(statistic, value); } } + + /** + * Creates a new Tracing context before entering the retry loop of a rest operation. + * This will ensure all rest operations have unique + * tracing context that will be used for all the retries. + * @param tracingContext original tracingContext. + * @return tracingContext new tracingContext object created from original one. + */ + @VisibleForTesting + public TracingContext createNewTracingContext(final TracingContext tracingContext) { + return new TracingContext(tracingContext); + } + + /** + * Returns the tracing contest used for last rest operation made. + * @return tracingContext lasUserTracingContext. + */ + @VisibleForTesting + public final TracingContext getLastTracingContext() { + return lastUsedTracingContext; + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 1f0ff667522da..57f5702f74fab 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -29,10 +29,14 @@ import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; +import org.mockito.Mockito; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient; @@ -61,7 +65,6 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; import static org.apache.hadoop.test.LambdaTestUtils.intercept; - /** * Test delete operation. */ @@ -257,14 +260,15 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { // Case 2: Mimic retried case // Idempotency check on Delete always returns success - AbfsRestOperation idempotencyRetOp = ITestAbfsClient.getRestOp( + AbfsRestOperation idempotencyRetOp = Mockito.spy(ITestAbfsClient.getRestOp( DeletePath, mockClient, HTTP_METHOD_DELETE, ITestAbfsClient.getTestUrl(mockClient, "/NonExistingPath"), - ITestAbfsClient.getTestRequestHeaders(mockClient)); + ITestAbfsClient.getTestRequestHeaders(mockClient))); idempotencyRetOp.hardSetResult(HTTP_OK); doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any()); TracingContext tracingContext = getTestTracingContext(fs, false); + doReturn(tracingContext).when(idempotencyRetOp).createNewTracingContext(any()); when(mockClient.deletePath("/NonExistingPath", false, null, tracingContext)) .thenCallRealMethod(); @@ -283,4 +287,25 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { mockStore.delete(new Path("/NonExistingPath"), false, getTestTracingContext(fs, false)); } + @Test + public void deleteBlobDirParallelThreadToDeleteOnDifferentTracingContext() + throws Exception { + Configuration configuration = getRawConfiguration(); + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.newInstance(configuration)); + AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); + AbfsClient spiedClient = Mockito.spy(fs.getAbfsClient()); + + Mockito.doReturn(spiedStore).when(fs).getAbfsStore(); + spiedStore.setClient(spiedClient); + + fs.mkdirs(new Path("/testDir")); + fs.create(new Path("/testDir/file1")); + fs.create(new Path("/testDir/file2")); + + AbfsClientTestUtil.hookOnRestOpsForTracingContextSingularity(spiedClient); + + fs.delete(new Path("/testDir"), true); + fs.close(); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java index 8d1330b5ea7dd..e7f57b8af54d0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -20,6 +20,7 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.net.SocketTimeoutException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; @@ -28,6 +29,8 @@ import java.util.concurrent.Future; import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.stubbing.Stubber; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -36,16 +39,30 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.contract.ContractTestUtils; +import static java.net.HttpURLConnection.HTTP_OK; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_LIST_MAX_RESULTS; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_JDK_MESSAGE; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; import static org.apache.hadoop.fs.contract.ContractTestUtils.rename; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.when; /** * Test listStatus operation. @@ -53,6 +70,7 @@ public class ITestAzureBlobFileSystemListStatus extends AbstractAbfsIntegrationTest { private static final int TEST_FILES_NUMBER = 6000; + private static final String TEST_CONTINUATION_TOKEN = "continuation"; public ITestAzureBlobFileSystemListStatus() throws Exception { super(); @@ -62,34 +80,105 @@ public ITestAzureBlobFileSystemListStatus() throws Exception { public void testListPath() throws Exception { Configuration config = new Configuration(this.getRawConfiguration()); config.set(AZURE_LIST_MAX_RESULTS, "5000"); - final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem - .newInstance(getFileSystem().getUri(), config); - final List> tasks = new ArrayList<>(); - - ExecutorService es = Executors.newFixedThreadPool(10); - for (int i = 0; i < TEST_FILES_NUMBER; i++) { - final Path fileName = new Path("/test" + i); - Callable callable = new Callable() { - @Override - public Void call() throws Exception { - touch(fileName); - return null; - } - }; - - tasks.add(es.submit(callable)); - } - - for (Future task : tasks) { - task.get(); + try (final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem + .newInstance(getFileSystem().getUri(), config)) { + final List> tasks = new ArrayList<>(); + + ExecutorService es = Executors.newFixedThreadPool(10); + for (int i = 0; i < TEST_FILES_NUMBER; i++) { + final Path fileName = new Path("/test" + i); + Callable callable = new Callable() { + @Override + public Void call() throws Exception { + touch(fileName); + return null; + } + }; + + tasks.add(es.submit(callable)); + } + + for (Future task : tasks) { + task.get(); + } + + es.shutdownNow(); + fs.registerListener( + new TracingHeaderValidator(getConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.LISTSTATUS, true, 0)); + FileStatus[] files = fs.listStatus(new Path("/")); + assertEquals(TEST_FILES_NUMBER, files.length /* user directory */); } + } - es.shutdownNow(); - fs.registerListener( - new TracingHeaderValidator(getConfiguration().getClientCorrelationId(), - fs.getFileSystemId(), FSOperationType.LISTSTATUS, true, 0)); - FileStatus[] files = fs.listStatus(new Path("/")); - assertEquals(TEST_FILES_NUMBER, files.length /* user directory */); + /** + * Test to verify that each paginated call to ListBlobs uses a new tracing context. + * @throws Exception + */ + @Test + public void testListPathTracingContext() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); + final AbfsClient spiedClient = Mockito.spy(fs.getAbfsClient()); + final TracingContext spiedTracingContext = Mockito.spy( + new TracingContext( + fs.getClientCorrelationId(), fs.getFileSystemId(), + FSOperationType.LISTSTATUS, true, TracingHeaderFormat.ALL_ID_FORMAT, null)); + + Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); + spiedStore.setClient(spiedClient); + spiedFs.setWorkingDirectory(new Path("/")); + + AbfsClientTestUtil.setMockAbfsRestOperationForListPathOperation(spiedClient, + (httpOperation) -> { + + ListResultEntrySchema entry = new ListResultEntrySchema() + .withName("a") + .withIsDirectory(true); + List paths = new ArrayList<>(); + paths.add(entry); + paths.clear(); + entry = new ListResultEntrySchema() + .withName("abc.txt") + .withIsDirectory(false); + paths.add(entry); + ListResultSchema schema1 = new ListResultSchema().withPaths(paths); + ListResultSchema schema2 = new ListResultSchema().withPaths(paths); + + when(httpOperation.getListResultSchema()).thenReturn(schema1) + .thenReturn(schema2); + when(httpOperation.getResponseHeader( + HttpHeaderConfigurations.X_MS_CONTINUATION)) + .thenReturn(TEST_CONTINUATION_TOKEN) + .thenReturn(EMPTY_STRING); + + Stubber stubber = Mockito.doThrow( + new SocketTimeoutException(CONNECTION_TIMEOUT_JDK_MESSAGE)); + stubber.doNothing().when(httpOperation).processResponse( + nullable(byte[].class), nullable(int.class), nullable(int.class)); + + when(httpOperation.getStatusCode()).thenReturn(-1).thenReturn(HTTP_OK); + return httpOperation; + }); + + List fileStatuses = new ArrayList<>(); + spiedStore.listStatus(new Path("/"), "", fileStatuses, true, null, spiedTracingContext); + + // Assert that there were 2 paginated ListPath calls were made 1 and 2. + // 1. Without continuation token + Mockito.verify(spiedClient, times(1)).listPath( + "/", false, + spiedFs.getAbfsStore().getAbfsConfiguration().getListMaxResults(), + null, spiedTracingContext); + // 2. With continuation token + Mockito.verify(spiedClient, times(1)).listPath( + "/", false, + spiedFs.getAbfsStore().getAbfsConfiguration().getListMaxResults(), + TEST_CONTINUATION_TOKEN, spiedTracingContext); + + // Assert that none of the API calls used the same tracing header. + Mockito.verify(spiedTracingContext, times(0)).constructHeader(any(), any()); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 23e65ed2dd246..2da530364c13b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -83,44 +83,46 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, boolean includeInHeader) throws Exception { Configuration conf = getRawConfiguration(); conf.set(FS_AZURE_CLIENT_CORRELATIONID, clientCorrelationId); - AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(conf); - - String correlationID = fs.getClientCorrelationId(); - if (includeInHeader) { - Assertions.assertThat(correlationID) - .describedAs("Correlation ID should match config when valid") - .isEqualTo(clientCorrelationId); - } else { - Assertions.assertThat(correlationID) - .describedAs("Invalid ID should be replaced with empty string") - .isEqualTo(EMPTY_STRING); + try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(conf)) { + + String correlationID = fs.getClientCorrelationId(); + if (includeInHeader) { + Assertions.assertThat(correlationID) + .describedAs("Correlation ID should match config when valid") + .isEqualTo(clientCorrelationId); + } else { + Assertions.assertThat(correlationID) + .describedAs("Invalid ID should be replaced with empty string") + .isEqualTo(EMPTY_STRING); + } + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fs.getFileSystemId(), FSOperationType.TEST_OP, + TracingHeaderFormat.ALL_ID_FORMAT, null); + boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); + String path = getRelativePath(new Path("/testDir")); + String permission = isNamespaceEnabled + ? getOctalNotation(FsPermission.getDirDefault()) + : null; + String umask = isNamespaceEnabled + ? getOctalNotation(FsPermission.getUMask(fs.getConf())) + : null; + + //request should not fail for invalid clientCorrelationID + AbfsRestOperation op = fs.getAbfsClient() + .createPath(path, false, true, permission, umask, false, null, + tracingContext); + + int statusCode = op.getResult().getStatusCode(); + Assertions.assertThat(statusCode).describedAs("Request should not fail") + .isEqualTo(HTTP_CREATED); + + String requestHeader = op.getResult().getClientRequestId().replace("[", "") + .replace("]", ""); + Assertions.assertThat(requestHeader) + .describedAs("Client Request Header should match TracingContext") + .isEqualTo(op.getLastTracingContext().getHeader()); + } - TracingContext tracingContext = new TracingContext(clientCorrelationId, - fs.getFileSystemId(), FSOperationType.TEST_OP, - TracingHeaderFormat.ALL_ID_FORMAT, null); - boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); - String path = getRelativePath(new Path("/testDir")); - String permission = isNamespaceEnabled - ? getOctalNotation(FsPermission.getDirDefault()) - : null; - String umask = isNamespaceEnabled - ? getOctalNotation(FsPermission.getUMask(fs.getConf())) - : null; - - //request should not fail for invalid clientCorrelationID - AbfsRestOperation op = fs.getAbfsClient() - .createPath(path, false, true, permission, umask, false, null, - tracingContext); - - int statusCode = op.getResult().getStatusCode(); - Assertions.assertThat(statusCode).describedAs("Request should not fail") - .isEqualTo(HTTP_CREATED); - - String requestHeader = op.getResult().getClientRequestId().replace("[", "") - .replace("]", ""); - Assertions.assertThat(requestHeader) - .describedAs("Client Request Header should match TracingContext") - .isEqualTo(tracingContext.getHeader()); } @Ignore diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java new file mode 100644 index 0000000000000..875682fe20320 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java @@ -0,0 +1,162 @@ +/** + * 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.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.locks.ReentrantLock; + +import org.assertj.core.api.Assertions; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.util.functional.FunctionRaisingIOE; + +import static java.net.HttpURLConnection.HTTP_OK; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_GET; +import static org.apache.hadoop.fs.azurebfs.services.AuthType.OAuth; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.nullable; + +/** + * Utility class to help defining mock behavior on AbfsClient and AbfsRestOperation + * objects which are protected inside services package. + */ +public final class AbfsClientTestUtil { + + private AbfsClientTestUtil() { + + } + + public static void setMockAbfsRestOperationForListPathOperation( + final AbfsClient spiedClient, + FunctionRaisingIOE functionRaisingIOE) + throws Exception { + ExponentialRetryPolicy retryPolicy = Mockito.mock(ExponentialRetryPolicy.class); + AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class); + AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation( + AbfsRestOperationType.ListPaths, + spiedClient, + HTTP_METHOD_GET, + null, + new ArrayList<>() + )); + + Mockito.doReturn(abfsRestOperation).when(spiedClient).getAbfsRestOperation( + eq(AbfsRestOperationType.ListPaths), any(), any(), any()); + + addGeneralMockBehaviourToAbfsClient(spiedClient, retryPolicy); + addGeneralMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); + + functionRaisingIOE.apply(httpOperation); + } + + /** + * Adding general mock behaviour to AbfsRestOperation and AbfsHttpOperation + * to avoid any NPE occurring. These will avoid any network call made and + * will return the relevant exception or return value directly. + * @param abfsRestOperation to be mocked + * @param httpOperation to be mocked + * @throws IOException + */ + public static void addGeneralMockBehaviourToRestOpAndHttpOp(final AbfsRestOperation abfsRestOperation, + final AbfsHttpOperation httpOperation) throws IOException { + HttpURLConnection httpURLConnection = Mockito.mock(HttpURLConnection.class); + Mockito.doNothing().when(httpURLConnection) + .setRequestProperty(nullable(String.class), nullable(String.class)); + Mockito.doReturn(httpURLConnection).when(httpOperation).getConnection(); + Mockito.doReturn("").when(abfsRestOperation).getClientLatency(); + Mockito.doReturn(httpOperation).when(abfsRestOperation).createHttpOperation(); + } + + /** + * Adding general mock behaviour to AbfsClient to avoid any NPE occurring. + * These will avoid any network call made and will return the relevant exception or return value directly. + * @param abfsClient to be mocked + * @param retryPolicy to be mocked + * @throws IOException + */ + public static void addGeneralMockBehaviourToAbfsClient(final AbfsClient abfsClient, + final ExponentialRetryPolicy retryPolicy) throws IOException { + Mockito.doReturn(OAuth).when(abfsClient).getAuthType(); + Mockito.doReturn("").when(abfsClient).getAccessToken(); + AbfsThrottlingIntercept intercept = Mockito.mock( + AbfsThrottlingIntercept.class); + Mockito.doReturn(intercept).when(abfsClient).getIntercept(); + Mockito.doNothing() + .when(intercept) + .sendingRequest(any(), nullable(AbfsCounters.class)); + Mockito.doNothing().when(intercept).updateMetrics(any(), any()); + + Mockito.doReturn(retryPolicy).when(abfsClient).getRetryPolicy(); + Mockito.doReturn(true) + .when(retryPolicy) + .shouldRetry(nullable(Integer.class), nullable(Integer.class)); + Mockito.doReturn(false).when(retryPolicy).shouldRetry(0, HTTP_OK); + Mockito.doReturn(false).when(retryPolicy).shouldRetry(1, HTTP_OK); + Mockito.doReturn(false).when(retryPolicy).shouldRetry(2, HTTP_OK); + } + + public static void hookOnRestOpsForTracingContextSingularity(AbfsClient client) { + Set tracingContextSet = new HashSet<>(); + ReentrantLock lock = new ReentrantLock(); + Answer answer = new Answer() { + @Override + public Object answer(final InvocationOnMock invocationOnMock) + throws Throwable { + AbfsRestOperation op = Mockito.spy((AbfsRestOperation) invocationOnMock.callRealMethod()); + Mockito.doAnswer(completeExecuteInvocation -> { + lock.lock(); + try { + TracingContext context = completeExecuteInvocation.getArgument(0); + Assertions.assertThat(tracingContextSet).doesNotContain(context); + tracingContextSet.add(context); + } finally { + lock.unlock(); + } + return completeExecuteInvocation.callRealMethod(); + }).when(op).completeExecute(Mockito.any(TracingContext.class)); + return op; + } + }; + + Mockito.doAnswer(answer) + .when(client) + .getAbfsRestOperation(Mockito.any(AbfsRestOperationType.class), + Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList(), + Mockito.nullable(byte[].class), Mockito.anyInt(), Mockito.anyInt(), + Mockito.nullable(String.class)); + Mockito.doAnswer(answer) + .when(client) + .getAbfsRestOperation(Mockito.any(AbfsRestOperationType.class), + Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList()); + Mockito.doAnswer(answer) + .when(client) + .getAbfsRestOperation(Mockito.any(AbfsRestOperationType.class), + Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList(), + Mockito.nullable(String.class)); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java index 18d1e3917f24e..6707c593f5a76 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java @@ -58,6 +58,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -592,7 +593,7 @@ public void testExpectHundredContinue() throws Exception { // Mock the restOperation for the client. Mockito.doReturn(op) .when(testClient) - .getAbfsRestOperationForAppend(Mockito.any(), + .getAbfsRestOperation(eq(AbfsRestOperationType.Append), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.nullable(int.class), Mockito.nullable(int.class), Mockito.any()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java index 6ffe2e2773bbf..6574a808f92bd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java @@ -290,6 +290,7 @@ public void testExpectHundredContinue() throws Exception { TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", "abcde", FSOperationType.APPEND, TracingHeaderFormat.ALL_ID_FORMAT, null)); + Mockito.doReturn(tracingContext).when(op).createNewTracingContext(Mockito.any()); switch (errorType) { case WRITE: diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java index bfa524a25e600..b302a1fa939e7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java @@ -18,9 +18,7 @@ package org.apache.hadoop.fs.azurebfs.services; -import java.io.IOException; import java.io.InterruptedIOException; -import java.net.HttpURLConnection; import java.net.SocketException; import java.net.SocketTimeoutException; import java.net.UnknownHostException; @@ -39,7 +37,8 @@ import static java.net.HttpURLConnection.HTTP_UNAVAILABLE; import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.EGRESS_OVER_ACCOUNT_LIMIT; import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.INGRESS_OVER_ACCOUNT_LIMIT; -import static org.apache.hadoop.fs.azurebfs.services.AuthType.OAuth; +import static org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil.addGeneralMockBehaviourToAbfsClient; +import static org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil.addGeneralMockBehaviourToRestOpAndHttpOp; import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_ABBREVIATION; import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_MESSAGE; import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_ABBREVIATION; @@ -166,7 +165,7 @@ private void testClientRequestIdForStatusRetry(int status, AbfsClient abfsClient = Mockito.mock(AbfsClient.class); ExponentialRetryPolicy retryPolicy = Mockito.mock( ExponentialRetryPolicy.class); - addMockBehaviourToAbfsClient(abfsClient, retryPolicy); + addGeneralMockBehaviourToAbfsClient(abfsClient, retryPolicy); AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation( @@ -178,7 +177,7 @@ private void testClientRequestIdForStatusRetry(int status, )); AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class); - addMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); + addGeneralMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); Mockito.doNothing() .doNothing() @@ -202,6 +201,8 @@ private void testClientRequestIdForStatusRetry(int status, TracingContext tracingContext = Mockito.mock(TracingContext.class); Mockito.doNothing().when(tracingContext).setRetryCount(nullable(int.class)); + Mockito.doReturn(tracingContext) + .when(abfsRestOperation).createNewTracingContext(any()); int[] count = new int[1]; count[0] = 0; @@ -225,7 +226,7 @@ private void testClientRequestIdForTimeoutRetry(Exception[] exceptions, AbfsClient abfsClient = Mockito.mock(AbfsClient.class); ExponentialRetryPolicy retryPolicy = Mockito.mock( ExponentialRetryPolicy.class); - addMockBehaviourToAbfsClient(abfsClient, retryPolicy); + addGeneralMockBehaviourToAbfsClient(abfsClient, retryPolicy); AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation( @@ -237,7 +238,7 @@ private void testClientRequestIdForTimeoutRetry(Exception[] exceptions, )); AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class); - addMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); + addGeneralMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); Stubber stubber = Mockito.doThrow(exceptions[0]); for (int iteration = 1; iteration < len; iteration++) { @@ -253,6 +254,7 @@ private void testClientRequestIdForTimeoutRetry(Exception[] exceptions, TracingContext tracingContext = Mockito.mock(TracingContext.class); Mockito.doNothing().when(tracingContext).setRetryCount(nullable(int.class)); + Mockito.doReturn(tracingContext).when(abfsRestOperation).createNewTracingContext(any()); int[] count = new int[1]; count[0] = 0; @@ -268,35 +270,4 @@ private void testClientRequestIdForTimeoutRetry(Exception[] exceptions, abfsRestOperation.execute(tracingContext); Assertions.assertThat(count[0]).isEqualTo(len + 1); } - - private void addMockBehaviourToRestOpAndHttpOp(final AbfsRestOperation abfsRestOperation, - final AbfsHttpOperation httpOperation) throws IOException { - HttpURLConnection httpURLConnection = Mockito.mock(HttpURLConnection.class); - Mockito.doNothing() - .when(httpURLConnection) - .setRequestProperty(nullable(String.class), nullable(String.class)); - Mockito.doReturn(httpURLConnection).when(httpOperation).getConnection(); - Mockito.doReturn("").when(abfsRestOperation).getClientLatency(); - Mockito.doReturn(httpOperation).when(abfsRestOperation).createHttpOperation(); - } - - private void addMockBehaviourToAbfsClient(final AbfsClient abfsClient, - final ExponentialRetryPolicy retryPolicy) throws IOException { - Mockito.doReturn(OAuth).when(abfsClient).getAuthType(); - Mockito.doReturn("").when(abfsClient).getAccessToken(); - AbfsThrottlingIntercept intercept = Mockito.mock( - AbfsThrottlingIntercept.class); - Mockito.doReturn(intercept).when(abfsClient).getIntercept(); - Mockito.doNothing() - .when(intercept) - .sendingRequest(any(), nullable(AbfsCounters.class)); - Mockito.doNothing().when(intercept).updateMetrics(any(), any()); - - Mockito.doReturn(retryPolicy).when(abfsClient).getRetryPolicy(); - Mockito.doReturn(true) - .when(retryPolicy) - .shouldRetry(nullable(Integer.class), nullable(Integer.class)); - Mockito.doReturn(false).when(retryPolicy).shouldRetry(1, HTTP_OK); - Mockito.doReturn(false).when(retryPolicy).shouldRetry(2, HTTP_OK); - } } From 2fa7d4fe8600b3d1b735734cc557757afa70a73b Mon Sep 17 00:00:00 2001 From: Vinod Anandan Date: Wed, 15 Nov 2023 16:27:59 +0200 Subject: [PATCH 05/10] HADOOP-18964. Update plugin for SBOM generation to 2.7.10 (#6235). Contributed by Vinod Anandan. Signed-off-by: Ayush Saxena --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index b86af01172f42..0c175d8da03cc 100644 --- a/pom.xml +++ b/pom.xml @@ -118,7 +118,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x 4.2.0 1.1.1 3.10.1 - 2.7.6 + 2.7.10 bash From 513e6dcf14e032e8681eeecfe4bf9535fb38ed68 Mon Sep 17 00:00:00 2001 From: Jian Zhang <38941777+KeeProMise@users.noreply.github.com> Date: Fri, 17 Nov 2023 01:13:48 +0800 Subject: [PATCH 06/10] HDFS-17232. RBF: Fix NoNamenodesAvailableException for a long time, when use observer. (#6208) --- .../resolver/MembershipNamenodeResolver.java | 53 ++- .../federation/router/RouterRpcClient.java | 64 ++- .../federation/MiniRouterDFSCluster.java | 31 +- .../TestNoNamenodesAvailableLongTime.java | 430 ++++++++++++++++++ 4 files changed, 529 insertions(+), 49 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestNoNamenodesAvailableLongTime.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java index c0e800e0430d4..933ecf070091a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java @@ -79,7 +79,7 @@ public class MembershipNamenodeResolver * name and a boolean indicating if observer namenodes should be listed first. * If true, observer namenodes are listed first. If false, active namenodes are listed first. * Invalidated on cache refresh. */ - private Map, List> cacheNS; + private Map, List> cacheNS; /** Cached lookup of NN for block pool. Invalidated on cache refresh. */ private Map> cacheBP; @@ -483,9 +483,9 @@ public void setRouterId(String router) { * Rotate cache, make the current namenode have the lowest priority, * to ensure that the current namenode will not be accessed first next time. * - * @param nsId name service id - * @param namenode namenode contexts - * @param listObserversFirst Observer read case, observer NN will be ranked first + * @param nsId name service id. + * @param namenode namenode contexts. + * @param listObserversFirst Observer read case, observer NN will be ranked first. */ @Override public void rotateCache( @@ -494,29 +494,32 @@ public void rotateCache( if (namenodeContexts == null || namenodeContexts.size() <= 1) { return namenodeContexts; } - FederationNamenodeContext firstNamenodeContext = namenodeContexts.get(0); - /* - * If the first nn in the cache is active, the active nn priority cannot be lowered. - * This happens when other threads have already updated the cache. - */ - if (firstNamenodeContext.getState().equals(ACTIVE)) { - return namenodeContexts; + + // If there is active nn, rotateCache is not needed + // because the router has already loaded the cache. + for (FederationNamenodeContext namenodeContext : namenodeContexts) { + if (namenodeContext.getState() == ACTIVE) { + return namenodeContexts; + } } - /* - * If the first nn in the cache at this time is not the nn - * that needs to be lowered in priority, there is no need to rotate. - * This happens when other threads have already rotated the cache. - */ - if (firstNamenodeContext.getRpcAddress().equals(namenode.getRpcAddress())) { - List rotatedNnContexts = new ArrayList<>(namenodeContexts); - Collections.rotate(rotatedNnContexts, -1); - String firstNamenodeId = namenodeContexts.get(0).getNamenodeId(); - LOG.info("Rotate cache of pair , put namenode: {} in the " + - "first position of the cache and namenode: {} in the last position of the cache", - nsId, listObserversFirst, firstNamenodeId, namenode.getNamenodeId()); - return rotatedNnContexts; + + // If the last namenode in the cache at this time + // is the namenode whose priority needs to be lowered. + // No need to rotate cache, because other threads have already rotated the cache. + FederationNamenodeContext lastNamenode = namenodeContexts.get(namenodeContexts.size()-1); + if (lastNamenode.getRpcAddress().equals(namenode.getRpcAddress())) { + return namenodeContexts; } - return namenodeContexts; + + // Move the inaccessible namenode to the end of the cache, + // to ensure that the namenode will not be accessed first next time. + List rotateNamenodeContexts = + (List) namenodeContexts; + rotateNamenodeContexts.remove(namenode); + rotateNamenodeContexts.add(namenode); + LOG.info("Rotate cache of pair<{}, {}> -> {}", + nsId, listObserversFirst, rotateNamenodeContexts); + return rotateNamenodeContexts; }); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java index b38900c3bc264..ad95e9532a3e1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java @@ -457,14 +457,17 @@ private static IOException toIOException(Exception e) { * @param ioe IOException reported. * @param retryCount Number of retries. * @param nsId Nameservice ID. + * @param namenode namenode context. + * @param listObserverFirst Observer read case, observer NN will be ranked first. * @return Retry decision. - * @throws NoNamenodesAvailableException Exception that the retry policy - * generates for no available namenodes. + * @throws IOException An IO Error occurred. */ - private RetryDecision shouldRetry(final IOException ioe, final int retryCount, - final String nsId) throws IOException { + private RetryDecision shouldRetry( + final IOException ioe, final int retryCount, final String nsId, + final FederationNamenodeContext namenode, + final boolean listObserverFirst) throws IOException { // check for the case of cluster unavailable state - if (isClusterUnAvailable(nsId)) { + if (isClusterUnAvailable(nsId, namenode, listObserverFirst)) { // we allow to retry once if cluster is unavailable if (retryCount == 0) { return RetryDecision.RETRY; @@ -538,7 +541,7 @@ public Object invokeMethod( ProxyAndInfo client = connection.getClient(); final Object proxy = client.getProxy(); - ret = invoke(nsId, 0, method, proxy, params); + ret = invoke(nsId, namenode, useObserver, 0, method, proxy, params); if (failover && FederationNamenodeServiceState.OBSERVER != namenode.getState()) { // Success on alternate server, update @@ -594,13 +597,16 @@ public Object invokeMethod( se.initCause(ioe); throw se; } else if (ioe instanceof NoNamenodesAvailableException) { + IOException cause = (IOException) ioe.getCause(); if (this.rpcMonitor != null) { this.rpcMonitor.proxyOpNoNamenodes(nsId); } LOG.error("Cannot get available namenode for {} {} error: {}", nsId, rpcAddress, ioe.getMessage()); // Rotate cache so that client can retry the next namenode in the cache - this.namenodeResolver.rotateCache(nsId, namenode, shouldUseObserver); + if (shouldRotateCache(cause)) { + this.namenodeResolver.rotateCache(nsId, namenode, useObserver); + } // Throw RetriableException so that client can retry throw new RetriableException(ioe); } else { @@ -708,7 +714,9 @@ private void addClientInfoToCallerContext(UserGroupInformation ugi) { * @return Response from the remote server * @throws IOException If error occurs. */ - private Object invoke(String nsId, int retryCount, final Method method, + private Object invoke( + String nsId, FederationNamenodeContext namenode, Boolean listObserverFirst, + int retryCount, final Method method, final Object obj, final Object... params) throws IOException { try { return method.invoke(obj, params); @@ -721,14 +729,14 @@ private Object invoke(String nsId, int retryCount, final Method method, IOException ioe = (IOException) cause; // Check if we should retry. - RetryDecision decision = shouldRetry(ioe, retryCount, nsId); + RetryDecision decision = shouldRetry(ioe, retryCount, nsId, namenode, listObserverFirst); if (decision == RetryDecision.RETRY) { if (this.rpcMonitor != null) { this.rpcMonitor.proxyOpRetries(); } // retry - return invoke(nsId, ++retryCount, method, obj, params); + return invoke(nsId, namenode, listObserverFirst, ++retryCount, method, obj, params); } else if (decision == RetryDecision.FAILOVER_AND_RETRY) { // failover, invoker looks for standby exceptions for failover. if (ioe instanceof StandbyException) { @@ -772,13 +780,23 @@ public static boolean isUnavailableException(IOException ioe) { * Check if the cluster of given nameservice id is available. * * @param nsId nameservice ID. + * @param namenode namenode context. + * @param listObserverFirst Observer read case, observer NN will be ranked first. * @return true if the cluster with given nameservice id is available. * @throws IOException if error occurs. */ - private boolean isClusterUnAvailable(String nsId) throws IOException { + private boolean isClusterUnAvailable( + String nsId, FederationNamenodeContext namenode, + boolean listObserverFirst) throws IOException { + // If the operation is an observer read + // and the namenode that caused the exception is an observer, + // false is returned so that the observer can be marked as unavailable,so other observers + // or active namenode which is standby in the cache of the router can be retried. + if (listObserverFirst && namenode.getState() == FederationNamenodeServiceState.OBSERVER) { + return false; + } List nnState = this.namenodeResolver - .getNamenodesForNameserviceId(nsId, false); - + .getNamenodesForNameserviceId(nsId, listObserverFirst); if (nnState != null) { for (FederationNamenodeContext nnContext : nnState) { // Once we find one NN is in active state, we assume this @@ -1830,4 +1848,24 @@ private LongAccumulator getTimeOfLastCallToActive(String namespaceId) { return lastActiveNNRefreshTimes .computeIfAbsent(namespaceId, key -> new LongAccumulator(Math::max, 0)); } + + /** + * Determine whether router rotated cache is required when NoNamenodesAvailableException occurs. + * + * @param ioe cause of the NoNamenodesAvailableException. + * @return true if NoNamenodesAvailableException occurs due to + * {@link RouterRpcClient#isUnavailableException(IOException) unavailable exception}, + * otherwise false. + */ + private boolean shouldRotateCache(IOException ioe) { + if (isUnavailableException(ioe)) { + return true; + } + if (ioe instanceof RemoteException) { + RemoteException re = (RemoteException) ioe; + ioe = re.unwrapRemoteException(); + ioe = getCleanException(ioe); + } + return isUnavailableException(ioe); + } } 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 bf22cf01148a3..ea19dd1e1ba33 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 @@ -132,9 +132,9 @@ public class MiniRouterDFSCluster { /** Mini cluster. */ private MiniDFSCluster cluster; - protected static final long DEFAULT_HEARTBEAT_INTERVAL_MS = + public static final long DEFAULT_HEARTBEAT_INTERVAL_MS = TimeUnit.SECONDS.toMillis(5); - protected static final long DEFAULT_CACHE_INTERVAL_MS = + public static final long DEFAULT_CACHE_INTERVAL_MS = TimeUnit.SECONDS.toMillis(5); /** Heartbeat interval in milliseconds. */ private long heartbeatInterval; @@ -240,17 +240,26 @@ public FileSystem getFileSystem(Configuration configuration) throws IOException } public FileSystem getFileSystemWithObserverReadProxyProvider() throws IOException { - Configuration observerReadConf = new Configuration(conf); - observerReadConf.set(DFS_NAMESERVICES, - observerReadConf.get(DFS_NAMESERVICES)+ ",router-service"); - observerReadConf.set(DFS_HA_NAMENODES_KEY_PREFIX + ".router-service", "router1"); - observerReadConf.set(DFS_NAMENODE_RPC_ADDRESS_KEY+ ".router-service.router1", + return getFileSystemWithProxyProvider(ObserverReadProxyProvider.class.getName()); + } + + public FileSystem getFileSystemWithConfiguredFailoverProxyProvider() throws IOException { + return getFileSystemWithProxyProvider(ConfiguredFailoverProxyProvider.class.getName()); + } + + private FileSystem getFileSystemWithProxyProvider( + String proxyProviderClassName) throws IOException { + conf.set(DFS_NAMESERVICES, + conf.get(DFS_NAMESERVICES)+ ",router-service"); + conf.set(DFS_HA_NAMENODES_KEY_PREFIX + ".router-service", "router1"); + conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY+ ".router-service.router1", getFileSystemURI().toString()); - observerReadConf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX - + "." + "router-service", ObserverReadProxyProvider.class.getName()); - DistributedFileSystem.setDefaultUri(observerReadConf, "hdfs://router-service"); - return DistributedFileSystem.get(observerReadConf); + conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + + "." + "router-service", proxyProviderClassName); + DistributedFileSystem.setDefaultUri(conf, "hdfs://router-service"); + + return DistributedFileSystem.get(conf); } public DFSClient getClient(UserGroupInformation user) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestNoNamenodesAvailableLongTime.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestNoNamenodesAvailableLongTime.java new file mode 100644 index 0000000000000..ad5eb2c8d25fa --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestNoNamenodesAvailableLongTime.java @@ -0,0 +1,430 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.federation.router; + +import static org.apache.hadoop.fs.permission.AclEntryType.USER; +import static org.apache.hadoop.fs.permission.FsAction.ALL; +import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.DFSConfigKeys; +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; +import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster; +import org.apache.hadoop.hdfs.server.federation.metrics.FederationRPCMetrics; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.util.Lists; +import org.junit.After; +import org.junit.Test; + + +import java.io.IOException; +import java.util.Collection; +import java.util.List; + +import static org.apache.hadoop.ha.HAServiceProtocol.HAServiceState.ACTIVE; +import static org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.DEFAULT_HEARTBEAT_INTERVAL_MS; +import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; + +/** + * When failover occurs, the router may record that the ns has no active namenode + * even if there is actually an active namenode. + * Only when the router updates the cache next time can the memory status be updated, + * causing the router to report NoNamenodesAvailableException for a long time, + * + * @see org.apache.hadoop.hdfs.server.federation.router.NoNamenodesAvailableException + */ +public class TestNoNamenodesAvailableLongTime { + + // router load cache interval 10s + private static final long CACHE_FLUSH_INTERVAL_MS = 10000; + private StateStoreDFSCluster cluster; + private FileSystem fileSystem; + private RouterContext routerContext; + private FederationRPCMetrics rpcMetrics; + + @After + public void cleanup() throws IOException { + rpcMetrics = null; + routerContext = null; + if (fileSystem != null) { + fileSystem.close(); + fileSystem = null; + } + if (cluster != null) { + cluster.shutdown(); + cluster = null; + } + } + + /** + * Set up state store cluster. + * + * @param numNameservices number of name services + * @param numberOfObserver number of observer + * @param useObserver whether to use observer + */ + private void setupCluster(int numNameservices, int numberOfObserver, boolean useObserver) + throws Exception { + if (!useObserver) { + numberOfObserver = 0; + } + int numberOfNamenode = 2 + numberOfObserver; + cluster = new StateStoreDFSCluster(true, numNameservices, numberOfNamenode, + DEFAULT_HEARTBEAT_INTERVAL_MS, CACHE_FLUSH_INTERVAL_MS); + Configuration routerConf = new RouterConfigBuilder() + .stateStore() + .metrics() + .admin() + .rpc() + .heartbeat() + .build(); + + // Set router observer related configs + if (useObserver) { + routerConf.setBoolean(RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_DEFAULT_KEY, true); + routerConf.setBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY, true); + routerConf.set(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, "0ms"); + } + + // Reduce the number of RPC clients threads to overload the Router easy + routerConf.setInt(RBFConfigKeys.DFS_ROUTER_CLIENT_THREADS_SIZE, 4); + + // No need for datanodes + cluster.setNumDatanodesPerNameservice(0); + cluster.addRouterOverrides(routerConf); + + cluster.startCluster(); + + // Making one Namenode active per nameservice + if (cluster.isHighAvailability()) { + for (String ns : cluster.getNameservices()) { + List nnList = cluster.getNamenodes(ns); + cluster.switchToActive(ns, nnList.get(0).getNamenodeId()); + cluster.switchToStandby(ns, nnList.get(1).getNamenodeId()); + for (int i = 2; i < numberOfNamenode; i++) { + cluster.switchToObserver(ns, nnList.get(i).getNamenodeId()); + } + } + } + + cluster.startRouters(); + cluster.waitClusterUp(); + } + + /** + * Initialize the test environment and start the cluster so that + * there is no active namenode record in the router cache, + * but the second non-observer namenode in the router cache is actually active. + */ + private void initEnv(int numberOfObserver, boolean useObserver) throws Exception { + setupCluster(1, numberOfObserver, useObserver); + // Transition all namenodes in the cluster are standby. + transitionActiveToStandby(); + // + allRoutersHeartbeat(); + allRoutersLoadCache(); + + List namenodes = cluster.getNamenodes(); + + // Make sure all namenodes are in standby state + for (MiniRouterDFSCluster.NamenodeContext namenodeContext : namenodes) { + assertNotEquals(ACTIVE.ordinal(), namenodeContext.getNamenode().getNameNodeState()); + } + + routerContext = cluster.getRandomRouter(); + + // Get the second namenode in the router cache and make it active + setSecondNonObserverNamenodeInTheRouterCacheActive(numberOfObserver, false); + allRoutersHeartbeat(); + + // Get router metrics + rpcMetrics = routerContext.getRouter().getRpcServer().getRPCMetrics(); + + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", useObserver)); + + // Retries is 2 (see FailoverOnNetworkExceptionRetry#shouldRetry, will fail + // when reties > max.attempts), so total access is 3. + routerContext.getConf().setInt("dfs.client.retry.max.attempts", 1); + + if (useObserver) { + fileSystem = routerContext.getFileSystemWithObserverReadProxyProvider(); + } else { + fileSystem = routerContext.getFileSystemWithConfiguredFailoverProxyProvider(); + } + } + + /** + * If NoNamenodesAvailableException occurs due to + * {@link RouterRpcClient#isUnavailableException(IOException) unavailable exception}, + * should rotated Cache. + */ + @Test + public void testShouldRotatedCache() throws Exception { + // 2 namenodes: 1 active, 1 standby. + // But there is no active namenode in router cache. + initEnv(0, false); + // At this time, the router has recorded 2 standby namenodes in memory. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", false)); + + Path path = new Path("/test.file"); + // The first create operation will cause NoNamenodesAvailableException and RotatedCache. + // After retrying, create and complete operation will be executed successfully. + fileSystem.create(path); + assertEquals(1, rpcMetrics.getProxyOpNoNamenodes()); + + // At this time, the router has recorded 2 standby namenodes in memory, + // the operation can be successful without waiting for the router load cache. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", false)); + } + + /** + * If a request still fails even if it is sent to active, + * then the change operation itself is illegal, + * the cache should not be rotated due to illegal operations. + */ + @Test + public void testShouldNotBeRotatedCache() throws Exception { + testShouldRotatedCache(); + long proxyOpNoNamenodes = rpcMetrics.getProxyOpNoNamenodes(); + Path path = new Path("/test.file"); + /* + * we have put the actually active namenode at the front of the cache by rotating the cache. + * Therefore, the setPermission operation does not cause NoNamenodesAvailableException. + */ + fileSystem.setPermission(path, FsPermission.createImmutable((short)0640)); + assertEquals(proxyOpNoNamenodes, rpcMetrics.getProxyOpNoNamenodes()); + + // At this time, the router has recorded 2 standby namenodes in memory + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", false)); + + /* + * Even if the router transfers the illegal request to active, + * NoNamenodesAvailableException will still be generated. + * Therefore, rotated cache is not needed. + */ + List aclSpec = Lists.newArrayList(aclEntry(DEFAULT, USER, "foo", ALL)); + try { + fileSystem.setAcl(path, aclSpec); + }catch (RemoteException e) { + assertTrue(e.getMessage().contains( + "org.apache.hadoop.hdfs.server.federation.router.NoNamenodesAvailableException: " + + "No namenodes available under nameservice ns0")); + assertTrue(e.getMessage().contains( + "org.apache.hadoop.hdfs.protocol.AclException: Invalid ACL: " + + "only directories may have a default ACL. Path: /test.file")); + } + // Retries is 2 (see FailoverOnNetworkExceptionRetry#shouldRetry, will fail + // when reties > max.attempts), so total access is 3. + assertEquals(proxyOpNoNamenodes + 3, rpcMetrics.getProxyOpNoNamenodes()); + proxyOpNoNamenodes = rpcMetrics.getProxyOpNoNamenodes(); + + // So legal operations can be accessed normally without reporting NoNamenodesAvailableException. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", false)); + fileSystem.getFileStatus(path); + assertEquals(proxyOpNoNamenodes, rpcMetrics.getProxyOpNoNamenodes()); + + // At this time, the router has recorded 2 standby namenodes in memory, + // the operation can be successful without waiting for the router load cache. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", false)); + } + + /** + * In the observer scenario, NoNamenodesAvailableException occurs, + * the operation can be successful without waiting for the router load cache. + */ + @Test + public void testUseObserver() throws Exception { + // 4 namenodes: 2 observers, 1 active, 1 standby. + // But there is no active namenode in router cache. + initEnv(2, true); + + Path path = new Path("/"); + // At this time, the router has recorded 2 standby namenodes in memory. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", true)); + + // The first msync operation will cause NoNamenodesAvailableException and RotatedCache. + // After retrying, msync and getFileInfo operation will be executed successfully. + fileSystem.getFileStatus(path); + assertEquals(1, rpcMetrics.getObserverProxyOps()); + assertEquals(1, rpcMetrics.getProxyOpNoNamenodes()); + + // At this time, the router has recorded 2 standby namenodes in memory, + // the operation can be successful without waiting for the router load cache. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", true)); + } + + /** + * In a multi-observer environment, if at least one observer is normal, + * read requests can still succeed even if NoNamenodesAvailableException occurs. + */ + @Test + public void testAtLeastOneObserverNormal() throws Exception { + // 4 namenodes: 2 observers, 1 active, 1 standby. + // But there is no active namenode in router cache. + initEnv(2, true); + // Shutdown one observer. + stopObserver(1); + + /* + * The first msync operation will cause NoNamenodesAvailableException and RotatedCache. + * After retrying, msync operation will be executed successfully. + * Each read request will shuffle the observer, + * if the getFileInfo operation is sent to the downed observer, + * it will cause NoNamenodesAvailableException, + * at this time, the request can be retried to the normal observer, + * no NoNamenodesAvailableException will be generated and the operation will be successful. + */ + fileSystem.getFileStatus(new Path("/")); + assertEquals(1, rpcMetrics.getProxyOpNoNamenodes()); + assertEquals(1, rpcMetrics.getObserverProxyOps()); + + // At this time, the router has recorded 2 standby namenodes in memory, + // the operation can be successful without waiting for the router load cache. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", true)); + } + + /** + * If all obervers are down, read requests can succeed, + * even if a NoNamenodesAvailableException occurs. + */ + @Test + public void testAllObserverAbnormality() throws Exception { + // 4 namenodes: 2 observers, 1 active, 1 standby. + // But there is no active namenode in router cache. + initEnv(2, true); + // Shutdown all observers. + stopObserver(2); + + /* + * The first msync operation will cause NoNamenodesAvailableException and RotatedCache. + * After retrying, msync operation will be executed successfully. + * The getFileInfo operation retried 2 namenodes, both causing UnavailableException, + * and continued to retry to the standby namenode, + * causing NoNamenodesAvailableException and RotatedCache, + * and the execution was successful after retrying. + */ + fileSystem.getFileStatus(new Path("/")); + assertEquals(2, rpcMetrics.getProxyOpFailureCommunicate()); + assertEquals(2, rpcMetrics.getProxyOpNoNamenodes()); + + // At this time, the router has recorded 2 standby namenodes in memory, + // the operation can be successful without waiting for the router load cache. + assertTrue(routerCacheNoActiveNamenode(routerContext, "ns0", true)); + } + + /** + * Determine whether cache of the router has an active namenode. + * + * @return true if no active namenode, otherwise false. + */ + private boolean routerCacheNoActiveNamenode( + RouterContext context, String nsId, boolean useObserver) throws IOException { + List namenodes + = context.getRouter().getNamenodeResolver().getNamenodesForNameserviceId(nsId, useObserver); + for (FederationNamenodeContext namenode : namenodes) { + if (namenode.getState() == FederationNamenodeServiceState.ACTIVE){ + return false; + } + } + return true; + } + + /** + * All routers in the cluster force loadcache. + */ + private void allRoutersLoadCache() { + for (MiniRouterDFSCluster.RouterContext context : cluster.getRouters()) { + // Update service cache + context.getRouter().getStateStore().refreshCaches(true); + } + } + + /** + * Set the second non-observer state namenode in the router cache to active. + */ + private void setSecondNonObserverNamenodeInTheRouterCacheActive( + int numberOfObserver, boolean useObserver) throws IOException { + List ns0 = routerContext.getRouter() + .getNamenodeResolver() + .getNamenodesForNameserviceId("ns0", useObserver); + + String nsId = ns0.get(numberOfObserver+1).getNamenodeId(); + cluster.switchToActive("ns0", nsId); + assertEquals(ACTIVE.ordinal(), + cluster.getNamenode("ns0", nsId).getNamenode().getNameNodeState()); + + } + + /** + * All routers in the cluster force heartbeat. + */ + private void allRoutersHeartbeat() throws IOException { + for (RouterContext context : cluster.getRouters()) { + // Manually trigger the heartbeat, but the router does not manually load the cache + Collection heartbeatServices = context + .getRouter().getNamenodeHeartbeatServices(); + for (NamenodeHeartbeatService service : heartbeatServices) { + service.periodicInvoke(); + } + } + } + + /** + * Transition the active namenode in the cluster to standby. + */ + private void transitionActiveToStandby() { + if (cluster.isHighAvailability()) { + for (String ns : cluster.getNameservices()) { + List nnList = cluster.getNamenodes(ns); + for (MiniRouterDFSCluster.NamenodeContext namenodeContext : nnList) { + if (namenodeContext.getNamenode().isActiveState()) { + cluster.switchToStandby(ns, namenodeContext.getNamenodeId()); + } + } + } + } + } + + /** + * Shutdown observer namenode in the cluster. + * + * @param num The number of shutdown observer. + */ + private void stopObserver(int num) { + int nnIndex; + int numNns = cluster.getNamenodes().size(); + for (nnIndex = 0; nnIndex < numNns && num > 0; nnIndex++) { + NameNode nameNode = cluster.getCluster().getNameNode(nnIndex); + if (nameNode != null && nameNode.isObserverState()) { + cluster.getCluster().shutdownNameNode(nnIndex); + num--; + } + } + } +} From 469f20a52b7b59f1c5dfffe88193502d65e05c6e Mon Sep 17 00:00:00 2001 From: huhaiyang Date: Fri, 17 Nov 2023 11:50:08 +0800 Subject: [PATCH 07/10] HDFS-17172. Support FSNamesystemLock Parameters reconfigurable. (#6002). Contributed by Haiyang Hu. Reviewed-by: ZanderXu Signed-off-by: He Xiaoqiao --- .../hdfs/server/namenode/FSNamesystem.java | 27 +++++++ .../server/namenode/FSNamesystemLock.java | 33 +++++++- .../hadoop/hdfs/server/namenode/NameNode.java | 57 +++++++++++++- .../namenode/TestNameNodeReconfigure.java | 75 +++++++++++++++++++ .../hadoop/hdfs/tools/TestDFSAdmin.java | 6 +- 5 files changed, 191 insertions(+), 7 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 3c6efb78dd1fd..dea9c57f65f58 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 @@ -9124,4 +9124,31 @@ Path getEnclosingRoot(final String srcArg) throws IOException { return new Path("/"); } } + + public void setMetricsEnabled(boolean metricsEnabled) { + this.fsLock.setMetricsEnabled(metricsEnabled); + } + + @VisibleForTesting + public boolean isMetricsEnabled() { + return this.fsLock.isMetricsEnabled(); + } + + public void setReadLockReportingThresholdMs(long readLockReportingThresholdMs) { + this.fsLock.setReadLockReportingThresholdMs(readLockReportingThresholdMs); + } + + @VisibleForTesting + public long getReadLockReportingThresholdMs() { + return this.fsLock.getReadLockReportingThresholdMs(); + } + + public void setWriteLockReportingThresholdMs(long writeLockReportingThresholdMs) { + this.fsLock.setWriteLockReportingThresholdMs(writeLockReportingThresholdMs); + } + + @VisibleForTesting + public long getWriteLockReportingThresholdMs() { + return this.fsLock.getWriteLockReportingThresholdMs(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java index e510bd625e738..af641e964f46c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java @@ -68,7 +68,7 @@ class FSNamesystemLock { @VisibleForTesting protected ReentrantReadWriteLock coarseLock; - private final boolean metricsEnabled; + private volatile boolean metricsEnabled; private final MutableRatesWithAggregation detailedHoldTimeMetrics; private final Timer timer; @@ -79,14 +79,14 @@ class FSNamesystemLock { private final long lockSuppressWarningIntervalMs; /** Threshold (ms) for long holding write lock report. */ - private final long writeLockReportingThresholdMs; + private volatile long writeLockReportingThresholdMs; /** Last time stamp for write lock. Keep the longest one for multi-entrance.*/ private long writeLockHeldTimeStampNanos; /** Frequency limiter used for reporting long write lock hold times. */ private final LogThrottlingHelper writeLockReportLogger; /** Threshold (ms) for long holding read lock report. */ - private final long readLockReportingThresholdMs; + private volatile long readLockReportingThresholdMs; /** * Last time stamp for read lock. Keep the longest one for * multi-entrance. This is ThreadLocal since there could be @@ -462,6 +462,33 @@ private static String getMetricName(String operationName, boolean isWrite) { LOCK_METRIC_SUFFIX; } + @VisibleForTesting + public void setMetricsEnabled(boolean metricsEnabled) { + this.metricsEnabled = metricsEnabled; + } + + public boolean isMetricsEnabled() { + return metricsEnabled; + } + + public void setReadLockReportingThresholdMs(long readLockReportingThresholdMs) { + this.readLockReportingThresholdMs = readLockReportingThresholdMs; + } + + @VisibleForTesting + public long getReadLockReportingThresholdMs() { + return readLockReportingThresholdMs; + } + + public void setWriteLockReportingThresholdMs(long writeLockReportingThresholdMs) { + this.writeLockReportingThresholdMs = writeLockReportingThresholdMs; + } + + @VisibleForTesting + public long getWriteLockReportingThresholdMs() { + return writeLockReportingThresholdMs; + } + /** * Read lock Held Info. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java index df490ea0d9fe0..cd74511e6bbca 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java @@ -139,6 +139,12 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NN_NOT_BECOME_ACTIVE_IN_SAFEMODE_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LOCK_DETAILED_METRICS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BLOCKPLACEMENTPOLICY_MIN_BLOCKS_FOR_WRITE_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BLOCKPLACEMENTPOLICY_MIN_BLOCKS_FOR_WRITE_KEY; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT; @@ -371,7 +377,10 @@ public enum OperationCategory { DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_BLOCKS_PER_LOCK, DFS_NAMENODE_BLOCKPLACEMENTPOLICY_MIN_BLOCKS_FOR_WRITE_KEY, IPC_SERVER_LOG_SLOW_RPC, - IPC_SERVER_LOG_SLOW_RPC_THRESHOLD_MS_KEY)); + IPC_SERVER_LOG_SLOW_RPC_THRESHOLD_MS_KEY, + DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, + DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY, + DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY)); private static final String USAGE = "Usage: hdfs namenode [" + StartupOption.BACKUP.getName() + "] | \n\t[" @@ -2378,6 +2387,10 @@ protected String reconfigurePropertyImpl(String property, String newVal) } else if (property.equals(IPC_SERVER_LOG_SLOW_RPC) || (property.equals(IPC_SERVER_LOG_SLOW_RPC_THRESHOLD_MS_KEY))) { return reconfigureLogSlowRPC(property, newVal); + } else if (property.equals(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY) + || property.equals(DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY) + || property.equals(DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY)) { + return reconfigureFSNamesystemLockMetricsParameters(property, newVal); } else { throw new ReconfigurationException(property, newVal, getConf().get( property)); @@ -2736,6 +2749,48 @@ private String reconfigureMinBlocksForWrite(String property, String newValue) } } + private String reconfigureFSNamesystemLockMetricsParameters(final String property, + final String newVal) throws ReconfigurationException { + String result; + try { + switch (property) { + case DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY: { + if (newVal != null && !newVal.equalsIgnoreCase("true") && + !newVal.equalsIgnoreCase("false")) { + throw new IllegalArgumentException(newVal + " is not boolean value"); + } + boolean enable = (newVal == null ? + DFS_NAMENODE_LOCK_DETAILED_METRICS_DEFAULT : Boolean.parseBoolean(newVal)); + result = Boolean.toString(enable); + namesystem.setMetricsEnabled(enable); + break; + } + case DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY: { + long readLockReportingThresholdMs = (newVal == null ? + DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT : Long.parseLong(newVal)); + result = Long.toString(readLockReportingThresholdMs); + namesystem.setReadLockReportingThresholdMs(readLockReportingThresholdMs); + break; + } + case DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY: { + long writeLockReportingThresholdMs = (newVal == null ? + DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT : Long.parseLong(newVal)); + result = Long.toString(writeLockReportingThresholdMs); + namesystem.setWriteLockReportingThresholdMs(writeLockReportingThresholdMs); + break; + } + default: { + throw new IllegalArgumentException("Unexpected property " + property + " in " + + "reconfigureFSNamesystemLockMetricsParameters"); + } + } + LOG.info("RECONFIGURE* changed FSNamesystemLockMetricsParameters {} to {}", property, result); + return result; + } catch (IllegalArgumentException e){ + throw new ReconfigurationException(property, newVal, getConf().get(property), e); + } + } + @Override // ReconfigurableBase protected Configuration getNewConf() { return new HdfsConfiguration(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java index 5a0f62a8117e0..d116da5a8167b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java @@ -34,6 +34,9 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_SERVER_LOG_SLOW_RPC_THRESHOLD_MS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_NODES_TO_REPORT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY; import static org.junit.Assert.*; import org.slf4j.Logger; @@ -747,6 +750,78 @@ public void testReconfigureLogSlowRPC() throws ReconfigurationException { assertEquals(nnrs.getClientRpcServer().getLogSlowRPCThresholdTime(), 20000); } + @Test + public void testReconfigureFSNamesystemLockMetricsParameters() + throws ReconfigurationException, IOException { + Configuration conf = new HdfsConfiguration(); + conf.setBoolean(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, false); + long defaultReadLockMS = 1000L; + conf.setLong(DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY, defaultReadLockMS); + long defaultWriteLockMS = 1000L; + conf.setLong(DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY, defaultWriteLockMS); + + try (MiniDFSCluster newCluster = new MiniDFSCluster.Builder(conf).build()) { + newCluster.waitActive(); + final NameNode nameNode = newCluster.getNameNode(); + final FSNamesystem fsNamesystem = nameNode.getNamesystem(); + // verify default value. + assertFalse(fsNamesystem.isMetricsEnabled()); + assertEquals(defaultReadLockMS, fsNamesystem.getReadLockReportingThresholdMs()); + assertEquals(defaultWriteLockMS, fsNamesystem.getWriteLockReportingThresholdMs()); + + // try invalid metricsEnabled. + try { + nameNode.reconfigurePropertyImpl(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, + "non-boolean"); + fail("should not reach here"); + } catch (ReconfigurationException e) { + assertEquals( + "Could not change property dfs.namenode.lock.detailed-metrics.enabled from " + + "'false' to 'non-boolean'", e.getMessage()); + } + + // try correct metricsEnabled. + nameNode.reconfigurePropertyImpl(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, "true"); + assertTrue(fsNamesystem.isMetricsEnabled()); + + nameNode.reconfigurePropertyImpl(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, null); + assertFalse(fsNamesystem.isMetricsEnabled()); + + // try invalid readLockMS. + try { + nameNode.reconfigureProperty(DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY, + "non-numeric"); + fail("Should not reach here"); + } catch (ReconfigurationException e) { + assertEquals("Could not change property " + + "dfs.namenode.read-lock-reporting-threshold-ms from '" + + defaultReadLockMS + "' to 'non-numeric'", e.getMessage()); + } + + // try correct readLockMS. + nameNode.reconfigureProperty(DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY, + "20000"); + assertEquals(fsNamesystem.getReadLockReportingThresholdMs(), 20000); + + + // try invalid writeLockMS. + try { + nameNode.reconfigureProperty( + DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY, "non-numeric"); + fail("Should not reach here"); + } catch (ReconfigurationException e) { + assertEquals("Could not change property " + + "dfs.namenode.write-lock-reporting-threshold-ms from '" + + defaultWriteLockMS + "' to 'non-numeric'", e.getMessage()); + } + + // try correct writeLockMS. + nameNode.reconfigureProperty( + DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY, "100000"); + assertEquals(fsNamesystem.getWriteLockReportingThresholdMs(), 100000); + } + } + @After public void shutDown() throws IOException { if (cluster != null) { 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 1712c620d2c82..2a3fba83ffcb2 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 @@ -99,7 +99,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY; import static org.apache.hadoop.hdfs.client.HdfsAdmin.TRASH_PERMISSION; import static org.hamcrest.CoreMatchers.allOf; import static org.hamcrest.CoreMatchers.anyOf; @@ -442,7 +442,7 @@ public void testNameNodeGetReconfigurableProperties() throws IOException, Interr final List outs = Lists.newArrayList(); final List errs = Lists.newArrayList(); getReconfigurableProperties("namenode", address, outs, errs); - assertEquals(25, outs.size()); + assertEquals(28, outs.size()); assertTrue(outs.get(0).contains("Reconfigurable properties:")); assertEquals(DFS_BLOCK_INVALIDATE_LIMIT_KEY, outs.get(1)); assertEquals(DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY, outs.get(2)); @@ -457,7 +457,7 @@ public void testNameNodeGetReconfigurableProperties() throws IOException, Interr assertEquals(DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_BLOCKS_PER_LOCK, outs.get(11)); assertEquals(DFS_NAMENODE_DECOMMISSION_BACKOFF_MONITOR_PENDING_LIMIT, outs.get(12)); assertEquals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, outs.get(13)); - assertEquals(DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY, outs.get(14)); + assertEquals(DFS_NAMENODE_LOCK_DETAILED_METRICS_KEY, outs.get(14)); assertEquals(errs.size(), 0); } From 28068aa3208442dc304e4854b6367683b3811abb Mon Sep 17 00:00:00 2001 From: QI Jiale Date: Fri, 17 Nov 2023 11:59:04 +0800 Subject: [PATCH 08/10] HDFS-17063. Support to configure different capacity reserved for each disk of DataNode. (#5793). Contributed by QI Jiale. Reviewed-by : Tao Li Signed-off-by: He Xiaoqiao --- .../datanode/fsdataset/impl/FsVolumeImpl.java | 3 +- .../impl/ReservedSpaceCalculator.java | 43 +++++++++----- .../src/main/resources/hdfs-default.xml | 20 ++++++- .../impl/TestReservedSpaceCalculator.java | 56 ++++++++++++++++++- 4 files changed, 104 insertions(+), 18 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java index bf68f96a6dab0..2935e6ae3221a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java @@ -171,7 +171,8 @@ public class FsVolumeImpl implements FsVolumeSpi { this.usage = usage; if (this.usage != null) { reserved = new ReservedSpaceCalculator.Builder(conf) - .setUsage(this.usage).setStorageType(storageType).build(); + .setUsage(this.usage).setStorageType(storageType) + .setDir(currentDir != null ? currentDir.getParent() : "NULL").build(); boolean fixedSizeVolume = conf.getBoolean( DFSConfigKeys.DFS_DATANODE_FIXED_VOLUME_SIZE_KEY, DFSConfigKeys.DFS_DATANODE_FIXED_VOLUME_SIZE_DEFAULT); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReservedSpaceCalculator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReservedSpaceCalculator.java index 749e16e659c44..0ab4032104ac9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReservedSpaceCalculator.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReservedSpaceCalculator.java @@ -46,6 +46,8 @@ public static class Builder { private DF usage; private StorageType storageType; + private String dir; + public Builder(Configuration conf) { this.conf = conf; } @@ -61,6 +63,11 @@ public Builder setStorageType( return this; } + public Builder setDir(String newDir) { + this.dir = newDir; + return this; + } + ReservedSpaceCalculator build() { try { Class clazz = conf.getClass( @@ -69,10 +76,10 @@ ReservedSpaceCalculator build() { ReservedSpaceCalculator.class); Constructor constructor = clazz.getConstructor( - Configuration.class, DF.class, StorageType.class); + Configuration.class, DF.class, StorageType.class, String.class); return (ReservedSpaceCalculator) constructor.newInstance( - conf, usage, storageType); + conf, usage, storageType, dir); } catch (Exception e) { throw new IllegalStateException( "Error instantiating ReservedSpaceCalculator", e); @@ -84,20 +91,30 @@ ReservedSpaceCalculator build() { private final Configuration conf; private final StorageType storageType; + private final String dir; + ReservedSpaceCalculator(Configuration conf, DF usage, - StorageType storageType) { + StorageType storageType, String dir) { this.usage = usage; this.conf = conf; this.storageType = storageType; + this.dir = dir; } DF getUsage() { return usage; } + String getDir() { + return dir; + } + long getReservedFromConf(String key, long defaultValue) { - return conf.getLong(key + "." + StringUtils.toLowerCase( - storageType.toString()), conf.getLongBytes(key, defaultValue)); + return conf.getLong( + key + "." + getDir() + "." + StringUtils.toLowerCase(storageType.toString()), + conf.getLong(key + "." + getDir(), + conf.getLong(key + "." + StringUtils.toLowerCase(storageType.toString()), + conf.getLongBytes(key, defaultValue)))); } /** @@ -117,8 +134,8 @@ public static class ReservedSpaceCalculatorAbsolute extends private final long reservedBytes; public ReservedSpaceCalculatorAbsolute(Configuration conf, DF usage, - StorageType storageType) { - super(conf, usage, storageType); + StorageType storageType, String dir) { + super(conf, usage, storageType, dir); this.reservedBytes = getReservedFromConf(DFS_DATANODE_DU_RESERVED_KEY, DFS_DATANODE_DU_RESERVED_DEFAULT); } @@ -138,8 +155,8 @@ public static class ReservedSpaceCalculatorPercentage extends private final long reservedPct; public ReservedSpaceCalculatorPercentage(Configuration conf, DF usage, - StorageType storageType) { - super(conf, usage, storageType); + StorageType storageType, String dir) { + super(conf, usage, storageType, dir); this.reservedPct = getReservedFromConf( DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY, DFS_DATANODE_DU_RESERVED_PERCENTAGE_DEFAULT); @@ -162,8 +179,8 @@ public static class ReservedSpaceCalculatorConservative extends private final long reservedPct; public ReservedSpaceCalculatorConservative(Configuration conf, DF usage, - StorageType storageType) { - super(conf, usage, storageType); + StorageType storageType, String dir) { + super(conf, usage, storageType, dir); this.reservedBytes = getReservedFromConf(DFS_DATANODE_DU_RESERVED_KEY, DFS_DATANODE_DU_RESERVED_DEFAULT); this.reservedPct = getReservedFromConf( @@ -197,8 +214,8 @@ public static class ReservedSpaceCalculatorAggressive extends private final long reservedPct; public ReservedSpaceCalculatorAggressive(Configuration conf, DF usage, - StorageType storageType) { - super(conf, usage, storageType); + StorageType storageType, String dir) { + super(conf, usage, storageType, dir); this.reservedBytes = getReservedFromConf(DFS_DATANODE_DU_RESERVED_KEY, DFS_DATANODE_DU_RESERVED_DEFAULT); this.reservedPct = getReservedFromConf( 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 52075a24f1e32..3e9f54f58ff8f 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 @@ -397,12 +397,19 @@ dfs.datanode.du.reserved 0 Reserved space in bytes per volume. Always leave this much space free for non dfs use. + Specific directory based reservation is supported. The property can be followed with directory + name which is set at 'dfs.datanode.data.dir'. For example, reserved space for /data/hdfs1/data + can be configured using property 'dfs.datanode.du.reserved./data/hdfs1/data'. If specific directory + reservation is not configured then dfs.datanode.du.reserved will be used. Specific storage type based reservation is also supported. The property can be followed with corresponding storage types ([ssd]/[disk]/[archive]/[ram_disk]/[nvdimm]) for cluster with heterogeneous storage. For example, reserved space for RAM_DISK storage can be configured using property 'dfs.datanode.du.reserved.ram_disk'. If specific storage type reservation is not configured then dfs.datanode.du.reserved will be used. Support multiple size unit suffix(case insensitive), - as described in dfs.blocksize. + as described in dfs.blocksize. Use directory name and storage type based reservation at the + same time is also allowed if both are configured. + Property priority example: dfs.datanode.du.reserved./data/hdfs1/data.ram_disk > + dfs.datanode.du.reserved./data/hdfs1/data > dfs.datanode.du.reserved.ram_disk > dfs.datanode.du.reserved Note: In case of using tune2fs to set reserved-blocks-percentage, or other filesystem tools, then you can possibly run into out of disk errors because hadoop will not check those external tool configurations. @@ -414,12 +421,19 @@ 0 Reserved space in percentage. Read dfs.datanode.du.reserved.calculator to see when this takes effect. The actual number of bytes reserved will be calculated by using the - total capacity of the data directory in question. Specific storage type based reservation + total capacity of the data directory in question. Specific directory based reservation is + supported. The property can be followed with directory name which is set at 'dfs.datanode.data.dir'. + For example, reserved percentage space for /data/hdfs1/data can be configured using property + 'dfs.datanode.du.reserved.pct./data/hdfs1/data'. If specific directory reservation is not + configured then dfs.datanode.du.reserved.pct will be used. Specific storage type based reservation is also supported. The property can be followed with corresponding storage types ([ssd]/[disk]/[archive]/[ram_disk]/[nvdimm]) for cluster with heterogeneous storage. For example, reserved percentage space for RAM_DISK storage can be configured using property 'dfs.datanode.du.reserved.pct.ram_disk'. If specific storage type reservation is not configured - then dfs.datanode.du.reserved.pct will be used. + then dfs.datanode.du.reserved.pct will be used. Use directory and storage type based reservation + is also allowed if both are configured. + Priority example: dfs.datanode.du.reserved.pct./data/hdfs1/data.ram_disk > dfs.datanode.du.reserved.pct./data/hdfs1/data + > dfs.datanode.du.reserved.pct.ram_disk > dfs.datanode.du.reserved.pct diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReservedSpaceCalculator.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReservedSpaceCalculator.java index fa666f2a691a7..10136863964b9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReservedSpaceCalculator.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReservedSpaceCalculator.java @@ -168,6 +168,55 @@ public void testReservedSpaceAggresivePerStorageType() { checkReserved(StorageType.ARCHIVE, 100000, 5000); } + @Test + public void testReservedSpaceAbsolutePerDir() { + conf.setClass(DFS_DATANODE_DU_RESERVED_CALCULATOR_KEY, ReservedSpaceCalculatorAbsolute.class, + ReservedSpaceCalculator.class); + + String dir1 = "/data/hdfs1/data"; + String dir2 = "/data/hdfs2/data"; + String dir3 = "/data/hdfs3/data"; + + conf.setLong(DFS_DATANODE_DU_RESERVED_KEY + "." + dir1 + ".ssd", 900); + conf.setLong(DFS_DATANODE_DU_RESERVED_KEY + "." + dir1, 1800); + conf.setLong(DFS_DATANODE_DU_RESERVED_KEY + "." + dir2, 2700); + conf.setLong(DFS_DATANODE_DU_RESERVED_KEY + ".ssd", 3600); + conf.setLong(DFS_DATANODE_DU_RESERVED_KEY, 4500); + + checkReserved(StorageType.SSD, 10000, 900, dir1); + checkReserved(StorageType.DISK, 10000, 1800, dir1); + checkReserved(StorageType.SSD, 10000, 2700, dir2); + checkReserved(StorageType.DISK, 10000, 2700, dir2); + checkReserved(StorageType.SSD, 10000, 3600, dir3); + checkReserved(StorageType.DISK, 10000, 4500, dir3); + } + + @Test + public void testReservedSpacePercentagePerDir() { + conf.setClass(DFS_DATANODE_DU_RESERVED_CALCULATOR_KEY, + ReservedSpaceCalculatorPercentage.class, + ReservedSpaceCalculator.class); + + String dir1 = "/data/hdfs1/data"; + String dir2 = "/data/hdfs2/data"; + String dir3 = "/data/hdfs3/data"; + + // Set percentage reserved values for different directories + conf.setLong(DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY + "." + dir1 + ".ssd", 20); + conf.setLong(DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY + "." + dir1, 10); + conf.setLong(DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY + "." + dir2, 25); + conf.setLong(DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY + ".ssd", 30); + conf.setLong(DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY, 40); + + // Verify reserved space calculations for different directories and storage types + checkReserved(StorageType.SSD, 10000, 2000, dir1); + checkReserved(StorageType.DISK, 10000, 1000, dir1); + checkReserved(StorageType.SSD, 10000, 2500, dir2); + checkReserved(StorageType.DISK, 10000, 2500, dir2); + checkReserved(StorageType.SSD, 10000, 3000, dir3); + checkReserved(StorageType.DISK, 10000, 4000, dir3); + } + @Test(expected = IllegalStateException.class) public void testInvalidCalculator() { conf.set(DFS_DATANODE_DU_RESERVED_CALCULATOR_KEY, "INVALIDTYPE"); @@ -179,10 +228,15 @@ public void testInvalidCalculator() { private void checkReserved(StorageType storageType, long totalCapacity, long reservedExpected) { + checkReserved(storageType, totalCapacity, reservedExpected, "NULL"); + } + + private void checkReserved(StorageType storageType, + long totalCapacity, long reservedExpected, String dir) { when(usage.getCapacity()).thenReturn(totalCapacity); reserved = new ReservedSpaceCalculator.Builder(conf).setUsage(usage) - .setStorageType(storageType).build(); + .setStorageType(storageType).setDir(dir).build(); assertEquals(reservedExpected, reserved.getReserved()); } } \ No newline at end of file From 7a55442297228dbc2e24b6c0fb9a513e93c8435c Mon Sep 17 00:00:00 2001 From: Istvan Fajth Date: Fri, 17 Nov 2023 10:51:43 +0100 Subject: [PATCH 09/10] HADOOP-18956. Zookeeper SSL/TLS support in ZKDelegationTokenSecretManager and ZKSignerSecretProvider (#6263) --- .../util/ZKSignerSecretProvider.java | 117 ++-- .../authentication/util/ZookeeperClient.java | 318 +++++++++++ .../src/site/markdown/Configuration.md | 15 + .../util/TestZookeeperClientCreation.java | 498 ++++++++++++++++++ .../hadoop/fs/CommonConfigurationKeys.java | 5 + .../ZKDelegationTokenSecretManager.java | 197 +++---- .../src/main/resources/core-default.xml | 12 + .../hdfs/tools/DFSZKFailoverController.java | 7 +- .../src/main/resources/hdfs-default.xml | 2 + .../src/main/resources/yarn-default.xml | 6 +- ...tiveStandbyElectorBasedElectorService.java | 6 +- .../resourcemanager/ResourceManager.java | 8 +- 12 files changed, 982 insertions(+), 209 deletions(-) create mode 100644 hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZookeeperClient.java create mode 100644 hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZookeeperClientCreation.java diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java index 53b6f4d239f98..b0604c85c39f4 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java @@ -16,25 +16,13 @@ import org.apache.hadoop.classification.VisibleForTesting; import java.nio.ByteBuffer; import java.security.SecureRandom; -import java.util.Collections; -import java.util.List; import java.util.Properties; import java.util.Random; -import javax.security.auth.login.Configuration; import javax.servlet.ServletContext; -import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.framework.api.ACLProvider; -import org.apache.curator.framework.imps.DefaultACLProvider; -import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooDefs.Perms; -import org.apache.zookeeper.client.ZKClientConfig; -import org.apache.zookeeper.data.ACL; -import org.apache.zookeeper.data.Id; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -92,6 +80,16 @@ public class ZKSignerSecretProvider extends RolloverSignerSecretProvider { public static final String ZOOKEEPER_KERBEROS_PRINCIPAL = CONFIG_PREFIX + "kerberos.principal"; + public static final String ZOOKEEPER_SSL_ENABLED = CONFIG_PREFIX + "ssl.enabled"; + public static final String ZOOKEEPER_SSL_KEYSTORE_LOCATION = + CONFIG_PREFIX + "ssl.keystore.location"; + public static final String ZOOKEEPER_SSL_KEYSTORE_PASSWORD = + CONFIG_PREFIX + "ssl.keystore.password"; + public static final String ZOOKEEPER_SSL_TRUSTSTORE_LOCATION = + CONFIG_PREFIX + "ssl.truststore.location"; + public static final String ZOOKEEPER_SSL_TRUSTSTORE_PASSWORD = + CONFIG_PREFIX + "ssl.truststore.password"; + /** * Constant for the property that specifies whether or not the Curator client * should disconnect from ZooKeeper on shutdown. The default is "true". Only @@ -350,80 +348,33 @@ protected byte[] generateRandomSecret() { * This method creates the Curator client and connects to ZooKeeper. * @param config configuration properties * @return A Curator client - * @throws Exception thrown if an error occurred */ - protected CuratorFramework createCuratorClient(Properties config) - throws Exception { - String connectionString = config.getProperty( - ZOOKEEPER_CONNECTION_STRING, "localhost:2181"); - - RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); - ACLProvider aclProvider; + protected CuratorFramework createCuratorClient(Properties config) { + String connectionString = config.getProperty(ZOOKEEPER_CONNECTION_STRING, "localhost:2181"); String authType = config.getProperty(ZOOKEEPER_AUTH_TYPE, "none"); - if (authType.equals("sasl")) { - LOG.info("Connecting to ZooKeeper with SASL/Kerberos" - + "and using 'sasl' ACLs"); - String principal = setJaasConfiguration(config); - System.setProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY, - JAAS_LOGIN_ENTRY_NAME); - System.setProperty("zookeeper.authProvider.1", - "org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); - aclProvider = new SASLOwnerACLProvider(principal); - } else { // "none" - LOG.info("Connecting to ZooKeeper without authentication"); - aclProvider = new DefaultACLProvider(); // open to everyone - } - CuratorFramework cf = CuratorFrameworkFactory.builder() - .connectString(connectionString) - .retryPolicy(retryPolicy) - .aclProvider(aclProvider) - .build(); - cf.start(); - return cf; - } - - private String setJaasConfiguration(Properties config) throws Exception { - String keytabFile = config.getProperty(ZOOKEEPER_KERBEROS_KEYTAB).trim(); - if (keytabFile == null || keytabFile.length() == 0) { - throw new IllegalArgumentException(ZOOKEEPER_KERBEROS_KEYTAB - + " must be specified"); - } - String principal = config.getProperty(ZOOKEEPER_KERBEROS_PRINCIPAL) - .trim(); - if (principal == null || principal.length() == 0) { - throw new IllegalArgumentException(ZOOKEEPER_KERBEROS_PRINCIPAL - + " must be specified"); - } + String keytab = config.getProperty(ZOOKEEPER_KERBEROS_KEYTAB, "").trim(); + String principal = config.getProperty(ZOOKEEPER_KERBEROS_PRINCIPAL, "").trim(); - // This is equivalent to writing a jaas.conf file and setting the system - // property, "java.security.auth.login.config", to point to it - JaasConfiguration jConf = - new JaasConfiguration(JAAS_LOGIN_ENTRY_NAME, principal, keytabFile); - Configuration.setConfiguration(jConf); - return principal.split("[/@]")[0]; - } + boolean sslEnabled = Boolean.parseBoolean(config.getProperty(ZOOKEEPER_SSL_ENABLED, "false")); + String keystoreLocation = config.getProperty(ZOOKEEPER_SSL_KEYSTORE_LOCATION, ""); + String keystorePassword = config.getProperty(ZOOKEEPER_SSL_KEYSTORE_PASSWORD, ""); + String truststoreLocation = config.getProperty(ZOOKEEPER_SSL_TRUSTSTORE_LOCATION, ""); + String truststorePassword = config.getProperty(ZOOKEEPER_SSL_TRUSTSTORE_PASSWORD, ""); - /** - * Simple implementation of an {@link ACLProvider} that simply returns an ACL - * that gives all permissions only to a single principal. - */ - private static class SASLOwnerACLProvider implements ACLProvider { - - private final List saslACL; - - private SASLOwnerACLProvider(String principal) { - this.saslACL = Collections.singletonList( - new ACL(Perms.ALL, new Id("sasl", principal))); - } - - @Override - public List getDefaultAcl() { - return saslACL; - } - - @Override - public List getAclForPath(String path) { - return saslACL; - } + CuratorFramework zkClient = + ZookeeperClient.configure() + .withConnectionString(connectionString) + .withAuthType(authType) + .withKeytab(keytab) + .withPrincipal(principal) + .withJaasLoginEntryName(JAAS_LOGIN_ENTRY_NAME) + .enableSSL(sslEnabled) + .withKeystore(keystoreLocation) + .withKeystorePassword(keystorePassword) + .withTruststore(truststoreLocation) + .withTruststorePassword(truststorePassword) + .create(); + zkClient.start(); + return zkClient; } } diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZookeeperClient.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZookeeperClient.java new file mode 100644 index 0000000000000..38b06a0ac7d9b --- /dev/null +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZookeeperClient.java @@ -0,0 +1,318 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ + +package org.apache.hadoop.security.authentication.util; + +import org.apache.curator.RetryPolicy; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.api.ACLProvider; +import org.apache.curator.framework.imps.DefaultACLProvider; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.utils.ConfigurableZookeeperFactory; +import org.apache.curator.utils.ZookeeperFactory; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.client.ZKClientConfig; +import org.apache.zookeeper.common.ClientX509Util; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Id; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.login.Configuration; +import java.util.Collections; +import java.util.List; + +/** + * Utility class to create a CuratorFramework object that can be used to connect to Zookeeper + * based on configuration values that can be supplied from different configuration properties. + * It is used from ZKDelegationTokenSecretManager in hadoop-common, and from + * {@link ZKSignerSecretProvider}. + * + * The class implements a fluid API to set up all the different properties. A very basic setup + * would seem like: + *
+ *   ZookeeperClient.configure()
+ *     .withConnectionString(<connectionString>)
+ *     .create();
+ * 
+ * + * Mandatory parameters to be set: + *
    + *
  • connectionString: A Zookeeper connection string.
  • + *
  • if authentication type is set to 'sasl': + *
      + *
    • keytab: the location of the keytab to be used for Kerberos authentication
    • + *
    • principal: the Kerberos principal to be used from the supplied Kerberos keytab file.
    • + *
    • jaasLoginEntryName: the login entry name in the JAAS configuration that is created for + * the KerberosLoginModule to be used by the Zookeeper client code.
    • + *
    + *
  • + *
  • if SSL is enabled: + *
      + *
    • the location of the Truststore file to be used
    • + *
    • the location of the Keystore file to be used
    • + *
    • if the Truststore is protected by a password, then the password of the Truststore
    • + *
    • if the Keystore is protected by a password, then the password if the Keystore
    • + *
    + *
  • + *
+ * + * When using 'sasl' authentication type, the JAAS configuration to be used by the Zookeeper client + * withing CuratorFramework is set to use the supplied keytab and principal for Kerberos login, + * moreover an ACL provider is set to provide a default ACL that requires SASL auth and the same + * principal to have access to the used paths. + * + * When using SSL/TLS, the Zookeeper client will set to use the secure channel towards Zookeeper, + * with the specified Keystore and Truststore. + * + * Default values: + *
    + *
  • authentication type: 'none'
  • + *
  • sessionTimeout: either the system property curator-default-session-timeout, or 60 + * seconds
  • + *
  • connectionTimeout: either the system property curator-default-connection-timeout, or 15 + * seconds
  • + *
  • retryPolicy: an ExponentialBackoffRetry, with a starting interval of 1 seconds and 3 + * retries
  • + *
  • zkFactory: a ConfigurableZookeeperFactory instance, to allow SSL setup via + * ZKClientConfig
  • + *
+ * + * @see ZKSignerSecretProvider + */ +public class ZookeeperClient { + + private static final Logger LOG = LoggerFactory.getLogger(ZookeeperClient.class); + + private String connectionString; + private String namespace; + + private String authenticationType = "none"; + private String keytab; + private String principal; + private String jaasLoginEntryName; + + private int sessionTimeout = + Integer.getInteger("curator-default-session-timeout", 60 * 1000); + private int connectionTimeout = + Integer.getInteger("curator-default-connection-timeout", 15 * 1000); + + private RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); + + private ZookeeperFactory zkFactory = new ConfigurableZookeeperFactory(); + + private boolean isSSLEnabled; + private String keystoreLocation; + private String keystorePassword; + private String truststoreLocation; + private String truststorePassword; + + public static ZookeeperClient configure() { + return new ZookeeperClient(); + } + + public ZookeeperClient withConnectionString(String conn) { + connectionString = conn; + return this; + } + + public ZookeeperClient withNamespace(String ns) { + this.namespace = ns; + return this; + } + + public ZookeeperClient withAuthType(String authType) { + this.authenticationType = authType; + return this; + } + + public ZookeeperClient withKeytab(String keytabPath) { + this.keytab = keytabPath; + return this; + } + + public ZookeeperClient withPrincipal(String princ) { + this.principal = princ; + return this; + } + + public ZookeeperClient withJaasLoginEntryName(String entryName) { + this.jaasLoginEntryName = entryName; + return this; + } + + public ZookeeperClient withSessionTimeout(int timeoutMS) { + this.sessionTimeout = timeoutMS; + return this; + } + + public ZookeeperClient withConnectionTimeout(int timeoutMS) { + this.connectionTimeout = timeoutMS; + return this; + } + + public ZookeeperClient withRetryPolicy(RetryPolicy policy) { + this.retryPolicy = policy; + return this; + } + + public ZookeeperClient withZookeeperFactory(ZookeeperFactory factory) { + this.zkFactory = factory; + return this; + } + + public ZookeeperClient enableSSL(boolean enable) { + this.isSSLEnabled = enable; + return this; + } + + public ZookeeperClient withKeystore(String keystorePath) { + this.keystoreLocation = keystorePath; + return this; + } + + public ZookeeperClient withKeystorePassword(String keystorePass) { + this.keystorePassword = keystorePass; + return this; + } + + public ZookeeperClient withTruststore(String truststorePath) { + this.truststoreLocation = truststorePath; + return this; + } + + public ZookeeperClient withTruststorePassword(String truststorePass) { + this.truststorePassword = truststorePass; + return this; + } + + public CuratorFramework create() { + checkNotNull(connectionString, "Zookeeper connection string cannot be null!"); + checkNotNull(retryPolicy, "Zookeeper connection retry policy cannot be null!"); + + return createFrameworkFactoryBuilder() + .connectString(connectionString) + .zookeeperFactory(zkFactory) + .namespace(namespace) + .sessionTimeoutMs(sessionTimeout) + .connectionTimeoutMs(connectionTimeout) + .retryPolicy(retryPolicy) + .aclProvider(aclProvider()) + .zkClientConfig(zkClientConfig()) + .build(); + } + + @VisibleForTesting + CuratorFrameworkFactory.Builder createFrameworkFactoryBuilder() { + return CuratorFrameworkFactory.builder(); + } + + private ACLProvider aclProvider() { + // AuthType has to be explicitly set to 'none' or 'sasl' + checkNotNull(authenticationType, "Zookeeper authType cannot be null!"); + checkArgument(authenticationType.equals("sasl") || authenticationType.equals("none"), + "Zookeeper authType must be one of [none, sasl]!"); + + ACLProvider aclProvider; + if (authenticationType.equals("sasl")) { + LOG.info("Connecting to ZooKeeper with SASL/Kerberos and using 'sasl' ACLs."); + + checkArgument(!isEmpty(keytab), "Zookeeper client's Kerberos Keytab must be specified!"); + checkArgument(!isEmpty(principal), + "Zookeeper client's Kerberos Principal must be specified!"); + checkArgument(!isEmpty(jaasLoginEntryName), "JAAS Login Entry name must be specified!"); + + JaasConfiguration jConf = new JaasConfiguration(jaasLoginEntryName, principal, keytab); + Configuration.setConfiguration(jConf); + System.setProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY, jaasLoginEntryName); + System.setProperty("zookeeper.authProvider.1", + "org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); + aclProvider = new SASLOwnerACLProvider(principal.split("[/@]")[0]); + } else { // "none" + LOG.info("Connecting to ZooKeeper without authentication."); + aclProvider = new DefaultACLProvider(); // open to everyone + } + return aclProvider; + } + + private ZKClientConfig zkClientConfig() { + ZKClientConfig zkClientConfig = new ZKClientConfig(); + if (isSSLEnabled){ + LOG.info("Zookeeper client will use SSL connection. (keystore = {}; truststore = {};)", + keystoreLocation, truststoreLocation); + checkArgument(!isEmpty(keystoreLocation), + "The keystore location parameter is empty for the ZooKeeper client connection."); + checkArgument(!isEmpty(truststoreLocation), + "The truststore location parameter is empty for the ZooKeeper client connection."); + + try (ClientX509Util sslOpts = new ClientX509Util()) { + zkClientConfig.setProperty(ZKClientConfig.SECURE_CLIENT, "true"); + zkClientConfig.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, + "org.apache.zookeeper.ClientCnxnSocketNetty"); + zkClientConfig.setProperty(sslOpts.getSslKeystoreLocationProperty(), keystoreLocation); + zkClientConfig.setProperty(sslOpts.getSslKeystorePasswdProperty(), keystorePassword); + zkClientConfig.setProperty(sslOpts.getSslTruststoreLocationProperty(), truststoreLocation); + zkClientConfig.setProperty(sslOpts.getSslTruststorePasswdProperty(), truststorePassword); + } + } else { + LOG.info("Zookeeper client will use Plain connection."); + } + return zkClientConfig; + } + + /** + * Simple implementation of an {@link ACLProvider} that simply returns an ACL + * that gives all permissions only to a single principal. + */ + @VisibleForTesting + static final class SASLOwnerACLProvider implements ACLProvider { + + private final List saslACL; + + private SASLOwnerACLProvider(String principal) { + this.saslACL = Collections.singletonList( + new ACL(ZooDefs.Perms.ALL, new Id("sasl", principal))); + } + + @Override + public List getDefaultAcl() { + return saslACL; + } + + @Override + public List getAclForPath(String path) { + return saslACL; + } + } + + private boolean isEmpty(String str) { + return str == null || str.length() == 0; + } + + //Preconditions allowed to be imported from hadoop-common, but that results + // in a circular dependency + private void checkNotNull(Object reference, String errorMessage) { + if (reference == null) { + throw new NullPointerException(errorMessage); + } + } + + private void checkArgument(boolean expression, String errorMessage) { + if (!expression) { + throw new IllegalArgumentException(errorMessage); + } + } +} diff --git a/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md b/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md index 43597b688112d..147ba52055cc4 100644 --- a/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md +++ b/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md @@ -404,6 +404,21 @@ The following configuration properties are specific to the `zookeeper` implement * `signer.secret.provider.zookeeper.kerberos.principal`: Set this to the Kerberos principal to use. This only required if using Kerberos. +* `signer.secret.provider.zookeeper.ssl.enabled` : Set this to true to enable SSL/TLS + communication between the server and Zookeeper, if the SignerSecretProvider is zookeeper. + +* `signer.secret.provider.zookeeper.ssl.keystore.location` : Specifies the location of the + Zookeeper client's keystore file. + +* `signer.secret.provider.zookeeper.ssl.keystore.password` : Specifies the location of the + Zookeeper client's keystore password. + +* `signer.secret.provider.zookeeper.ssl.truststore.location` : Specifies the location of the + Zookeeper client's truststore file. + +* `signer.secret.provider.zookeeper.ssl.truststore.password` : Specifies the location of the + Zookeeper client's truststore password. + * `signer.secret.provider.zookeeper.disconnect.on.shutdown`: Whether to close the ZooKeeper connection when the provider is shutdown. The default value is `true`. Only set this to `false` if a custom Curator client is being provided and diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZookeeperClientCreation.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZookeeperClientCreation.java new file mode 100644 index 0000000000000..65cf3939ab44b --- /dev/null +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestZookeeperClientCreation.java @@ -0,0 +1,498 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ + +package org.apache.hadoop.security.authentication.util; + +import org.apache.curator.RetryPolicy; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.imps.DefaultACLProvider; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.utils.ConfigurableZookeeperFactory; +import org.apache.curator.utils.ZookeeperFactory; +import org.apache.hadoop.security.authentication.util.ZookeeperClient.SASLOwnerACLProvider; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.client.ZKClientConfig; +import org.apache.zookeeper.common.ClientX509Util; +import org.hamcrest.Matcher; +import org.hamcrest.core.IsNull; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; + +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; + +import java.util.Arrays; + +import static org.hamcrest.CoreMatchers.anyOf; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentCaptor.forClass; +import static org.mockito.ArgumentMatchers.isA; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests for ZookeeperClient class, to check if it creates CuratorFramework by providing expected + * parameter values to the CuratorFrameworkFactory.Builder instance. + */ +public class TestZookeeperClientCreation { + + private ZookeeperClient clientConfigurer; + private CuratorFrameworkFactory.Builder cfBuilder; + + @BeforeEach + public void setup() { + clientConfigurer = spy(ZookeeperClient.configure()); + clientConfigurer.withConnectionString("dummy"); + cfBuilder = spy(CuratorFrameworkFactory.builder()); + + when(clientConfigurer.createFrameworkFactoryBuilder()).thenReturn(cfBuilder); + } + + //Positive tests + @Test + public void testConnectionStringSet() { + clientConfigurer.withConnectionString("conn").create(); + + verify(cfBuilder).connectString("conn"); + + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + @Test + public void testZookeeperFactorySet() { + ZookeeperFactory zkFactory = mock(ZookeeperFactory.class); + clientConfigurer.withZookeeperFactory(zkFactory).create(); + + verify(cfBuilder).zookeeperFactory(zkFactory); + + verifyDummyConnectionString(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + @Test + public void testNameSpaceSet() { + clientConfigurer.withNamespace("someNS/someSubSpace").create(); + + verify(cfBuilder).namespace("someNS/someSubSpace"); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + @Test + public void testSessionTimeoutSet() { + clientConfigurer.withSessionTimeout(20000).create(); + + verify(cfBuilder).sessionTimeoutMs(20000); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + + @Test + public void testDefaultSessionTimeoutIsAffectedBySystemProperty() { + System.setProperty("curator-default-session-timeout", "20000"); + setup(); + clientConfigurer.create(); + + verify(cfBuilder).sessionTimeoutMs(20000); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + System.clearProperty("curator-default-session-timeout"); + } + + @Test + public void testConnectionTimeoutSet() { + clientConfigurer.withConnectionTimeout(50).create(); + + verify(cfBuilder).connectionTimeoutMs(50); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + @Test + public void testDefaultConnectionTimeoutIsAffectedBySystemProperty() { + System.setProperty("curator-default-connection-timeout", "50"); + setup(); + clientConfigurer.create(); + + verify(cfBuilder).connectionTimeoutMs(50); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + System.clearProperty("curator-default-connection-timeout"); + } + + @Test + public void testRetryPolicySet() { + RetryPolicy policy = mock(RetryPolicy.class); + clientConfigurer.withRetryPolicy(policy).create(); + + verify(cfBuilder).retryPolicy(policy); + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultAclProvider(); + verifyDefaultZKClientConfig(); + } + + @Test + public void testSaslAutTypeWithIBMJava() { + testSaslAuthType("IBMJava"); + } + + @Test + public void testSaslAuthTypeWithNonIBMJava() { + testSaslAuthType("OracleJava"); + } + + @Test + public void testSSLConfiguration() { + clientConfigurer + .enableSSL(true) + .withKeystore("keystoreLoc") + .withKeystorePassword("ksPass") + .withTruststore("truststoreLoc") + .withTruststorePassword("tsPass") + .create(); + + ArgumentCaptor clientConfCaptor = forClass(ZKClientConfig.class); + verify(cfBuilder).zkClientConfig(clientConfCaptor.capture()); + ZKClientConfig conf = clientConfCaptor.getValue(); + + assertThat(conf.getProperty(ZKClientConfig.SECURE_CLIENT), is("true")); + assertThat(conf.getProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET), + is("org.apache.zookeeper.ClientCnxnSocketNetty")); + try (ClientX509Util sslOpts = new ClientX509Util()) { + assertThat(conf.getProperty(sslOpts.getSslKeystoreLocationProperty()), is("keystoreLoc")); + assertThat(conf.getProperty(sslOpts.getSslKeystorePasswdProperty()), is("ksPass")); + assertThat(conf.getProperty(sslOpts.getSslTruststoreLocationProperty()), is("truststoreLoc")); + assertThat(conf.getProperty(sslOpts.getSslTruststorePasswdProperty()), is("tsPass")); + } + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultAclProvider(); + } + + //Negative tests + @Test + public void testNoConnectionString(){ + clientConfigurer.withConnectionString(null); + + Throwable t = assertThrows(NullPointerException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), containsString("Zookeeper connection string cannot be null!")); + } + + @Test + public void testNoRetryPolicy() { + clientConfigurer.withRetryPolicy(null); + + Throwable t = assertThrows(NullPointerException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), containsString("Zookeeper connection retry policy cannot be null!")); + } + + @Test + public void testNoAuthType() { + clientConfigurer.withAuthType(null); + + Throwable t = assertThrows(NullPointerException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), containsString("Zookeeper authType cannot be null!")); + } + + @Test + public void testUnrecognizedAuthType() { + clientConfigurer.withAuthType("something"); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("Zookeeper authType must be one of [none, sasl]!")); + } + + @Test + public void testSaslAuthTypeWithoutKeytab() { + clientConfigurer.withAuthType("sasl"); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("Zookeeper client's Kerberos Keytab must be specified!")); + } + + @Test + public void testSaslAuthTypeWithEmptyKeytab() { + clientConfigurer + .withAuthType("sasl") + .withKeytab(""); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("Zookeeper client's Kerberos Keytab must be specified!")); + } + + @Test + public void testSaslAuthTypeWithoutPrincipal() { + clientConfigurer + .withAuthType("sasl") + .withKeytab("keytabLoc"); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("Zookeeper client's Kerberos Principal must be specified!")); + } + + @Test + public void testSaslAuthTypeWithEmptyPrincipal() { + clientConfigurer + .withAuthType("sasl") + .withKeytab("keytabLoc") + .withPrincipal(""); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("Zookeeper client's Kerberos Principal must be specified!")); + } + + @Test + public void testSaslAuthTypeWithoutJaasLoginEntryName() { + clientConfigurer + .withAuthType("sasl") + .withKeytab("keytabLoc") + .withPrincipal("principal") + .withJaasLoginEntryName(null); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("JAAS Login Entry name must be specified!")); + } + + @Test + public void testSaslAuthTypeWithEmptyJaasLoginEntryName() { + clientConfigurer + .withAuthType("sasl") + .withKeytab("keytabLoc") + .withPrincipal("principal") + .withJaasLoginEntryName(""); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), is("JAAS Login Entry name must be specified!")); + } + + @Test + public void testSSLWithoutKeystore() { + clientConfigurer + .enableSSL(true); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), + is("The keystore location parameter is empty for the ZooKeeper client connection.")); + } + + @Test + public void testSSLWithEmptyKeystore() { + clientConfigurer + .enableSSL(true) + .withKeystore(""); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), + is("The keystore location parameter is empty for the ZooKeeper client connection.")); + } + + @Test + public void testSSLWithoutTruststore() { + clientConfigurer + .enableSSL(true) + .withKeystore("keyStoreLoc"); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), + is("The truststore location parameter is empty for the ZooKeeper client connection.")); + } + + @Test + public void testSSLWithEmptyTruststore() { + clientConfigurer + .enableSSL(true) + .withKeystore("keyStoreLoc") + .withTruststore(""); + + Throwable t = assertThrows(IllegalArgumentException.class, () -> clientConfigurer.create()); + assertThat(t.getMessage(), + is("The truststore location parameter is empty for the ZooKeeper client connection.")); + } + + private void testSaslAuthType(String vendor) { + String origVendor = System.getProperty("java.vendor"); + System.setProperty("java.vendor", vendor); + Configuration origConf = Configuration.getConfiguration(); + + try { + clientConfigurer + .withAuthType("sasl") + .withKeytab("keytabLoc") + .withPrincipal("principal@some.host/SOME.REALM") + .withJaasLoginEntryName("TestEntry") + .create(); + + ArgumentCaptor aclProviderCaptor = forClass(SASLOwnerACLProvider.class); + verify(cfBuilder).aclProvider(aclProviderCaptor.capture()); + SASLOwnerACLProvider aclProvider = aclProviderCaptor.getValue(); + + assertThat(aclProvider.getDefaultAcl().size(), is(1)); + assertThat(aclProvider.getDefaultAcl().get(0).getId().getScheme(), is("sasl")); + assertThat(aclProvider.getDefaultAcl().get(0).getId().getId(), is("principal")); + assertThat(aclProvider.getDefaultAcl().get(0).getPerms(), is(ZooDefs.Perms.ALL)); + + Arrays.stream(new String[] {"/", "/foo", "/foo/bar/baz", "/random/path"}) + .forEach(s -> { + assertThat(aclProvider.getAclForPath(s).size(), is(1)); + assertThat(aclProvider.getAclForPath(s).get(0).getId().getScheme(), is("sasl")); + assertThat(aclProvider.getAclForPath(s).get(0).getId().getId(), is("principal")); + assertThat(aclProvider.getAclForPath(s).get(0).getPerms(), is(ZooDefs.Perms.ALL)); + }); + + assertThat(System.getProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY), is("TestEntry")); + assertThat(System.getProperty("zookeeper.authProvider.1"), + is("org.apache.zookeeper.server.auth.SASLAuthenticationProvider")); + + Configuration config = Configuration.getConfiguration(); + assertThat(config.getAppConfigurationEntry("TestEntry").length, is(1)); + AppConfigurationEntry entry = config.getAppConfigurationEntry("TestEntry")[0]; + assertThat(entry.getOptions().get("keyTab"), is("keytabLoc")); + assertThat(entry.getOptions().get("principal"), is("principal@some.host/SOME.REALM")); + assertThat(entry.getOptions().get("useKeyTab"), is("true")); + assertThat(entry.getOptions().get("storeKey"), is("true")); + assertThat(entry.getOptions().get("useTicketCache"), is("false")); + assertThat(entry.getOptions().get("refreshKrb5Config"), is("true")); + if (System.getProperty("java.vendor").contains("IBM")){ + assertThat(entry.getLoginModuleName(), is("com.ibm.security.auth.module.Krb5LoginModule")); + } else { + assertThat(entry.getLoginModuleName(), is("com.sun.security.auth.module.Krb5LoginModule")); + } + } finally { + Configuration.setConfiguration(origConf); + System.setProperty("java.vendor", origVendor); + } + + verifyDummyConnectionString(); + verifyDefaultZKFactory(); + verifyDefaultNamespace(); + verifyDefaultSessionTimeout(); + verifyDefaultConnectionTimeout(); + verifyDefaultRetryPolicy(); + verifyDefaultZKClientConfig(); + } + + private void verifyDummyConnectionString() { + verify(cfBuilder).connectString("dummy"); + } + + private void verifyDefaultNamespace() { + verify(cfBuilder).namespace(null); + } + + private void verifyDefaultZKFactory() { + verify(cfBuilder).zookeeperFactory(isA(ConfigurableZookeeperFactory.class)); + } + + private void verifyDefaultSessionTimeout() { + verify(cfBuilder).sessionTimeoutMs(60000); + } + + private void verifyDefaultConnectionTimeout() { + verify(cfBuilder).connectionTimeoutMs(15000); + } + + private void verifyDefaultRetryPolicy() { + ArgumentCaptor retry = forClass(ExponentialBackoffRetry.class); + verify(cfBuilder).retryPolicy(retry.capture()); + ExponentialBackoffRetry policy = retry.getValue(); + + assertThat(policy.getBaseSleepTimeMs(), is(1000)); + assertThat(policy.getN(), is(3)); + } + + private void verifyDefaultAclProvider() { + verify(cfBuilder).aclProvider(isA(DefaultACLProvider.class)); + } + + private void verifyDefaultZKClientConfig() { + ArgumentCaptor clientConfCaptor = forClass(ZKClientConfig.class); + verify(cfBuilder).zkClientConfig(clientConfCaptor.capture()); + ZKClientConfig conf = clientConfCaptor.getValue(); + + assertThat(conf.getProperty(ZKClientConfig.SECURE_CLIENT), isEmptyOrFalse()); + try (ClientX509Util sslOpts = new ClientX509Util()) { + assertThat(conf.getProperty(sslOpts.getSslKeystoreLocationProperty()), isEmpty()); + assertThat(conf.getProperty(sslOpts.getSslKeystorePasswdProperty()), isEmpty()); + assertThat(conf.getProperty(sslOpts.getSslTruststoreLocationProperty()), isEmpty()); + assertThat(conf.getProperty(sslOpts.getSslTruststorePasswdProperty()), isEmpty()); + } + } + + private Matcher isEmptyOrFalse() { + return anyOf(isEmpty(), is("false")); + } + + private Matcher isEmpty() { + return anyOf(new IsNull<>(), is("")); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index 63f494a20ac37..31b6654afc578 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -417,6 +417,10 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { /** How often to retry a ZooKeeper operation in milliseconds. */ public static final String ZK_RETRY_INTERVAL_MS = ZK_PREFIX + "retry-interval-ms"; + + /** SSL enablement for all Hadoop->ZK communication. */ + //Note: except ZKSignerSecretProvider in hadoop-auth to avoid circular dependency. + public static final String ZK_CLIENT_SSL_ENABLED = ZK_PREFIX + "ssl.enabled"; /** Keystore location for ZooKeeper client connection over SSL. */ public static final String ZK_SSL_KEYSTORE_LOCATION = ZK_PREFIX + "ssl.keystore.location"; /** Keystore password for ZooKeeper client connection over SSL. */ @@ -425,6 +429,7 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { public static final String ZK_SSL_TRUSTSTORE_LOCATION = ZK_PREFIX + "ssl.truststore.location"; /** Truststore password for ZooKeeper client connection over SSL. */ public static final String ZK_SSL_TRUSTSTORE_PASSWORD = ZK_PREFIX + "ssl.truststore.password"; + public static final int ZK_RETRY_INTERVAL_MS_DEFAULT = 1000; /** Default domain name resolver for hadoop to use. */ public static final String HADOOP_DOMAINNAME_RESOLVER_IMPL = diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java index 925bc030c2cd8..0642d3d581066 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java @@ -24,18 +24,12 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.UncheckedIOException; -import java.util.Collections; -import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Stream; -import org.apache.curator.ensemble.fixed.FixedEnsembleProvider; +import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.framework.CuratorFrameworkFactory.Builder; -import org.apache.curator.framework.api.ACLProvider; -import org.apache.curator.framework.imps.DefaultACLProvider; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.CuratorCache; import org.apache.curator.framework.recipes.cache.CuratorCacheBridge; @@ -43,28 +37,28 @@ import org.apache.curator.framework.recipes.shared.SharedCount; import org.apache.curator.framework.recipes.shared.VersionedValue; import org.apache.curator.retry.RetryNTimes; +import org.apache.curator.utils.ZookeeperFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.authentication.util.JaasConfiguration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.security.SecurityUtil.TruststoreKeystore; +import org.apache.hadoop.security.authentication.util.ZookeeperClient; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager; + +import static org.apache.hadoop.security.SecurityUtil.getServerPrincipal; import static org.apache.hadoop.util.Time.now; -import org.apache.hadoop.util.curator.ZKCuratorManager; + +import org.apache.hadoop.util.curator.ZKCuratorManager.HadoopZookeeperFactory; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; -import org.apache.zookeeper.ZooDefs.Perms; -import org.apache.zookeeper.client.ZKClientConfig; -import org.apache.zookeeper.data.ACL; -import org.apache.zookeeper.data.Id; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.util.Preconditions; /** * An implementation of {@link AbstractDelegationTokenSecretManager} that @@ -104,6 +98,16 @@ public abstract class ZKDelegationTokenSecretManager saslACL; - - private SASLOwnerACLProvider(String principal) { - this.saslACL = Collections.singletonList( - new ACL(Perms.ALL, new Id("sasl", principal))); - } - - @Override - public List getDefaultAcl() { - return saslACL; - } - - @Override - public List getAclForPath(String path) { - return saslACL; - } - } - @VisibleForTesting @Private @Unstable 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 c86fd8b98609b..7333f840f37e7 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 @@ -4255,6 +4255,18 @@ The switch to turn S3A auditing on or off. + + hadoop.zk.ssl.enabled + + Enable SSL/TLS encryption for the ZooKeeper communication. + Note: this setting overrides dfs.ha.zkfc.client.ssl.enabled, + yarn.resourcemanager.zk-client-ssl.enabled and also + hadoop.kms.authentication.zk-dt-secret-manager.ssl.enabled in order to unify the SSL based + Zookeeper access across Hadoop. Leaving this property empty ensures that service specific + enablement can be done separately. + + + hadoop.zk.ssl.keystore.location diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java index 2b09f81301dde..7860129196656 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java @@ -30,6 +30,7 @@ import java.util.List; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.HadoopIllegalArgumentException; @@ -297,8 +298,8 @@ public List getAllOtherNodes() { @Override protected boolean isSSLEnabled() { - return conf.getBoolean( - DFSConfigKeys.ZK_CLIENT_SSL_ENABLED, - DFSConfigKeys.DEFAULT_ZK_CLIENT_SSL_ENABLED); + return conf.getBoolean(CommonConfigurationKeys.ZK_CLIENT_SSL_ENABLED, + conf.getBoolean(DFSConfigKeys.ZK_CLIENT_SSL_ENABLED, + DFSConfigKeys.DEFAULT_ZK_CLIENT_SSL_ENABLED)); } } 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 3e9f54f58ff8f..b64a7a664b918 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 @@ -3802,6 +3802,8 @@ false Enable SSL/TLS encryption for the ZooKeeper communication from ZKFC. + Note: if hadoop.zk.ssl.enabled is set to a value, then that central setting has precedence, + and this value will be overridden by the value of hadoop.zk.ssl.enabled. 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 0e317712f826c..b8892e4fc12ee 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 @@ -742,7 +742,11 @@ - Enable SSL/TLS encryption for the ZooKeeper communication. + + Enable SSL/TLS encryption for the ZooKeeper communication. + Note: if hadoop.zk.ssl.enabled is set to a value, then that central setting has precedence, + and this value will be overridden by the value of hadoop.zk.ssl.enabled. + yarn.resourcemanager.zk-client-ssl.enabled false diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java index 989c9c53cd0a9..024bfeffdea2e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java @@ -105,8 +105,10 @@ protected void serviceInit(Configuration conf) conf.getInt(YarnConfiguration.RM_HA_FC_ELECTOR_ZK_RETRIES_KEY, conf .getInt(CommonConfigurationKeys.HA_FC_ELECTOR_ZK_OP_RETRIES_KEY, CommonConfigurationKeys.HA_FC_ELECTOR_ZK_OP_RETRIES_DEFAULT)); - boolean isSSLEnabled = conf.getBoolean(YarnConfiguration.RM_ZK_CLIENT_SSL_ENABLED, - YarnConfiguration.DEFAULT_RM_ZK_CLIENT_SSL_ENABLED); + boolean isSSLEnabled = + conf.getBoolean(CommonConfigurationKeys.ZK_CLIENT_SSL_ENABLED, + conf.getBoolean(YarnConfiguration.RM_ZK_CLIENT_SSL_ENABLED, + YarnConfiguration.DEFAULT_RM_ZK_CLIENT_SSL_ENABLED)); SecurityUtil.TruststoreKeystore truststoreKeystore = isSSLEnabled ? new SecurityUtil.TruststoreKeystore(conf) : null; elector = new ActiveStandbyElector(zkQuorum, (int) zkSessionTimeout, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index f358a22cee715..61442e4c291e3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager; import org.apache.commons.lang3.math.NumberUtils; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.classification.VisibleForTesting; import com.sun.jersey.spi.container.servlet.ServletContainer; @@ -427,8 +428,11 @@ public ZKCuratorManager createAndStartZKManager(Configuration authInfos.add(authInfo); } - manager.start(authInfos, config.getBoolean(YarnConfiguration.RM_ZK_CLIENT_SSL_ENABLED, - YarnConfiguration.DEFAULT_RM_ZK_CLIENT_SSL_ENABLED)); + boolean isSSLEnabled = + config.getBoolean(CommonConfigurationKeys.ZK_CLIENT_SSL_ENABLED, + config.getBoolean(YarnConfiguration.RM_ZK_CLIENT_SSL_ENABLED, + YarnConfiguration.DEFAULT_RM_ZK_CLIENT_SSL_ENABLED)); + manager.start(authInfos, isSSLEnabled); return manager; } From 0c10bab7bb77aa4ea3ca26c899ab28131561e052 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Fri, 17 Nov 2023 23:34:36 +0800 Subject: [PATCH 10/10] YARN-11483. [Federation] Router AdminCLI Supports Clean Finish Apps. (#6251) Contributed by Shilun Fan. Reviewed-by: Inigo Goiri Signed-off-by: Shilun Fan --- ...ResourceManagerAdministrationProtocol.java | 15 +++ .../DeleteFederationApplicationRequest.java | 51 ++++++++++ .../DeleteFederationApplicationResponse.java | 47 +++++++++ ...ourcemanager_administration_protocol.proto | 1 + ...erver_resourcemanager_service_protos.proto | 8 ++ .../hadoop/yarn/client/cli/RouterCLI.java | 74 ++++++++++++++ .../hadoop/yarn/client/cli/TestRouterCLI.java | 9 +- ...gerAdministrationProtocolPBClientImpl.java | 19 ++++ ...erAdministrationProtocolPBServiceImpl.java | 22 +++++ ...eteFederationApplicationRequestPBImpl.java | 97 +++++++++++++++++++ ...teFederationApplicationResponsePBImpl.java | 97 +++++++++++++++++++ .../server/MockResourceManagerFacade.java | 8 ++ .../server/resourcemanager/AdminService.java | 10 ++ .../yarn/server/router/RouterMetrics.java | 32 ++++++ .../DefaultRMAdminRequestInterceptor.java | 9 ++ .../rmadmin/FederationRMAdminInterceptor.java | 38 ++++++++ .../router/rmadmin/RouterRMAdminService.java | 9 ++ .../PassThroughRMAdminRequestInterceptor.java | 8 ++ .../TestFederationRMAdminInterceptor.java | 28 ++++++ 19 files changed, 581 insertions(+), 1 deletion(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationRequest.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationResponse.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationRequestPBImpl.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationResponsePBImpl.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java index 916dcd50aa0d9..0b4f1ad6429fa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java @@ -64,6 +64,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; @Private public interface ResourceManagerAdministrationProtocol extends GetUserMappingsProtocol { @@ -218,4 +220,17 @@ BatchSaveFederationQueuePoliciesResponse batchSaveFederationQueuePolicies( @Idempotent QueryFederationQueuePoliciesResponse listFederationQueuePolicies( QueryFederationQueuePoliciesRequest request) throws YarnException, IOException; + + /** + * In YARN-Federation mode, this method provides a way to delete federation application. + * + * @param request DeleteFederationApplicationRequest Request. + * @return Response from deleteFederationApplication. + * @throws YarnException exceptions from yarn servers. + * @throws IOException if an IO error occurred. + */ + @Private + @Idempotent + DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationRequest.java new file mode 100644 index 0000000000000..a035aa8275c12 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationRequest.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.api.protocolrecords; + +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.yarn.util.Records; + +/** + * This class is used for cleaning up an application that exists in the FederationStateStore. + * This is a user-specified operation; + * we typically use this command to clean up an expired application. + * However, it can also be used to clean up non-expired application, although it is not recommended. + */ +@Private +@Unstable +public abstract class DeleteFederationApplicationRequest { + + @Private + @Unstable + public static DeleteFederationApplicationRequest newInstance(String application) { + DeleteFederationApplicationRequest request = + Records.newRecord(DeleteFederationApplicationRequest.class); + request.setApplication(application); + return request; + } + + @Public + @Unstable + public abstract String getApplication(); + + @Public + @Unstable + public abstract void setApplication(String application); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationResponse.java new file mode 100644 index 0000000000000..d75515ff633ec --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/DeleteFederationApplicationResponse.java @@ -0,0 +1,47 @@ +/** + * 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.api.protocolrecords; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.util.Records; + +@Private +@Unstable +public abstract class DeleteFederationApplicationResponse { + + public static DeleteFederationApplicationResponse newInstance() { + return Records.newRecord(DeleteFederationApplicationResponse.class); + } + + public static DeleteFederationApplicationResponse newInstance(String msg) { + DeleteFederationApplicationResponse response = + Records.newRecord(DeleteFederationApplicationResponse.class); + response.setMessage(msg); + return response; + } + + @Public + @Unstable + public abstract String getMessage(); + + @Public + @Unstable + public abstract void setMessage(String msg); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto index fcae14128d80f..f0f3fa563f59d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto @@ -51,4 +51,5 @@ service ResourceManagerAdministrationProtocolService { rpc saveFederationQueuePolicy(SaveFederationQueuePolicyRequestProto) returns (SaveFederationQueuePolicyResponseProto); rpc batchSaveFederationQueuePolicies(BatchSaveFederationQueuePoliciesRequestProto) returns (BatchSaveFederationQueuePoliciesResponseProto); rpc listFederationQueuePolicies(QueryFederationQueuePoliciesRequestProto) returns (QueryFederationQueuePoliciesResponseProto); + rpc deleteFederationApplication(DeleteFederationApplicationRequestProto) returns (DeleteFederationApplicationResponseProto); } 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 a2945f1eb1ee1..ba18e514a7dfa 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 @@ -203,6 +203,14 @@ message QueryFederationQueuePoliciesResponseProto { repeated FederationQueueWeightProto federationQueueWeights = 5; } +message DeleteFederationApplicationRequestProto { + optional string application = 1; +} + +message DeleteFederationApplicationResponseProto { + optional string message = 1; +} + ////////////////////////////////////////////////////////////////// ///////////// RM Failover related records //////////////////////// ////////////////////////////////////////////////////////////////// diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java index 2da584f9d61bb..9a68794078e2e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.client.cli; import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.MissingArgumentException; import org.apache.commons.cli.Option; @@ -42,6 +43,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.DeregisterSubClusterRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.DeregisterSubClusterResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.DeregisterSubClusters; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.SaveFederationQueuePolicyRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.SaveFederationQueuePolicyResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesRequest; @@ -227,12 +230,37 @@ public class RouterCLI extends Configured implements Tool { .addExample(POLICY_LIST_USAGE.args, POLICY_LIST_USAGE_EXAMPLE_1) .addExample(POLICY_LIST_USAGE.args, POLICY_LIST_USAGE_EXAMPLE_2); + // Command3: application + private static final String CMD_APPLICATION = "-application"; + + // Application Delete + protected final static UsageInfo APPLICATION_DELETE_USAGE = new UsageInfo( + "--delete ", + "This command is used to delete the specified application."); + + protected final static String APPLICATION_DELETE_USAGE_EXAMPLE_DESC = + "If we want to delete application_1440536969523_0001."; + + protected final static String APPLICATION_DELETE_USAGE_EXAMPLE_1 = + "yarn routeradmin -application --delete application_1440536969523_0001"; + + protected final static RouterCmdUsageInfos APPLICATION_USAGEINFOS = new RouterCmdUsageInfos() + // application delete + .addUsageInfo(APPLICATION_DELETE_USAGE) + .addExampleDescs(APPLICATION_DELETE_USAGE.args, APPLICATION_DELETE_USAGE_EXAMPLE_DESC) + .addExample(APPLICATION_DELETE_USAGE.args, APPLICATION_DELETE_USAGE_EXAMPLE_1); + + // delete application + private static final String OPTION_DELETE_APP = "delete"; + protected final static Map ADMIN_USAGE = ImmutableMap.builder() // Command1: deregisterSubCluster .put(CMD_DEREGISTERSUBCLUSTER, DEREGISTER_SUBCLUSTER_USAGEINFOS) // Command2: policy .put(CMD_POLICY, POLICY_USAGEINFOS) + // Command3: application + .put(CMD_APPLICATION, APPLICATION_USAGEINFOS) .build(); public RouterCLI() { @@ -814,6 +842,50 @@ protected int handListPolicies(int pageSize, int currentPage, String queue, List } } + private int handleDeleteApplication(String application) { + LOG.info("Delete Application = {}.", application); + try { + DeleteFederationApplicationRequest request = + DeleteFederationApplicationRequest.newInstance(application); + ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol(); + DeleteFederationApplicationResponse response = + adminProtocol.deleteFederationApplication(request); + System.out.println(response.getMessage()); + return EXIT_SUCCESS; + } catch (Exception e) { + LOG.error("handleSavePolicy error.", e); + return EXIT_ERROR; + } + } + + private int handleApplication(String[] args) + throws IOException, YarnException, ParseException { + // Prepare Options. + Options opts = new Options(); + opts.addOption("application", false, + "We provide a set of commands to query and clean applications."); + Option deleteOpt = new Option(null, OPTION_DELETE_APP, true, + "We will clean up the provided application."); + opts.addOption(deleteOpt); + + // Parse command line arguments. + CommandLine cliParser; + try { + cliParser = new DefaultParser().parse(opts, args); + } catch (MissingArgumentException ex) { + System.out.println("Missing argument for options"); + printUsage(args[0]); + return EXIT_ERROR; + } + + if (cliParser.hasOption(OPTION_DELETE_APP)) { + String application = cliParser.getOptionValue(OPTION_DELETE_APP); + return handleDeleteApplication(application); + } + + return 0; + } + @Override public int run(String[] args) throws Exception { YarnConfiguration yarnConf = getConf() == null ? @@ -839,6 +911,8 @@ public int run(String[] args) throws Exception { return handleDeregisterSubCluster(args); } else if (CMD_POLICY.equals(cmd)) { return handlePolicy(args); + } else if (CMD_APPLICATION.equals(cmd)) { + return handleApplication(args); } else { System.out.println("No related commands found."); printHelp(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java index a86878dac3f81..8e697dd6c01db 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java @@ -94,6 +94,7 @@ public void setup() throws Exception { return QueryFederationQueuePoliciesResponse.newInstance(1, 1, 1, 10, weights); }); + Configuration config = new Configuration(); config.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true); @@ -271,7 +272,7 @@ public void testListPolicies() throws Exception { @Test public void testBuildHelpMsg() throws Exception { Map adminUsage = rmAdminCLI.getAdminUsage(); - assertEquals(2, adminUsage.size()); + assertEquals(3, adminUsage.size()); RouterCLI.RouterCmdUsageInfos deregisterSubClusterUsageInfos = adminUsage.get("-deregisterSubCluster"); @@ -291,5 +292,11 @@ public void testBuildHelpMsg() throws Exception { policyExamplesMap.forEach((cmd, cmdExamples) -> { assertEquals(2, cmdExamples.size()); }); + + RouterCLI.RouterCmdUsageInfos applicationUsageInfos = adminUsage.get("-application"); + assertNotNull(applicationUsageInfos); + Map> applicationExamplesMap = applicationUsageInfos.getExamples(); + assertNotNull(applicationExamplesMap); + assertEquals(1, applicationExamplesMap.size()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java index c7c6dc510424c..9d9c018a46962 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java @@ -48,6 +48,7 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.SaveFederationQueuePolicyRequestProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.BatchSaveFederationQueuePoliciesRequestProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.QueryFederationQueuePoliciesRequestProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationRequestProto; import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol; import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocolPB; import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest; @@ -86,6 +87,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesRequestPBImpl; @@ -122,6 +125,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.BatchSaveFederationQueuePoliciesResponsePBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederationQueuePoliciesRequestPBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederationQueuePoliciesResponsePBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationRequestPBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationResponsePBImpl; import org.apache.hadoop.thirdparty.protobuf.ServiceException; @@ -420,4 +425,18 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( } return null; } + + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + DeleteFederationApplicationRequestProto requestProto = + ((DeleteFederationApplicationRequestPBImpl) request).getProto(); + try { + return new DeleteFederationApplicationResponsePBImpl( + proxy.deleteFederationApplication(null, requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + } + return null; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java index 0eab80b9ad770..1e0b09a619568 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java @@ -60,6 +60,8 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeregisterSubClusterResponseProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.SaveFederationQueuePolicyRequestProto; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.SaveFederationQueuePolicyResponseProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationResponseProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationRequestProto; import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol; import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocolPB; import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse; @@ -87,6 +89,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesRequestPBImpl; @@ -123,6 +127,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.BatchSaveFederationQueuePoliciesResponsePBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederationQueuePoliciesRequestPBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederationQueuePoliciesResponsePBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationRequestPBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationResponsePBImpl; import org.apache.hadoop.thirdparty.protobuf.RpcController; import org.apache.hadoop.thirdparty.protobuf.ServiceException; @@ -445,4 +451,20 @@ public QueryFederationQueuePoliciesResponseProto listFederationQueuePolicies( throw new ServiceException(e); } } + + @Override + public DeleteFederationApplicationResponseProto deleteFederationApplication( + RpcController controller, DeleteFederationApplicationRequestProto proto) + throws ServiceException { + DeleteFederationApplicationRequest request = + new DeleteFederationApplicationRequestPBImpl(proto); + try { + DeleteFederationApplicationResponse response = real.deleteFederationApplication(request); + return ((DeleteFederationApplicationResponsePBImpl) response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationRequestPBImpl.java new file mode 100644 index 0000000000000..0309632c40ac2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationRequestPBImpl.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.yarn.server.api.protocolrecords.impl.pb; + +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.thirdparty.protobuf.TextFormat; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationRequestProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationRequestProtoOrBuilder; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; + +@Private +@Unstable +public class DeleteFederationApplicationRequestPBImpl extends DeleteFederationApplicationRequest { + + private DeleteFederationApplicationRequestProto proto = + DeleteFederationApplicationRequestProto.getDefaultInstance(); + private DeleteFederationApplicationRequestProto.Builder builder = null; + private boolean viaProto = false; + + public DeleteFederationApplicationRequestPBImpl() { + builder = DeleteFederationApplicationRequestProto.newBuilder(); + } + + public DeleteFederationApplicationRequestPBImpl(DeleteFederationApplicationRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + private synchronized void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = DeleteFederationApplicationRequestProto.newBuilder(proto); + } + viaProto = false; + } + + public DeleteFederationApplicationRequestProto getProto() { + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof DeleteFederationApplicationRequest)) { + return false; + } + DeleteFederationApplicationRequestPBImpl otherImpl = this.getClass().cast(other); + return new EqualsBuilder().append(this.getProto(), otherImpl.getProto()).isEquals(); + } + + @Override + public String getApplication() { + DeleteFederationApplicationRequestProtoOrBuilder p = viaProto ? proto : builder; + boolean hasApplication = p.hasApplication(); + if (hasApplication) { + return p.getApplication(); + } + return null; + } + + @Override + public void setApplication(String application) { + maybeInitBuilder(); + if (application == null) { + builder.clearApplication(); + return; + } + builder.setApplication(application); + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationResponsePBImpl.java new file mode 100644 index 0000000000000..f21f6fbde18f0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DeleteFederationApplicationResponsePBImpl.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.yarn.server.api.protocolrecords.impl.pb; + +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.thirdparty.protobuf.TextFormat; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationResponseProto; +import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationResponseProtoOrBuilder; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; + +@Private +@Unstable +public class DeleteFederationApplicationResponsePBImpl extends DeleteFederationApplicationResponse { + + private DeleteFederationApplicationResponseProto proto = + DeleteFederationApplicationResponseProto.getDefaultInstance(); + private DeleteFederationApplicationResponseProto.Builder builder = null; + private boolean viaProto = false; + + public DeleteFederationApplicationResponsePBImpl() { + builder = DeleteFederationApplicationResponseProto.newBuilder(); + } + + public DeleteFederationApplicationResponsePBImpl(DeleteFederationApplicationResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + public DeleteFederationApplicationResponseProto getProto() { + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof DeleteFederationApplicationResponse)) { + return false; + } + DeleteFederationApplicationResponsePBImpl otherImpl = this.getClass().cast(other); + return new EqualsBuilder().append(this.getProto(), otherImpl.getProto()).isEquals(); + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + @Override + public String getMessage() { + DeleteFederationApplicationResponseProtoOrBuilder p = viaProto ? proto : builder; + boolean hasMessage = p.hasMessage(); + if (hasMessage) { + return p.getMessage(); + } + return null; + } + + private synchronized void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = DeleteFederationApplicationResponseProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public void setMessage(String msg) { + maybeInitBuilder(); + if (msg == null) { + builder.clearMessage(); + return; + } + builder.setMessage(msg); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java index ed66dadefb497..0823660bc0404 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java @@ -181,6 +181,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; /** @@ -988,6 +990,12 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( return null; } + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + return null; + } + @VisibleForTesting public HashMap> getApplicationContainerIdMap() { return applicationContainerIdMap; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java index 8f9e8caa53670..c45c34dc85c80 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java @@ -103,6 +103,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils; import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem; import org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration; @@ -1105,6 +1107,14 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( " Please call Router's listFederationQueuePolicies to list Policies."); } + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + throw new YarnException("It is not allowed to call the RM's " + + " deleteFederationApplication. " + + " Please call Router's deleteFederationApplication to delete Application."); + } + private void validateAttributesExists( List nodesToAttributes) throws IOException { NodeAttributesManager nodeAttributesManager = 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 d0e4825fed39b..66157797a936e 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 @@ -155,6 +155,8 @@ public final class RouterMetrics { private MutableGaugeInt numBatchSaveFederationQueuePoliciesFailedRetrieved; @Metric("# of listFederationQueuePolicies failed to be retrieved") private MutableGaugeInt numListFederationQueuePoliciesFailedRetrieved; + @Metric("# of deleteFederationApplication failed to be retrieved") + private MutableGaugeInt numDeleteFederationApplicationFailedRetrieved; @Metric("# of refreshAdminAcls failed to be retrieved") private MutableGaugeInt numRefreshAdminAclsFailedRetrieved; @Metric("# of refreshServiceAcls failed to be retrieved") @@ -307,6 +309,8 @@ public final class RouterMetrics { private MutableRate totalSucceededBatchSaveFederationQueuePoliciesRetrieved; @Metric("Total number of successful Retrieved ListFederationQueuePolicies and latency(ms)") private MutableRate totalSucceededListFederationQueuePoliciesFailedRetrieved; + @Metric("Total number of successful Retrieved DeleteFederationApplication and latency(ms)") + private MutableRate totalSucceededDeleteFederationApplicationFailedRetrieved; @Metric("Total number of successful Retrieved RefreshAdminAcls and latency(ms)") private MutableRate totalSucceededRefreshAdminAclsRetrieved; @Metric("Total number of successful Retrieved RefreshServiceAcls and latency(ms)") @@ -396,6 +400,7 @@ public final class RouterMetrics { private MutableQuantiles saveFederationQueuePolicyLatency; private MutableQuantiles batchSaveFederationQueuePoliciesLatency; private MutableQuantiles listFederationQueuePoliciesLatency; + private MutableQuantiles deleteFederationApplicationLatency; private MutableQuantiles refreshAdminAclsLatency; private MutableQuantiles refreshServiceAclsLatency; private MutableQuantiles replaceLabelsOnNodesLatency; @@ -618,6 +623,10 @@ private RouterMetrics() { "listFederationQueuePoliciesLatency", "latency of list federationqueuepolicies timeouts", "ops", "latency", 10); + deleteFederationApplicationLatency = registry.newQuantiles( + "deleteFederationApplicationLatency", + "latency of delete FederationApplication timeouts", "ops", "latency", 10); + refreshAdminAclsLatency = registry.newQuantiles("refreshAdminAclsLatency", "latency of refresh admin acls timeouts", "ops", "latency", 10); @@ -962,6 +971,11 @@ public long getNumSucceededListFederationQueuePoliciesFailedRetrieved() { return totalSucceededListFederationQueuePoliciesFailedRetrieved.lastStat().numSamples(); } + @VisibleForTesting + public long getNumSucceededDeleteFederationApplicationFailedRetrieved() { + return totalSucceededDeleteFederationApplicationFailedRetrieved.lastStat().numSamples(); + } + @VisibleForTesting public long getNumSucceededRefreshAdminAclsRetrieved() { return totalSucceededRefreshAdminAclsRetrieved.lastStat().numSamples(); @@ -1322,6 +1336,11 @@ public double getLatencySucceededListFederationQueuePoliciesRetrieved() { return totalSucceededListFederationQueuePoliciesFailedRetrieved.lastStat().mean(); } + @VisibleForTesting + public double getLatencySucceededDeleteFederationApplicationFailedRetrieved() { + return totalSucceededDeleteFederationApplicationFailedRetrieved.lastStat().mean(); + } + @VisibleForTesting public double getLatencySucceededRefreshAdminAclsRetrieved() { return totalSucceededRefreshAdminAclsRetrieved.lastStat().mean(); @@ -1629,6 +1648,10 @@ public int getListFederationQueuePoliciesFailedRetrieved() { return numListFederationQueuePoliciesFailedRetrieved.value(); } + public int getDeleteFederationApplicationFailedRetrieved() { + return numDeleteFederationApplicationFailedRetrieved.value(); + } + public int getNumRefreshAdminAclsFailedRetrieved() { return numRefreshAdminAclsFailedRetrieved.value(); } @@ -1996,6 +2019,11 @@ public void succeededListFederationQueuePoliciesRetrieved(long duration) { listFederationQueuePoliciesLatency.add(duration); } + public void succeededDeleteFederationApplicationFailedRetrieved(long duration) { + totalSucceededDeleteFederationApplicationFailedRetrieved.add(duration); + deleteFederationApplicationLatency.add(duration); + } + public void succeededRefreshAdminAclsRetrieved(long duration) { totalSucceededRefreshAdminAclsRetrieved.add(duration); refreshAdminAclsLatency.add(duration); @@ -2286,6 +2314,10 @@ public void incrListFederationQueuePoliciesFailedRetrieved() { numListFederationQueuePoliciesFailedRetrieved.incr(); } + public void incrDeleteFederationApplicationFailedRetrieved() { + numDeleteFederationApplicationFailedRetrieved.incr(); + } + public void incrRefreshAdminAclsFailedRetrieved() { numRefreshAdminAclsFailedRetrieved.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/DefaultRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java index 23517b97b4e19..2633a90d62cc0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java @@ -64,6 +64,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -233,4 +235,11 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( QueryFederationQueuePoliciesRequest request) throws YarnException, IOException { return rmAdminProxy.listFederationQueuePolicies(request); } + + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) + throws YarnException, IOException { + return rmAdminProxy.deleteFederationApplication(request); + } } 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 30b36909fb6d8..7ade0d4007974 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 @@ -28,6 +28,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.util.Time; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -71,6 +72,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo; @@ -1088,6 +1091,41 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( throw new YarnException("Unable to listFederationQueuePolicies."); } + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + + // Parameter validation. + if (request == null) { + routerMetrics.incrDeleteFederationApplicationFailedRetrieved(); + RouterServerUtil.logAndThrowException( + "Missing deleteFederationApplication Request.", null); + } + + String application = request.getApplication(); + if (StringUtils.isBlank(application)) { + routerMetrics.incrDeleteFederationApplicationFailedRetrieved(); + RouterServerUtil.logAndThrowException( + "ApplicationId cannot be null.", null); + } + + // Try calling deleteApplicationHomeSubCluster to delete the application. + try { + long startTime = clock.getTime(); + ApplicationId applicationId = ApplicationId.fromString(application); + federationFacade.deleteApplicationHomeSubCluster(applicationId); + long stopTime = clock.getTime(); + routerMetrics.succeededDeleteFederationApplicationFailedRetrieved(stopTime - startTime); + return DeleteFederationApplicationResponse.newInstance( + "applicationId = " + applicationId + " delete success."); + } catch (Exception e) { + RouterServerUtil.logAndThrowException(e, + "Unable to deleteFederationApplication due to exception. " + e.getMessage()); + } + + throw new YarnException("Unable to deleteFederationApplication."); + } + /** * According to the configuration information of the queue filtering queue, * this part should only return 1 result. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java index fc2278d3bb318..718abd2894493 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java @@ -71,6 +71,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.router.RouterServerUtil; import org.apache.hadoop.yarn.server.router.security.authorize.RouterPolicyProvider; import org.apache.hadoop.yarn.util.LRUCacheHashMap; @@ -419,4 +421,11 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( RequestInterceptorChainWrapper pipeline = getInterceptorChain(); return pipeline.getRootInterceptor().listFederationQueuePolicies(request); } + + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + RequestInterceptorChainWrapper pipeline = getInterceptorChain(); + return pipeline.getRootInterceptor().deleteFederationApplication(request); + } } 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/PassThroughRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java index 6fe218c7b2a00..0dab931a7ab97 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java @@ -58,6 +58,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; /** * Mock interceptor that does not do anything other than forwarding it to the @@ -185,4 +187,10 @@ public QueryFederationQueuePoliciesResponse listFederationQueuePolicies( QueryFederationQueuePoliciesRequest request) throws YarnException, IOException { return getNextInterceptor().listFederationQueuePolicies(request); } + + @Override + public DeleteFederationApplicationResponse deleteFederationApplication( + DeleteFederationApplicationRequest request) throws YarnException, IOException { + return getNextInterceptor().deleteFederationApplication(request); + } } 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 520c25d22cb09..62f1eee845b56 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 @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.test.LambdaTestUtils; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.DecommissionType; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; @@ -64,12 +65,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse; import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo; import org.apache.hadoop.yarn.server.federation.policies.manager.WeightedLocalityPolicyManager; import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo; import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration; +import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreTestUtil; import org.junit.Assert; @@ -989,4 +993,28 @@ public void testFilterPoliciesConfigurationsByQueues() throws Exception { "The index of the records to be retrieved has exceeded the maximum index.", () -> interceptor.listFederationQueuePolicies(request8)); } + + + @Test + public void testDeleteFederationApplication() throws Exception { + ApplicationId applicationId = ApplicationId.newInstance(10, 1); + DeleteFederationApplicationRequest request1 = + DeleteFederationApplicationRequest.newInstance(applicationId.toString()); + LambdaTestUtils.intercept(YarnException.class, + "Application application_10_0001 does not exist.", + () -> interceptor.deleteFederationApplication(request1)); + + ApplicationId applicationId2 = ApplicationId.newInstance(10, 2); + SubClusterId homeSubCluster = SubClusterId.newInstance("SC-1"); + ApplicationHomeSubCluster appHomeSubCluster = + ApplicationHomeSubCluster.newInstance(applicationId2, homeSubCluster); + facade.addApplicationHomeSubCluster(appHomeSubCluster); + DeleteFederationApplicationRequest request2 = + DeleteFederationApplicationRequest.newInstance(applicationId2.toString()); + DeleteFederationApplicationResponse deleteFederationApplicationResponse = + interceptor.deleteFederationApplication(request2); + assertNotNull(deleteFederationApplicationResponse); + assertEquals("applicationId = " + applicationId2 + " delete success.", + deleteFederationApplicationResponse.getMessage()); + } }