瀏覽代碼

HDFS-16217. RBF: Set default value of hdfs.fedbalance.procedure.scheduler.journal.uri by adding appropriate config resources (#3434)

Signed-off-by: Akira Ajisaka <aajisaka@apache.org>
Viraj Jasani 3 年之前
父節點
當前提交
7c25a77911

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

@@ -17,12 +17,15 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.service.CompositeService.CompositeServiceShutdownHook;
+import org.apache.hadoop.tools.fedbalance.FedBalance;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
@@ -65,7 +68,7 @@ public final class DFSRouter {
       ShutdownHookManager.get().addShutdownHook(
           new CompositeServiceShutdownHook(router), SHUTDOWN_HOOK_PRIORITY);
 
-      Configuration conf = new HdfsConfiguration();
+      Configuration conf = getConfiguration();
       router.init(conf);
       router.start();
     } catch (Throwable e) {
@@ -73,4 +76,13 @@ public final class DFSRouter {
       terminate(1, e);
     }
   }
+
+  @VisibleForTesting
+  static Configuration getConfiguration() {
+    Configuration conf = new HdfsConfiguration();
+    conf.addResource(FedBalance.FED_BALANCE_DEFAULT_XML);
+    conf.addResource(FedBalance.FED_BALANCE_SITE_XML);
+    return conf;
+  }
+
 }

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

@@ -436,7 +436,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol,
       URI journalUri;
       try {
         journalUri = new URI(sConf.get(SCHEDULER_JOURNAL_URI));
-      } catch (URISyntaxException e) {
+      } catch (URISyntaxException | NullPointerException e) {
         throw new IOException("Bad journal uri. Please check configuration for "
             + SCHEDULER_JOURNAL_URI);
       }

+ 39 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestDFSRouter.java

@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.fedbalance.FedBalanceConfigs;
+
+public class TestDFSRouter {
+
+  @Test
+  public void testDefaultConfigs() {
+    Configuration configuration = DFSRouter.getConfiguration();
+    String journalUri =
+        configuration.get(FedBalanceConfigs.SCHEDULER_JOURNAL_URI);
+    int workerThreads =
+        configuration.getInt(FedBalanceConfigs.WORK_THREAD_NUM, -1);
+    Assert.assertEquals("hdfs://localhost:8020/tmp/procedure", journalUri);
+    Assert.assertEquals(10, workerThreads);
+  }
+
+}