Browse Source

HDFS-15410. Add separated config file hdfs-fedbalance-default.xml for fedbalance tool. Contributed by Jinglun.

Yiqun Lin 4 years ago
parent
commit
de2cb86260

+ 32 - 16
hadoop-tools/hadoop-federation-balance/src/main/java/org/apache/hadoop/tools/fedbalance/FedBalance.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.tools.fedbalance;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -25,7 +26,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.tools.fedbalance.procedure.BalanceProcedure;
 import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
 import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
@@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.tools.fedbalance.procedure.BalanceJob;
 import org.apache.hadoop.tools.fedbalance.procedure.BalanceProcedureScheduler;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
@@ -45,14 +44,13 @@ import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.hadoop.tools.fedbalance.DistCpBalanceOptions.ROUTER;
-import static org.apache.hadoop.tools.fedbalance.DistCpBalanceOptions.FORCE_CLOSE_OPEN;
-import static org.apache.hadoop.tools.fedbalance.DistCpBalanceOptions.MAP;
-import static org.apache.hadoop.tools.fedbalance.DistCpBalanceOptions.BANDWIDTH;
-import static org.apache.hadoop.tools.fedbalance.DistCpBalanceOptions.TRASH;
-import static org.apache.hadoop.tools.fedbalance.DistCpBalanceOptions.DELAY_DURATION;
-import static org.apache.hadoop.tools.fedbalance.DistCpBalanceOptions.CLI_OPTIONS;
-import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.FEDERATION_BALANCE_CLASS;
+import static org.apache.hadoop.tools.fedbalance.FedBalanceOptions.ROUTER;
+import static org.apache.hadoop.tools.fedbalance.FedBalanceOptions.FORCE_CLOSE_OPEN;
+import static org.apache.hadoop.tools.fedbalance.FedBalanceOptions.MAP;
+import static org.apache.hadoop.tools.fedbalance.FedBalanceOptions.BANDWIDTH;
+import static org.apache.hadoop.tools.fedbalance.FedBalanceOptions.TRASH;
+import static org.apache.hadoop.tools.fedbalance.FedBalanceOptions.DELAY_DURATION;
+import static org.apache.hadoop.tools.fedbalance.FedBalanceOptions.CLI_OPTIONS;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.TrashOption;
 
 /**
@@ -73,6 +71,10 @@ public class FedBalance extends Configured implements Tool {
   private static final String MOUNT_TABLE_PROCEDURE = "mount-table-procedure";
   private static final String TRASH_PROCEDURE = "trash-procedure";
 
+  private static final String FED_BALANCE_DEFAULT_XML =
+      "hdfs-fedbalance-default.xml";
+  private static final String FED_BALANCE_SITE_XML = "hdfs-fedbalance-site.xml";
+
   /**
    * This class helps building the balance job.
    */
