You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by si...@apache.org on 2012/09/26 13:04:08 UTC

svn commit: r1390401 - in /zookeeper/bookkeeper/trunk: ./ hedwig-server/bin/ hedwig-server/conf/ hedwig-server/src/main/java/org/apache/hedwig/server/netty/ hedwig-server/src/main/java/org/apache/hedwig/server/regions/ hedwig-server/src/test/java/org/a...

Author: sijie
Date: Wed Sep 26 11:04:08 2012
New Revision: 1390401

URL: http://svn.apache.org/viewvc?rev=1390401&view=rev
Log:
BOOKKEEPER-397: Make the hedwig client in RegionManager configurable. (Aniruddha via sijie)

Added:
    zookeeper/bookkeeper/trunk/hedwig-server/conf/hw_region_client.conf   (with props)
Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/hedwig-server/bin/hedwig
    zookeeper/bookkeeper/trunk/hedwig-server/conf/hwenv.sh
    zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/netty/PubSubServer.java
    zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/regions/HedwigHubClientFactory.java
    zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/netty/TestPubSubServer.java

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1390401&r1=1390400&r2=1390401&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Wed Sep 26 11:04:08 2012
@@ -164,6 +164,8 @@ Trunk (unreleased changes)
 
         BOOKKEEPER-252: Hedwig: provide a subscription mode to kill other subscription channel when hedwig client is used as a proxy-style server. (sijie via ivank)
 
+        BOOKKEEPER-397: Make the hedwig client in RegionManager configurable. (Aniruddha via sijie)
+
       hedwig-client:
 
         BOOKKEEPER-306: Change C++ client to use gtest for testing (ivank via sijie)

Modified: zookeeper/bookkeeper/trunk/hedwig-server/bin/hedwig
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-server/bin/hedwig?rev=1390401&r1=1390400&r2=1390401&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-server/bin/hedwig (original)
+++ zookeeper/bookkeeper/trunk/hedwig-server/bin/hedwig Wed Sep 26 11:04:08 2012
@@ -53,6 +53,7 @@ BINDIR=`dirname "$0"`
 HW_HOME=`cd $BINDIR/..;pwd`
 
 DEFAULT_CONF=$HW_HOME/conf/hw_server.conf
+DEFAULT_REGION_CLIENT_CONF=$HW_HOME/conf/hw_region_client.conf
 DEFAULT_LOG_CONF=$HW_HOME/conf/log4j.properties
 
 . $HW_HOME/conf/hwenv.sh
@@ -108,6 +109,8 @@ or command is the full name of a class w
 
 Environment variables:
    HEDWIG_SERVER_CONF           Hedwig server configuration file (default $DEFAULT_CONF)
+   HEDWIG_REGION_CLIENT_CONF           Configuration file for the hedwig client used by the
+                                region manager (default $DEFAULT_REGION_CLIENT_CONF)
    HEDWIG_CONSOLE_SERVER_CONF   Server part configuration for hedwig console,
                                 used for metadata management (defaults to HEDWIG_SERVER_CONF)
    HEDWIG_CONSOLE_CLIENT_CONF   Client part configuration for hedwig console,
@@ -136,6 +139,10 @@ if [ -z "$HEDWIG_SERVER_CONF" ]; then
     HEDWIG_SERVER_CONF=$DEFAULT_CONF;
 fi
 
+if [ -z "$HEDWIG_REGION_CLIENT_CONF" ]; then
+    HEDWIG_REGION_CLIENT_CONF=$DEFAULT_REGION_CLIENT_CONF;
+fi
+
 if [ -z "$HEDWIG_LOG_CONF" ]; then
     HEDWIG_LOG_CONF=$DEFAULT_LOG_CONF
 fi
@@ -164,7 +171,7 @@ OPTS="$OPTS -Dhedwig.log.file=$HEDWIG_LO
 # Change to HW_HOME to support relative paths
 cd "$BK_HOME"
 if [ $COMMAND == "server" ]; then
-    exec java $OPTS $JMX_ARGS org.apache.hedwig.server.netty.PubSubServer $HEDWIG_SERVER_CONF $@
+    exec java $OPTS $JMX_ARGS org.apache.hedwig.server.netty.PubSubServer $HEDWIG_SERVER_CONF $HEDWIG_REGION_CLIENT_CONF $@
 elif [ $COMMAND == "console" ]; then
     # hedwig console configuration server part
     if [ -z "$HEDWIG_CONSOLE_SERVER_CONF" ]; then

