Browse Source

HDFS-8582. Support getting a list of reconfigurable config properties and do not generate spurious reconfig warnings (Lei (Eddy) Xu via Colin P. McCabe)

Colin Patrick Mccabe 10 năm trước cách đây
mục cha
commit
d112d18324

+ 9 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java

@@ -88,6 +88,11 @@ public abstract class ReconfigurableBase
     reconfigurationUtil = Preconditions.checkNotNull(ru);
   }
 
+  /**
+   * Create a new configuration.
+   */
+  protected abstract Configuration getNewConf();
+
   @VisibleForTesting
   public Collection<PropertyChange> getChangedProperties(
       Configuration newConf, Configuration oldConf) {
@@ -108,17 +113,16 @@ public abstract class ReconfigurableBase
     public void run() {
       LOG.info("Starting reconfiguration task.");
       Configuration oldConf = this.parent.getConf();
-      Configuration newConf = new Configuration();
+      Configuration newConf = this.parent.getNewConf();
       Collection<PropertyChange> changes =
           this.parent.getChangedProperties(newConf, oldConf);
       Map<PropertyChange, Optional<String>> results = Maps.newHashMap();
       for (PropertyChange change : changes) {
         String errorMessage = null;
         if (!this.parent.isPropertyReconfigurable(change.prop)) {
-          errorMessage = "Property " + change.prop +
-              " is not reconfigurable";
-          LOG.info(errorMessage);
-          results.put(change, Optional.of(errorMessage));
+          LOG.info(String.format(
+              "Property %s is not configurable: old value: %s, new value: %s",
+              change.prop, change.oldVal, change.newVal));
           continue;
         }
         LOG.info("Change property: " + change.prop + " from \""

+ 11 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java

@@ -118,6 +118,11 @@ public class TestReconfiguration {
       super(conf);
     }
 
+    @Override
+    protected Configuration getNewConf() {
+      return new Configuration();
+    }
+
     @Override 
     public Collection<String> getReconfigurableProperties() {
       return Arrays.asList(PROP1, PROP2, PROP4);
@@ -336,6 +341,11 @@ public class TestReconfiguration {
       super(conf);
     }
 
+    @Override
+    protected Configuration getNewConf() {
+      return new Configuration();
+    }
+
     final CountDownLatch latch = new CountDownLatch(1);
 
     @Override
@@ -397,7 +407,7 @@ public class TestReconfiguration {
 
     waitAsyncReconfigureTaskFinish(dummy);
     ReconfigurationTaskStatus status = dummy.getReconfigurationTaskStatus();
-    assertEquals(3, status.getStatus().size());
+    assertEquals(2, status.getStatus().size());
     for (Map.Entry<PropertyChange, Optional<String>> result :
         status.getStatus().entrySet()) {
       PropertyChange change = result.getKey();

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -659,6 +659,10 @@ Release 2.8.0 - UNRELEASED
     HDFS-8605. Merge Refactor of DFSOutputStream from HDFS-7285 branch.
     (vinayakumarb via wang)
 
+    HDFS-8582. Support getting a list of reconfigurable config properties and
+    do not generate spurious reconfig warnings (Lei (Eddy) Xu via Colin P.
+    McCabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java

@@ -158,6 +158,11 @@ public interface ClientDatanodeProtocol {
    */
   ReconfigurationTaskStatus getReconfigurationStatus() throws IOException;
 
+  /**
+   * Get a list of allowed properties for reconfiguration.
+   */
+  List<String> listReconfigurableProperties() throws IOException;
+
   /**
    * Trigger a new block report.
    */

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java

@@ -44,6 +44,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdf
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto.Builder;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ListReconfigurablePropertiesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ListReconfigurablePropertiesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ShutdownDatanodeRequestProto;
@@ -208,6 +210,23 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
     return START_RECONFIG_RESP;
   }
 
+  @Override
+  public ListReconfigurablePropertiesResponseProto listReconfigurableProperties(
+        RpcController controller,
+        ListReconfigurablePropertiesRequestProto request)
+      throws ServiceException {
+    ListReconfigurablePropertiesResponseProto.Builder builder =
+        ListReconfigurablePropertiesResponseProto.newBuilder();
+    try {
+      for (String name : impl.listReconfigurableProperties()) {
+        builder.addName(name);
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return builder.build();
+  }
+
   @Override
   public GetReconfigurationStatusResponseProto getReconfigurationStatus(
       RpcController unused, GetReconfigurationStatusRequestProto request)

+ 18 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -52,6 +52,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ListReconfigurablePropertiesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ListReconfigurablePropertiesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReconfigurationStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReconfigurationStatusResponseProto;
@@ -59,7 +61,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetRec
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ShutdownDatanodeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.StartReconfigurationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.TriggerBlockReportRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.TriggerBlockReportResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -101,6 +102,9 @@ public class ClientDatanodeProtocolTranslatorPB implements
       GetReconfigurationStatusRequestProto.newBuilder().build();
   private final static StartReconfigurationRequestProto VOID_START_RECONFIG =
       StartReconfigurationRequestProto.newBuilder().build();
+  private static final ListReconfigurablePropertiesRequestProto
+      VOID_LIST_RECONFIGURABLE_PROPERTIES =
+      ListReconfigurablePropertiesRequestProto.newBuilder().build();
 
   public ClientDatanodeProtocolTranslatorPB(DatanodeID datanodeid,
       Configuration conf, int socketTimeout, boolean connectToDnViaHostname,
@@ -337,6 +341,19 @@ public class ClientDatanodeProtocolTranslatorPB implements
     return new ReconfigurationTaskStatus(startTime, endTime, statusMap);
   }
 
+  @Override
+  public List<String> listReconfigurableProperties()
+      throws IOException {
+    ListReconfigurablePropertiesResponseProto response;
+    try {
+      response = rpcProxy.listReconfigurableProperties(NULL_CONTROLLER,
+          VOID_LIST_RECONFIGURABLE_PROPERTIES);
+      return response.getNameList();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   @Override
   public void triggerBlockReport(BlockReportOptions options)
       throws IOException {

+ 18 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -272,6 +272,11 @@ public class DataNode extends ReconfigurableBase
 
   static final int CURRENT_BLOCK_FORMAT_VERSION = 1;
 
+  /** A list of property that are reconfigurable at runtime. */
+  private static final List<String> RECONFIGURABLE_PROPERTIES =
+      Collections.unmodifiableList(
+          Arrays.asList(DFS_DATANODE_DATA_DIR_KEY));
+
   /**
    * Use {@link NetUtils#createSocketAddr(String)} instead.
    */
@@ -449,6 +454,11 @@ public class DataNode extends ReconfigurableBase
             });
   }
 
+  @Override  // ReconfigurableBase
+  protected Configuration getNewConf() {
+    return new HdfsConfiguration();
+  }
+
   @Override
   public void reconfigurePropertyImpl(String property, String newVal)
       throws ReconfigurationException {
@@ -469,11 +479,9 @@ public class DataNode extends ReconfigurableBase
   /**
    * Get a list of the keys of the re-configurable properties in configuration.
    */
-  @Override
+  @Override // Reconfigurable
   public Collection<String> getReconfigurableProperties() {
-    List<String> reconfigurable =
-        Collections.unmodifiableList(Arrays.asList(DFS_DATANODE_DATA_DIR_KEY));
-    return reconfigurable;
+    return RECONFIGURABLE_PROPERTIES;
   }
 
   /**
@@ -3051,6 +3059,12 @@ public class DataNode extends ReconfigurableBase
     return getReconfigurationTaskStatus();
   }
 
+  @Override // ClientDatanodeProtocol
+  public List<String> listReconfigurableProperties()
+      throws IOException {
+    return RECONFIGURABLE_PROPERTIES;
+  }
+
   @Override // ClientDatanodeProtocol
   public void triggerBlockReport(BlockReportOptions options)
       throws IOException {

+ 47 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -407,7 +407,7 @@ public class DFSAdmin extends FsShell {
     "\t[-refreshSuperUserGroupsConfiguration]\n" +
     "\t[-refreshCallQueue]\n" +
     "\t[-refresh <host:ipc_port> <key> [arg1..argn]\n" +
-    "\t[-reconfig <datanode|...> <host:ipc_port> <start|status>]\n" +
+    "\t[-reconfig <datanode|...> <host:ipc_port> <start|status|properties>]\n" +
     "\t[-printTopology]\n" +
     "\t[-refreshNamenodes datanode_host:ipc_port]\n"+
     "\t[-deleteBlockPool datanode_host:ipc_port blockpoolId [force]]\n"+
@@ -979,8 +979,9 @@ public class DFSAdmin extends FsShell {
 
     String refreshCallQueue = "-refreshCallQueue: Reload the call queue from config\n";
 
-    String reconfig = "-reconfig <datanode|...> <host:ipc_port> <start|status>:\n" +
-        "\tStarts reconfiguration or gets the status of an ongoing reconfiguration.\n" +
+    String reconfig = "-reconfig <datanode|...> <host:ipc_port> <start|status|properties>:\n" +
+        "\tStarts or gets the status of a reconfiguration operation, \n" +
+        "\tor gets a list of reconfigurable properties.\n" +
         "\tThe second parameter specifies the node type.\n" +
         "\tCurrently, only reloading DataNode's configuration is supported.\n";
 
@@ -1439,6 +1440,9 @@ public class DFSAdmin extends FsShell {
       return startReconfiguration(nodeType, address);
     } else if ("status".equals(op)) {
       return getReconfigurationStatus(nodeType, address, System.out, System.err);
+    } else if ("properties".equals(op)) {
+      return getReconfigurableProperties(
+          nodeType, address, System.out, System.err);
     }
     System.err.println("Unknown operation: " + op);
     return -1;
@@ -1476,18 +1480,24 @@ public class DFSAdmin extends FsShell {
 
         out.println(" and finished at " +
             new Date(status.getEndTime()).toString() + ".");
+        if (status.getStatus() == null) {
+          // Nothing to report.
+          return 0;
+        }
         for (Map.Entry<PropertyChange, Optional<String>> result :
             status.getStatus().entrySet()) {
           if (!result.getValue().isPresent()) {
-            out.print("SUCCESS: ");
+            out.printf(
+                "SUCCESS: Changed property %s%n\tFrom: \"%s\"%n\tTo: \"%s\"%n",
+                result.getKey().prop, result.getKey().oldVal,
+                result.getKey().newVal);
           } else {
-            out.print("FAILED: ");
-          }
-          out.printf("Change property %s%n\tFrom: \"%s\"%n\tTo: \"%s\"%n",
-              result.getKey().prop, result.getKey().oldVal,
-              result.getKey().newVal);
-          if (result.getValue().isPresent()) {
-            out.println("\tError: " + result.getValue().get() + ".");
+            final String errorMsg = result.getValue().get();
+            out.printf(
+                  "FAILED: Change property %s%n\tFrom: \"%s\"%n\tTo: \"%s\"%n",
+                  result.getKey().prop, result.getKey().oldVal,
+                  result.getKey().newVal);
+            out.println("\tError: " + errorMsg + ".");
           }
         }
       } catch (IOException e) {
@@ -1495,7 +1505,32 @@ public class DFSAdmin extends FsShell {
         return 1;
       }
     } else {
-      err.println("Node type " + nodeType + " does not support reconfiguration.");
+      err.println("Node type " + nodeType +
+          " does not support reconfiguration.");
+      return 1;
+    }
+    return 0;
+  }
+
+  int getReconfigurableProperties(String nodeType, String address,
+      PrintStream out, PrintStream err) throws IOException {
+    if ("datanode".equals(nodeType)) {
+      ClientDatanodeProtocol dnProxy = getDataNodeProxy(address);
+      try {
+        List<String> properties =
+            dnProxy.listReconfigurableProperties();
+        out.println(
+            "Configuration properties that are allowed to be reconfigured:");
+        for (String name : properties) {
+          out.println(name);
+        }
+      } catch (IOException e) {
+        err.println("DataNode reconfiguration: " + e + ".");
+        return 1;
+      }
+    } else {
+      err.println("Node type " + nodeType +
+          " does not support reconfiguration.");
       return 1;
     }
     return 0;

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto

@@ -180,6 +180,14 @@ message GetReconfigurationStatusResponseProto {
   repeated GetReconfigurationStatusConfigChangeProto changes = 3;
 }
 
+message ListReconfigurablePropertiesRequestProto {
+}
+
+/** Query the reconfigurable properties on DataNode. */
+message ListReconfigurablePropertiesResponseProto {
+  repeated string name = 1;
+}
+
 /**
  * Protocol used from client to the Datanode.
  * See the request and response for details of rpc call.
@@ -230,6 +238,10 @@ service ClientDatanodeProtocolService {
   rpc startReconfiguration(StartReconfigurationRequestProto)
       returns(StartReconfigurationResponseProto);
 
+  rpc listReconfigurableProperties(
+      ListReconfigurablePropertiesRequestProto)
+      returns(ListReconfigurablePropertiesResponseProto);
+
   rpc triggerBlockReport(TriggerBlockReportRequestProto)
       returns(TriggerBlockReportResponseProto);
 }

+ 90 - 27
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java

@@ -22,6 +22,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.ReconfigurationUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -42,6 +43,7 @@ import static org.hamcrest.CoreMatchers.allOf;
 import static org.hamcrest.CoreMatchers.anyOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.not;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.hamcrest.CoreMatchers.containsString;
@@ -50,18 +52,17 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 public class TestDFSAdmin {
+  private Configuration conf = null;
   private MiniDFSCluster cluster;
   private DFSAdmin admin;
   private DataNode datanode;
 
   @Before
   public void setUp() throws Exception {
-    Configuration conf = new Configuration();
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
-    cluster.waitActive();
+    conf = new Configuration();
+    restartCluster();
 
     admin = new DFSAdmin();
-    datanode = cluster.getDataNodes().get(0);
   }
 
   @After
@@ -72,6 +73,15 @@ public class TestDFSAdmin {
     }
   }
 
+  private void restartCluster() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    datanode = cluster.getDataNodes().get(0);
+  }
+
   private List<String> getReconfigureStatus(String nodeType, String address)
       throws IOException {
     ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
@@ -87,16 +97,26 @@ public class TestDFSAdmin {
     return outputs;
   }
 
-  @Test(timeout = 30000)
-  public void testGetReconfigureStatus()
+  /**
+   * Test reconfiguration and check the status outputs.
+   * @param expectedSuccuss set true if the reconfiguration task should success.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void testGetReconfigurationStatus(boolean expectedSuccuss)
       throws IOException, InterruptedException {
     ReconfigurationUtil ru = mock(ReconfigurationUtil.class);
     datanode.setReconfigurationUtil(ru);
 
     List<ReconfigurationUtil.PropertyChange> changes =
-        new ArrayList<ReconfigurationUtil.PropertyChange>();
+        new ArrayList<>();
     File newDir = new File(cluster.getDataDirectory(), "data_new");
-    newDir.mkdirs();
+    if (expectedSuccuss) {
+      newDir.mkdirs();
+    } else {
+      // Inject failure.
+      newDir.createNewFile();
+    }
     changes.add(new ReconfigurationUtil.PropertyChange(
         DFS_DATANODE_DATA_DIR_KEY, newDir.toString(),
         datanode.getConf().get(DFS_DATANODE_DATA_DIR_KEY)));
@@ -121,31 +141,74 @@ public class TestDFSAdmin {
       Thread.sleep(100);
     }
     assertTrue(count > 0);
-    assertThat(outputs.size(), is(8));  // 3 (SUCCESS) + 4 (FAILED)
+    if (expectedSuccuss) {
+      assertThat(outputs.size(), is(4));
+    } else {
+      assertThat(outputs.size(), is(6));
+    }
 
     List<StorageLocation> locations = DataNode.getStorageLocations(
         datanode.getConf());
-    assertThat(locations.size(), is(1));
-    assertThat(locations.get(0).getFile(), is(newDir));
-    // Verify the directory is appropriately formatted.
-    assertTrue(new File(newDir, Storage.STORAGE_DIR_CURRENT).isDirectory());
-
-    int successOffset = outputs.get(1).startsWith("SUCCESS:") ? 1 : 5;
-    int failedOffset = outputs.get(1).startsWith("FAILED:") ? 1: 4;
-    assertThat(outputs.get(successOffset),
-        containsString("Change property " + DFS_DATANODE_DATA_DIR_KEY));
-    assertThat(outputs.get(successOffset + 1),
+    if (expectedSuccuss) {
+      assertThat(locations.size(), is(1));
+      assertThat(locations.get(0).getFile(), is(newDir));
+      // Verify the directory is appropriately formatted.
+      assertTrue(new File(newDir, Storage.STORAGE_DIR_CURRENT).isDirectory());
+    } else {
+      assertTrue(locations.isEmpty());
+    }
+
+    int offset = 1;
+    if (expectedSuccuss) {
+      assertThat(outputs.get(offset),
+          containsString("SUCCESS: Changed property " +
+              DFS_DATANODE_DATA_DIR_KEY));
+    } else {
+      assertThat(outputs.get(offset),
+          containsString("FAILED: Change property " +
+              DFS_DATANODE_DATA_DIR_KEY));
+    }
+    assertThat(outputs.get(offset + 1),
         is(allOf(containsString("From:"), containsString("data1"),
             containsString("data2"))));
-    assertThat(outputs.get(successOffset + 2),
+    assertThat(outputs.get(offset + 2),
         is(not(anyOf(containsString("data1"), containsString("data2")))));
-    assertThat(outputs.get(successOffset + 2),
+    assertThat(outputs.get(offset + 2),
         is(allOf(containsString("To"), containsString("data_new"))));
-    assertThat(outputs.get(failedOffset),
-        containsString("Change property randomKey"));
-    assertThat(outputs.get(failedOffset + 1),
-        containsString("From: \"old456\""));
-    assertThat(outputs.get(failedOffset + 2),
-        containsString("To: \"new123\""));
+  }
+
+  @Test(timeout = 30000)
+  public void testGetReconfigurationStatus()
+      throws IOException, InterruptedException {
+    testGetReconfigurationStatus(true);
+    restartCluster();
+    testGetReconfigurationStatus(false);
+  }
+
+  private List<String> getReconfigurationAllowedProperties(
+      String nodeType, String address)
+      throws IOException {
+    ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
+    PrintStream out = new PrintStream(bufOut);
+    ByteArrayOutputStream bufErr = new ByteArrayOutputStream();
+    PrintStream err = new PrintStream(bufErr);
+    admin.getReconfigurableProperties(nodeType, address, out, err);
+    Scanner scanner = new Scanner(bufOut.toString());
+    List<String> outputs = Lists.newArrayList();
+    while (scanner.hasNextLine()) {
+      outputs.add(scanner.nextLine());
+    }
+    return outputs;
+  }
+
+  @Test(timeout = 30000)
+  public void testGetReconfigAllowedProperties() throws IOException {
+    final int port = datanode.getIpcPort();
+    final String address = "localhost:" + port;
+    List<String> outputs =
+        getReconfigurationAllowedProperties("datanode", address);
+    assertEquals(2, outputs.size());
+    assertEquals(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+        outputs.get(1));
   }
 }