@@ -210,7 +212,7 @@ public class FedBalance extends Configured implements Tool {
   public int run(String[] args) throws Exception {
     CommandLineParser parser = new GnuParser();
     CommandLine command =
-        parser.parse(DistCpBalanceOptions.CLI_OPTIONS, args, true);
+        parser.parse(FedBalanceOptions.CLI_OPTIONS, args, true);
     String[] leftOverArgs = command.getArgs();
     if (leftOverArgs == null || leftOverArgs.length < 1) {
       printUsage();
@@ -355,19 +357,33 @@ public class FedBalance extends Configured implements Tool {
         CLI_OPTIONS);
   }
 
+  /**
+   * Loads properties from hdfs-fedbalance-default.xml into configuration
+   * object.
+   *
+   * @return Configuration which includes properties from
+   *         hdfs-fedbalance-default.xml and hdfs-fedbalance-site.xml
+   */
+  @VisibleForTesting
+  static Configuration getDefaultConf() {
+    Configuration config = new Configuration();
+    config.addResource(FED_BALANCE_DEFAULT_XML);
+    config.addResource(FED_BALANCE_SITE_XML);
+    return config;
+  }
+
   /**
    * Main function of the FedBalance program. Parses the input arguments and
    * invokes the FedBalance::run() method, via the ToolRunner.
    * @param argv Command-line arguments sent to FedBalance.
    */
   public static void main(String[] argv) {
-    Configuration conf = new HdfsConfiguration();
-    Class<Tool> balanceClazz = (Class<Tool>) conf
-        .getClass(FEDERATION_BALANCE_CLASS, FedBalance.class);
-    Tool balancer = ReflectionUtils.newInstance(balanceClazz, conf);
+    Configuration conf = getDefaultConf();
+    FedBalance fedBalance = new FedBalance();
+    fedBalance.setConf(conf);
     int exitCode;
     try {
-      exitCode = ToolRunner.run(balancer, argv);
+      exitCode = ToolRunner.run(fedBalance, argv);
     } catch (Exception e) {
       LOG.warn("Couldn't complete FedBalance operation.", e);
       exitCode = -1;

+ 8 - 9
hadoop-tools/hadoop-federation-balance/src/main/java/org/apache/hadoop/tools/fedbalance/FedBalanceConfigs.java

@@ -24,9 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public final class FedBalanceConfigs {
-  /* The class used for federation balance */
-  public static final String FEDERATION_BALANCE_CLASS =
-      "federation.balance.class";
+
   public static final String LAST_SNAPSHOT_NAME = "DISTCP-BALANCE-CURRENT";
   public static final String CURRENT_SNAPSHOT_NAME = "DISTCP-BALANCE-NEXT";
   /* Specify the behaviour of trash. */
@@ -34,17 +32,18 @@ public final class FedBalanceConfigs {
     TRASH, DELETE, SKIP
   }
 
-  /* The worker threads number of the BalanceProcedureScheduler */
+  /* The worker threads number of the BalanceProcedureScheduler.
+   BalanceProcedureScheduler is responsible for scheduling a balance job,
+   including submit, run, delay and recover. */
   public static final String WORK_THREAD_NUM =
-      "hadoop.hdfs.procedure.work.thread.num";
+      "hdfs.fedbalance.procedure.work.thread.num";
   public static final int WORK_THREAD_NUM_DEFAULT = 10;
-  /* The uri of the journal */
+  /* The uri of the journal, the journal file is used for handling the job
+   persistence and recover. */
   public static final String SCHEDULER_JOURNAL_URI =
-      "hadoop.hdfs.procedure.scheduler.journal.uri";
+      "hdfs.fedbalance.procedure.scheduler.journal.uri";
   public static final String JOB_PREFIX = "JOB-";
   public static final String TMP_TAIL = ".tmp";
-  public static final String JOURNAL_CLASS =
-      "hadoop.hdfs.procedure.journal.class";
 
   private FedBalanceConfigs(){}
 }

+ 8 - 8
hadoop-tools/hadoop-federation-balance/src/main/java/org/apache/hadoop/tools/fedbalance/DistCpBalanceOptions.java → hadoop-tools/hadoop-federation-balance/src/main/java/org/apache/hadoop/tools/fedbalance/FedBalanceOptions.java

@@ -23,23 +23,23 @@ import org.apache.commons.cli.Options;
 /**
  * Command line options of FedBalance.
  */
-public final class DistCpBalanceOptions {
+public final class FedBalanceOptions {
 
   /**
    * The private construct protects this class from being instantiated.
    */
-  private DistCpBalanceOptions() {}
+  private FedBalanceOptions() {}
 
   /**
    * Run in router-based federation mode.
    */
   final static Option ROUTER = new Option("router", false,
-      "If `true` the command runs in router mode. The source path is "
-          + "taken as a mount point. It will disable write by setting the mount"
-          + " point readonly. Otherwise the command works in normal federation"
-          + " mode. The source path is taken as the full path. It will disable"
-          + " write by cancelling all permissions of the source path. The"
-          + " default value is `true`.");
+      "If this option is set then the command runs in router mode."
+          + " The source path is taken as a mount point. It will disable write"
+          + " by setting the mount point readonly. Otherwise the command works"
+          + " in normal federation mode. The source path is taken as the full"
+          + " path. It will disable write by cancelling all permissions of the"
+          + " source path.");
 
   /**
    * If true, in DIFF_DISTCP stage it will force close all open files when

+ 2 - 5
hadoop-tools/hadoop-federation-balance/src/main/java/org/apache/hadoop/tools/fedbalance/procedure/BalanceProcedureScheduler.java

@@ -35,14 +35,12 @@ import org.apache.commons.lang3.builder.CompareToBuilder;
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.WORK_THREAD_NUM;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.WORK_THREAD_NUM_DEFAULT;
-import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.JOURNAL_CLASS;
 /**
  * <pre>
  * The state machine framework consist of:
@@ -115,9 +113,8 @@ public class BalanceProcedureScheduler {
     this.readerThread.start();
 
     // init journal.
-    Class<BalanceJournal> clazz = (Class<BalanceJournal>) conf
-        .getClass(JOURNAL_CLASS, BalanceJournalInfoHDFS.class);
-    journal = ReflectionUtils.newInstance(clazz, conf);
+    journal = new BalanceJournalInfoHDFS();
+    journal.setConf(conf);
 
     if (recoverJobs) {
       recoverAllJobs();

+ 41 - 0
hadoop-tools/hadoop-federation-balance/src/main/resources/hdfs-fedbalance-default.xml

@@ -0,0 +1,41 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  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.
+-->
+
+<!-- Do not modify this file directly. Anything that need to be overwritten,
+     need to be done so, through -D switches or customized conf -->
+
+<configuration>
+
+    <property>
+        <name>hdfs.fedbalance.procedure.scheduler.journal.uri</name>
+        <value>hdfs://localhost:8020/tmp/procedure</value>
+        <description>
+            The uri of the journal, the journal file is used for handling the
+            job persistence and recover.
+        </description>
+    </property>
+
+    <property>
+        <name>hdfs.fedbalance.procedure.work.thread.num</name>
+        <value>10</value>
+        <description>
+            The worker threads number of the BalanceProcedureScheduler.
+            BalanceProcedureScheduler is responsible for scheduling a balance
+            job, including submit, run, delay and recover.
+        </description>
+    </property>
+
+</configuration>

+ 34 - 0
hadoop-tools/hadoop-federation-balance/src/test/java/org/apache/hadoop/tools/fedbalance/TestFedBalance.java

@@ -0,0 +1,34 @@
+/**
+ * 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.tools.fedbalance;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+import static junit.framework.TestCase.assertNotNull;
+import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.SCHEDULER_JOURNAL_URI;
+import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.WORK_THREAD_NUM;
+
+public class TestFedBalance {
+  @Test
+  public void testLoadFedBalanceDefaultConf() {
+    Configuration conf = FedBalance.getDefaultConf();
+    assertNotNull(conf.get(SCHEDULER_JOURNAL_URI));
+    assertNotNull(conf.get(WORK_THREAD_NUM));
+  }
+}