Added: zookeeper/bookkeeper/trunk/hedwig-server/conf/hw_region_client.conf
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-server/conf/hw_region_client.conf?rev=1390401&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-server/conf/hw_region_client.conf (added)
+++ zookeeper/bookkeeper/trunk/hedwig-server/conf/hw_region_client.conf Wed Sep 26 11:04:08 2012
@@ -0,0 +1,42 @@
+#   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.
+
+# This is the configuration file for the hedwig client used by the region manager
+
+# This parameter is a boolean flag indicating if communication with the
+# server should be done via SSL for encryption. The Hedwig server hubs also
+# need to be SSL enabled for this to work.
+# ssl_enabled=false
+
+# The maximum message size in bytes
+# max_message_size=2097152
+
+# The maximum number of redirects we permit before signalling an error
+# max_server_redirects=2
+
+# A flag indicating whether the client library should automatically send
+# consume messages to the server
+# auto_send_consume_message_enabled=true
+
+# The number of messages we buffer before sending a consume message
+# to the server
+# consumed_messages_buffer_size=5
+
+# Support for client side throttling.
+# max_outstanding_messages=10
+
+# The timeout in milliseconds before we error out any existing
+# requests
+# server_ack_response_timeout=30000

Propchange: zookeeper/bookkeeper/trunk/hedwig-server/conf/hw_region_client.conf
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: zookeeper/bookkeeper/trunk/hedwig-server/conf/hwenv.sh
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-server/conf/hwenv.sh?rev=1390401&r1=1390400&r2=1390401&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-server/conf/hwenv.sh (original)
+++ zookeeper/bookkeeper/trunk/hedwig-server/conf/hwenv.sh Wed Sep 26 11:04:08 2012
@@ -21,7 +21,13 @@
 # */
 
 # default settings for starting hedwig
-#HEDWIG_SERVER_CONF=
+# HEDWIG_SERVER_CONF=
+
+# default settings for the region manager's hedwig client
+# HEDWIG_REGION_CLIENT_CONF=
+
+# default settings for the region manager's hedwig client
+# HEDWIG_CLIENT_CONF=
 
 # Server part configuration for hedwig console,
 # used for metadata management

