You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by dr...@apache.org on 2022/08/16 13:18:36 UTC

[ratis] 07/12: RATIS-1661. Support configurable hostname in GrpcService (#707)

This is an automated email from the ASF dual-hosted git repository.

dragonyliu pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/ratis.git

commit 396d696540d9acb8f959b594f8dcb44bc27cbbe1
Author: William Song <48...@users.noreply.github.com>
AuthorDate: Tue Aug 9 01:02:47 2022 +0800

    RATIS-1661. Support configurable hostname in GrpcService (#707)
    
    
    (cherry picked from commit 485c7eccd036ea9afb3750ba77ab325bc76b0037)
---
 .../java/org/apache/ratis/grpc/GrpcConfigKeys.java | 34 ++++++++++++++++++++++
 .../org/apache/ratis/grpc/server/GrpcService.java  | 30 ++++++++++++-------
 2 files changed, 53 insertions(+), 11 deletions(-)

diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java
index b227dfb37..8163d18de 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java
@@ -97,6 +97,16 @@ public interface GrpcConfigKeys {
   interface Admin {
     String PREFIX = GrpcConfigKeys.PREFIX + ".admin";
 
+    String HOST_KEY = PREFIX + ".host";
+    String HOST_DEFAULT = null;
+    static String host(RaftProperties properties) {
+      final String fallbackServerHost = Server.host(properties, null);
+      return get(properties::get, HOST_KEY, HOST_DEFAULT, Server.HOST_KEY, fallbackServerHost, getDefaultLog());
+    }
+    static void setHost(RaftProperties properties, String host) {
+      set(properties::set, HOST_KEY, host);
+    }
+
     String PORT_KEY = PREFIX + ".port";
     int PORT_DEFAULT = -1;
     static int port(RaftProperties properties) {
@@ -121,6 +131,16 @@ public interface GrpcConfigKeys {
   interface Client {
     String PREFIX = GrpcConfigKeys.PREFIX + ".client";
 
+    String HOST_KEY = PREFIX + ".host";
+    String HOST_DEFAULT = null;
+    static String host(RaftProperties properties) {
+      final String fallbackServerHost = Server.host(properties, null);
+      return get(properties::get, HOST_KEY, HOST_DEFAULT, Server.HOST_KEY, fallbackServerHost, getDefaultLog());
+    }
+    static void setHost(RaftProperties properties, String host) {
+      set(properties::set, HOST_KEY, host);
+    }
+
     String PORT_KEY = PREFIX + ".port";
     int PORT_DEFAULT = -1;
     static int port(RaftProperties properties) {
@@ -145,6 +165,20 @@ public interface GrpcConfigKeys {
   interface Server {
     String PREFIX = GrpcConfigKeys.PREFIX + ".server";
 
+    String HOST_KEY = PREFIX + ".host";
+    String HOST_DEFAULT = null;
+    static String host(RaftProperties properties) {
+      return host(properties, getDefaultLog());
+    }
+
+    static String host(RaftProperties properties, Consumer<String> logger) {
+      return get(properties::get, HOST_KEY, HOST_DEFAULT, logger);
+    }
+
+    static void setHost(RaftProperties properties, String host) {
+      set(properties::set, HOST_KEY, host);
+    }
+
     String PORT_KEY = PREFIX + ".port";
     int PORT_DEFAULT = 0;
     static int port(RaftProperties properties) {
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java
index 9d65cba39..40e413915 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java
@@ -120,9 +120,15 @@ public final class GrpcService extends RaftServerRpcWithProxy<GrpcServerProtocol
   private GrpcService(RaftServer server,
       GrpcTlsConfig adminTlsConfig, GrpcTlsConfig clientTlsConfig, GrpcTlsConfig serverTlsConfig) {
     this(server, server::getId,
-        GrpcConfigKeys.Admin.port(server.getProperties()), adminTlsConfig,
-        GrpcConfigKeys.Client.port(server.getProperties()), clientTlsConfig,
-        GrpcConfigKeys.Server.port(server.getProperties()), serverTlsConfig,
+        GrpcConfigKeys.Admin.host(server.getProperties()),
+        GrpcConfigKeys.Admin.port(server.getProperties()),
+        adminTlsConfig,
+        GrpcConfigKeys.Client.host(server.getProperties()),
+        GrpcConfigKeys.Client.port(server.getProperties()),
+        clientTlsConfig,
+        GrpcConfigKeys.Server.host(server.getProperties()),
+        GrpcConfigKeys.Server.port(server.getProperties()),
+        serverTlsConfig,
         GrpcConfigKeys.messageSizeMax(server.getProperties(), LOG::info),
         RaftServerConfigKeys.Log.Appender.bufferByteLimit(server.getProperties()),
         GrpcConfigKeys.flowControlWindow(server.getProperties(), LOG::info),
@@ -132,9 +138,9 @@ public final class GrpcService extends RaftServerRpcWithProxy<GrpcServerProtocol
 
   @SuppressWarnings("checkstyle:ParameterNumber") // private constructor
   private GrpcService(RaftServer raftServer, Supplier<RaftPeerId> idSupplier,
-      int adminPort, GrpcTlsConfig adminTlsConfig,
-      int clientPort, GrpcTlsConfig clientTlsConfig,
-      int serverPort, GrpcTlsConfig serverTlsConfig,
+      String adminHost, int adminPort, GrpcTlsConfig adminTlsConfig,
+      String clientHost, int clientPort, GrpcTlsConfig clientTlsConfig,
+      String serverHost, int serverPort, GrpcTlsConfig serverTlsConfig,
       SizeInBytes grpcMessageSizeMax, SizeInBytes appenderBufferSize,
       SizeInBytes flowControlWindow,TimeDuration requestTimeoutDuration,
       boolean useSeparateHBChannel) {
@@ -163,7 +169,7 @@ public final class GrpcService extends RaftServerRpcWithProxy<GrpcServerProtocol
     final boolean separateClientServer = clientPort != serverPort && clientPort > 0;
 
     final NettyServerBuilder serverBuilder =
-        startBuildingNettyServer(serverPort, serverTlsConfig, grpcMessageSizeMax, flowControlWindow);
+        startBuildingNettyServer(serverHost, serverPort, serverTlsConfig, grpcMessageSizeMax, flowControlWindow);
     serverBuilder.addService(ServerInterceptors.intercept(
         new GrpcServerProtocolService(idSupplier, raftServer), serverInterceptor));
     if (!separateAdminServer) {
@@ -179,7 +185,7 @@ public final class GrpcService extends RaftServerRpcWithProxy<GrpcServerProtocol
 
     if (separateAdminServer) {
       final NettyServerBuilder builder =
-          startBuildingNettyServer(adminPort, adminTlsConfig, grpcMessageSizeMax, flowControlWindow);
+          startBuildingNettyServer(adminHost, adminPort, adminTlsConfig, grpcMessageSizeMax, flowControlWindow);
       addAdminService(raftServer, builder);
       final Server adminServer = builder.build();
       servers.put(GrpcAdminProtocolService.class.getName(), adminServer);
@@ -190,7 +196,7 @@ public final class GrpcService extends RaftServerRpcWithProxy<GrpcServerProtocol
 
     if (separateClientServer) {
       final NettyServerBuilder builder =
-          startBuildingNettyServer(clientPort, clientTlsConfig, grpcMessageSizeMax, flowControlWindow);
+          startBuildingNettyServer(clientHost, clientPort, clientTlsConfig, grpcMessageSizeMax, flowControlWindow);
       addClientService(builder);
       final Server clientServer = builder.build();
       servers.put(GrpcClientProtocolService.class.getName(), clientServer);
@@ -214,9 +220,11 @@ public final class GrpcService extends RaftServerRpcWithProxy<GrpcServerProtocol
           serverInterceptor));
   }
 
-  private static NettyServerBuilder startBuildingNettyServer(int port, GrpcTlsConfig tlsConfig,
+  private static NettyServerBuilder startBuildingNettyServer(String hostname, int port, GrpcTlsConfig tlsConfig,
       SizeInBytes grpcMessageSizeMax, SizeInBytes flowControlWindow) {
-    NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forPort(port)
+    InetSocketAddress address = hostname == null || hostname.isEmpty() ?
+        new InetSocketAddress(port) : new InetSocketAddress(hostname, port);
+    NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forAddress(address)
         .withChildOption(ChannelOption.SO_REUSEADDR, true)
         .maxInboundMessageSize(grpcMessageSizeMax.getSizeInt())
         .flowControlWindow(flowControlWindow.getSizeInt());