Explorar el Código

HDFS-15040. RBF: Secured Router should not run when SecretManager is not running. (#1745)

Takanobu Asanuma hace 5 años
padre
commit
c4733377d0

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/RouterSecurityManager.java

@@ -58,7 +58,7 @@ public class RouterSecurityManager {
         AuthenticationMethod.KERBEROS;
     if (authMethodConfigured.equals(authMethodToInit)) {
       this.dtSecretManager = FederationUtil.newSecretManager(conf);
-      if (this.dtSecretManager == null) {
+      if (this.dtSecretManager == null || !this.dtSecretManager.isRunning()) {
         throw new IOException("Failed to create SecretManager");
       }
     }

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/security/MockNotRunningSecretManager.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.federation.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+
+import java.io.IOException;
+
+/**
+ * Mock functionality of AbstractDelegationTokenSecretManager.
+ * Test case that SecretManager is not running.
+ */
+public class MockNotRunningSecretManager
+    extends AbstractDelegationTokenSecretManager<DelegationTokenIdentifier> {
+
+  public MockNotRunningSecretManager(Configuration conf)
+      throws IOException {
+    super(100000, 100000, 100000, 100000);
+    // It doesn't execute startThreads() to keep the running status false.
+  }
+
+  @Override
+  public DelegationTokenIdentifier createIdentifier() {
+    return new DelegationTokenIdentifier();
+  }
+}

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/security/TestRouterSecurityManager.java

@@ -201,4 +201,14 @@ public class TestRouterSecurityManager {
     intercept(ServiceStateException.class, "Failed to create SecretManager",
         () -> router.init(conf));
   }
+
+  @Test
+  public void testNotRunningSecretManager() throws Exception {
+    Configuration conf = initSecurity();
+    conf.set(DFS_ROUTER_DELEGATION_TOKEN_DRIVER_CLASS,
+        MockNotRunningSecretManager.class.getName());
+    Router router = new Router();
+    intercept(ServiceStateException.class, "Failed to create SecretManager",
+        () -> router.init(conf));
+  }
 }