You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by sz...@apache.org on 2017/02/18 23:35:15 UTC

incubator-ratis git commit: RATIS-24 Move out the Hadoop RPC config keys from RaftServerConfigKeys.

Repository: incubator-ratis
Updated Branches:
  refs/heads/master 5565425f0 -> bcb806d1c


RATIS-24 Move out the Hadoop RPC config keys from RaftServerConfigKeys.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/bcb806d1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/bcb806d1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/bcb806d1

Branch: refs/heads/master
Commit: bcb806d1c1ca30214eb16a324d822088aaf6be37
Parents: 5565425
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Sat Feb 18 15:28:30 2017 -0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Sat Feb 18 15:28:30 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/ratis/conf/ConfUtils.java   | 50 ++++++++++++++++
 .../server/HadoopRpcServerConfigKeys.java       | 56 ++++++++++++++++++
 .../hadooprpc/server/HadoopRpcService.java      | 20 ++-----
 .../hadooprpc/MiniRaftClusterWithHadoopRpc.java |  8 +--
 .../ratis/server/RaftServerConfigKeys.java      | 61 --------------------
 5 files changed, 115 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/bcb806d1/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java
new file mode 100644
index 0000000..e43ac60
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java
@@ -0,0 +1,50 @@
+package org.apache.ratis.conf;
+
+import org.apache.ratis.util.NetUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetSocketAddress;
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+
+public abstract class ConfUtils {
+  static Logger LOG = LoggerFactory.getLogger(ConfUtils.class);
+
+  public static int getInt(
+      BiFunction<String, Integer, Integer> getInt,
+      String key, int defaultValue, Integer min, Integer max) {
+    final int value = getInt.apply(key, defaultValue);
+    final String s = key + " = " + value;
+    LOG.info(s);
+
+    if (min != null && value < min) {
+      throw new IllegalArgumentException(s + " < min = " + min);
+    }
+    if (max != null && value > max) {
+      throw new IllegalArgumentException(s + " > max = " + max);
+    }
+    return value;
+  }
+
+  public static String getString(
+      BiFunction<String, String, String> getString,
+      String key, String defaultValue) {
+    final String value = getString.apply(key, defaultValue);
+    LOG.info(key + " = " + value);
+    return value;
+  }
+
+  public static InetSocketAddress getInetSocketAddress(
+      BiFunction<String, String, String> getString,
+      String key, String defaultValue) {
+    return NetUtils.createSocketAddr(getString(getString, key, defaultValue));
+  }
+
+  public static void setString(
+      BiConsumer<String, String> setString,
+      String key, String value) {
+    setString.accept(key, value);
+    LOG.info("set " + key + " = " + value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/bcb806d1/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcServerConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcServerConfigKeys.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcServerConfigKeys.java
new file mode 100644
index 0000000..4cdbdb3
--- /dev/null
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcServerConfigKeys.java
@@ -0,0 +1,56 @@
+/**
+ * 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.ratis.hadooprpc.server;
+
+import org.apache.ratis.conf.ConfUtils;
+
+import java.net.InetSocketAddress;
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+
+public interface HadoopRpcServerConfigKeys {
+  String PREFIX = "raft.hadooprpc";
+
+  /** IPC server configurations */
+  abstract class Ipc {
+    public static final String PREFIX = HadoopRpcServerConfigKeys.PREFIX + ".ipc";
+
+    public static final String ADDRESS_KEY = PREFIX + ".address";
+    public static final int DEFAULT_PORT = 10718;
+    public static final String ADDRESS_DEFAULT = "0.0.0.0:" + DEFAULT_PORT;
+
+    public static final String HANDLERS_KEY = PREFIX + ".handlers";
+    public static final int HANDLERS_DEFAULT = 10;
+
+    public static int handlers(BiFunction<String, Integer, Integer> getInt) {
+      return ConfUtils.getInt(getInt,
+          HANDLERS_KEY, HANDLERS_DEFAULT, 1, null);
+    }
+
+    public static InetSocketAddress address(BiFunction<String, String, String> getTrimmed) {
+      return ConfUtils.getInetSocketAddress(getTrimmed,
+          ADDRESS_KEY, ADDRESS_DEFAULT);
+    }
+
+    public static void setAddress(
+        BiConsumer<String, String> setString,
+        String address) {
+      ConfUtils.setString(setString, ADDRESS_KEY, address);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/bcb806d1/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
index 7f4a251..93f896c 100644
--- a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
@@ -28,7 +28,6 @@ import org.apache.ratis.protocol.RaftClientProtocol;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.RaftServer;
-import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.RaftServerRpc;
 import org.apache.ratis.server.protocol.RaftServerProtocol;
 import org.apache.ratis.shaded.com.google.protobuf.BlockingService;
@@ -59,6 +58,9 @@ public class HadoopRpcService implements RaftServerRpc {
     }
 
     public Configuration getConf() {
+      if (conf == null) {
+        conf = new Configuration();
+      }
       return conf;
     }
 
@@ -114,20 +116,8 @@ public class HadoopRpcService implements RaftServerRpc {
 
   private RPC.Server newRpcServer(RaftServerProtocol serverProtocol, final Configuration conf)
       throws IOException {
-    final RaftServerConfigKeys.Get get = new RaftServerConfigKeys.Get() {
-      @Override
-      protected int getInt(String key, int defaultValue) {
-        return conf.getInt(key, defaultValue);
-      }
-
-      @Override
-      protected String getTrimmed(String key, String defaultValue) {
-        return conf.getTrimmed(key, defaultValue);
-      }
-    };
-
-    final int handlerCount = get.ipc().handlers();
-    final InetSocketAddress address = get.ipc().address();
+    final int handlerCount = HadoopRpcServerConfigKeys.Ipc.handlers(conf::getInt);
+    final InetSocketAddress address = HadoopRpcServerConfigKeys.Ipc.address(conf::getTrimmed);
 
     final BlockingService service
         = RaftServerProtocolService.newReflectiveBlockingService(

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/bcb806d1/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java
index b8c69d5..f1247fd 100644
--- a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java
+++ b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java
@@ -24,9 +24,9 @@ import org.apache.ratis.RaftTestUtil;
 import org.apache.ratis.client.RaftClientRequestSender;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.hadooprpc.client.HadoopClientRequestSender;
+import org.apache.ratis.hadooprpc.server.HadoopRpcServerConfigKeys;
 import org.apache.ratis.hadooprpc.server.HadoopRpcService;
 import org.apache.ratis.protocol.RaftPeer;
-import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.DelayLocalExecutionInjection;
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.slf4j.Logger;
@@ -46,7 +46,7 @@ public class MiniRaftClusterWithHadoopRpc extends MiniRaftCluster.RpcBase {
     public MiniRaftClusterWithHadoopRpc newCluster(
         String[] ids, RaftProperties prop, boolean formatted) throws IOException {
       final Configuration conf = new Configuration();
-      conf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, "0.0.0.0:0");
+      HadoopRpcServerConfigKeys.Ipc.setAddress(conf::set, "0.0.0.0:0");
       return new MiniRaftClusterWithHadoopRpc(ids, prop, conf, formatted);
     }
   };
@@ -86,7 +86,7 @@ public class MiniRaftClusterWithHadoopRpc extends MiniRaftCluster.RpcBase {
   @Override
   protected RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException {
     Configuration hconf = new Configuration(hadoopConf);
-    hconf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, peer.getAddress());
+    HadoopRpcServerConfigKeys.Ipc.setAddress(hconf::set, peer.getAddress());
 
     RaftServerImpl server = servers.get(peer.getId());
     final HadoopRpcService rpc = HadoopRpcService.newBuilder()
@@ -96,7 +96,7 @@ public class MiniRaftClusterWithHadoopRpc extends MiniRaftCluster.RpcBase {
     Preconditions.checkState(
         rpc.getInetSocketAddress().toString().contains(peer.getAddress()),
         "address in the raft conf: %s, address in rpc server: %s",
-        peer.getAddress(), rpc.getInetSocketAddress().toString());
+        peer.getAddress(), rpc.getInetSocketAddress());
     server.setServerRpc(rpc);
     return server;
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/bcb806d1/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
index 09c77a9..8e6d54f 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
@@ -28,34 +28,6 @@ public interface RaftServerConfigKeys {
 
   String PREFIX = "raft.server";
 
-  /** IPC server configurations */
-  interface Ipc {
-    String PREFIX  = RaftServerConfigKeys.PREFIX + ".ipc";
-
-    String ADDRESS_KEY = PREFIX + ".address";
-    int    DEFAULT_PORT = 10718;
-    String ADDRESS_DEFAULT = "0.0.0.0:" + DEFAULT_PORT;
-
-    String HANDLERS_KEY = PREFIX + ".handlers";
-    int    HANDLERS_DEFAULT = 10;
-
-    class Getters {
-      private final Get get;
-
-      Getters(Get get) {
-        this.get = get;
-      }
-
-      public int handlers() {
-        return get.getInt(HANDLERS_KEY, HANDLERS_DEFAULT, 1, null);
-      }
-
-      public InetSocketAddress address() {
-        return get.getInetSocketAddress(ADDRESS_KEY, ADDRESS_DEFAULT);
-      }
-    }
-  }
-
   enum Factory {
     NETTY("org.apache.ratis.server.impl.ServerFactory$BaseFactory"),
     GRPC("org.apache.ratis.grpc.server.GrpcServerFactory"),
@@ -150,37 +122,4 @@ public interface RaftServerConfigKeys {
   String RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_KEY = "raft.server.log.appender.batch.enabled";
   boolean RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_DEFAULT = false;
 
-  /** An utility class to get conf values. */
-  abstract class Get {
-    static Logger LOG = LoggerFactory.getLogger(RaftServerConfigKeys.class);
-
-    private final Ipc.Getters ipc = new Ipc.Getters(this);
-
-    protected abstract int getInt(String key, int defaultValue);
-
-    int getInt(String key, int defaultValue, Integer min, Integer max) {
-      final int value = getInt(key, defaultValue);
-      final String s = key + " = " + value;
-      if (min != null && value < min) {
-        throw new IllegalArgumentException(s + " < min = " + min);
-      }
-      if (max != null && value > max) {
-        throw new IllegalArgumentException(s + " > max = " + max);
-      }
-      LOG.info(s);
-      return value;
-    }
-
-    protected abstract String getTrimmed(String key, String defaultValue);
-
-    InetSocketAddress getInetSocketAddress(String key, String defaultValue) {
-      final String address = getTrimmed(key, defaultValue);
-      LOG.info(key + " = " + address);
-      return NetUtils.createSocketAddr(address);
-    }
-
-    public Ipc.Getters ipc() {
-      return ipc;
-    }
-  }
 }