Modified: zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/netty/PubSubServer.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/netty/PubSubServer.java?rev=1390401&r1=1390400&r2=1390401&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/netty/PubSubServer.java (original)
+++ zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/netty/PubSubServer.java Wed Sep 26 11:04:08 2012
@@ -94,6 +94,7 @@ public class PubSubServer {
     ServerSocketChannelFactory serverChannelFactory;
     ClientSocketChannelFactory clientChannelFactory;
     ServerConfiguration conf;
+    org.apache.hedwig.client.conf.ClientConfiguration clientConfiguration;
     ChannelGroup allChannels;
 
     // Manager components that make up the PubSubServer
@@ -159,7 +160,8 @@ public class PubSubServer {
     }
 
     protected RegionManager instantiateRegionManager(PersistenceManager pm, ScheduledExecutorService scheduler) {
-        return new RegionManager(pm, conf, zk, scheduler, new HedwigHubClientFactory(conf, clientChannelFactory));
+        return new RegionManager(pm, conf, zk, scheduler, new HedwigHubClientFactory(conf, clientConfiguration,
+                clientChannelFactory));
     }
 
     protected void instantiateZookeeperClient() throws Exception {
@@ -349,13 +351,18 @@ public class PubSubServer {
      * @throws InterruptedException
      * @throws ConfigurationException
      */
-    public PubSubServer(final ServerConfiguration conf,
+    public PubSubServer(final ServerConfiguration serverConfiguration,
+                        final org.apache.hedwig.client.conf.ClientConfiguration clientConfiguration,
                         final Thread.UncaughtExceptionHandler exceptionHandler)
             throws ConfigurationException {
 
-        // First validate the conf
-        this.conf = conf;
-        conf.validate();
+        // First validate the serverConfiguration
+        this.conf = serverConfiguration;
+        serverConfiguration.validate();
+
+        // Validate the client configuration
+        this.clientConfiguration = clientConfiguration;
+        clientConfiguration.validate();
 
         // We need a custom thread group, so that we can override the uncaught
         // exception method
@@ -425,8 +432,13 @@ public class PubSubServer {
         }
     }
 
-    public PubSubServer(ServerConfiguration conf) throws Exception {
-        this(conf, new TerminateJVMExceptionHandler());
+    public PubSubServer(ServerConfiguration serverConfiguration,
+                        org.apache.hedwig.client.conf.ClientConfiguration clientConfiguration) throws Exception {
+        this(serverConfiguration, clientConfiguration, new TerminateJVMExceptionHandler());
+    }
+
+    public PubSubServer(ServerConfiguration serverConfiguration) throws Exception {
+        this(serverConfiguration, new org.apache.hedwig.client.conf.ClientConfiguration());
     }
 
     /**
@@ -451,22 +463,38 @@ public class PubSubServer {
     public static void main(String[] args) {
 
         logger.info("Attempting to start Hedwig");
-        ServerConfiguration conf = new ServerConfiguration();
+        ServerConfiguration serverConfiguration = new ServerConfiguration();
+        // The client configuration for the hedwig client in the region manager.
+        org.apache.hedwig.client.conf.ClientConfiguration regionMgrClientConfiguration
+                = new org.apache.hedwig.client.conf.ClientConfiguration();
         if (args.length > 0) {
             String confFile = args[0];
             try {
-                conf.loadConf(new File(confFile).toURI().toURL());
+                serverConfiguration.loadConf(new File(confFile).toURI().toURL());
             } catch (MalformedURLException e) {
-                String msg = "Could not open configuration file: " + confFile;
+                String msg = "Could not open server configuration file: " + confFile;
                 errorMsgAndExit(msg, e, RC_INVALID_CONF_FILE);
             } catch (ConfigurationException e) {
-                String msg = "Malformed configuration file: " + confFile;
+                String msg = "Malformed server configuration file: " + confFile;
                 errorMsgAndExit(msg, e, RC_MISCONFIGURED);
             }
             logger.info("Using configuration file " + confFile);
         }
+        if (args.length > 1) {
+            // args[1] is the client configuration file.
+            String confFile = args[1];
+            try {
+                regionMgrClientConfiguration.loadConf(new File(confFile).toURI().toURL());
+            } catch (MalformedURLException e) {
+                String msg = "Could not open client configuration file: " + confFile;
+                errorMsgAndExit(msg, e, RC_INVALID_CONF_FILE);
+            } catch (ConfigurationException e) {
+                String msg = "Malformed client configuration file: " + confFile;
+                errorMsgAndExit(msg, e, RC_MISCONFIGURED);
+            }
+        }
         try {
-            new PubSubServer(conf).start();
+            new PubSubServer(serverConfiguration, regionMgrClientConfiguration).start();
         } catch (Throwable t) {
             errorMsgAndExit("Error during startup", t, RC_OTHER);
         }

Modified: zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/regions/HedwigHubClientFactory.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/regions/HedwigHubClientFactory.java?rev=1390401&r1=1390400&r2=1390401&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/regions/HedwigHubClientFactory.java (original)
+++ zookeeper/bookkeeper/trunk/hedwig-server/src/main/java/org/apache/hedwig/server/regions/HedwigHubClientFactory.java Wed Sep 26 11:04:08 2012
@@ -17,21 +17,28 @@
  */
 package org.apache.hedwig.server.regions;
 
+import org.apache.commons.configuration.ConfigurationException;
 import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
 
 import org.apache.hedwig.client.conf.ClientConfiguration;
 import org.apache.hedwig.server.common.ServerConfiguration;
 import org.apache.hedwig.util.HedwigSocketAddress;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class HedwigHubClientFactory {
 
     private final ServerConfiguration cfg;
+    private final ClientConfiguration clientConfiguration;
     private final ClientSocketChannelFactory channelFactory;
+    private static final Logger logger = LoggerFactory.getLogger(HedwigHubClientFactory.class);
 
-    // Constructor that takes in a ServerConfiguration and a ChannelFactory
+    // Constructor that takes in a ServerConfiguration, ClientConfiguration and a ChannelFactory
     // so we can reuse it for all Clients created here.
-    public HedwigHubClientFactory(ServerConfiguration cfg, ClientSocketChannelFactory channelFactory) {
+    public HedwigHubClientFactory(ServerConfiguration cfg, ClientConfiguration clientConfiguration,
+                                  ClientSocketChannelFactory channelFactory) {
         this.cfg = cfg;
+        this.clientConfiguration = clientConfiguration;
         this.channelFactory = channelFactory;
     }
 
@@ -44,7 +51,7 @@ public class HedwigHubClientFactory {
      */
     HedwigHubClient create(final HedwigSocketAddress hub) {
         // Create a hub specific version of the client to use
-        return new HedwigHubClient(new ClientConfiguration() {
+        ClientConfiguration hubClientConfiguration = new ClientConfiguration() {
             @Override
             protected HedwigSocketAddress getDefaultServerHedwigSocketAddress() {
                 return hub;
@@ -52,9 +59,16 @@ public class HedwigHubClientFactory {
 
             @Override
             public boolean isSSLEnabled() {
-                return cfg.isInterRegionSSLEnabled();
+                return cfg.isInterRegionSSLEnabled() || clientConfiguration.isSSLEnabled();
             }
-        }, channelFactory);
+        };
+        try {
+            hubClientConfiguration.addConf(this.clientConfiguration.getConf());
+        } catch (ConfigurationException e) {
+            String msg = "Configuration exception while loading the client configuration for the region manager.";
+            logger.error(msg);
+            throw new RuntimeException(msg);
+        }
+        return new HedwigHubClient(hubClientConfiguration, channelFactory);
     }
-
 }

Modified: zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/netty/TestPubSubServer.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/netty/TestPubSubServer.java?rev=1390401&r1=1390400&r2=1390401&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/netty/TestPubSubServer.java (original)
+++ zookeeper/bookkeeper/trunk/hedwig-server/src/test/java/org/apache/hedwig/server/netty/TestPubSubServer.java Wed Sep 26 11:04:08 2012
@@ -85,7 +85,7 @@ public class TestPubSubServer extends Pu
                 return port;
             }
 
-        }, uncaughtExceptionHandler) {
+        }, new ClientConfiguration(), uncaughtExceptionHandler) {
 
             @Override
             protected TopicManager instantiateTopicManager() throws IOException {