Forráskód Böngészése

HDFS-9414. Refactor reconfiguration of ClientDatanodeProtocol for reusability. Contributed by Xiaobing Zhou.

(cherry picked from commit 303f80e6567023985f00e5f33ff41a7bbb4d6072)
cnauroth 9 éve
szülő
commit
1072be12b9

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReconfigurationProtocol.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.conf.ReconfigurationTaskStatus;
 @InterfaceStability.Evolving
 public interface ReconfigurationProtocol {
 
-  long versionID = 1L;
+  long VERSIONID = 1L;
 
   /**
    * Asynchronously reload configuration on disk and apply changes.

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolPB.java

@@ -24,13 +24,6 @@ import org.apache.hadoop.ipc.ProtocolInfo;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.ReconfigurationProtocolService;
 
-@KerberosInfo(serverPrincipal =
-    CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
-@ProtocolInfo(
-    protocolName = "org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol",
-    protocolVersion = 1)
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
 /**
  * Protocol that clients use to communicate with the NN/DN to do
  * reconfiguration on the fly.
@@ -38,6 +31,13 @@ import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.Recon
  * Note: This extends the protocolbuffer service based interface to
  * add annotations required for security.
  */
+@KerberosInfo(serverPrincipal =
+    CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
+@ProtocolInfo(
+    protocolName = "org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol",
+    protocolVersion = 1)
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
 public interface ReconfigurationProtocolPB extends
     ReconfigurationProtocolService.BlockingInterface {
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolUtils.java

@@ -32,7 +32,7 @@ import com.google.common.collect.Maps;
  * This is a client side utility class that handles
  * common logic to to parameter reconfiguration.
  */
-public class ReconfigurationProtocolUtils {
+public final class ReconfigurationProtocolUtils {
   private ReconfigurationProtocolUtils() {
   }
 

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java

@@ -0,0 +1,18 @@
+/**
+ * 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.protocolPB;

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

@@ -32,7 +32,7 @@ import com.google.common.base.Optional;
  * This is a server side utility class that handles
  * common logic to to parameter reconfiguration.
  */
-public class ReconfigurationProtocolServerSideUtils {
+public final class ReconfigurationProtocolServerSideUtils {
   private ReconfigurationProtocolServerSideUtils() {
   }
 

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

@@ -925,7 +925,7 @@ public class DataNode extends ReconfigurableBase
         .setSecretManager(blockPoolTokenSecretManager).build();
 
     ReconfigurationProtocolServerSideTranslatorPB reconfigurationProtocolXlator
-      = new ReconfigurationProtocolServerSideTranslatorPB(this);
+        = new ReconfigurationProtocolServerSideTranslatorPB(this);
     service = ReconfigurationProtocolService
         .newReflectiveBlockingService(reconfigurationProtocolXlator);
     DFSUtil.addPBProtocol(conf, ReconfigurationProtocolPB.class, service,